hang the runner, agent=new sheriff (#270)

* fix docker build

this is trivially incorrect since glide doesn't actually provide reproducible
builds. the idea is to build with the deps that we have checked into git, so
that we actually know what code is executing so that we might debug it...

all for multi stage build instead of what we had, but adding the glide step is
wrong. i added a loud warning so as to discourage this behavior in the future.

* hang the runner, agent=new sheriff

tl;dr agent is now runner, with a hopefully saner api

the general idea is get rid of all the various 'task' structs now, change our
terminology to only be 'calls' now, push a lot of the http construction of a
call into the agent, allow calls to mutate their state around their execution
easily and to simplify the number of code paths, channels and context timeouts
in something [hopefully] easy to understand.

this introduces the idea of 'slots' which are either hot or cold and are
separate from reserving memory (memory is denominated in 'tokens' now).
a 'slot' is essentially a container that is ready for execution of a call, be
it hot or cold (it just means different things based on hotness). taking a
look into Submit should make these relatively easy to grok.

sorry, things were pretty broken especially wrt timings. I tried to keep good
notes (maybe too good), to highlight stuff so that we don't make the same
mistakes again (history repeating itself blah blah quote). even now, there is
lots of work to do :)

I encourage just reading the agent.go code, Submit is really simple and
there's a description of how the whole thing works at the head of the file
(after TODOs). call.go contains code for constructing calls, as well as Start
/ End (small atm). I did some amount of code massaging to try to make things
simple / straightforward / fit reasonable mental model, but as always am open
to critique (the more negative the better) as I'm just one guy and wth do i
know...

-----------------------------------------------------------------------------

below enumerates a number of changes as briefly as possible (heh..):

models.Call all the things

removes models.Task as models.Call is now what it previously was.
models.FnCall is now rid of in favor of models.Call, despite the datastore
only storing a few fields of it [for now]. we should probably store entire
calls in the db, since app & route configurations can change at any given
moment, it would be nice to see the parameters of each call (costs db space,
obviously).

this removes the endpoints for getting & deleting messages, we were just
looping back to localhost to call the MQ (wtf? this was for iron integration i
think) and just calls the MQ.

changes the name of the FnLog to LogStore, confusing cause there's also a
`FuncLogger` which uses the Logstore (punting). removes other `Fn` prefixed
structs (redundant naming convention).

removes some unused and/or weird structs (IDStatus, CompleteTime)

updates the swagger

makes the db methods consistent to use 'Call' nomenclature.

remove runner nuisances:

* push down registry stuff to docker driver
* remove Environment / Stats stuff of yore
* remove unused writers (now in FuncLogger)
* remove 2 of the task types, old hot stuff, runner, etc

fixes ram available calculation on startup to not always be 300GB (helps a lot
on a laptop!)

format for DOCKER_AUTH env now is not a list but a map (there are no docs,
would prefer to get rid of this altogether anyway). the ~/.docker/cfg expected
format is unchanged.

removes arbitrary task queue, if a machine is out of ram we can probably just
time out without queueing... (can open separate discussion) in any case the
old one didn't really account well for hot tasks, it just lined everyone up in
the task queue if there wasn't a place to run hot and then timed them out
[even if a slot became free].

removes HEADER_ prefixing on any headers in the request to a invoke a call.
(this was inconsistent with cli for test anyway)

removes TASK_ID header sent in to hot only (this is a dupe of FN_CALL_ID,
which has not been removed)

now user functions can reply directly to the client. this means that for
cold containers if they write to stdout it will send a 200 + headers. for
hot containers, the user can reply directly to the client from the container,
i.e. with its preferred status code / headers (vs. always getting a 200).
the dispatch itself is a little http specific atm, i think we can add an
interchange format but the current version is easily extended to add json for
now, separate discussion. this eliminates a lot of the request/response
rewriting and buffering we were doing (yey). now Dispatch ONLY does input and
output, vs. managing the call timeout and having access to a call's fields.

cache is pushed down into agent now instead of in the front end, would like to
push it down to the datastore actually but it's here for now anyway. cache
delete functions removed (b/c fn is distributed anyway?). added app caching,
should help with latency.

in general, a lot of server/runner.go got pushed down into the agent. i think
it will be useful in testing to be able to construct calls without having to
invoke http handlers + async also needs to construct calls without a handler.

safe shutdown actually works now for everything (leaked / didn't wait on
certain things before)

now we're waiting for hot slots to open up while we're attempting to get ram
to launch a container if we didn't find any hot slots to run the call in
immediately. we can change this policy really easily now (no more channel
jungle; still some channels). also looking for somewhere else to go while the
container is launching now. slots now get sent _out_ of a container, vs.
a container receiving calls, which makes this kind of policy easier to
implement. this fixes a number of bugs around things like trying to execute
calls against containers that have not and may never start and trying to
launch a bazillion containers when there are no free containers. the driver api
underwent some changes to make this possible (relatively minimal, added Wait).
the easiest way to think about this is that allocating ram has moved 'up'
instead of just wrapping launching containers, so that we can select on a
channel trying to find ram.

not dispatching hot calls to containers that died anymore either...

the timeout is now started at the beginning of Submit, rather than Dispatch or
the container itself having to manage the call timeout, which was an
inaccurate way of doing things since finding a slot / allocating ram / pulling
image can all take a non-trivial (timeout amount, even!) amount of time. this
makes for much more reasonable response times from fn under load, there's
still a little TODO about handling cold+timeout container removal response
times but it's much improved.

if call.Start is called with < call.timeout/2 time left, then the call will
not be executed and return a timeout. we can discuss. this makes async play
_a lot_ nicer, specifically. for large timeouts / 2 makes less sense.

env is no longer getting upper cased (admittedly, this can look a little weird
now). our whole route.Config/app.Config/env/headers stuff probably deserves a
whole discussion...

sync output no longer has the call id in json if there's an error / timeout.
we could add this back to signify that it's _us_ writing these but this was
out of place. FN_CALL_ID is still shipped out to get the id for sync calls,
and async [server] output remains unchanged.

async logs are now an entire raw http request (so that a user can write a 400
or something from their hot async container)

async hot now 'just works'

cold sync calls can now reply to the client before container removal, which
shaves a lot of latency off of those (still eat start). still need to figure
out async removal if timeout or something.

-----------------------------------------------------------------------------

i've located a number of bugs that were generally inherited, and also added
a number of TODOs in the head of the agent.go file according to robustness we
probably need to add. this is at least at parity with the previous
implementation, to my knowledge (hopefully/likely a good bit ahead). I can
memorialize these to github quickly enough, not that anybody searches before
adding bugs anyway (sigh).

the big thing to work on next imo is async being a lot more robust,
specifically to survive fn server failures / network issues.

thanks for review (gulp)
This commit is contained in:
Reed Allman
2017-09-05 10:32:51 -07:00
committed by Denis Makogon
parent 1b1b64436f
commit 71a88a991c
100 changed files with 2151 additions and 4121 deletions

View File

@@ -1,2 +0,0 @@
vendor/
_vendor*/

View File

@@ -3,9 +3,18 @@ FROM golang:alpine AS build-env
RUN apk --no-cache add build-base git bzr mercurial gcc RUN apk --no-cache add build-base git bzr mercurial gcc
ENV D=/go/src/github.com/fnproject/fn ENV D=/go/src/github.com/fnproject/fn
# If dep ever gets decent enough to use, try `dep ensure --vendor-only` from here: https://medium.com/travis-on-docker/triple-stage-docker-builds-with-go-and-angular-1b7d2006cb88 # If dep ever gets decent enough to use, try `dep ensure --vendor-only` from here: https://medium.com/travis-on-docker/triple-stage-docker-builds-with-go-and-angular-1b7d2006cb88
RUN go get -u github.com/Masterminds/glide # IF WE DO GLIDE INSTALL THEN THE DEPS DON'T MATCH WHAT WE HAVE IN GIT!!!!!!!
ADD glide.* $D/ # IF WE DO GLIDE INSTALL THEN THE DEPS DON'T MATCH WHAT WE HAVE IN GIT!!!!!!!
RUN cd $D && glide install -v # IF WE DO GLIDE INSTALL THEN THE DEPS DON'T MATCH WHAT WE HAVE IN GIT!!!!!!!
# IF WE DO GLIDE INSTALL THEN THE DEPS DON'T MATCH WHAT WE HAVE IN GIT!!!!!!!
# IF WE DO GLIDE INSTALL THEN THE DEPS DON'T MATCH WHAT WE HAVE IN GIT!!!!!!!
# IF WE DO GLIDE INSTALL THEN THE DEPS DON'T MATCH WHAT WE HAVE IN GIT!!!!!!!
# IF WE DO GLIDE INSTALL THEN THE DEPS DON'T MATCH WHAT WE HAVE IN GIT!!!!!!!
# IF WE DO GLIDE INSTALL THEN THE DEPS DON'T MATCH WHAT WE HAVE IN GIT!!!!!!!
#RUN go get -u github.com/Masterminds/glide
#ADD glide.* $D/
#RUN cd $D && glide install -v
ADD . $D ADD . $D
RUN cd $D && go build -o fn-alpine && cp fn-alpine /tmp/ RUN cd $D && go build -o fn-alpine && cp fn-alpine /tmp/

View File

@@ -36,7 +36,7 @@ docker-dep:
docker run --rm -it -v ${CURDIR}:/go/src/github.com/fnproject/fn -w /go/src/github.com/fnproject/fn treeder/glide install -v docker run --rm -it -v ${CURDIR}:/go/src/github.com/fnproject/fn -w /go/src/github.com/fnproject/fn treeder/glide install -v
docker-build: docker-build:
docker build --build-arg HTTP_PROXY -t fnproject/functions:latest . docker build --build-arg HTTPS_PROXY --build-arg HTTP_PROXY -t fnproject/functions:latest .
docker-run: docker-build docker-run: docker-build
docker run --rm --privileged -it -e NO_PROXY -e HTTP_PROXY -e LOG_LEVEL=debug -e "DB_URL=sqlite3:///app/data/fn.db" -v ${CURDIR}/data:/app/data -p 8080:8080 funcy/functions docker run --rm --privileged -it -e NO_PROXY -e HTTP_PROXY -e LOG_LEVEL=debug -e "DB_URL=sqlite3:///app/data/fn.db" -v ${CURDIR}/data:/app/data -p 8080:8080 funcy/functions

750
api/agent/agent.go Normal file
View File

@@ -0,0 +1,750 @@
package agent
import (
"context"
"crypto/sha1"
"errors"
"fmt"
"io"
"sort"
"sync"
"time"
"github.com/fnproject/fn/api/agent/drivers"
"github.com/fnproject/fn/api/agent/drivers/docker"
"github.com/fnproject/fn/api/agent/protocol"
"github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/common/singleflight"
"github.com/fnproject/fn/api/id"
"github.com/fnproject/fn/api/models"
"github.com/opentracing/opentracing-go"
"github.com/patrickmn/go-cache"
"github.com/sirupsen/logrus"
)
// TODO remove ccirello/supervisor from vendor/
// TODO make sure some errors that user should see (like image doesn't exist) bubble up
// TODO we should prob store async calls in db immediately since we're returning id (will 404 until post-execution)
// TODO async calls need to add route.Headers as well
// TODO need to shut off reads/writes in dispatch to the pipes when call times out so that
// 2 calls don't have the same container's pipes...
// TODO fix ram token / cold slot racy races (highly unlikely, still fix)
// TODO add spans back around container launching for hot (follows from?) + other more granular spans
// TODO handle timeouts / no response in sync & async (sync is json+503 atm, not 504, async is empty log+status)
// see also: server/runner.go wrapping the response writer there, but need to handle async too (push down?)
// TODO herd launch prevention part deux
// TODO plumb FXLB-WAIT back - can we use headers now? maybe let's use api
// TODO none of the Datastore methods actually use the ctx for timeouts :(
// TODO not adding padding if call times out to store appropriately (ctx timed out, happenstance it works now cuz of ^)
// TODO all Datastore methods need to take unit of tenancy (app or route) at least (e.g. not just call id)
// TODO limit the request body length when making calls
// TODO discuss concrete policy for hot launch or timeout / timeout vs time left
// TODO can we get rid of 'GET url?payload' weirdness?
// TODO call env need to be map[string][]string to match headers behavior...
// TODO it may be nice to have an interchange type for Dispatch that can have
// all the info we need to build e.g. http req, grpc req, json, etc. so that
// we can easily do e.g. http->grpc, grpc->http, http->json. ofc grpc<->http is
// weird for proto specifics like e.g. proto version, method, headers, et al.
// discuss.
// TODO if we don't cap the number of any one container we could get into a situation
// where the machine is full but all the containers are idle up to the idle timeout. meh.
// TODO async is still broken, but way less so. we need to modify mq semantics
// 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 we should prob not be logging all async output to the logs by default...
// TODO the call api should fill in all the fields
// TODO the log api should be plaintext (or at least offer it)
// TODO func logger needs to be hanged, dragged and quartered. in reverse order.
// TODO we should probably differentiate ran-but-timeout vs timeout-before-run
// TODO push down the app/route cache into Datastore that decorates
// TODO between calls, logs and stderr can contain output/ids from previous call. need elegant solution. grossness.
// 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
// about why that may be so (say, whatever it is takes longer than the timeout, e.g.)
// TODO if an image is not found or similar issues in getting a slot, then async should probably
// mark the call as errored rather than forever trying & failing to run it
// TODO it would be really nice if we made the ramToken wrap the driver cookie (less brittle,
// if those leak the container leaks too...) -- not the allocation, but the token.Close and cookie.Close
// TODO if machine is out of ram, just timeout immediately / wait for hot slot? (discuss policy)
//
// Agent exposes an api to create calls from various parameters and then submit
// 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)
// * execute calls against containers
// * invoke Start and End for each call appropriately
// * check the mq for any async calls, and submit them
//
// 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.
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.
GetCall(...CallOpt) (Call, error)
// Submit will attempt to execute a call locally, a Call may store information
// about itself in its Start and End methods, which will be called in Submit
// immediately before and after the Call is executed, respectively. An error
// will be returned if there is an issue executing the call or the error
// may be from the call's execution itself (if, say, the container dies,
// or the call times out).
Submit(Call) error
// Close will wait for any outstanding calls to complete and then exit.
// Close is not safe to be called from multiple threads.
io.Closer
// Stats should be burned at the stake. adding so as to not ruffle feathers.
// TODO this should be derived from our metrics
Stats() Stats
}
type agent struct {
// TODO maybe these should be on GetCall? idk. was getting bloated.
mq models.MessageQueue
ds models.Datastore
driver drivers.Driver
hMu sync.RWMutex // protects hot
hot map[string]chan slot
// TODO could make this separate too...
// cache for apps and routes
cache *cache.Cache
singleflight singleflight.SingleFlight // singleflight assists Datastore // TODO rename
// TODO we could make a separate struct for the memory stuff
// cond protects access to ramUsed
cond *sync.Cond
// ramTotal is the total accessible memory by this process
ramTotal uint64
// ramUsed is ram reserved for running containers. idle hot containers
// count against ramUsed.
ramUsed uint64
// used to track running calls / safe shutdown
wg sync.WaitGroup // TODO rename
shutdown chan struct{}
stats // TODO kill me
}
func New(ds models.Datastore, mq models.MessageQueue) Agent {
// TODO: Create drivers.New(runnerConfig)
driver := docker.NewDocker(drivers.Config{})
a := &agent{
ds: ds,
mq: mq,
driver: driver,
hot: make(map[string]chan slot),
cache: cache.New(5*time.Second, 1*time.Minute),
cond: sync.NewCond(new(sync.Mutex)),
ramTotal: getAvailableMemory(),
shutdown: make(chan struct{}),
}
go a.asyncDequeue() // safe shutdown can nanny this fine
return a
}
func (a *agent) Close() error {
select {
case <-a.shutdown:
default:
close(a.shutdown)
}
a.wg.Wait()
return nil
}
func (a *agent) Submit(callI Call) error {
a.wg.Add(1)
defer a.wg.Done()
select {
case <-a.shutdown:
return errors.New("agent shut down")
default:
}
a.stats.Enqueue()
call := callI.(*call)
ctx := call.req.Context()
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_submit")
defer span.Finish()
// start the timer STAT! TODO add some wiggle room
ctx, cancel := context.WithTimeout(ctx, time.Duration(call.Timeout)*time.Second)
call.req = call.req.WithContext(ctx)
defer cancel()
slot, err := a.getSlot(ctx, call) // find ram available / running
if err != nil {
return err
}
// TODO if the call times out & container is created, we need
// to make this remove the container asynchronously?
defer slot.Close() // notify our slot is free once we're done
// TODO Start is checking the timer now, we could do it here, too.
err = call.Start(ctx)
if err != nil {
return err
}
a.stats.Start()
err = slot.exec(ctx, call)
// pass this error (nil or otherwise) to end directly, to store status, etc
// End may rewrite the error or elect to return it
a.stats.Complete()
// TODO if the context is timed out here we need to allocate some more time...
// right now this only works b/c the db isn't using the context
return call.End(ctx, err)
}
// getSlot must ensure that if it receives a slot, it will be returned, otherwise
// a container will be locked up forever waiting for slot to free.
func (a *agent) getSlot(ctx context.Context, call *call) (slot, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_get_slot")
defer span.Finish()
if protocol.IsStreamable(protocol.Protocol(call.Format)) {
return a.hotSlot(ctx, call)
}
// make new channel and launch 1 for cold
ch := make(chan slot)
return a.launchOrSlot(ctx, ch, call)
}
// launchOrSlot will launch a container that will send slots on the provided channel when it
// is free if no slots are available on that channel first. the returned slot may or may not
// be from the launched container. if there is an error launching a new container (if necessary),
// then that will be returned rather than a slot, if no slot is free first.
func (a *agent) launchOrSlot(ctx context.Context, slots chan slot, call *call) (slot, error) {
var errCh <-chan error
// check if any slot immediately without trying to get a ram token
select {
case s := <-slots:
return s, nil
case <-ctx.Done():
return nil, ctx.Err()
default:
}
// if nothing free, wait for ram token or a slot
select {
case s := <-slots:
return s, nil
case tok := <-a.ramToken(call.Memory * 1024 * 1024): // convert MB TODO mangle
errCh = a.launch(ctx, slots, call, tok) // TODO mangle
case <-ctx.Done():
return nil, ctx.Err()
}
// wait for launch err or a slot to open up (possibly from launch)
select {
case err := <-errCh:
// if we get a launch err, try to return to user (e.g. image not found)
return nil, err
case slot := <-slots:
return slot, nil
case <-ctx.Done():
return nil, ctx.Err()
}
}
func (a *agent) hotSlot(ctx context.Context, call *call) (slot, error) {
slots := a.slots(hotKey(call))
// TODO if we track avg run time we could know how long to wait or
// if we need to launch instead of waiting.
// if we can get a slot in a reasonable amount of time, use it
select {
case s := <-slots:
return s, nil
case <-time.After(100 * time.Millisecond): // XXX(reed): precise^
// TODO this means the first launched container if none are running eats
// this. yes it sucks but there are a lot of other fish to fry, opening a
// policy discussion...
}
// then wait for a slot or try to launch...
return a.launchOrSlot(ctx, slots, call)
}
// TODO this should be a LIFO stack of channels, perhaps. a queue (channel)
// will always send the least recently used, not ideal.
func (a *agent) slots(key string) chan slot {
a.hMu.RLock()
slots, ok := a.hot[key]
a.hMu.RUnlock()
if !ok {
a.hMu.Lock()
slots, ok = a.hot[key]
if !ok {
slots = make(chan slot) // should not be buffered
a.hot[key] = slots
}
a.hMu.Unlock()
}
return slots
}
func hotKey(call *call) string {
// return a sha1 hash of a (hopefully) unique string of all the config
// values, to make map lookups quicker [than the giant unique string]
hash := sha1.New()
fmt.Fprint(hash, call.AppName, "\x00")
fmt.Fprint(hash, call.Path, "\x00")
fmt.Fprint(hash, call.Image, "\x00")
fmt.Fprint(hash, call.Timeout, "\x00")
fmt.Fprint(hash, call.IdleTimeout, "\x00")
fmt.Fprint(hash, call.Memory, "\x00")
fmt.Fprint(hash, call.Format, "\x00")
// we have to sort these before printing, yay. TODO do better
keys := make([]string, 0, len(call.BaseEnv))
for k := range call.BaseEnv {
keys = append(keys, k)
}
sort.Strings(keys)
for _, k := range keys {
fmt.Fprint(hash, k, "\x00", call.BaseEnv[k], "\x00")
}
var buf [sha1.Size]byte
return string(hash.Sum(buf[:0]))
}
// TODO we could rename this more appropriately (ideas?)
type Token interface {
// Close must be called by any thread that receives a token.
io.Closer
}
type token struct {
decrement func()
}
func (t *token) Close() error {
t.decrement()
return nil
}
// NOTE in theory goroutines spawned here could run forever (i.e. leak), if the provided value
// is unable to be satisfied. the calling thread will time out waiting for it and the value
// properly adjusted if ever satisfied, but we could trivially provide a ctx here and time
// out if the calling thread times out if we want to [just to prevent leaks].
//
// the received token should be passed directly to launch (unconditionally), launch
// will close this token (i.e. the receiver should not call Close)
func (a *agent) ramToken(memory uint64) <-chan Token {
// TODO we could do an initial check here and return in the same thread so
// that a calling thread could call this and have a filled channel
// immediately so that a a select default case could be used to determine
// whether machine is at capacity (and caller can decide whether to wait) --
// right now this is a race if used as described.
c := a.cond
ch := make(chan Token)
go func() {
c.L.Lock()
for (a.ramUsed + memory) > a.ramTotal {
// TODO we could add ctx here
c.Wait()
}
a.ramUsed += memory
c.L.Unlock()
t := &token{decrement: func() {
c.L.Lock()
a.ramUsed -= memory
c.L.Unlock()
c.Broadcast()
}}
select {
// TODO fix this race. caller needs to provide channel since we could get here
// before ramToken has returned. :( or something better, idk
case ch <- t:
default:
// if we can't send b/c nobody is waiting anymore, need to decrement here
t.Close()
}
}()
return ch
}
// asyncRam will send a signal on the returned channel when at least half of
// the available RAM on this machine is free.
func (a *agent) asyncRam() chan struct{} {
ch := make(chan struct{})
c := a.cond
go func() {
c.L.Lock()
for (a.ramTotal/2)-a.ramUsed < 0 {
c.Wait()
}
c.L.Unlock()
ch <- struct{}{}
// TODO this could leak forever (only in shutdown, blech)
}()
return ch
}
type slot interface {
exec(ctx context.Context, call *call) error
io.Closer
}
// implements Slot
type coldSlot struct {
cookie drivers.Cookie
tok Token
stderr io.Closer
}
func (s *coldSlot) exec(ctx context.Context, call *call) error {
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_cold_exec")
defer span.Finish()
waiter, err := s.cookie.Run(ctx)
if err != nil {
return err
}
res, err := waiter.Wait(ctx)
if err != nil {
return err
} else if res.Error() != "" {
// check for call error (oom/exit) and beam it up
return res
}
// nil or timed out (Wait will silently return nil if it encounters a timeout, maybe TODO)
return ctx.Err()
}
func (s *coldSlot) Close() error {
if s.cookie != nil {
// call this from here so that in exec we don't have to eat container
// removal latency
s.cookie.Close(context.Background()) // ensure container removal, separate ctx
}
s.tok.Close()
s.stderr.Close()
return nil
}
// implements Slot
type hotSlot struct {
done chan<- struct{} // signal we are done with slot
proto protocol.ContainerIO
errC <-chan error // container error
container *container // TODO mask this
}
func (s *hotSlot) Close() error { close(s.done); return nil }
func (s *hotSlot) exec(ctx context.Context, call *call) error {
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_hot_exec")
defer span.Finish()
stderr := NewFuncLogger(ctx, call.AppName, call.Path, call.Image, call.ID, call.ds)
if call.w == nil {
// send STDOUT to logs if no writer given (async...)
// TODO fuck func logger, change it to not need a context and make calls
// require providing their own stderr and writer instead of this crap. punting atm.
call.w = stderr
}
// link the container id and id in the logs [for us!]
// TODO go is broke idk why logrus.Fields doesn't work
common.Logger(ctx).WithField("container_id", s.container.id).WithField("id", call.ID).Info("starting call")
// swap in the new id and the new stderr logger
s.container.swap(stderr)
defer stderr.Close() // TODO shove in Close / elsewhere (to upload logs after exec exits)
errApp := make(chan error, 1)
go func() {
// TODO make sure stdin / stdout not blocked if container dies or we leak goroutine
// we have to make sure this gets shut down or 2 threads will be reading/writing in/out
errApp <- s.proto.Dispatch(call.w, call.req)
}()
select {
case err := <-s.errC: // error from container
return err
case err := <-errApp:
return err
case <-ctx.Done(): // call timeout
return ctx.Err()
}
// TODO we REALLY need to wait for dispatch to return before conceding our slot
}
// this will work for hot & cold (woo)
// if launch encounters a non-nil error it will send it on the returned channel,
// this can be useful if an image doesn't exist, e.g.
// TODO i don't like how this came out and it's slightly racy w/ unbuffered channels since
// the enclosing thread may not be ready to receive (up to go scheduler), need to tidy up, but
// this race is unlikely (still need to fix, yes)
func (a *agent) launch(ctx context.Context, slots chan<- slot, call *call, tok Token) <-chan error {
ch := make(chan error, 1)
if !protocol.IsStreamable(protocol.Protocol(call.Format)) {
// TODO no
go func() {
err := a.prepCold(ctx, slots, call, tok)
if err != nil {
ch <- err
}
}()
return ch
}
go func() {
err := a.runHot(slots, call, tok)
if err != nil {
ch <- err
}
}()
return ch
}
func (a *agent) prepCold(ctx context.Context, slots chan<- slot, call *call, tok Token) error {
// TODO dupe stderr code, reduce me
stderr := NewFuncLogger(ctx, call.AppName, call.Path, call.Image, call.ID, call.ds)
if call.w == nil {
// send STDOUT to logs if no writer given (async...)
// TODO fuck func logger, change it to not need a context and make calls
// require providing their own stderr and writer instead of this crap. punting atm.
call.w = stderr
}
container := &container{
id: id.New().String(), // XXX we could just let docker generate ids...
image: call.Image,
env: call.EnvVars, // full env
memory: call.Memory,
timeout: time.Duration(call.Timeout) * time.Second, // this is unnecessary, but in case removal fails...
stdin: call.req.Body,
stdout: call.w,
stderr: stderr,
}
// 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)
if err != nil {
tok.Close() // TODO make this less brittle
return err
}
slot := &coldSlot{cookie, tok, stderr}
select {
case slots <- slot: // TODO need to make sure receiver will be ready (go routine race)
default:
slot.Close() // if we can't send this slot, need to take care of it ourselves
}
return nil
}
// TODO add ctx back but be careful to only use for logs/spans
func (a *agent) runHot(slots chan<- slot, call *call, tok Token) error {
if tok == nil {
// TODO we should panic, probably ;)
return errors.New("no token provided, not giving you a slot")
}
defer tok.Close()
// TODO we have to make sure we flush these pipes or we will deadlock
stdinRead, stdinWrite := io.Pipe()
stdoutRead, stdoutWrite := io.Pipe()
proto := protocol.New(protocol.Protocol(call.Format), stdinWrite, stdoutRead)
// we don't want to timeout in here. this is inside of a goroutine and the
// caller can timeout this Call appropriately. e.g. w/ hot if it takes 20
// minutes to pull, then timing out calls for 20 minutes and eventually
// having the image is ideal vs. never getting the image pulled.
// TODO this ctx needs to inherit logger, etc
ctx, shutdownContainer := context.WithCancel(context.Background())
defer shutdownContainer() // close this if our waiter returns
// set up the stderr for the first one to capture any logs before the slot is
// executed.
// TODO need to figure out stderr logging for hot functions at a high level
stderr := &ghostWriter{inner: newLineWriter(&logWriter{ctx: ctx, appName: call.AppName, path: call.Path, image: call.Image, reqID: call.ID})}
container := &container{
id: id.New().String(), // XXX we could just let docker generate ids...
image: call.Image,
env: call.BaseEnv, // only base env
memory: call.Memory,
stdin: stdinRead,
stdout: stdoutWrite,
stderr: stderr,
}
logger := logrus.WithFields(logrus.Fields{"id": container.id, "app": call.AppName, "route": call.Path, "image": call.Image, "memory": call.Memory, "format": call.Format, "idle_timeout": call.IdleTimeout})
cookie, err := a.driver.Prepare(ctx, container)
if err != nil {
return err
}
defer cookie.Close(context.Background()) // ensure container removal, separate ctx
waiter, err := cookie.Run(ctx)
if err != nil {
return err
}
// container is running
// buffered, in case someone has slot when waiter returns but isn't yet listening
errC := make(chan error, 1)
go func() {
for {
select { // make sure everything is up before trying to send slot
case <-ctx.Done(): // container shutdown
return
case <-a.shutdown: // server shutdown
shutdownContainer()
return
default: // ok
}
done := make(chan struct{})
slot := &hotSlot{done, proto, errC, container}
select {
case slots <- slot:
case <-time.After(time.Duration(call.IdleTimeout) * time.Second):
logger.Info("Canceling inactive hot function")
shutdownContainer()
return
case <-ctx.Done(): // container shutdown
return
case <-a.shutdown: // server shutdown
shutdownContainer()
return
}
// wait for this call to finish
// NOTE do NOT select with shutdown / other channels. slot handles this.
<-done
}
}()
// we can discard the result, mostly for timeouts / cancels.
_, err = waiter.Wait(ctx)
if err != nil {
errC <- err
}
logger.WithError(err).Info("hot function terminated")
return err
}
// container implements drivers.ContainerTask container is the execution of a
// single container, which may run multiple functions [consecutively]. the id
// and stderr can be swapped out by new calls in the container. input and
// output must be copied in and out.
type container struct {
id string // contrived
image string
env map[string]string
memory uint64
timeout time.Duration // cold only (superfluous, but in case)
stdin io.Reader
stdout io.Writer
stderr io.Writer
}
func (c *container) swap(stderr io.Writer) {
// TODO meh, maybe shouldn't bury this
gw, ok := c.stderr.(*ghostWriter)
if ok {
gw.swap(stderr)
}
}
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) 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() {}
func (c *container) WriteStat(drivers.Stat) {}
func (c *container) Image() string { return c.image }
func (c *container) Timeout() time.Duration { return c.timeout }
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) DockerAuth() (docker.AuthConfiguration, error) {
// Implementing the docker.AuthConfiguration interface.
// TODO per call could implement this stored somewhere (vs. configured on host)
//}
// ghostWriter is a writer who will pass writes to an inner writer
// (that may be changed at will).
type ghostWriter struct {
sync.Mutex
inner io.Writer
}
func (g *ghostWriter) swap(w io.Writer) {
g.Lock()
g.inner = w
g.Unlock()
}
func (g *ghostWriter) Write(b []byte) (int, error) {
// we don't need to serialize writes but swapping g.inner could be a race if unprotected
g.Lock()
w := g.inner
g.Unlock()
return w.Write(b)
}

63
api/agent/async.go Normal file
View File

@@ -0,0 +1,63 @@
package agent
import (
"context"
"time"
"github.com/sirupsen/logrus"
)
func (a *agent) asyncDequeue() {
a.wg.Add(1)
defer a.wg.Done() // we can treat this thread like one big task and get safe shutdown fo free
for {
select {
case <-a.shutdown:
return
case <-a.asyncRam():
// TODO we _could_ return a token here to reserve the ram so that there's
// not a race between here and Submit but we're single threaded
// dequeueing and retries handled gracefully inside of Submit if we run
// out of RAM so..
}
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) // TODO ???
model, err := a.mq.Reserve(ctx)
cancel()
if err != nil || model == nil {
if err != nil {
logrus.WithError(err).Error("error fetching queued calls")
}
time.Sleep(1 * time.Second) // backoff a little
continue
}
// TODO output / logger should be here too...
a.wg.Add(1) // need to add 1 in this thread to ensure safe shutdown
go func() {
defer a.wg.Done() // can shed it after this is done, Submit will add 1 too but it's fine
call, err := a.GetCall(FromModel(model))
if err != nil {
logrus.WithError(err).Error("error getting async call")
return
}
// TODO if the task is cold and doesn't require reading STDIN, it could
// run but we may not listen for output since the task timed out. these
// are at least once semantics, which is really preferable to at most
// once, so let's do it for now
err = a.Submit(call)
if err != nil {
// NOTE: these could be errors / timeouts from the call that we're
// logging here (i.e. not our fault), but it's likely better to log
// these than suppress them so...
id := call.Model().ID
logrus.WithFields(logrus.Fields{"id": id}).WithError(err).Error("error running async call")
}
}()
}
}

359
api/agent/call.go Normal file
View File

@@ -0,0 +1,359 @@
package agent
import (
"context"
"errors"
"fmt"
"io"
"net/http"
"strings"
"time"
"github.com/fnproject/fn/api/id"
"github.com/fnproject/fn/api/models"
"github.com/go-openapi/strfmt"
"github.com/opentracing/opentracing-go"
"github.com/patrickmn/go-cache"
"github.com/sirupsen/logrus"
)
type Call interface {
// Model will return the underlying models.Call configuration for this call.
// TODO we could respond to async correctly from agent but layering, this
// is only because the front end has different responses based on call type.
// try to discourage use elsewhere until this gets pushed down more...
Model() *models.Call
// Start will be called before this call is executed, it may be used to
// guarantee mutual exclusion, check docker permissions, update timestamps,
// etc.
// TODO Start and End can likely be unexported as they are only used in the agent,
// and on a type which is constructed in a specific agent. meh.
Start(ctx context.Context) error
// End will be called immediately after attempting a call execution,
// regardless of whether the execution failed or not. An error will be passed
// to End, which if nil indicates a successful execution. Any error returned
// from End will be returned as the error from Submit.
End(ctx context.Context, err error) error
}
// TODO build w/o closures... lazy
type CallOpt func(a *agent, c *call) error
func FromRequest(appName, path string, req *http.Request) CallOpt {
return func(a *agent, c *call) error {
// TODO we need to add a little timeout to these 2 things
app, err := a.app(req.Context(), appName)
if err != nil {
return err
}
route, err := a.route(req.Context(), appName, path)
if err != nil {
return err
}
params, match := matchRoute(route.Path, path)
if !match {
return errors.New("route does not match") // TODO wtf, can we ignore match?
}
if route.Format == "" {
route.Format = "default"
}
id := id.New().String()
// baseVars are the vars on the route & app, not on this specific request [for hot functions]
baseVars := make(map[string]string, len(app.Config)+len(route.Config)+3)
baseVars["FN_FORMAT"] = route.Format
baseVars["APP_NAME"] = appName
baseVars["ROUTE"] = route.Path
baseVars["MEMORY_MB"] = fmt.Sprintf("%d", route.Memory)
// app config
for k, v := range app.Config {
k = toEnvName("", k)
baseVars[k] = v
}
for k, v := range route.Config {
k = toEnvName("", k)
baseVars[k] = v
}
// envVars contains the full set of env vars, per request + base
envVars := make(map[string]string, len(baseVars)+len(params)+len(req.Header)+3)
for k, v := range baseVars {
envVars[k] = v
}
envVars["CALL_ID"] = id
envVars["METHOD"] = req.Method
envVars["REQUEST_URL"] = fmt.Sprintf("%v://%v%v", func() string {
if req.TLS == nil {
return "http"
}
return "https"
}(), req.Host, req.URL.String())
// params
for _, param := range params {
envVars[toEnvName("PARAM", param.Key)] = param.Value
}
// add all the env vars we build to the request headers
// TODO should we save req.Headers and copy OVER app.Config / route.Config ?
for k, v := range envVars {
req.Header.Add(k, v)
}
// TODO this relies on ordering of opts, but tests make sure it works, probably re-plumb/destroy headers
if rw, ok := c.w.(http.ResponseWriter); ok {
rw.Header().Add("FN_CALL_ID", id)
for k, vs := range route.Headers {
for _, v := range vs {
// pre-write in these headers to response
rw.Header().Add(k, v)
}
}
}
c.Call = &models.Call{
ID: id,
AppName: appName,
Path: route.Path,
Image: route.Image,
// Delay: 0,
Type: route.Type,
Format: route.Format,
// Payload: TODO,
Priority: new(int32), // TODO this is crucial, apparently
Timeout: route.Timeout,
IdleTimeout: route.IdleTimeout,
Memory: route.Memory,
BaseEnv: baseVars,
EnvVars: envVars,
CreatedAt: strfmt.DateTime(time.Now()),
URL: req.URL.String(), // TODO we should probably strip host/port
Method: req.Method,
}
// TODO if these made it to here we have a problemo. error instead?
if c.Timeout <= 0 {
c.Timeout = models.DefaultRouteTimeout
}
if c.IdleTimeout <= 0 {
c.IdleTimeout = models.DefaultIdleTimeout
}
c.req = req
return nil
}
}
func FromModel(mCall *models.Call) CallOpt {
return func(a *agent, c *call) error {
c.Call = mCall
// NOTE this adds content length based on payload length
req, err := http.NewRequest(c.Method, c.URL, strings.NewReader(c.Payload))
if err != nil {
return err
}
for k, v := range c.EnvVars {
// TODO if we don't store env as []string headers are messed up
req.Header.Set(k, v)
}
c.req = req
// TODO anything else really?
return nil
}
}
// TODO this should be required
func WithWriter(w io.Writer) CallOpt {
return func(a *agent, c *call) error {
c.w = w
return nil
}
}
// GetCall builds a Call that can be used to submit jobs to the agent.
//
// TODO we could make this package level just moving the cache around. meh.
// TODO where to put this? async and sync both call this
func (a *agent) GetCall(opts ...CallOpt) (Call, error) {
var c call
for _, o := range opts {
err := o(a, &c)
if err != nil {
return nil, err
}
}
// TODO typed errors to test
if c.req == nil || c.Call == nil {
return nil, errors.New("no model or request provided for call")
}
// TODO move func logger here
// TODO add log store interface (yagni?)
c.ds = a.ds
c.mq = a.mq
return &c, nil
}
type call struct {
*models.Call
ds models.Datastore
mq models.MessageQueue
w io.Writer
req *http.Request
stderr io.WriteCloser
}
func (c *call) Model() *models.Call { return c.Call }
func (c *call) Start(ctx context.Context) error {
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_call_start")
defer span.Finish()
// TODO discuss this policy. cold has not yet started the container,
// hot just has to dispatch
//
// make sure we have at least half our timeout to run, or timeout here
deadline, ok := ctx.Deadline()
need := time.Now().Add(time.Duration(c.Timeout) * time.Second) // > deadline, always
// need.Sub(deadline) = elapsed time
if ok && need.Sub(deadline) > (time.Duration(c.Timeout)*time.Second)/2 {
return context.DeadlineExceeded
}
c.StartedAt = strfmt.DateTime(time.Now())
c.Status = "running"
if c.Type == models.TypeAsync {
// XXX (reed): make sure MQ reservation is lengthy. to skirt MQ semantics,
// we could add a new message to MQ w/ delay of call.Timeout and delete the
// old one (in that order), after marking the call as running in the db
// (see below)
// XXX (reed): should we store the updated started_at + status? we could
// use this so that if we pick up a call from mq and find its status is
// running to avoid running the call twice and potentially mark it as
// errored (built in long running task detector, so to speak...)
err := c.mq.Delete(ctx, c.Call)
if err != nil {
return err // let another thread try this
}
}
return nil
}
func (c *call) End(ctx context.Context, err error) error {
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_call_end")
defer span.Finish()
c.CompletedAt = strfmt.DateTime(time.Now())
switch err {
case nil:
c.Status = "success"
case context.DeadlineExceeded:
c.Status = "timeout"
default:
// XXX (reed): should we append the error to logs? Error field?
c.Status = "error"
}
if c.Type == models.TypeAsync {
// XXX (reed): delete MQ message, eventually
}
// TODO since the function itself can reply directly to a client (or logs),
// this means that we could potentially store an error / timeout status for a
// call that ran successfully [by a user's perspective]
// TODO this should be update, really
if err := c.ds.InsertCall(ctx, c.Call); err != nil {
// TODO we should just log this error not return it to user? just issue storing call status but call is run
logrus.WithError(err).Error("error inserting call into datastore")
}
// return the original error so that this is returned from Submit (for sync)
// TODO we could just skip over (and log) and End errors and return slot.exec error
// in submit instead of doing this, it's a bit odd. thoughts?
return err
}
func (a *agent) route(ctx context.Context, appName, path string) (*models.Route, error) {
key := routeCacheKey(appName, path)
route, ok := a.cache.Get(key)
if ok {
return route.(*models.Route), nil
}
resp, err := a.singleflight.Do(key,
func() (interface{}, error) { return a.ds.GetRoute(ctx, appName, path) },
)
if err != nil {
return nil, err
}
route = resp.(*models.Route)
a.cache.Set(key, route, cache.DefaultExpiration)
return route.(*models.Route), nil
}
func (a *agent) app(ctx context.Context, appName string) (*models.App, error) {
key := appCacheKey(appName)
app, ok := a.cache.Get(key)
if ok {
return app.(*models.App), nil
}
resp, err := a.singleflight.Do(key,
func() (interface{}, error) { return a.ds.GetApp(ctx, appName) },
)
if err != nil {
return nil, err
}
app = resp.(*models.App)
a.cache.Set(key, app, cache.DefaultExpiration)
return app.(*models.App), nil
}
func routeCacheKey(appname, path string) string {
return "r:" + appname + "\x00" + path
}
func appCacheKey(appname string) string {
return "a:" + appname
}
func fakeHandler(http.ResponseWriter, *http.Request, Params) {}
// TODO what is this stuff anyway?
func matchRoute(baseRoute, route string) (Params, bool) {
tree := &node{}
tree.addRoute(baseRoute, fakeHandler)
handler, p, _ := tree.getValue(route)
if handler == nil {
return nil, false
}
return p, true
}
func toEnvName(envtype, name string) string {
name = strings.Replace(name, "-", "_", -1)
if envtype == "" {
return name
}
return fmt.Sprintf("%s_%s", envtype, name)
}

View File

@@ -7,13 +7,13 @@ import (
"os" "os"
"path" "path"
"strings" "strings"
"time"
"github.com/sirupsen/logrus" "github.com/fnproject/fn/api/agent/drivers"
"github.com/fnproject/fn/api/runner/common" "github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/runner/drivers"
"github.com/fsouza/go-dockerclient" "github.com/fsouza/go-dockerclient"
"github.com/opentracing/opentracing-go" "github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/log"
"github.com/sirupsen/logrus"
) )
// A drivers.ContainerTask should implement the Auther interface if it would // A drivers.ContainerTask should implement the Auther interface if it would
@@ -34,7 +34,6 @@ type Auther interface {
type runResult struct { type runResult struct {
error error
status string status string
start time.Time
} }
func (r *runResult) Error() string { func (r *runResult) Error() string {
@@ -44,33 +43,48 @@ func (r *runResult) Error() string {
return r.error.Error() return r.error.Error()
} }
func (r *runResult) Status() string { return r.status } func (r *runResult) Status() string { return r.status }
func (r *runResult) UserVisible() bool { return common.IsUserVisibleError(r.error) } func (r *runResult) UserVisible() bool { return common.IsUserVisibleError(r.error) }
func (r *runResult) StartTime() time.Time { return r.start }
type DockerDriver struct { type DockerDriver struct {
conf drivers.Config conf drivers.Config
docker dockerClient // retries on *docker.Client, restricts ad hoc *docker.Client usage / retries docker dockerClient // retries on *docker.Client, restricts ad hoc *docker.Client usage / retries
hostname string hostname string
auths map[string]docker.AuthConfiguration
*common.Environment
} }
// implements drivers.Driver // implements drivers.Driver
func NewDocker(env *common.Environment, conf drivers.Config) *DockerDriver { func NewDocker(conf drivers.Config) *DockerDriver {
hostname, err := os.Hostname() hostname, err := os.Hostname()
if err != nil { if err != nil {
logrus.WithError(err).Fatal("couldn't resolve hostname") logrus.WithError(err).Fatal("couldn't resolve hostname")
} }
return &DockerDriver{ return &DockerDriver{
conf: conf, conf: conf,
docker: newClient(env), docker: newClient(),
hostname: hostname, hostname: hostname,
Environment: env, auths: registryFromEnv(),
} }
} }
func registryFromEnv() map[string]docker.AuthConfiguration {
var auths *docker.AuthConfigurations
var err error
if reg := os.Getenv("DOCKER_AUTH"); reg != "" {
// TODO docker does not use this itself, we should get rid of env docker config (nor is this documented..)
auths, err = docker.NewAuthConfigurations(strings.NewReader(reg))
} else {
auths, err = docker.NewAuthConfigurationsFromDockerCfg()
}
if err != nil {
logrus.WithError(err).Error("error getting docker auths from config files")
return nil
}
return auths.Configs
}
func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask) (drivers.Cookie, error) { func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask) (drivers.Cookie, error) {
ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"stack": "Prepare"}) ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"stack": "Prepare"})
var cmd []string var cmd []string
@@ -85,9 +99,8 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
envvars = append(envvars, name+"="+val) envvars = append(envvars, name+"="+val)
} }
containerName := newContainerID(task)
container := docker.CreateContainerOptions{ container := docker.CreateContainerOptions{
Name: containerName, Name: task.Id(),
Config: &docker.Config{ Config: &docker.Config{
Env: envvars, Env: envvars,
Cmd: cmd, Cmd: cmd,
@@ -96,7 +109,6 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
Hostname: drv.hostname, Hostname: drv.hostname,
Image: task.Image(), Image: task.Image(),
Volumes: map[string]struct{}{}, Volumes: map[string]struct{}{},
Labels: task.Labels(),
OpenStdin: true, OpenStdin: true,
AttachStdin: true, AttachStdin: true,
StdinOnce: true, StdinOnce: true,
@@ -131,7 +143,7 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
if err != docker.ErrContainerAlreadyExists { if err != docker.ErrContainerAlreadyExists {
log.WithFields(logrus.Fields{"call_id": task.Id(), "command": container.Config.Cmd, "memory": container.Config.Memory, log.WithFields(logrus.Fields{"call_id": task.Id(), "command": container.Config.Cmd, "memory": container.Config.Memory,
"cpu_shares": container.Config.CPUShares, "hostname": container.Config.Hostname, "name": container.Name, "cpu_shares": container.Config.CPUShares, "hostname": container.Config.Hostname, "name": container.Name,
"image": container.Config.Image, "volumes": container.Config.Volumes, "binds": container.HostConfig.Binds, "container": containerName, "image": container.Config.Image, "volumes": container.Config.Volumes, "binds": container.HostConfig.Binds, "container": container.Name,
}).WithError(err).Error("Could not create container") }).WithError(err).Error("Could not create container")
if ce := containerConfigError(err); ce != nil { if ce := containerConfigError(err); ce != nil {
@@ -142,7 +154,7 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
} }
// discard removal error // discard removal error
return &cookie{id: containerName, task: task, drv: drv}, nil return &cookie{id: task.Id(), task: task, drv: drv}, nil
} }
type cookie struct { type cookie struct {
@@ -151,9 +163,11 @@ type cookie struct {
drv *DockerDriver drv *DockerDriver
} }
func (c *cookie) Close(ctx context.Context) error { return c.drv.removeContainer(ctx, c.id) } func (c *cookie) Close(ctx context.Context) error {
return c.drv.removeContainer(ctx, c.id)
}
func (c *cookie) Run(ctx context.Context) (drivers.RunResult, error) { func (c *cookie) Run(ctx context.Context) (drivers.WaitResult, error) {
return c.drv.run(ctx, c.id, c.task) return c.drv.run(ctx, c.id, c.task)
} }
@@ -174,6 +188,15 @@ func (drv *DockerDriver) ensureImage(ctx context.Context, task drivers.Container
// validate creds even if the image is downloaded. // validate creds even if the image is downloaded.
var config docker.AuthConfiguration // default, tries docker hub w/o user/pass var config docker.AuthConfiguration // default, tries docker hub w/o user/pass
// if any configured host auths match task registry, try them (task docker auth can override)
for _, v := range drv.auths {
// TODO doubt this works. copied to attempt to keep parity. nobody using so... glhf
if strings.HasSuffix(v.ServerAddress, reg) {
config = v
}
}
if task, ok := task.(Auther); ok { if task, ok := task.(Auther); ok {
var err error var err error
span, _ := opentracing.StartSpanFromContext(ctx, "docker_auth") span, _ := opentracing.StartSpanFromContext(ctx, "docker_auth")
@@ -230,7 +253,7 @@ func (drv *DockerDriver) pullImage(ctx context.Context, task drivers.ContainerTa
// Run executes the docker container. If task runs, drivers.RunResult will be returned. If something fails outside the task (ie: Docker), it will return error. // Run executes the docker container. If task runs, drivers.RunResult will be returned. If something fails outside the task (ie: Docker), it will return error.
// The docker driver will attempt to cast the task to a Auther. If that succeeds, private image support is available. See the Auther interface for how to implement this. // The docker driver will attempt to cast the task to a Auther. If that succeeds, private image support is available. See the Auther interface for how to implement this.
func (drv *DockerDriver) run(ctx context.Context, container string, task drivers.ContainerTask) (drivers.RunResult, error) { func (drv *DockerDriver) run(ctx context.Context, container string, task drivers.ContainerTask) (drivers.WaitResult, error) {
timeout := task.Timeout() timeout := task.Timeout()
var cancel context.CancelFunc var cancel context.CancelFunc
@@ -253,22 +276,34 @@ func (drv *DockerDriver) run(ctx context.Context, container string, task drivers
return nil, err return nil, err
} }
start := time.Now()
err = drv.startTask(ctx, container) err = drv.startTask(ctx, container)
if err != nil && ctx.Err() == nil { if err != nil && ctx.Err() == nil {
// if there's just a timeout making the docker calls, drv.wait below will rewrite it to timeout // if there's just a timeout making the docker calls, drv.wait below will rewrite it to timeout
return nil, err return nil, err
} }
return &waitResult{
container: container,
waiter: waiter,
drv: drv,
}, nil
}
// implements drivers.WaitResult
type waitResult struct {
container string
waiter docker.CloseWaiter
drv *DockerDriver
}
func (w *waitResult) Wait(ctx context.Context) (drivers.RunResult, error) {
defer func() { defer func() {
waiter.Close() w.waiter.Close()
waiter.Wait() // make sure we gather all logs w.waiter.Wait() // make sure we gather all logs
}() }()
status, err := drv.wait(ctx, container) status, err := w.drv.wait(ctx, w.container)
return &runResult{ return &runResult{
start: start,
status: status, status: status,
error: err, error: err,
}, nil }, nil
@@ -363,11 +398,6 @@ func cherryPick(ds *docker.Stats) drivers.Stat {
} }
} }
// Introduces some randomness to prevent container name clashes where task ID remains the same.
func newContainerID(task drivers.ContainerTask) string {
return fmt.Sprintf("task-%d-%s", time.Now().UnixNano(), task.Id())
}
func (drv *DockerDriver) startTask(ctx context.Context, container string) error { func (drv *DockerDriver) startTask(ctx context.Context, container string) error {
log := common.Logger(ctx) log := common.Logger(ctx)
log.WithFields(logrus.Fields{"container": container}).Debug("Starting container execution") log.WithFields(logrus.Fields{"container": container}).Debug("Starting container execution")
@@ -408,11 +438,11 @@ func (drv *DockerDriver) wait(ctx context.Context, container string) (status str
switch exitCode { switch exitCode {
default: default:
return drivers.StatusError, common.UserError(fmt.Errorf("exit code %d", exitCode)) return drivers.StatusError, common.UserError(fmt.Errorf("container exit code %d", exitCode))
case 0: case 0:
return drivers.StatusSuccess, nil return drivers.StatusSuccess, nil
case 137: // OOM case 137: // OOM
drv.Inc("docker", "oom", 1, 1) opentracing.SpanFromContext(ctx).LogFields(log.String("docker", "oom"))
return drivers.StatusKilled, drivers.ErrOutOfMemory return drivers.StatusKilled, drivers.ErrOutOfMemory
} }
} }

View File

@@ -12,11 +12,11 @@ import (
"strings" "strings"
"time" "time"
"github.com/sirupsen/logrus" "github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/runner/common"
"github.com/fsouza/go-dockerclient" "github.com/fsouza/go-dockerclient"
"github.com/opentracing/opentracing-go" "github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/log" "github.com/opentracing/opentracing-go/log"
"github.com/sirupsen/logrus"
) )
const ( const (
@@ -42,7 +42,7 @@ type dockerClient interface {
} }
// TODO: switch to github.com/docker/engine-api // TODO: switch to github.com/docker/engine-api
func newClient(env *common.Environment) dockerClient { func newClient() dockerClient {
// TODO this was much easier, don't need special settings at the moment // TODO this was much easier, don't need special settings at the moment
// docker, err := docker.NewClient(conf.Docker) // docker, err := docker.NewClient(conf.Docker)
client, err := docker.NewClientFromEnv() client, err := docker.NewClientFromEnv()
@@ -87,13 +87,12 @@ func newClient(env *common.Environment) dockerClient {
logrus.WithError(err).Fatal("couldn't connect to other docker daemon") logrus.WithError(err).Fatal("couldn't connect to other docker daemon")
} }
return &dockerWrap{client, clientNoTimeout, env} return &dockerWrap{client, clientNoTimeout}
} }
type dockerWrap struct { type dockerWrap struct {
docker *docker.Client docker *docker.Client
dockerNoTimeout *docker.Client dockerNoTimeout *docker.Client
*common.Environment
} }
func (d *dockerWrap) retry(ctx context.Context, f func() error) error { func (d *dockerWrap) retry(ctx context.Context, f func() error) error {
@@ -106,8 +105,8 @@ func (d *dockerWrap) retry(ctx context.Context, f func() error) error {
for ; ; i++ { for ; ; i++ {
select { select {
case <-ctx.Done(): case <-ctx.Done():
d.Inc("task", "fail.docker", 1, 1) span.LogFields(log.String("task", "fail.docker"))
logger.WithError(ctx.Err()).Warnf("retrying on docker errors timed out, restart docker or rotate this instance?") logger.WithError(ctx.Err()).Warnf("docker call timed out")
return ctx.Err() return ctx.Err()
default: default:
} }
@@ -116,11 +115,11 @@ func (d *dockerWrap) retry(ctx context.Context, f func() error) error {
if common.IsTemporary(err) || isDocker50x(err) { if common.IsTemporary(err) || isDocker50x(err) {
logger.WithError(err).Warn("docker temporary error, retrying") logger.WithError(err).Warn("docker temporary error, retrying")
b.Sleep() b.Sleep()
d.Inc("task", "error.docker", 1, 1) span.LogFields(log.String("task", "tmperror.docker"))
continue continue
} }
if err != nil { if err != nil {
d.Inc("task", "error.docker", 1, 1) span.LogFields(log.String("task", "error.docker"))
} }
return err return err
} }

View File

@@ -8,8 +8,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/fnproject/fn/api/runner/common" "github.com/fnproject/fn/api/agent/drivers"
"github.com/fnproject/fn/api/runner/drivers"
"github.com/fsouza/go-dockerclient" "github.com/fsouza/go-dockerclient"
) )
@@ -37,8 +36,7 @@ func (f *taskDockerTest) Close() {}
func (f *taskDockerTest) Input() io.Reader { return f.input } func (f *taskDockerTest) Input() io.Reader { return f.input }
func TestRunnerDocker(t *testing.T) { func TestRunnerDocker(t *testing.T) {
env := common.NewEnvironment(func(e *common.Environment) {}) dkr := NewDocker(drivers.Config{})
dkr := NewDocker(env, drivers.Config{})
ctx := context.Background() ctx := context.Background()
task := &taskDockerTest{"test-docker", nil, nil} task := &taskDockerTest{"test-docker", nil, nil}
@@ -49,7 +47,12 @@ func TestRunnerDocker(t *testing.T) {
} }
defer cookie.Close(ctx) defer cookie.Close(ctx)
result, err := cookie.Run(ctx) waiter, err := cookie.Run(ctx)
if err != nil {
t.Fatal(err)
}
result, err := waiter.Wait(ctx)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@@ -60,8 +63,7 @@ func TestRunnerDocker(t *testing.T) {
} }
func TestRunnerDockerStdin(t *testing.T) { func TestRunnerDockerStdin(t *testing.T) {
env := common.NewEnvironment(func(e *common.Environment) {}) dkr := NewDocker(drivers.Config{})
dkr := NewDocker(env, drivers.Config{})
ctx := context.Background() ctx := context.Background()
input := `{"name": "test"}` input := `{"name": "test"}`
@@ -75,7 +77,12 @@ func TestRunnerDockerStdin(t *testing.T) {
} }
defer cookie.Close(ctx) defer cookie.Close(ctx)
result, err := cookie.Run(ctx) waiter, err := cookie.Run(ctx)
if err != nil {
t.Fatal(err)
}
result, err := waiter.Wait(ctx)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }

View File

@@ -19,7 +19,7 @@ import (
"github.com/docker/distribution/registry/client/auth" "github.com/docker/distribution/registry/client/auth"
"github.com/docker/distribution/registry/client/auth/challenge" "github.com/docker/distribution/registry/client/auth/challenge"
"github.com/docker/distribution/registry/client/transport" "github.com/docker/distribution/registry/client/transport"
"github.com/fnproject/fn/api/runner/drivers" "github.com/fnproject/fn/api/agent/drivers"
docker "github.com/fsouza/go-dockerclient" docker "github.com/fsouza/go-dockerclient"
) )

View File

@@ -28,7 +28,15 @@ type Cookie interface {
// //
// Run() MUST monitor the context. task cancellation is indicated by // Run() MUST monitor the context. task cancellation is indicated by
// cancelling the context. // cancelling the context.
Run(ctx context.Context) (RunResult, error) Run(ctx context.Context) (WaitResult, error)
}
type WaitResult interface {
// Wait may be called to await the result of a container's execution. If the
// provided context is canceled and the container does not return first, the
// resulting status will be 'canceled'. If the provided context times out
// then the resulting status will be 'timeout'.
Wait(context.Context) (RunResult, error)
} }
type Driver interface { type Driver interface {
@@ -51,11 +59,6 @@ type RunResult interface {
// Status should return the current status of the task. // Status should return the current status of the task.
// Only valid options are {"error", "success", "timeout", "killed", "cancelled"}. // Only valid options are {"error", "success", "timeout", "killed", "cancelled"}.
Status() string Status() string
// StartTime returns the time just before beginning execution of a task,
// for example including the time to pull a container image and doing any
// other setup. This should not include a container's execution duration.
StartTime() time.Time
} }
// The ContainerTask interface guides task execution across a wide variety of // The ContainerTask interface guides task execution across a wide variety of
@@ -73,9 +76,6 @@ type ContainerTask interface {
// Input feeds the container with data // Input feeds the container with data
Input() io.Reader Input() io.Reader
// Labels returns container label key-value pairs.
Labels() map[string]string
// The id to assign the container // The id to assign the container
Id() string Id() string

View File

@@ -5,7 +5,7 @@ import (
"fmt" "fmt"
"time" "time"
"github.com/fnproject/fn/api/runner/drivers" "github.com/fnproject/fn/api/agent/drivers"
) )
func New() drivers.Driver { func New() drivers.Driver {
@@ -26,7 +26,7 @@ type cookie struct {
func (c *cookie) Close(context.Context) error { return nil } func (c *cookie) Close(context.Context) error { return nil }
func (c *cookie) Run(ctx context.Context) (drivers.RunResult, error) { func (c *cookie) Run(ctx context.Context) (drivers.WaitResult, error) {
c.m.count++ c.m.count++
if c.m.count%100 == 0 { if c.m.count%100 == 0 {
return nil, fmt.Errorf("Mocker error! Bad.") return nil, fmt.Errorf("Mocker error! Bad.")
@@ -44,5 +44,6 @@ type runResult struct {
start time.Time start time.Time
} }
func (r *runResult) Status() string { return r.status } func (r *runResult) Wait(context.Context) (drivers.RunResult, error) { return r, nil }
func (r *runResult) StartTime() time.Time { return r.start } func (r *runResult) Status() string { return r.status }
func (r *runResult) StartTime() time.Time { return r.start }

View File

@@ -1,4 +1,4 @@
package runner package agent
import ( import (
"bytes" "bytes"
@@ -8,45 +8,32 @@ import (
"io" "io"
"sync" "sync"
"github.com/sirupsen/logrus" "github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/common" "github.com/sirupsen/logrus"
) )
// TODO kind of no reason to have FuncLogger interface... we can just do the thing. var (
bufPool = &sync.Pool{New: func() interface{} { return new(bytes.Buffer) }}
logPool = &sync.Pool{New: func() interface{} { return new(bytes.Buffer) }}
)
type FuncLogger interface { // TODO we can have different types of these func loggers later
Writer(ctx context.Context, appName, path, image, reqID string) io.WriteCloser // TODO move this to a different package
}
func NewFuncLogger(logDB models.FnLog) FuncLogger {
// TODO we should probably make it somehow configurable to log to stderr and/or db but meh
return &DefaultFuncLogger{
logDB: logDB,
bufPool: &sync.Pool{New: func() interface{} { return new(bytes.Buffer) }},
logPool: &sync.Pool{New: func() interface{} { return new(bytes.Buffer) }},
}
}
// DefaultFuncLogger returns a WriteCloser that writes STDERR output from a // DefaultFuncLogger returns a WriteCloser that writes STDERR output from a
// container and outputs it in a parsed structured log format to attached // container and outputs it in a parsed structured log format to attached
// STDERR as well as writing the log to the db when Close is called. // STDERR as well as writing the log to the db when Close is called.
type DefaultFuncLogger struct { func NewFuncLogger(ctx context.Context, appName, path, image, reqID string, logDB models.LogStore) io.WriteCloser {
logDB models.FnLog lbuf := bufPool.Get().(*bytes.Buffer)
bufPool *sync.Pool // these are usually small, for buffering lines dbuf := logPool.Get().(*bytes.Buffer)
logPool *sync.Pool // these are usually large, for buffering whole logs
}
func (l *DefaultFuncLogger) Writer(ctx context.Context, appName, path, image, reqID string) io.WriteCloser {
lbuf := l.bufPool.Get().(*bytes.Buffer)
dbuf := l.logPool.Get().(*bytes.Buffer)
close := func() error { close := func() error {
// TODO we may want to toss out buffers that grow to grotesque size but meh they will prob get GC'd // TODO we may want to toss out buffers that grow to grotesque size but meh they will prob get GC'd
lbuf.Reset() lbuf.Reset()
dbuf.Reset() dbuf.Reset()
l.bufPool.Put(lbuf) bufPool.Put(lbuf)
l.logPool.Put(dbuf) logPool.Put(dbuf)
return nil return nil
} }
@@ -64,7 +51,7 @@ func (l *DefaultFuncLogger) Writer(ctx context.Context, appName, path, image, re
// we don't need to log per line to db, but we do need to limit it // we don't need to log per line to db, but we do need to limit it
limitw := newLimitWriter(MB, &dbWriter{ limitw := newLimitWriter(MB, &dbWriter{
Buffer: dbuf, Buffer: dbuf,
db: l.logDB, db: logDB,
ctx: ctx, ctx: ctx,
reqID: reqID, reqID: reqID,
}) })
@@ -189,7 +176,7 @@ func (li *lineWriter) Close() error {
type dbWriter struct { type dbWriter struct {
*bytes.Buffer *bytes.Buffer
db models.FnLog db models.LogStore
ctx context.Context ctx context.Context
reqID string reqID string
} }

97
api/agent/mem.go Normal file
View File

@@ -0,0 +1,97 @@
package agent
import (
"bufio"
"errors"
"fmt"
"io/ioutil"
"os"
"runtime"
"strconv"
"strings"
"github.com/sirupsen/logrus"
)
func getAvailableMemory() uint64 {
const tooBig = 322122547200 // #300GB or 0, biggest aws instance is 244GB
var availableMemory uint64 = tooBig
if runtime.GOOS == "linux" {
var err error
availableMemory, err = checkCgroup()
if err != nil {
logrus.WithError(err).Error("Error checking for cgroup memory limits, falling back to host memory available..")
}
if availableMemory >= tooBig || availableMemory <= 0 {
// Then -m flag probably wasn't set, so use max available on system
availableMemory, err = checkProc()
if availableMemory >= tooBig || availableMemory <= 0 {
logrus.WithError(err).Fatal("Cannot get the proper memory information to size server. You must specify the maximum available memory by passing the -m command with docker run when starting the server via docker, eg: `docker run -m 2G ...`")
}
}
} else {
// This still lets 10-20 functions execute concurrently assuming a 2GB machine.
availableMemory = 2 * 1024 * 1024 * 1024
}
logrus.WithFields(logrus.Fields{"ram": availableMemory}).Info("available memory")
return availableMemory
}
func checkCgroup() (uint64, error) {
f, err := os.Open("/sys/fs/cgroup/memory/memory.limit_in_bytes")
if err != nil {
return 0, err
}
defer f.Close()
b, err := ioutil.ReadAll(f)
limBytes := string(b)
limBytes = strings.TrimSpace(limBytes)
if err != nil {
return 0, err
}
return strconv.ParseUint(limBytes, 10, 64)
}
var errCantReadMemInfo = errors.New("Didn't find MemAvailable in /proc/meminfo, kernel is probably < 3.14")
func checkProc() (uint64, error) {
f, err := os.Open("/proc/meminfo")
if err != nil {
return 0, err
}
defer f.Close()
scanner := bufio.NewScanner(f)
for scanner.Scan() {
b := scanner.Text()
if !strings.HasPrefix(b, "MemAvailable") {
continue
}
// expect form:
// MemAvailable: 1234567890 kB
tri := strings.Fields(b)
if len(tri) != 3 {
return 0, fmt.Errorf("MemAvailable line has unexpected format: %v", b)
}
c, err := strconv.ParseUint(tri[1], 10, 64)
if err != nil {
return 0, fmt.Errorf("Could not parse MemAvailable: %v", b)
}
switch tri[2] { // convert units to bytes
case "kB":
c *= 1024
case "MB":
c *= 1024 * 1024
default:
return 0, fmt.Errorf("Unexpected units for MemAvailable in /proc/meminfo, need kB or MB, got: %v", tri[2])
}
return c, nil
}
return 0, errCantReadMemInfo
}

View File

@@ -0,0 +1,12 @@
package protocol
import (
"io"
"net/http"
)
// DefaultProtocol is the protocol used by cold-containers
type DefaultProtocol struct{}
func (p *DefaultProtocol) IsStreamable() bool { return false }
func (d *DefaultProtocol) Dispatch(w io.Writer, req *http.Request) error { return nil }

View File

@@ -1,24 +1,32 @@
package protocol package protocol
import ( import (
"context"
"errors" "errors"
"io" "io"
"net/http"
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/task"
) )
var errInvalidProtocol = errors.New("Invalid Protocol") var errInvalidProtocol = errors.New("Invalid Protocol")
type errorProto struct {
error
}
func (e errorProto) IsStreamable() bool { return false }
func (e errorProto) Dispatch(io.Writer, *http.Request) error { return e }
// ContainerIO defines the interface used to talk to a hot function. // ContainerIO defines the interface used to talk to a hot function.
// Internally, a protocol must know when to alternate between stdin and stdout. // Internally, a protocol must know when to alternate between stdin and stdout.
// It returns any protocol error, if present. // It returns any protocol error, if present.
type ContainerIO interface { type ContainerIO interface {
IsStreamable() bool IsStreamable() bool
// TODO this should take a drivers.ContainerTask? // Dispatch will handle sending stdin and stdout to a container. Implementers
Dispatch(ctx context.Context, t *task.Config) error // of Dispatch may format the input and output differently. Dispatch must respect
// the req.Context() timeout / cancellation.
Dispatch(w io.Writer, req *http.Request) error
} }
// Protocol defines all protocols that operates a ContainerIO. // Protocol defines all protocols that operates a ContainerIO.
@@ -53,12 +61,6 @@ func (p Protocol) MarshalJSON() ([]byte, error) {
return nil, errInvalidProtocol return nil, errInvalidProtocol
} }
// implements ContainerIO
type errorProto struct{}
func (e *errorProto) IsStreamable() bool { return false }
func (e *errorProto) Dispatch(ctx context.Context, t *task.Config) error { return errInvalidProtocol }
// New creates a valid protocol handler from a I/O pipe representing containers // New creates a valid protocol handler from a I/O pipe representing containers
// stdin/stdout. // stdin/stdout.
func New(p Protocol, in io.Writer, out io.Reader) ContainerIO { func New(p Protocol, in io.Writer, out io.Reader) ContainerIO {
@@ -68,12 +70,9 @@ func New(p Protocol, in io.Writer, out io.Reader) ContainerIO {
case Default, Empty: case Default, Empty:
return &DefaultProtocol{} return &DefaultProtocol{}
} }
return &errorProto{} // shouldn't make it past testing... return &errorProto{errInvalidProtocol}
} }
// IsStreamable says whether the given protocol can be used for streaming into // IsStreamable says whether the given protocol can be used for streaming into
// hot functions. // hot functions.
// TODO get rid of ContainerIO and just use Protocol func IsStreamable(p Protocol) bool { return New(p, nil, nil).IsStreamable() }
func IsStreamable(p Protocol) bool {
return New(p, nil, nil).IsStreamable()
}

144
api/agent/protocol/http.go Normal file
View File

@@ -0,0 +1,144 @@
package protocol
import (
"bufio"
"fmt"
"io"
"net/http"
"net/http/httputil"
"strings"
)
// HTTPProtocol converts stdin/stdout streams into HTTP/1.1 compliant
// communication. It relies on Content-Length to know when to stop reading from
// containers stdout. It also mandates valid HTTP headers back and forth, thus
// returning errors in case of parsing problems.
type HTTPProtocol struct {
in io.Writer
out io.Reader
}
func (p *HTTPProtocol) IsStreamable() bool { return true }
// this is just an http.Handler really
// TODO handle req.Context better with io.Copy. io.Copy could push us
// over the timeout.
// TODO maybe we should take io.Writer, io.Reader but then we have to
// dump the request to a buffer again :(
func (h *HTTPProtocol) Dispatch(w io.Writer, req *http.Request) error {
err := DumpRequestTo(h.in, req) // TODO timeout
if err != nil {
return err
}
if rw, ok := w.(http.ResponseWriter); ok {
// if we're writing directly to the response writer, we need to set headers
// and status code first since calling res.Write will just write the http
// response as the body (headers and all)
res, err := http.ReadResponse(bufio.NewReader(h.out), req) // TODO timeout
if err != nil {
return err
}
for k, v := range res.Header {
rw.Header()[k] = v
}
rw.WriteHeader(res.StatusCode)
// TODO should we TCP_CORK ?
io.Copy(rw, res.Body) // TODO timeout
res.Body.Close()
} else {
// logs can just copy the full thing in there, headers and all.
res, err := http.ReadResponse(bufio.NewReader(h.out), req) // TODO timeout
if err != nil {
return err
}
res.Write(w) // TODO timeout
}
return nil
}
// DumpRequestTo is httputil.DumpRequest with some modifications. It will
// dump the request to the provided io.Writer with the body always, consuming
// the body in the process.
//
// TODO we should support h2!
func DumpRequestTo(w io.Writer, req *http.Request) error {
// By default, print out the unmodified req.RequestURI, which
// is always set for incoming server requests. But because we
// previously used req.URL.RequestURI and the docs weren't
// always so clear about when to use DumpRequest vs
// DumpRequestOut, fall back to the old way if the caller
// provides a non-server Request.
reqURI := req.RequestURI
if reqURI == "" {
reqURI = req.URL.RequestURI()
}
fmt.Fprintf(w, "%s %s HTTP/%d.%d\r\n", valueOrDefault(req.Method, "GET"),
reqURI, req.ProtoMajor, req.ProtoMinor)
absRequestURI := strings.HasPrefix(req.RequestURI, "http://") || strings.HasPrefix(req.RequestURI, "https://")
if !absRequestURI {
host := req.Host
if host == "" && req.URL != nil {
host = req.URL.Host
}
if host != "" {
fmt.Fprintf(w, "Host: %s\r\n", host)
}
}
chunked := len(req.TransferEncoding) > 0 && req.TransferEncoding[0] == "chunked"
if len(req.TransferEncoding) > 0 {
fmt.Fprintf(w, "Transfer-Encoding: %s\r\n", strings.Join(req.TransferEncoding, ","))
}
if req.Close {
fmt.Fprintf(w, "Connection: close\r\n")
}
err := req.Header.WriteSubset(w, reqWriteExcludeHeaderDump)
if err != nil {
return err
}
io.WriteString(w, "\r\n")
if req.Body != nil {
var dest io.Writer = w
if chunked {
dest = httputil.NewChunkedWriter(dest)
}
// TODO copy w/ ctx
_, err = io.Copy(dest, req.Body)
if chunked {
dest.(io.Closer).Close()
io.WriteString(w, "\r\n")
}
}
return err
}
var reqWriteExcludeHeaderDump = map[string]bool{
"Host": true, // not in Header map anyway
"Transfer-Encoding": true,
"Trailer": true,
}
// Return value if nonempty, def otherwise.
func valueOrDefault(value, def string) string {
if value != "" {
return value
}
return def
}

View File

@@ -1,14 +1,16 @@
package runner package agent
import "sync" import "sync"
// TODO this should expose:
// * hot containers active
// * memory used / available
type stats struct { type stats struct {
mu sync.Mutex mu sync.Mutex
queue uint64 queue uint64
running uint64 running uint64
complete uint64 complete uint64
wait sync.WaitGroup
} }
type Stats struct { type Stats struct {
@@ -24,7 +26,6 @@ func (s *stats) Enqueue() {
} }
func (s *stats) Start() { func (s *stats) Start() {
s.wait.Add(1)
s.mu.Lock() s.mu.Lock()
s.queue-- s.queue--
s.running++ s.running++
@@ -32,7 +33,6 @@ func (s *stats) Start() {
} }
func (s *stats) Complete() { func (s *stats) Complete() {
s.wait.Done()
s.mu.Lock() s.mu.Lock()
s.running-- s.running--
s.complete++ s.complete++
@@ -48,5 +48,3 @@ func (s *stats) Stats() Stats {
s.mu.Unlock() s.mu.Unlock()
return stats return stats
} }
func (s *stats) Wait() { s.wait.Wait() }

View File

@@ -2,7 +2,10 @@
// Use of this source code is governed by a BSD-style license that can be found // Use of this source code is governed by a BSD-style license that can be found
// in the LICENSE file. // in the LICENSE file.
package server // TODO what is this, why do we have it, and where can we put it if we need it
// TODO move to common
package agent
import ( import (
"net/http" "net/http"

View File

@@ -6,7 +6,7 @@ import (
"syscall" "syscall"
) )
// Errors that can be directly exposed to task creators/users. // Errors that can be directly exposed to call creators/users.
type UserVisibleError interface { type UserVisibleError interface {
UserVisible() bool UserVisible() bool
} }

View File

@@ -1,4 +1,6 @@
package server package singleflight
// TODO figure out how this differs?
// Imported from https://github.com/golang/groupcache/blob/master/singleflight/singleflight.go // Imported from https://github.com/golang/groupcache/blob/master/singleflight/singleflight.go
@@ -13,7 +15,7 @@ type call struct {
err error err error
} }
type singleflight struct { type SingleFlight struct {
mu sync.Mutex // protects m mu sync.Mutex // protects m
m map[interface{}]*call // lazily initialized m map[interface{}]*call // lazily initialized
} }
@@ -22,7 +24,7 @@ type singleflight struct {
// sure that only one execution is in-flight for a given key at a // sure that only one execution is in-flight for a given key at a
// time. If a duplicate comes in, the duplicate caller waits for the // time. If a duplicate comes in, the duplicate caller waits for the
// original to complete and receives the same results. // original to complete and receives the same results.
func (g *singleflight) do(key interface{}, fn func() (interface{}, error)) (interface{}, error) { func (g *SingleFlight) Do(key interface{}, fn func() (interface{}, error)) (interface{}, error) {
g.mu.Lock() g.mu.Lock()
if g.m == nil { if g.m == nil {
g.m = make(map[interface{}]*call) g.m = make(map[interface{}]*call)

View File

@@ -33,47 +33,47 @@ func Test(t *testing.T, ds models.Datastore) {
ctx := context.Background() ctx := context.Background()
task := &models.Task{} call := new(models.Call)
task.CreatedAt = strfmt.DateTime(time.Now()) call.CreatedAt = strfmt.DateTime(time.Now())
task.Status = "success" call.Status = "success"
task.StartedAt = strfmt.DateTime(time.Now()) call.StartedAt = strfmt.DateTime(time.Now())
task.CompletedAt = strfmt.DateTime(time.Now()) call.CompletedAt = strfmt.DateTime(time.Now())
task.AppName = testApp.Name call.AppName = testApp.Name
task.Path = testRoute.Path call.Path = testRoute.Path
t.Run("call-insert", func(t *testing.T) { t.Run("call-insert", func(t *testing.T) {
task.ID = id.New().String() call.ID = id.New().String()
err := ds.InsertTask(ctx, task) err := ds.InsertCall(ctx, call)
if err != nil { if err != nil {
t.Log(buf.String()) t.Log(buf.String())
t.Fatalf("Test InsertTask(ctx, &task): unexpected error `%v`", err) t.Fatalf("Test InsertCall(ctx, &call): unexpected error `%v`", err)
} }
}) })
t.Run("call-get", func(t *testing.T) { t.Run("call-get", func(t *testing.T) {
task.ID = id.New().String() call.ID = id.New().String()
ds.InsertTask(ctx, task) ds.InsertCall(ctx, call)
newTask, err := ds.GetTask(ctx, task.ID) newCall, err := ds.GetCall(ctx, call.AppName, call.ID)
if err != nil { if err != nil {
t.Fatalf("Test GetTask(ctx, task.ID): unexpected error `%v`", err) t.Fatalf("Test GetCall(ctx, call.ID): unexpected error `%v`", err)
} }
if task.ID != newTask.ID { if call.ID != newCall.ID {
t.Log(buf.String()) t.Log(buf.String())
t.Fatalf("Test GetTask(ctx, task.ID): unexpected error `%v`", err) t.Fatalf("Test GetCall(ctx, call.ID): unexpected error `%v`", err)
} }
}) })
t.Run("calls-get", func(t *testing.T) { t.Run("calls-get", func(t *testing.T) {
filter := &models.CallFilter{AppName: task.AppName, Path: task.Path} filter := &models.CallFilter{AppName: call.AppName, Path: call.Path}
task.ID = id.New().String() call.ID = id.New().String()
ds.InsertTask(ctx, task) ds.InsertCall(ctx, call)
calls, err := ds.GetTasks(ctx, filter) calls, err := ds.GetCalls(ctx, filter)
if err != nil { if err != nil {
t.Fatalf("Test GetTasks(ctx, filter): unexpected error `%v`", err) t.Fatalf("Test GetCalls(ctx, filter): unexpected error `%v`", err)
} }
if len(calls) == 0 { if len(calls) == 0 {
t.Log(buf.String()) t.Log(buf.String())
t.Fatalf("Test GetTasks(ctx, filter): unexpected error `%v`", err) t.Fatalf("Test GetCalls(ctx, filter): unexpected error `%v`", err)
} }
}) })

View File

@@ -82,22 +82,22 @@ func (m *metricds) RemoveRoute(ctx context.Context, appName, routePath string) e
return m.ds.RemoveRoute(ctx, appName, routePath) return m.ds.RemoveRoute(ctx, appName, routePath)
} }
func (m *metricds) InsertTask(ctx context.Context, task *models.Task) error { func (m *metricds) InsertCall(ctx context.Context, call *models.Call) error {
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_insert_task") span, ctx := opentracing.StartSpanFromContext(ctx, "ds_insert_call")
defer span.Finish() defer span.Finish()
return m.ds.InsertTask(ctx, task) return m.ds.InsertCall(ctx, call)
} }
func (m *metricds) GetTask(ctx context.Context, callID string) (*models.FnCall, error) { func (m *metricds) GetCall(ctx context.Context, appName, callID string) (*models.Call, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_task") span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_call")
defer span.Finish() defer span.Finish()
return m.ds.GetTask(ctx, callID) return m.ds.GetCall(ctx, appName, callID)
} }
func (m *metricds) GetTasks(ctx context.Context, filter *models.CallFilter) (models.FnCalls, error) { func (m *metricds) GetCalls(ctx context.Context, filter *models.CallFilter) ([]*models.Call, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_tasks") span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_calls")
defer span.Finish() defer span.Finish()
return m.ds.GetTasks(ctx, filter) return m.ds.GetCalls(ctx, filter)
} }
func (m *metricds) InsertLog(ctx context.Context, callID string, callLog string) error { func (m *metricds) InsertLog(ctx context.Context, callID string, callLog string) error {
@@ -106,7 +106,7 @@ func (m *metricds) InsertLog(ctx context.Context, callID string, callLog string)
return m.ds.InsertLog(ctx, callID, callLog) return m.ds.InsertLog(ctx, callID, callLog)
} }
func (m *metricds) GetLog(ctx context.Context, callID string) (*models.FnCallLog, error) { func (m *metricds) GetLog(ctx context.Context, callID string) (*models.CallLog, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_log") span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_log")
defer span.Finish() defer span.Finish()
return m.ds.GetLog(ctx, callID) return m.ds.GetLog(ctx, callID)

View File

@@ -131,11 +131,11 @@ func (v *validator) RemoveRoute(ctx context.Context, appName, routePath string)
} }
// callID will never be empty. // callID will never be empty.
func (v *validator) GetTask(ctx context.Context, callID string) (*models.FnCall, error) { func (v *validator) GetCall(ctx context.Context, appName, callID string) (*models.Call, error) {
if callID == "" { if callID == "" {
return nil, models.ErrDatastoreEmptyTaskID return nil, models.ErrDatastoreEmptyCallID
} }
return v.Datastore.GetTask(ctx, callID) return v.Datastore.GetCall(ctx, appName, callID)
} }
func (v *validator) DeleteLog(ctx context.Context, callID string) error { func (v *validator) DeleteLog(ctx context.Context, callID string) error {

View File

@@ -11,31 +11,19 @@ import (
) )
type mock struct { type mock struct {
Apps models.Apps Apps []*models.App
Routes models.Routes Routes []*models.Route
Calls models.FnCalls Calls []*models.Call
data map[string][]byte data map[string][]byte
models.FnLog models.LogStore
} }
func NewMock() models.Datastore { func NewMock() models.Datastore {
return NewMockInit(nil, nil, nil, nil) return NewMockInit(nil, nil, nil)
} }
func NewMockInit(apps models.Apps, routes models.Routes, calls models.FnCalls, loggos []*models.FnCallLog) models.Datastore { func NewMockInit(apps []*models.App, routes []*models.Route, calls []*models.Call) models.Datastore {
if apps == nil {
apps = models.Apps{}
}
if routes == nil {
routes = models.Routes{}
}
if calls == nil {
calls = models.FnCalls{}
}
if loggos == nil {
loggos = []*models.FnCallLog{}
}
return datastoreutil.NewValidator(&mock{apps, routes, calls, make(map[string][]byte), logs.NewMock()}) return datastoreutil.NewValidator(&mock{apps, routes, calls, make(map[string][]byte), logs.NewMock()})
} }
@@ -150,13 +138,12 @@ func (m *mock) Get(ctx context.Context, key []byte) ([]byte, error) {
return m.data[string(key)], nil return m.data[string(key)], nil
} }
func (m *mock) InsertTask(ctx context.Context, task *models.Task) error { func (m *mock) InsertCall(ctx context.Context, call *models.Call) error {
var call *models.FnCall m.Calls = append(m.Calls, call)
m.Calls = append(m.Calls, call.FromTask(task))
return nil return nil
} }
func (m *mock) GetTask(ctx context.Context, callID string) (*models.FnCall, error) { func (m *mock) GetCall(ctx context.Context, appName, callID string) (*models.Call, error) {
for _, t := range m.Calls { for _, t := range m.Calls {
if t.ID == callID { if t.ID == callID {
return t, nil return t, nil
@@ -166,7 +153,7 @@ func (m *mock) GetTask(ctx context.Context, callID string) (*models.FnCall, erro
return nil, models.ErrCallNotFound return nil, models.ErrCallNotFound
} }
func (m *mock) GetTasks(ctx context.Context, filter *models.CallFilter) (models.FnCalls, error) { func (m *mock) GetCalls(ctx context.Context, filter *models.CallFilter) ([]*models.Call, error) {
return m.Calls, nil return m.Calls, nil
} }

View File

@@ -526,7 +526,7 @@ func (ds *sqlStore) Tx(f func(*sqlx.Tx) error) error {
return tx.Commit() return tx.Commit()
} }
func (ds *sqlStore) InsertTask(ctx context.Context, task *models.Task) error { func (ds *sqlStore) InsertCall(ctx context.Context, call *models.Call) error {
query := ds.db.Rebind(`INSERT INTO calls ( query := ds.db.Rebind(`INSERT INTO calls (
id, id,
created_at, created_at,
@@ -538,9 +538,9 @@ func (ds *sqlStore) InsertTask(ctx context.Context, task *models.Task) error {
) )
VALUES (?, ?, ?, ?, ?, ?, ?);`) VALUES (?, ?, ?, ?, ?, ?, ?);`)
_, err := ds.db.Exec(query, task.ID, task.CreatedAt.String(), _, err := ds.db.Exec(query, call.ID, call.CreatedAt.String(),
task.StartedAt.String(), task.CompletedAt.String(), call.StartedAt.String(), call.CompletedAt.String(),
task.Status, task.AppName, task.Path) call.Status, call.AppName, call.Path)
if err != nil { if err != nil {
return err return err
} }
@@ -548,12 +548,15 @@ func (ds *sqlStore) InsertTask(ctx context.Context, task *models.Task) error {
return nil return nil
} }
func (ds *sqlStore) GetTask(ctx context.Context, callID string) (*models.FnCall, error) { // TODO calls are not fully qualified in this backend currently. need to discuss,
query := fmt.Sprintf(`%s WHERE id=?`, callSelector) // if we store the whole thing then it adds a lot of disk space and then we can
// make async only queue hints instead of entire calls (mq a lot smaller space wise). pick.
func (ds *sqlStore) GetCall(ctx context.Context, appName, callID string) (*models.Call, error) {
query := fmt.Sprintf(`%s WHERE id=? AND app_name=?`, callSelector)
query = ds.db.Rebind(query) query = ds.db.Rebind(query)
row := ds.db.QueryRow(query, callID) row := ds.db.QueryRow(query, callID, appName)
var call models.FnCall var call models.Call
err := scanCall(row, &call) err := scanCall(row, &call)
if err != nil { if err != nil {
return nil, err return nil, err
@@ -561,8 +564,8 @@ func (ds *sqlStore) GetTask(ctx context.Context, callID string) (*models.FnCall,
return &call, nil return &call, nil
} }
func (ds *sqlStore) GetTasks(ctx context.Context, filter *models.CallFilter) (models.FnCalls, error) { func (ds *sqlStore) GetCalls(ctx context.Context, filter *models.CallFilter) ([]*models.Call, error) {
res := models.FnCalls{} res := []*models.Call{}
query, args := buildFilterCallQuery(filter) query, args := buildFilterCallQuery(filter)
query = fmt.Sprintf("%s %s", callSelector, query) query = fmt.Sprintf("%s %s", callSelector, query)
query = ds.db.Rebind(query) query = ds.db.Rebind(query)
@@ -573,7 +576,7 @@ func (ds *sqlStore) GetTasks(ctx context.Context, filter *models.CallFilter) (mo
defer rows.Close() defer rows.Close()
for rows.Next() { for rows.Next() {
var call models.FnCall var call models.Call
err := scanCall(rows, &call) err := scanCall(rows, &call)
if err != nil { if err != nil {
continue continue
@@ -592,7 +595,7 @@ func (ds *sqlStore) InsertLog(ctx context.Context, callID, callLog string) error
return err return err
} }
func (ds *sqlStore) GetLog(ctx context.Context, callID string) (*models.FnCallLog, error) { func (ds *sqlStore) GetLog(ctx context.Context, callID string) (*models.CallLog, error) {
query := ds.db.Rebind(`SELECT log FROM logs WHERE id=?`) query := ds.db.Rebind(`SELECT log FROM logs WHERE id=?`)
row := ds.db.QueryRow(query, callID) row := ds.db.QueryRow(query, callID)
@@ -605,7 +608,7 @@ func (ds *sqlStore) GetLog(ctx context.Context, callID string) (*models.FnCallLo
return nil, err return nil, err
} }
return &models.FnCallLog{ return &models.CallLog{
CallID: callID, CallID: callID,
Log: log, Log: log,
}, nil }, nil
@@ -622,7 +625,7 @@ type RowScanner interface {
Scan(dest ...interface{}) error Scan(dest ...interface{}) error
} }
func ScanLog(scanner RowScanner, log *models.FnCallLog) error { func ScanLog(scanner RowScanner, log *models.CallLog) error {
return scanner.Scan( return scanner.Scan(
&log.CallID, &log.CallID,
&log.Log, &log.Log,
@@ -746,7 +749,7 @@ func buildFilterCallQuery(filter *models.CallFilter) (string, []interface{}) {
return b.String(), args return b.String(), args
} }
func scanCall(scanner RowScanner, call *models.FnCall) error { func scanCall(scanner RowScanner, call *models.Call) error {
err := scanner.Scan( err := scanner.Scan(
&call.ID, &call.ID,
&call.CreatedAt, &call.CreatedAt,

View File

@@ -9,7 +9,7 @@ import (
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
) )
func New(dbURL string) (models.FnLog, error) { func New(dbURL string) (models.LogStore, error) {
u, err := url.Parse(dbURL) u, err := url.Parse(dbURL)
if err != nil { if err != nil {
logrus.WithError(err).WithFields(logrus.Fields{"url": dbURL}).Fatal("bad DB URL") logrus.WithError(err).WithFields(logrus.Fields{"url": dbURL}).Fatal("bad DB URL")

View File

@@ -7,19 +7,19 @@ import (
) )
type mock struct { type mock struct {
Logs map[string]*models.FnCallLog Logs map[string]*models.CallLog
ds models.Datastore ds models.Datastore
} }
func NewMock() models.FnLog { func NewMock() models.LogStore {
return NewMockInit(nil) return NewMockInit(nil)
} }
func NewMockInit(logs map[string]*models.FnCallLog) models.FnLog { func NewMockInit(logs map[string]*models.CallLog) models.LogStore {
if logs == nil { if logs == nil {
logs = map[string]*models.FnCallLog{} logs = map[string]*models.CallLog{}
} }
fnl := NewValidator(&mock{logs, nil}) fnl := &mock{logs, nil}
return fnl return fnl
} }
@@ -28,11 +28,11 @@ func (m *mock) SetDatastore(ctx context.Context, ds models.Datastore) {
} }
func (m *mock) InsertLog(ctx context.Context, callID string, callLog string) error { func (m *mock) InsertLog(ctx context.Context, callID string, callLog string) error {
m.Logs[callID] = &models.FnCallLog{CallID: callID, Log: callLog} m.Logs[callID] = &models.CallLog{CallID: callID, Log: callLog}
return nil return nil
} }
func (m *mock) GetLog(ctx context.Context, callID string) (*models.FnCallLog, error) { func (m *mock) GetLog(ctx context.Context, callID string) (*models.CallLog, error) {
logEntry := m.Logs[callID] logEntry := m.Logs[callID]
if logEntry == nil { if logEntry == nil {
return nil, errors.New("Call log not found") return nil, errors.New("Call log not found")

View File

@@ -23,68 +23,68 @@ var testRoute = &models.Route{
Format: "http", Format: "http",
} }
func SetUpTestTask() *models.Task { func SetupTestCall() *models.Call {
task := &models.Task{} var call models.Call
task.CreatedAt = strfmt.DateTime(time.Now()) call.CreatedAt = strfmt.DateTime(time.Now())
task.Status = "success" call.Status = "success"
task.StartedAt = strfmt.DateTime(time.Now()) call.StartedAt = strfmt.DateTime(time.Now())
task.CompletedAt = strfmt.DateTime(time.Now()) call.CompletedAt = strfmt.DateTime(time.Now())
task.AppName = testApp.Name call.AppName = testApp.Name
task.Path = testRoute.Path call.Path = testRoute.Path
return task return &call
} }
func Test(t *testing.T, fnl models.FnLog, ds models.Datastore) { func Test(t *testing.T, fnl models.LogStore, ds models.Datastore) {
ctx := context.Background() ctx := context.Background()
task := SetUpTestTask() call := SetupTestCall()
t.Run("call-log-insert", func(t *testing.T) { t.Run("call-log-insert", func(t *testing.T) {
task.ID = id.New().String() call.ID = id.New().String()
err := ds.InsertTask(ctx, task) err := ds.InsertCall(ctx, call)
if err != nil { if err != nil {
t.Fatalf("Test InsertTask(ctx, &task): unexpected error `%v`", err) t.Fatalf("Test InsertCall(ctx, &call): unexpected error `%v`", err)
} }
err = fnl.InsertLog(ctx, task.ID, "test") err = fnl.InsertLog(ctx, call.ID, "test")
if err != nil { if err != nil {
t.Fatalf("Test InsertLog(ctx, task.ID, logText): unexpected error during inserting log `%v`", err) t.Fatalf("Test InsertLog(ctx, call.ID, logText): unexpected error during inserting log `%v`", err)
} }
}) })
t.Run("call-log-insert-get", func(t *testing.T) { t.Run("call-log-insert-get", func(t *testing.T) {
task.ID = id.New().String() call.ID = id.New().String()
err := ds.InsertTask(ctx, task) err := ds.InsertCall(ctx, call)
logText := "test" logText := "test"
if err != nil { if err != nil {
t.Fatalf("Test InsertTask(ctx, &task): unexpected error `%v`", err) t.Fatalf("Test InsertCall(ctx, &call): unexpected error `%v`", err)
} }
err = fnl.InsertLog(ctx, task.ID, logText) err = fnl.InsertLog(ctx, call.ID, logText)
if err != nil { if err != nil {
t.Fatalf("Test InsertLog(ctx, task.ID, logText): unexpected error during inserting log `%v`", err) t.Fatalf("Test InsertLog(ctx, call.ID, logText): unexpected error during inserting log `%v`", err)
} }
logEntry, err := fnl.GetLog(ctx, task.ID) logEntry, err := fnl.GetLog(ctx, call.ID)
if !strings.Contains(logEntry.Log, logText) { if !strings.Contains(logEntry.Log, logText) {
t.Fatalf("Test GetLog(ctx, task.ID, logText): unexpected error, log mismatch. "+ t.Fatalf("Test GetLog(ctx, call.ID, logText): unexpected error, log mismatch. "+
"Expected: `%v`. Got `%v`.", logText, logEntry.Log) "Expected: `%v`. Got `%v`.", logText, logEntry.Log)
} }
}) })
t.Run("call-log-insert-get-delete", func(t *testing.T) { t.Run("call-log-insert-get-delete", func(t *testing.T) {
task.ID = id.New().String() call.ID = id.New().String()
err := ds.InsertTask(ctx, task) err := ds.InsertCall(ctx, call)
logText := "test" logText := "test"
if err != nil { if err != nil {
t.Fatalf("Test InsertTask(ctx, &task): unexpected error `%v`", err) t.Fatalf("Test InsertCall(ctx, &call): unexpected error `%v`", err)
} }
err = fnl.InsertLog(ctx, task.ID, logText) err = fnl.InsertLog(ctx, call.ID, logText)
if err != nil { if err != nil {
t.Fatalf("Test InsertLog(ctx, task.ID, logText): unexpected error during inserting log `%v`", err) t.Fatalf("Test InsertLog(ctx, call.ID, logText): unexpected error during inserting log `%v`", err)
} }
logEntry, err := fnl.GetLog(ctx, task.ID) logEntry, err := fnl.GetLog(ctx, call.ID)
if !strings.Contains(logEntry.Log, logText) { if !strings.Contains(logEntry.Log, logText) {
t.Fatalf("Test GetLog(ctx, task.ID, logText): unexpected error, log mismatch. "+ t.Fatalf("Test GetLog(ctx, call.ID, logText): unexpected error, log mismatch. "+
"Expected: `%v`. Got `%v`.", logText, logEntry.Log) "Expected: `%v`. Got `%v`.", logText, logEntry.Log)
} }
err = fnl.DeleteLog(ctx, task.ID) err = fnl.DeleteLog(ctx, call.ID)
if err != nil { if err != nil {
t.Fatalf("Test DeleteLog(ctx, task.ID): unexpected error during deleting log `%v`", err) t.Fatalf("Test DeleteLog(ctx, call.ID): unexpected error during deleting log `%v`", err)
} }
}) })
} }

View File

@@ -1,33 +0,0 @@
package logs
import (
"context"
"github.com/fnproject/fn/api/models"
)
type FnLog interface {
InsertLog(ctx context.Context, callID string, callLog string) error
GetLog(ctx context.Context, callID string) (*models.FnCallLog, error)
DeleteLog(ctx context.Context, callID string) error
}
type validator struct {
fnl FnLog
}
func NewValidator(fnl FnLog) models.FnLog {
return &validator{fnl}
}
func (v *validator) InsertLog(ctx context.Context, callID string, callLog string) error {
return v.fnl.InsertLog(ctx, callID, callLog)
}
func (v *validator) GetLog(ctx context.Context, callID string) (*models.FnCallLog, error) {
return v.fnl.GetLog(ctx, callID)
}
func (v *validator) DeleteLog(ctx context.Context, callID string) error {
return v.fnl.DeleteLog(ctx, callID)
}

View File

@@ -1,9 +1,5 @@
package models package models
type Apps []*App
type Tasks []*Task
type FnCalls []*FnCall
type App struct { type App struct {
Name string `json:"name"` Name string `json:"name"`
Routes Routes `json:"routes,omitempty"` Routes Routes `json:"routes,omitempty"`

134
api/models/call.go Normal file
View File

@@ -0,0 +1,134 @@
package models
import (
strfmt "github.com/go-openapi/strfmt"
)
const (
// TypeNone ...
TypeNone = ""
// TypeSync ...
TypeSync = "sync"
// TypeAsync ...
TypeAsync = "async"
)
const (
// FormatDefault ...
FormatDefault = "default"
// FormatHTTP ...
FormatHTTP = "http"
)
var possibleStatuses = [...]string{"delayed", "queued", "running", "success", "error", "cancelled"}
type CallLog struct {
CallID string `json:"call_id"`
Log string `json:"log"`
}
// Call is a representation of a specific invocation of a route.
type Call struct {
// Unique identifier representing a specific call.
ID string `json:"id"`
// NOTE: this is stale, retries are not implemented atm, but this is nice, so leaving
// States and valid transitions.
//
// +---------+
// +---------> delayed <----------------+
// +----+----+ |
// | |
// | |
// +----v----+ |
// +---------> queued <----------------+
// +----+----+ *
// | *
// | retry * creates new call
// +----v----+ *
// | running | *
// +--+-+-+--+ |
// +---------|-|-|-----+-------------+
// +---|---------+ | +-----|---------+ |
// | | | | | |
// +-----v---^-+ +--v-------^+ +--v---^-+
// | success | | cancelled | | error |
// +-----------+ +-----------+ +--------+
//
// * delayed - has a delay.
// * queued - Ready to be consumed when it's turn comes.
// * running - Currently consumed by a runner which will attempt to process it.
// * success - (or complete? success/error is common javascript terminology)
// * error - Something went wrong. In this case more information can be obtained
// by inspecting the "reason" field.
// - timeout
// - killed - forcibly killed by worker due to resource restrictions or access
// violations.
// - bad_exit - exited with non-zero status due to program termination/crash.
// * cancelled - cancelled via API. More information in the reason field.
// - client_request - Request was cancelled by a client.
Status string `json:"status"`
// App this call belongs to.
AppName string `json:"app_name"`
// Path of the route that is responsible for this call
Path string `json:"path"`
// Name of Docker image to use.
Image string `json:"image"`
// Number of seconds to wait before queueing the call for consumption for the
// first time. Must be a positive integer. Calls with a delay start in state
// "delayed" and transition to "running" after delay seconds.
Delay int32 `json:"delay,omitempty"`
// Type indicates whether a task is to be run synchronously or asynchronously.
Type string `json:"type,omitempty"`
// Format is the format to pass input into the function.
Format string `json:"format,omitempty"`
// Payload for the call. This is only used by async calls, to store their input.
// TODO should we copy it into here too for debugging sync?
Payload string `json:"payload,omitempty"`
// Full request url that spawned this invocation.
URL string `json:"url,omitempty"`
// Method of the http request used to make this call.
Method string `json:"method,omitempty"`
// Priority of the call. Higher has more priority. 3 levels from 0-2. Calls
// at same priority are processed in FIFO order.
Priority *int32 `json:"priority"`
// Maximum runtime in seconds.
Timeout int32 `json:"timeout,omitempty"`
// Hot function idle timeout in seconds before termination.
IdleTimeout int32 `json:"idle_timeout,omitempty"`
// Memory is the amount of RAM this call is allocated.
Memory uint64 `json:"memory,omitempty"`
// BaseEnv are the env vars for hot containers, not request specific.
BaseEnv map[string]string `json:"base_env,omitempty"`
// Env vars for the call. Comes from the ones set on the Route.
EnvVars map[string]string `json:"env_vars,omitempty"`
// Time when call completed, whether it was successul or failed. Always in UTC.
CompletedAt strfmt.DateTime `json:"completed_at,omitempty"`
// Time when call was submitted. Always in UTC.
CreatedAt strfmt.DateTime `json:"created_at,omitempty"`
// Time when call started execution. Always in UTC.
StartedAt strfmt.DateTime `json:"started_at,omitempty"`
}
type CallFilter struct {
Path string
AppName string
}

View File

@@ -1,27 +0,0 @@
package models
import "github.com/go-openapi/strfmt"
/*Complete complete
swagger:model Complete
*/
type Complete struct {
/* Time when task was completed. Always in UTC.
*/
CompletedAt strfmt.DateTime `json:"completed_at,omitempty"`
/* Error message, if status=error. Only used by the /error endpoint.
*/
Error string `json:"error,omitempty"`
/* Machine readable reason failure, if status=error. Only used by the /error endpoint.
*/
Reason string `json:"reason,omitempty"`
}
// Validate validates this complete
func (m *Complete) Validate(formats strfmt.Registry) error {
return nil
}

View File

@@ -38,11 +38,11 @@ type Datastore interface {
GetRoute(ctx context.Context, appName, routePath string) (*Route, error) GetRoute(ctx context.Context, appName, routePath string) (*Route, error)
// GetRoutes gets a slice of Routes, optionally filtered by filter. // GetRoutes gets a slice of Routes, optionally filtered by filter.
GetRoutes(ctx context.Context, filter *RouteFilter) (routes []*Route, err error) GetRoutes(ctx context.Context, filter *RouteFilter) ([]*Route, error)
// GetRoutesByApp gets a slice of routes for a appName, optionally filtering on filter (filter.AppName is ignored). // GetRoutesByApp gets a slice of routes for a appName, optionally filtering on filter (filter.AppName is ignored).
// Returns ErrDatastoreEmptyAppName if appName is empty. // Returns ErrDatastoreEmptyAppName if appName is empty.
GetRoutesByApp(ctx context.Context, appName string, filter *RouteFilter) (routes []*Route, err error) GetRoutesByApp(ctx context.Context, appName string, filter *RouteFilter) ([]*Route, error)
// InsertRoute inserts a route. Returns ErrDatastoreEmptyRoute when route is nil, and ErrDatastoreEmptyAppName // InsertRoute inserts a route. Returns ErrDatastoreEmptyRoute when route is nil, and ErrDatastoreEmptyAppName
// or ErrDatastoreEmptyRoutePath for empty AppName or Path. // or ErrDatastoreEmptyRoutePath for empty AppName or Path.
@@ -57,13 +57,19 @@ type Datastore interface {
// ErrDatastoreEmptyRoutePath when routePath is empty. Returns ErrRoutesNotFound when no route exists. // ErrDatastoreEmptyRoutePath when routePath is empty. Returns ErrRoutesNotFound when no route exists.
RemoveRoute(ctx context.Context, appName, routePath string) error RemoveRoute(ctx context.Context, appName, routePath string) error
// InsertTask inserts a task // InsertCall inserts a call into the datastore, it will error if the call already
InsertTask(ctx context.Context, task *Task) error // exists.
GetTask(ctx context.Context, callID string) (*FnCall, error) InsertCall(ctx context.Context, call *Call) error
GetTasks(ctx context.Context, filter *CallFilter) (FnCalls, error)
// Implement FnLog methods for convenience // GetCall returns a call at a certain id and app name.
FnLog GetCall(ctx context.Context, appName, callID string) (*Call, error)
// GetCalls returns a list of calls that satisfy the given CallFilter. If no
// calls exist, an empty list and a nil error are returned.
GetCalls(ctx context.Context, filter *CallFilter) ([]*Call, error)
// Implement LogStore methods for convenience
LogStore
// GetDatabase returns the underlying sqlx database implementation // GetDatabase returns the underlying sqlx database implementation
GetDatabase() *sqlx.DB GetDatabase() *sqlx.DB

View File

@@ -16,7 +16,7 @@ var (
code: http.StatusBadRequest, code: http.StatusBadRequest,
error: errors.New("Invalid JSON"), error: errors.New("Invalid JSON"),
} }
ErrRunnerTimeout = err{ ErrCallTimeout = err{
code: http.StatusGatewayTimeout, code: http.StatusGatewayTimeout,
error: errors.New("Timed out"), error: errors.New("Timed out"),
} }
@@ -72,9 +72,9 @@ var (
code: http.StatusBadRequest, code: http.StatusBadRequest,
error: errors.New("Missing key"), error: errors.New("Missing key"),
} }
ErrDatastoreEmptyTaskID = err{ ErrDatastoreEmptyCallID = err{
code: http.StatusBadRequest, code: http.StatusBadRequest,
error: errors.New("Missing task ID"), error: errors.New("Missing call ID"),
} }
ErrInvalidPayload = err{ ErrInvalidPayload = err{
code: http.StatusBadRequest, code: http.StatusBadRequest,

View File

@@ -1,102 +0,0 @@
package models
import (
"encoding/json"
strfmt "github.com/go-openapi/strfmt"
"github.com/go-openapi/swag"
"github.com/go-openapi/validate"
)
// TODO get rid of this. id and status are not more coupled than anything else?
// burn it at the stake
/*IDStatus Id status
swagger:model IdStatus
*/
type IDStatus struct {
/* Unique identifier representing a specific task.
Read Only: true
*/
ID string `json:"id,omitempty"`
/* States and valid transitions.
+---------+
+---------> delayed <----------------+
+----+----+ |
| |
| |
+----v----+ |
+---------> queued <----------------+
+----+----+ *
| *
| retry * creates new task
+----v----+ *
| running | *
+--+-+-+--+ |
+---------|-|-|-----+-------------+
+---|---------+ | +-----|---------+ |
| | | | | |
+-----v---^-+ +--v-------^+ +--v---^-+
| success | | cancelled | | error |
+-----------+ +-----------+ +--------+
* delayed - has a delay.
* queued - Ready to be consumed when it's turn comes.
* running - Currently consumed by a runner which will attempt to process it.
* success - (or complete? success/error is common javascript terminology)
* error - Something went wrong. In this case more information can be obtained
by inspecting the "reason" field.
- timeout
- killed - forcibly killed by worker due to resource restrictions or access
violations.
- bad_exit - exited with non-zero status due to program termination/crash.
* cancelled - cancelled via API. More information in the reason field.
- client_request - Request was cancelled by a client.
Read Only: true
*/
Status string `json:"status,omitempty"`
}
// Validate validates this Id status
func (m *IDStatus) Validate(formats strfmt.Registry) error { return m.validateStatus(formats) }
var idStatusTypeStatusPropEnum []interface{}
// prop value enum
func (m *IDStatus) validateStatusEnum(path, location string, value string) error {
if idStatusTypeStatusPropEnum == nil {
var res []string
if err := json.Unmarshal([]byte(`["delayed","queued","running","success","error","cancelled"]`), &res); err != nil {
return err
}
for _, v := range res {
idStatusTypeStatusPropEnum = append(idStatusTypeStatusPropEnum, v)
}
}
if err := validate.Enum(path, location, value, idStatusTypeStatusPropEnum); err != nil {
return err
}
return nil
}
func (m *IDStatus) validateStatus(formats strfmt.Registry) error {
if swag.IsZero(m.Status) { // not required
return nil
}
// value enum
if err := m.validateStatusEnum("status", "body", m.Status); err != nil {
return err
}
return nil
}

View File

@@ -4,14 +4,18 @@ import (
"context" "context"
) )
type FnLog interface { type LogStore interface {
// TODO TODO TODO BAD BUG BUG BUG WILL ROBINSON
// TODO these need to take an app name or users can provide ids for
// other users calls with their own app name and access their logs.
// InsertLog will insert the log at callID, overwriting if it previously // InsertLog will insert the log at callID, overwriting if it previously
// existed. // existed.
InsertLog(ctx context.Context, callID string, callLog string) error InsertLog(ctx context.Context, callID string, callLog string) error
// GetLog will return the log at callID, an error will be returned if the log // GetLog will return the log at callID, an error will be returned if the log
// cannot be found. // cannot be found.
GetLog(ctx context.Context, callID string) (*FnCallLog, error) GetLog(ctx context.Context, callID string) (*CallLog, error)
// DeleteLog will remove the log at callID, it will not return an error if // DeleteLog will remove the log at callID, it will not return an error if
// the log does not exist before removal. // the log does not exist before removal.

View File

@@ -3,18 +3,18 @@ package models
import "context" import "context"
// Message Queue is used to impose a total ordering on jobs that it will // Message Queue is used to impose a total ordering on jobs that it will
// execute in order. Tasks are added to the queue via the Push() interface. The // execute in order. calls are added to the queue via the Push() interface. The
// MQ must support a reserve-delete 2 step dequeue to allow implementing // MQ must support a reserve-delete 2 step dequeue to allow implementing
// timeouts and retries. // timeouts and retries.
// //
// The Reserve() operation must return a job based on this total ordering // The Reserve() operation must return a job based on this total ordering
// (described below). At this point, the MQ backend must start a timeout on the // (described below). At this point, the MQ backend must start a timeout on the
// job. If Delete() is not called on the Task within the timeout, the Task should // job. If Delete() is not called on the call within the timeout, the call should
// be restored to the queue. // be restored to the queue.
// //
// Total ordering: The queue should maintain an ordering based on priority and // Total ordering: The queue should maintain an ordering based on priority and
// logical time. Priorities are currently 0-2 and available in the Task's // logical time. Priorities are currently 0-2 and available in the call's
// priority field. Tasks with higher priority always get pulled off the queue // priority field. call with higher priority always get pulled off the queue
// first. Within the same priority, jobs should be available in FIFO order. // first. Within the same priority, jobs should be available in FIFO order.
// When a job is required to be restored to the queue, it should maintain it's // When a job is required to be restored to the queue, it should maintain it's
@@ -22,12 +22,12 @@ import "context"
// the head of the queue: // the head of the queue:
// Reserve() leads to A being passed to a consumer, and timeout started. // Reserve() leads to A being passed to a consumer, and timeout started.
// Next Reserve() leads to B being dequeued. This consumer finishes running the // Next Reserve() leads to B being dequeued. This consumer finishes running the
// task, leading to Delete() being called. B is now permanently erased from the // call, leading to Delete() being called. B is now permanently erased from the
// queue. // queue.
// A's timeout occurs before the job is finished. At this point the ordering // A's timeout occurs before the job is finished. At this point the ordering
// should be [A, C] and not [C, A]. // should be [A, C] and not [C, A].
type MessageQueue interface { type MessageQueue interface {
// Push a Task onto the queue. If any error is returned, the Task SHOULD not be // Push a call onto the queue. If any error is returned, the call SHOULD not be
// queued. Note that this does not completely avoid double queueing, that is // queued. Note that this does not completely avoid double queueing, that is
// OK, a check against the datastore will be performed after a dequeue. // OK, a check against the datastore will be performed after a dequeue.
// //
@@ -37,18 +37,16 @@ type MessageQueue interface {
// delays. That is, if jobs {A, C} are queued at t seconds, both with Delay // delays. That is, if jobs {A, C} are queued at t seconds, both with Delay
// = 5 seconds, and the same priority, then they may be available on the // = 5 seconds, and the same priority, then they may be available on the
// queue as [C, A] or [A, C]. // queue as [C, A] or [A, C].
Push(context.Context, *Task) (*Task, error) Push(context.Context, *Call) (*Call, error)
// Remove a job from the front of the queue, reserve it for a timeout and // Remove a job from the front of the queue, reserve it for a timeout and
// return it. MQ implementations MUST NOT lose jobs in case of errors. That // return it. MQ implementations MUST NOT lose jobs in case of errors. That
// is, in case of reservation failure, it should be possible to retrieve the // is, in case of reservation failure, it should be possible to retrieve the
// job on a future reservation. // job on a future reservation.
Reserve(context.Context) (*Task, error) Reserve(context.Context) (*Call, error)
// If a reservation is pending, consider it acknowledged and delete it. If // If a reservation is pending, consider it acknowledged and delete it. If
// the job does not have an outstanding reservation, error. If a job did not // the job does not have an outstanding reservation, error. If a job did not
// exist, succeed. // exist, succeed.
Delete(context.Context, *Task) error Delete(context.Context, *Call) error
} }
type Enqueue func(context.Context, MessageQueue, *Task) (*Task, error)

View File

@@ -8,8 +8,8 @@ import (
) )
const ( const (
defaultRouteTimeout = 30 // seconds DefaultRouteTimeout = 30 // seconds
htfnScaleDownTimeout = 30 // seconds DefaultIdleTimeout = 30 // seconds
) )
type Routes []*Route type Routes []*Route
@@ -50,11 +50,11 @@ func (r *Route) SetDefaults() {
} }
if r.Timeout == 0 { if r.Timeout == 0 {
r.Timeout = defaultRouteTimeout r.Timeout = DefaultRouteTimeout
} }
if r.IdleTimeout == 0 { if r.IdleTimeout == 0 {
r.IdleTimeout = htfnScaleDownTimeout r.IdleTimeout = DefaultIdleTimeout
} }
} }

View File

@@ -1,19 +0,0 @@
package models
import "github.com/go-openapi/strfmt"
/*Start start
swagger:model Start
*/
type Start struct {
/* Time when task started execution. Always in UTC.
*/
StartedAt strfmt.DateTime `json:"started_at,omitempty"`
}
// Validate validates this start
func (m *Start) Validate(formats strfmt.Registry) error {
return nil
}

View File

@@ -1,106 +0,0 @@
package models
import (
strfmt "github.com/go-openapi/strfmt"
)
const (
// TypeNone ...
TypeNone = ""
// TypeSync ...
TypeSync = "sync"
// TypeAsync ...
TypeAsync = "async"
)
const (
// FormatDefault ...
FormatDefault = "default"
// FormatHTTP ...
FormatHTTP = "http"
)
// TODO this should either be Task, or should be removed in favor of Task
type FnCall struct {
IDStatus
CompletedAt strfmt.DateTime `json:"completed_at,omitempty"`
CreatedAt strfmt.DateTime `json:"created_at,omitempty"`
StartedAt strfmt.DateTime `json:"started_at,omitempty"`
AppName string `json:"app_name,omitempty"`
Path string `json:"path"`
}
type FnCallLog struct {
CallID string `json:"call_id"`
Log string `json:"log"`
}
func (fnCall *FnCall) FromTask(task *Task) *FnCall {
return &FnCall{
CreatedAt: task.CreatedAt,
StartedAt: task.StartedAt,
CompletedAt: task.CompletedAt,
AppName: task.AppName,
Path: task.Path,
IDStatus: IDStatus{
ID: task.ID,
Status: task.Status,
},
}
}
// Task is a representation of a specific invocation of a route.
type Task struct {
IDStatus
// App this task belongs to.
AppName string `json:"app_name"`
// Path of the route that is responsible for this task
Path string `json:"path"`
// Name of Docker image to use.
Image string `json:"image"`
// Number of seconds to wait before queueing the task for consumption for the first time. Must be a positive integer. Tasks with a delay start in state "delayed" and transition to "running" after delay seconds.
Delay int32 `json:"delay,omitempty"`
// Payload for the task. This is only used by async tasks, to store their input.
Payload string `json:"payload,omitempty"`
// Priority of the task. Higher has more priority. 3 levels from 0-2. Tasks at same priority are processed in FIFO order.
Priority *int32 `json:"priority"`
// Maximum runtime in seconds.
Timeout int32 `json:"timeout,omitempty"`
// Hot function idle timeout in seconds before termination.
IdleTimeout int32 `json:"idle_timeout,omitempty"`
// Memory is the amount of RAM this task is allocated.
Memory uint64 `json:"memory,omitempty"`
// BaseEnv are the env vars for hot containers, not request specific.
BaseEnv map[string]string `json:"base_env,omitempty"`
// Env vars for the task. Comes from the ones set on the Route.
EnvVars map[string]string `json:"env_vars,omitempty"`
// Format is the format to pass input into the function.
// TODO plumb this in async land
// Format string `json:"format,omitempty"`
// Time when task completed, whether it was successul or failed. Always in UTC.
CompletedAt strfmt.DateTime `json:"completed_at,omitempty"`
// Time when task was submitted. Always in UTC.
CreatedAt strfmt.DateTime `json:"created_at,omitempty"`
// Time when task started execution. Always in UTC.
StartedAt strfmt.DateTime `json:"started_at,omitempty"`
}
type CallFilter struct {
Path string
AppName string
}

View File

@@ -13,8 +13,8 @@ import (
"github.com/sirupsen/logrus" "github.com/sirupsen/logrus"
"github.com/boltdb/bolt" "github.com/boltdb/bolt"
"github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/common"
) )
type BoltDbMQ struct { type BoltDbMQ struct {
@@ -180,7 +180,7 @@ func (mq *BoltDbMQ) Start() {
// We insert a "reservation" at readyAt, and store the json blob at the msg // We insert a "reservation" at readyAt, and store the json blob at the msg
// key. The timer loop plucks this out and puts it in the jobs bucket when the // key. The timer loop plucks this out and puts it in the jobs bucket when the
// time elapses. The value stored at the reservation key is the priority. // time elapses. The value stored at the reservation key is the priority.
func (mq *BoltDbMQ) delayTask(job *models.Task) (*models.Task, error) { func (mq *BoltDbMQ) delayCall(job *models.Call) (*models.Call, error) {
readyAt := time.Now().Add(time.Duration(job.Delay) * time.Second) readyAt := time.Now().Add(time.Duration(job.Delay) * time.Second)
err := mq.db.Update(func(tx *bolt.Tx) error { err := mq.db.Update(func(tx *bolt.Tx) error {
b := tx.Bucket(delayQueueName) b := tx.Bucket(delayQueueName)
@@ -204,12 +204,12 @@ func (mq *BoltDbMQ) delayTask(job *models.Task) (*models.Task, error) {
return job, err return job, err
} }
func (mq *BoltDbMQ) Push(ctx context.Context, job *models.Task) (*models.Task, error) { func (mq *BoltDbMQ) Push(ctx context.Context, job *models.Call) (*models.Call, error) {
ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID}) ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID})
log.Println("Pushed to MQ") log.Println("Pushed to MQ")
if job.Delay > 0 { if job.Delay > 0 {
return mq.delayTask(job) return mq.delayCall(job)
} }
err := mq.db.Update(func(tx *bolt.Tx) error { err := mq.db.Update(func(tx *bolt.Tx) error {
@@ -264,7 +264,7 @@ func resKeyToProperties(key []byte) (uint64, []byte) {
return reservedUntil, key[len(resKeyPrefix)+8:] return reservedUntil, key[len(resKeyPrefix)+8:]
} }
func (mq *BoltDbMQ) Reserve(ctx context.Context) (*models.Task, error) { func (mq *BoltDbMQ) Reserve(ctx context.Context) (*models.Call, error) {
// Start a writable transaction. // Start a writable transaction.
tx, err := mq.db.Begin(true) tx, err := mq.db.Begin(true)
if err != nil { if err != nil {
@@ -284,7 +284,7 @@ func (mq *BoltDbMQ) Reserve(ctx context.Context) (*models.Task, error) {
b.Delete(key) b.Delete(key)
var job models.Task var job models.Call
err = json.Unmarshal([]byte(value), &job) err = json.Unmarshal([]byte(value), &job)
if err != nil { if err != nil {
return nil, err return nil, err
@@ -324,7 +324,7 @@ func (mq *BoltDbMQ) Reserve(ctx context.Context) (*models.Task, error) {
return nil, nil return nil, nil
} }
func (mq *BoltDbMQ) Delete(ctx context.Context, job *models.Task) error { func (mq *BoltDbMQ) Delete(ctx context.Context, job *models.Call) error {
_, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID}) _, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID})
defer log.Println("Deleted") defer log.Println("Deleted")

View File

@@ -93,7 +93,7 @@ func NewIronMQ(url *url.URL) *IronMQ {
return mq return mq
} }
func (mq *IronMQ) Push(ctx context.Context, job *models.Task) (*models.Task, error) { func (mq *IronMQ) Push(ctx context.Context, job *models.Call) (*models.Call, error) {
if job.Priority == nil || *job.Priority < 0 || *job.Priority > 2 { if job.Priority == nil || *job.Priority < 0 || *job.Priority > 2 {
return nil, fmt.Errorf("IronMQ Push job %s: Bad priority", job.ID) return nil, fmt.Errorf("IronMQ Push job %s: Bad priority", job.ID)
} }
@@ -107,8 +107,8 @@ func (mq *IronMQ) Push(ctx context.Context, job *models.Task) (*models.Task, err
return job, err return job, err
} }
func (mq *IronMQ) Reserve(ctx context.Context) (*models.Task, error) { func (mq *IronMQ) Reserve(ctx context.Context) (*models.Call, error) {
var job models.Task var job models.Call
var messages []ironmq.Message var messages []ironmq.Message
var err error var err error
@@ -148,7 +148,7 @@ func (mq *IronMQ) Reserve(ctx context.Context) (*models.Task, error) {
return &job, nil return &job, nil
} }
func (mq *IronMQ) Delete(ctx context.Context, job *models.Task) error { func (mq *IronMQ) Delete(ctx context.Context, job *models.Call) error {
if job.Priority == nil || *job.Priority < 0 || *job.Priority > 2 { if job.Priority == nil || *job.Priority < 0 || *job.Priority > 2 {
return fmt.Errorf("IronMQ Delete job %s: Bad priority", job.ID) return fmt.Errorf("IronMQ Delete job %s: Bad priority", job.ID)
} }

View File

@@ -7,18 +7,18 @@ import (
"sync" "sync"
"time" "time"
"github.com/sirupsen/logrus" "github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/common"
"github.com/google/btree" "github.com/google/btree"
"github.com/sirupsen/logrus"
) )
type MemoryMQ struct { type MemoryMQ struct {
// WorkQueue A buffered channel that we can send work requests on. // WorkQueue A buffered channel that we can send work requests on.
PriorityQueues []chan *models.Task PriorityQueues []chan *models.Call
Ticker *time.Ticker Ticker *time.Ticker
BTree *btree.BTree BTree *btree.BTree
Timeouts map[string]*TaskItem Timeouts map[string]*callItem
// Protects B-tree and Timeouts // Protects B-tree and Timeouts
// If this becomes a bottleneck, consider separating the two mutexes. The // If this becomes a bottleneck, consider separating the two mutexes. The
// goroutine to clear up timed out messages could also become a bottleneck at // goroutine to clear up timed out messages could also become a bottleneck at
@@ -40,16 +40,16 @@ func randSeq(n int) string {
const NumPriorities = 3 const NumPriorities = 3
func NewMemoryMQ() *MemoryMQ { func NewMemoryMQ() *MemoryMQ {
var queues []chan *models.Task var queues []chan *models.Call
for i := 0; i < NumPriorities; i++ { for i := 0; i < NumPriorities; i++ {
queues = append(queues, make(chan *models.Task, 5000)) queues = append(queues, make(chan *models.Call, 5000))
} }
ticker := time.NewTicker(time.Second) ticker := time.NewTicker(time.Second)
mq := &MemoryMQ{ mq := &MemoryMQ{
PriorityQueues: queues, PriorityQueues: queues,
Ticker: ticker, Ticker: ticker,
BTree: btree.New(2), BTree: btree.New(2),
Timeouts: make(map[string]*TaskItem, 0), Timeouts: make(map[string]*callItem, 0),
} }
mq.start() mq.start()
logrus.Info("MemoryMQ initialized") logrus.Info("MemoryMQ initialized")
@@ -60,15 +60,15 @@ func (mq *MemoryMQ) start() {
// start goroutine to check for delayed jobs and put them onto regular queue when ready // start goroutine to check for delayed jobs and put them onto regular queue when ready
go func() { go func() {
for range mq.Ticker.C { for range mq.Ticker.C {
ji := &TaskItem{ ji := &callItem{
StartAt: time.Now(), StartAt: time.Now(),
} }
mq.Mutex.Lock() mq.Mutex.Lock()
mq.BTree.AscendLessThan(ji, func(a btree.Item) bool { mq.BTree.AscendLessThan(ji, func(a btree.Item) bool {
logrus.WithFields(logrus.Fields{"queue": a}).Debug("delayed job move to queue") logrus.WithFields(logrus.Fields{"queue": a}).Debug("delayed job move to queue")
ji2 := mq.BTree.Delete(a).(*TaskItem) ji2 := mq.BTree.Delete(a).(*callItem)
// put it onto the regular queue now // put it onto the regular queue now
_, err := mq.pushForce(ji2.Task) _, err := mq.pushForce(ji2.Call)
if err != nil { if err != nil {
logrus.WithError(err).Error("Couldn't push delayed message onto main queue") logrus.WithError(err).Error("Couldn't push delayed message onto main queue")
} }
@@ -81,14 +81,14 @@ func (mq *MemoryMQ) start() {
// TODO: this should be like the delayed messages above. Could even be the same thing as delayed messages, but remove them if job is completed. // TODO: this should be like the delayed messages above. Could even be the same thing as delayed messages, but remove them if job is completed.
go func() { go func() {
for range mq.Ticker.C { for range mq.Ticker.C {
ji := &TaskItem{ ji := &callItem{
StartAt: time.Now(), StartAt: time.Now(),
} }
mq.Mutex.Lock() mq.Mutex.Lock()
for _, jobItem := range mq.Timeouts { for _, jobItem := range mq.Timeouts {
if jobItem.Less(ji) { if jobItem.Less(ji) {
delete(mq.Timeouts, jobItem.Task.ID) delete(mq.Timeouts, jobItem.Call.ID)
_, err := mq.pushForce(jobItem.Task) _, err := mq.pushForce(jobItem.Call)
if err != nil { if err != nil {
logrus.WithError(err).Error("Couldn't push timed out message onto main queue") logrus.WithError(err).Error("Couldn't push timed out message onto main queue")
} }
@@ -99,19 +99,19 @@ func (mq *MemoryMQ) start() {
}() }()
} }
// TaskItem is for the Btree, implements btree.Item // callItem is for the Btree, implements btree.Item
type TaskItem struct { type callItem struct {
Task *models.Task Call *models.Call
StartAt time.Time StartAt time.Time
} }
func (ji *TaskItem) Less(than btree.Item) bool { func (ji *callItem) Less(than btree.Item) bool {
// TODO: this could lose jobs: https://godoc.org/github.com/google/btree#Item // TODO: this could lose jobs: https://godoc.org/github.com/google/btree#Item
ji2 := than.(*TaskItem) ji2 := than.(*callItem)
return ji.StartAt.Before(ji2.StartAt) return ji.StartAt.Before(ji2.StartAt)
} }
func (mq *MemoryMQ) Push(ctx context.Context, job *models.Task) (*models.Task, error) { func (mq *MemoryMQ) Push(ctx context.Context, job *models.Call) (*models.Call, error) {
_, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID}) _, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID})
log.Println("Pushed to MQ") log.Println("Pushed to MQ")
@@ -119,8 +119,8 @@ func (mq *MemoryMQ) Push(ctx context.Context, job *models.Task) (*models.Task, e
// job.MsgId = randSeq(20) // job.MsgId = randSeq(20)
if job.Delay > 0 { if job.Delay > 0 {
// then we'll put into short term storage until ready // then we'll put into short term storage until ready
ji := &TaskItem{ ji := &callItem{
Task: job, Call: job,
StartAt: time.Now().Add(time.Second * time.Duration(job.Delay)), StartAt: time.Now().Add(time.Second * time.Duration(job.Delay)),
} }
mq.Mutex.Lock() mq.Mutex.Lock()
@@ -136,10 +136,10 @@ func (mq *MemoryMQ) Push(ctx context.Context, job *models.Task) (*models.Task, e
return mq.pushForce(job) return mq.pushForce(job)
} }
func (mq *MemoryMQ) pushTimeout(job *models.Task) error { func (mq *MemoryMQ) pushTimeout(job *models.Call) error {
ji := &TaskItem{ ji := &callItem{
Task: job, Call: job,
StartAt: time.Now().Add(time.Minute), StartAt: time.Now().Add(time.Minute),
} }
mq.Mutex.Lock() mq.Mutex.Lock()
@@ -148,13 +148,13 @@ func (mq *MemoryMQ) pushTimeout(job *models.Task) error {
return nil return nil
} }
func (mq *MemoryMQ) pushForce(job *models.Task) (*models.Task, error) { func (mq *MemoryMQ) pushForce(job *models.Call) (*models.Call, error) {
mq.PriorityQueues[*job.Priority] <- job mq.PriorityQueues[*job.Priority] <- job
return job, nil return job, nil
} }
// This is recursive, so be careful how many channels you pass in. // This is recursive, so be careful how many channels you pass in.
func pickEarliestNonblocking(channels ...chan *models.Task) *models.Task { func pickEarliestNonblocking(channels ...chan *models.Call) *models.Call {
if len(channels) == 0 { if len(channels) == 0 {
return nil return nil
} }
@@ -167,7 +167,7 @@ func pickEarliestNonblocking(channels ...chan *models.Task) *models.Task {
} }
} }
func (mq *MemoryMQ) Reserve(ctx context.Context) (*models.Task, error) { func (mq *MemoryMQ) Reserve(ctx context.Context) (*models.Call, error) {
job := pickEarliestNonblocking(mq.PriorityQueues[2], mq.PriorityQueues[1], mq.PriorityQueues[0]) job := pickEarliestNonblocking(mq.PriorityQueues[2], mq.PriorityQueues[1], mq.PriorityQueues[0])
if job == nil { if job == nil {
return nil, nil return nil, nil
@@ -178,7 +178,7 @@ func (mq *MemoryMQ) Reserve(ctx context.Context) (*models.Task, error) {
return job, mq.pushTimeout(job) return job, mq.pushTimeout(job)
} }
func (mq *MemoryMQ) Delete(ctx context.Context, job *models.Task) error { func (mq *MemoryMQ) Delete(ctx context.Context, job *models.Call) error {
_, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID}) _, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID})
mq.Mutex.Lock() mq.Mutex.Lock()

View File

@@ -13,14 +13,14 @@ type Mock struct {
Routes []*models.Route Routes []*models.Route
} }
func (mock *Mock) Push(context.Context, *models.Task) (*models.Task, error) { func (mock *Mock) Push(context.Context, *models.Call) (*models.Call, error) {
return nil, nil return nil, nil
} }
func (mock *Mock) Reserve(context.Context) (*models.Task, error) { func (mock *Mock) Reserve(context.Context) (*models.Call, error) {
return nil, nil return nil, nil
} }
func (mock *Mock) Delete(context.Context, *models.Task) error { func (mock *Mock) Delete(context.Context, *models.Call) error {
return nil return nil
} }

View File

@@ -46,19 +46,19 @@ type metricMQ struct {
mq models.MessageQueue mq models.MessageQueue
} }
func (m *metricMQ) Push(ctx context.Context, t *models.Task) (*models.Task, error) { func (m *metricMQ) Push(ctx context.Context, t *models.Call) (*models.Call, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "mq_push") span, ctx := opentracing.StartSpanFromContext(ctx, "mq_push")
defer span.Finish() defer span.Finish()
return m.mq.Push(ctx, t) return m.mq.Push(ctx, t)
} }
func (m *metricMQ) Reserve(ctx context.Context) (*models.Task, error) { func (m *metricMQ) Reserve(ctx context.Context) (*models.Call, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "mq_reserve") span, ctx := opentracing.StartSpanFromContext(ctx, "mq_reserve")
defer span.Finish() defer span.Finish()
return m.mq.Reserve(ctx) return m.mq.Reserve(ctx)
} }
func (m *metricMQ) Delete(ctx context.Context, t *models.Task) error { func (m *metricMQ) Delete(ctx context.Context, t *models.Call) error {
span, ctx := opentracing.StartSpanFromContext(ctx, "mq_delete") span, ctx := opentracing.StartSpanFromContext(ctx, "mq_delete")
defer span.Finish() defer span.Finish()
return m.mq.Delete(ctx, t) return m.mq.Delete(ctx, t)

View File

@@ -9,10 +9,10 @@ import (
"strconv" "strconv"
"time" "time"
"github.com/sirupsen/logrus" "github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/common"
"github.com/garyburd/redigo/redis" "github.com/garyburd/redigo/redis"
"github.com/sirupsen/logrus"
) )
type RedisMQ struct { type RedisMQ struct {
@@ -100,7 +100,7 @@ func (mq *RedisMQ) processPendingReservations() {
return return
} }
var job models.Task var job models.Call
err = json.Unmarshal(response, &job) err = json.Unmarshal(response, &job)
if err != nil { if err != nil {
logrus.WithError(err).Error("error unmarshaling job json") logrus.WithError(err).Error("error unmarshaling job json")
@@ -114,7 +114,7 @@ func (mq *RedisMQ) processPendingReservations() {
redisPush(conn, mq.queueName, &job) redisPush(conn, mq.queueName, &job)
} }
func (mq *RedisMQ) processDelayedTasks() { func (mq *RedisMQ) processDelayedCalls() {
conn := mq.pool.Get() conn := mq.pool.Get()
defer conn.Close() defer conn.Close()
@@ -141,7 +141,7 @@ func (mq *RedisMQ) processDelayedTasks() {
continue continue
} }
var job models.Task var job models.Call
err = json.Unmarshal(buf, &job) err = json.Unmarshal(buf, &job)
if err != nil { if err != nil {
logrus.WithError(err).WithFields(logrus.Fields{"buf": buf, "reservationId": resId}).Error("Error unmarshaling job") logrus.WithError(err).WithFields(logrus.Fields{"buf": buf, "reservationId": resId}).Error("Error unmarshaling job")
@@ -164,12 +164,12 @@ func (mq *RedisMQ) start() {
go func() { go func() {
for range mq.ticker.C { for range mq.ticker.C {
mq.processPendingReservations() mq.processPendingReservations()
mq.processDelayedTasks() mq.processDelayedCalls()
} }
}() }()
} }
func redisPush(conn redis.Conn, queue string, job *models.Task) (*models.Task, error) { func redisPush(conn redis.Conn, queue string, job *models.Call) (*models.Call, error) {
buf, err := json.Marshal(job) buf, err := json.Marshal(job)
if err != nil { if err != nil {
return nil, err return nil, err
@@ -181,7 +181,7 @@ func redisPush(conn redis.Conn, queue string, job *models.Task) (*models.Task, e
return job, nil return job, nil
} }
func (mq *RedisMQ) delayTask(conn redis.Conn, job *models.Task) (*models.Task, error) { func (mq *RedisMQ) delayCall(conn redis.Conn, job *models.Call) (*models.Call, error) {
buf, err := json.Marshal(job) buf, err := json.Marshal(job)
if err != nil { if err != nil {
return nil, err return nil, err
@@ -209,7 +209,7 @@ func (mq *RedisMQ) delayTask(conn redis.Conn, job *models.Task) (*models.Task, e
return job, nil return job, nil
} }
func (mq *RedisMQ) Push(ctx context.Context, job *models.Task) (*models.Task, error) { func (mq *RedisMQ) Push(ctx context.Context, job *models.Call) (*models.Call, error) {
_, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID}) _, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID})
defer log.Println("Pushed to MQ") defer log.Println("Pushed to MQ")
@@ -217,7 +217,7 @@ func (mq *RedisMQ) Push(ctx context.Context, job *models.Task) (*models.Task, er
defer conn.Close() defer conn.Close()
if job.Delay > 0 { if job.Delay > 0 {
return mq.delayTask(conn, job) return mq.delayCall(conn, job)
} }
return redisPush(conn, mq.queueName, job) return redisPush(conn, mq.queueName, job)
} }
@@ -226,11 +226,11 @@ func (mq *RedisMQ) checkNilResponse(err error) bool {
} }
// Would be nice to switch to this model http://redis.io/commands/rpoplpush#pattern-reliable-queue // Would be nice to switch to this model http://redis.io/commands/rpoplpush#pattern-reliable-queue
func (mq *RedisMQ) Reserve(ctx context.Context) (*models.Task, error) { func (mq *RedisMQ) Reserve(ctx context.Context) (*models.Call, error) {
conn := mq.pool.Get() conn := mq.pool.Get()
defer conn.Close() defer conn.Close()
var job models.Task var job models.Call
var resp []byte var resp []byte
var err error var err error
for i := 2; i >= 0; i-- { for i := 2; i >= 0; i-- {
@@ -286,7 +286,7 @@ func (mq *RedisMQ) Reserve(ctx context.Context) (*models.Task, error) {
return &job, nil return &job, nil
} }
func (mq *RedisMQ) Delete(ctx context.Context, job *models.Task) error { func (mq *RedisMQ) Delete(ctx context.Context, job *models.Call) error {
_, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID}) _, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": job.ID})
defer log.Println("Deleted") defer log.Println("Deleted")

View File

@@ -1,198 +0,0 @@
package runner
import (
"bytes"
"context"
"crypto/tls"
"encoding/json"
"errors"
"fmt"
"io"
"io/ioutil"
"net"
"net/http"
"net/url"
"sync"
"time"
"github.com/sirupsen/logrus"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/common"
taskpkg "github.com/fnproject/fn/api/runner/task"
"github.com/opentracing/opentracing-go"
)
var client = &http.Client{
Transport: &http.Transport{
Proxy: http.ProxyFromEnvironment,
Dial: (&net.Dialer{
Timeout: 10 * time.Second,
KeepAlive: 120 * time.Second,
}).Dial,
MaxIdleConnsPerHost: 512,
TLSHandshakeTimeout: 10 * time.Second,
TLSClientConfig: &tls.Config{
ClientSessionCache: tls.NewLRUClientSessionCache(4096),
},
},
}
func getTask(ctx context.Context, url string) (*models.Task, error) {
// TODO shove this ctx into the request?
span, _ := opentracing.StartSpanFromContext(ctx, "get_task")
defer span.Finish()
req, _ := http.NewRequest(http.MethodGet, url, nil)
resp, err := client.Do(req.WithContext(ctx))
if err != nil {
return nil, err
}
defer func() {
io.Copy(ioutil.Discard, resp.Body)
resp.Body.Close()
}()
if resp.StatusCode != http.StatusOK {
return nil, errors.New(fmt.Sprintf("Unable to get task. Reason: %v", resp.Status))
}
var task models.Task
err = json.NewDecoder(resp.Body).Decode(&task)
if err != nil {
return nil, err
}
if task.ID == "" {
return nil, nil
}
return &task, nil
}
func deleteTask(ctx context.Context, url string, task *models.Task) error {
span, _ := opentracing.StartSpanFromContext(ctx, "delete_task")
defer span.Finish()
// Unmarshal task to be sent over as a json
body, err := json.Marshal(task)
if err != nil {
return err
}
// TODO use a reasonable http client..
// Send out Delete request to delete task from queue
req, err := http.NewRequest(http.MethodDelete, url, bytes.NewBuffer(body))
if err != nil {
return err
}
resp, err := client.Do(req)
if err != nil {
return err
}
defer func() {
io.Copy(ioutil.Discard, resp.Body)
resp.Body.Close()
}()
if resp.StatusCode != http.StatusAccepted {
body, err := ioutil.ReadAll(resp.Body)
if err != nil {
return err
}
return errors.New(string(body))
}
return nil
}
// RunAsyncRunner pulls tasks off a queue and processes them
func RunAsyncRunner(ctx context.Context, tasksrv string, rnr *Runner, ds models.Datastore) {
u := tasksrvURL(tasksrv)
startAsyncRunners(ctx, u, rnr, ds)
<-ctx.Done()
}
func startAsyncRunners(ctx context.Context, url string, rnr *Runner, ds models.Datastore) {
var wg sync.WaitGroup
ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"runner": "async"})
for {
select {
case <-ctx.Done():
wg.Wait()
return
default:
}
if !rnr.hasAsyncAvailableMemory() { // TODO this should be a channel to subscribe to
log.Debug("memory full")
time.Sleep(1 * time.Second)
continue
}
runAsyncTask(ctx, url, rnr, ds, &wg)
}
}
func runAsyncTask(ctx context.Context, url string, rnr *Runner, ds models.Datastore, wg *sync.WaitGroup) {
// start a new span altogether, unrelated to the shared global context
span := opentracing.GlobalTracer().StartSpan("async_task")
ctx = opentracing.ContextWithSpan(ctx, span)
defer span.Finish()
log := common.Logger(ctx)
task, err := getTask(ctx, url)
if err != nil {
if err, ok := err.(net.Error); ok && err.Timeout() {
log.WithError(err).Errorln("Could not fetch task, timeout.")
return
}
log.WithError(err).Error("Could not fetch task")
time.Sleep(1 * time.Second)
return
}
if task == nil {
time.Sleep(1 * time.Second)
return
}
ctx, log = common.LoggerWithFields(ctx, logrus.Fields{"call_id": task.ID})
log.Info("Running task async:", task.ID)
wg.Add(1)
go func() {
defer wg.Done()
// Process Task
_, err := rnr.RunTrackedTask(task, ctx, taskpkg.ConfigFromTask(task))
if err != nil {
log.WithError(err).Error("Cannot run task")
}
log.Debug("Processed task")
}()
// TODO this is so wrong... fix later+asap
// Delete task from queue
if err := deleteTask(ctx, url, task); err != nil {
log.WithError(err).Error("Cannot delete task")
return
}
// TODO uh, even if we don't delete it it still runs but w/e
log.Info("Task complete")
}
func tasksrvURL(tasksrv string) string {
parsed, err := url.Parse(tasksrv)
if err != nil {
logrus.WithError(err).Fatalln("cannot parse API_URL endpoint")
}
if parsed.Scheme == "" {
parsed.Scheme = "http"
}
if parsed.Path == "" || parsed.Path == "/" {
parsed.Path = "/tasks"
}
return parsed.String()
}

View File

@@ -1,242 +0,0 @@
package runner
import (
"bytes"
"context"
"encoding/json"
"fmt"
"io/ioutil"
"log"
"math/rand"
"net/http"
"net/http/httptest"
"testing"
"time"
"github.com/sirupsen/logrus"
"github.com/fnproject/fn/api/datastore"
"github.com/fnproject/fn/api/logs"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/mqs"
"github.com/fnproject/fn/api/runner/drivers"
"github.com/fnproject/fn/api/runner/task"
"github.com/gin-gonic/gin"
)
func setLogBuffer() *bytes.Buffer {
var buf bytes.Buffer
buf.WriteByte('\n')
logrus.SetOutput(&buf)
gin.DefaultErrorWriter = &buf
gin.DefaultWriter = &buf
log.SetOutput(&buf)
return &buf
}
func getMockTask() models.Task {
priority := int32(0)
image := fmt.Sprintf("Image-%d", rand.Int31()%1000)
task := &models.Task{}
task.Image = image
task.ID = fmt.Sprintf("ID-%d", rand.Int31()%1000)
task.AppName = fmt.Sprintf("RouteName-%d", rand.Int31()%1000)
task.Priority = &priority
return *task
}
func getTestServer(mockTasks []*models.Task) *httptest.Server {
ctx := context.TODO()
mq, err := mqs.New("memory://test")
if err != nil {
panic(err)
}
for _, mt := range mockTasks {
mq.Push(ctx, mt)
}
getHandler := func(c *gin.Context) {
task, err := mq.Reserve(ctx)
if err != nil {
logrus.WithError(err)
c.JSON(http.StatusInternalServerError, err)
return
}
c.JSON(http.StatusOK, task)
}
delHandler := func(c *gin.Context) {
body, err := ioutil.ReadAll(c.Request.Body)
if err != nil {
logrus.WithError(err)
c.JSON(http.StatusInternalServerError, err.Error())
return
}
var task models.Task
if err = json.Unmarshal(body, &task); err != nil {
logrus.WithError(err)
c.JSON(http.StatusInternalServerError, err.Error())
return
}
if err := mq.Delete(ctx, &task); err != nil {
logrus.WithError(err)
c.JSON(http.StatusInternalServerError, err.Error())
return
}
c.JSON(http.StatusAccepted, task)
}
r := gin.Default()
r.GET("/tasks", getHandler)
r.DELETE("/tasks", delHandler)
return httptest.NewServer(r)
}
func TestGetTask(t *testing.T) {
buf := setLogBuffer()
mockTask := getMockTask()
ts := getTestServer([]*models.Task{&mockTask})
defer ts.Close()
url := ts.URL + "/tasks"
task, err := getTask(context.Background(), url)
if err != nil {
t.Log(buf.String())
t.Error("expected no error, got", err)
}
if task.ID != mockTask.ID {
t.Log(buf.String())
t.Errorf("expected task ID '%s', got '%s'", task.ID, mockTask.ID)
}
}
func TestGetTaskError(t *testing.T) {
buf := setLogBuffer()
tests := []map[string]interface{}{
{
"url": "/invalid",
"task": getMockTask(),
"error": "Unable to get task. Reason: 404 Not Found",
},
}
var tasks []*models.Task
for _, v := range tests {
task := v["task"].(models.Task)
tasks = append(tasks, &task)
}
ts := getTestServer(tasks)
defer ts.Close()
for i, test := range tests {
url := ts.URL + test["url"].(string)
_, err := getTask(context.Background(), url)
if err == nil {
t.Log(buf.String())
t.Errorf("expected error '%s'", test["error"].(string))
}
if err.Error() != test["error"].(string) {
t.Log(buf.String())
t.Errorf("test %d: expected error '%s', got '%s'", i, test["error"].(string), err)
}
}
}
func TestDeleteTask(t *testing.T) {
buf := setLogBuffer()
mockTask := getMockTask()
ctx := context.Background()
ts := getTestServer([]*models.Task{&mockTask})
defer ts.Close()
url := ts.URL + "/tasks"
err := deleteTask(ctx, url, &mockTask)
if err == nil {
t.Log(buf.String())
t.Error("expected error 'Not reserver', got", err)
}
_, err = getTask(ctx, url)
if err != nil {
t.Log(buf.String())
t.Error("expected no error, got", err)
}
err = deleteTask(ctx, url, &mockTask)
if err != nil {
t.Log(buf.String())
t.Error("expected no error, got", err)
}
}
func TestTasksrvURL(t *testing.T) {
tests := []struct {
in, out string
}{
{"//localhost:8081", "http://localhost:8081/tasks"},
{"//localhost:8081/", "http://localhost:8081/tasks"},
{"http://localhost:8081", "http://localhost:8081/tasks"},
{"http://localhost:8081/", "http://localhost:8081/tasks"},
{"http://localhost:8081/endpoint", "http://localhost:8081/endpoint"},
}
for _, tt := range tests {
if got := tasksrvURL(tt.in); got != tt.out {
t.Errorf("tasksrv: %s\texpected: %s\tgot: %s\t", tt.in, tt.out, got)
}
}
}
func testRunner(t *testing.T) (*Runner, context.CancelFunc) {
ctx, cancel := context.WithCancel(context.Background())
ds := datastore.NewMock()
fnl := logs.NewMock()
r, err := New(ctx, NewFuncLogger(fnl), ds)
if err != nil {
t.Fatal("Test: failed to create new runner")
}
return r, cancel
}
type RunResult struct {
drivers.RunResult
}
func (r RunResult) Status() string {
return "success"
}
func TestAsyncRunnersGracefulShutdown(t *testing.T) {
buf := setLogBuffer()
mockTask := getMockTask()
ts := getTestServer([]*models.Task{&mockTask})
defer ts.Close()
tasks := make(chan task.Request)
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
defer close(tasks)
go func() {
for t := range tasks {
t.Response <- task.Response{
Result: RunResult{},
Err: nil,
}
}
}()
rnr, cancel := testRunner(t)
defer cancel()
startAsyncRunners(ctx, ts.URL+"/tasks", rnr, datastore.NewMock())
if err := ctx.Err(); err != context.DeadlineExceeded {
t.Log(buf.String())
t.Errorf("async runners stopped unexpectedly. context error: %v", err)
}
}

View File

@@ -1,24 +0,0 @@
package common
import (
"github.com/fnproject/fn/api/runner/common/stats"
)
// An Environment is a long lived object that carries around 'configuration'
// for the program. Other long-lived objects may embed an environment directly
// into their definition. Environments wrap common functionality like logging
// and metrics. For short-lived request-response like tasks use `Context`,
// which wraps an Environment.
type Environment struct {
stats.Statter
}
// Initializers are functions that may set up the environment as they like. By default the environment is 'inactive' in the sense that metrics aren't reported.
func NewEnvironment(initializers ...func(e *Environment)) *Environment {
env := &Environment{&stats.NilStatter{}}
for _, init := range initializers {
init(env)
}
return env
}

View File

@@ -1,174 +0,0 @@
package stats
import (
"sync"
"time"
)
type reporter interface {
report([]*collectedStat)
}
type collectedStat struct {
Name string
Counters map[string]int64
Values map[string]float64
Gauges map[string]int64
Timers map[string]time.Duration
avgCounts map[string]uint64
}
func newCollectedStatUnescaped(name string) *collectedStat {
return &collectedStat{
Name: name,
Counters: map[string]int64{},
Values: map[string]float64{},
Gauges: map[string]int64{},
Timers: map[string]time.Duration{},
avgCounts: map[string]uint64{},
}
}
// What do you call an alligator in a vest?
// Aggregator collects a stats and merges them together if they've been added
// previously. Useful for reporters that have low throughput ie stathat.
type Aggregator struct {
// Holds all of our stats based on stat.Name
sl sync.RWMutex
stats map[string]*statHolder
reporters []reporter
}
func newAggregator(reporters []reporter) *Aggregator {
return &Aggregator{
stats: make(map[string]*statHolder),
reporters: reporters,
}
}
type statHolder struct {
cl sync.RWMutex // Lock on Counters
vl sync.RWMutex // Lock on Values
s *collectedStat
}
func newStatHolder(st *collectedStat) *statHolder {
return &statHolder{s: st}
}
type kind int16
const (
counterKind kind = iota
valueKind
gaugeKind
durationKind
)
func (a *Aggregator) add(component, key string, kind kind, value interface{}) {
a.sl.RLock()
stat, ok := a.stats[component]
a.sl.RUnlock()
if !ok {
a.sl.Lock()
stat, ok = a.stats[component]
if !ok {
stat = newStatHolder(newCollectedStatUnescaped(component))
a.stats[component] = stat
}
a.sl.Unlock()
}
if kind == counterKind || kind == gaugeKind {
var mapPtr map[string]int64
if kind == counterKind {
mapPtr = stat.s.Counters
} else {
mapPtr = stat.s.Gauges
}
value := value.(int64)
stat.cl.Lock()
mapPtr[key] += value
stat.cl.Unlock()
}
/* TODO: this ends up ignoring tags so yeah gg
/ lets just calculate a running average for now. Can do percentiles later
/ Recalculated Average
/
/ currentAverage * currentCount + newValue
/ ------------------------------------------
/ (currentCount +1)
/
*/
if kind == valueKind || kind == durationKind {
var typedValue int64
if kind == valueKind {
typedValue = value.(int64)
} else {
typedValue = int64(value.(time.Duration))
}
stat.vl.Lock()
switch kind {
case valueKind:
oldAverage := stat.s.Values[key]
count := stat.s.avgCounts[key]
newAverage := (oldAverage*float64(count) + float64(typedValue)) / (float64(count + 1))
stat.s.avgCounts[key] = count + 1
stat.s.Values[key] = newAverage
case durationKind:
oldAverage := float64(stat.s.Timers[key])
count := stat.s.avgCounts[key]
newAverage := (oldAverage*float64(count) + float64(typedValue)) / (float64(count + 1))
stat.s.avgCounts[key] = count + 1
stat.s.Timers[key] = time.Duration(newAverage)
}
stat.vl.Unlock()
}
}
func (a *Aggregator) dump() []*collectedStat {
a.sl.Lock()
bucket := a.stats
// Clear out the maps, effectively resetting our average
a.stats = make(map[string]*statHolder)
a.sl.Unlock()
stats := make([]*collectedStat, 0, len(bucket))
for _, v := range bucket {
stats = append(stats, v.s)
}
return stats
}
func (a *Aggregator) report(st []*collectedStat) {
stats := a.dump()
stats = append(stats, st...)
for _, r := range a.reporters {
r.report(stats)
}
}
func (r *Aggregator) Inc(component string, stat string, value int64, rate float32) {
r.add(component, stat, counterKind, value)
}
func (r *Aggregator) Gauge(component string, stat string, value int64, rate float32) {
r.add(component, stat, gaugeKind, value)
}
func (r *Aggregator) Measure(component string, stat string, value int64, rate float32) {
r.add(component, stat, valueKind, value)
}
func (r *Aggregator) Time(component string, stat string, value time.Duration, rate float32) {
r.add(component, stat, durationKind, value)
}
func (r *Aggregator) NewTimer(component string, stat string, rate float32) *Timer {
return newTimer(r, component, stat, rate)
}

View File

@@ -1,81 +0,0 @@
package stats
import (
"fmt"
"math/rand"
"testing"
)
func TestAggregator(t *testing.T) {
ag := newAggregator([]reporter{})
var sum int64 = 0
var times int64 = 0
for i := 0; i < 100; i++ {
ag.add("mq push", "messages", counterKind, int64(1))
ag.add("mq push", "latency", valueKind, int64(i))
ag.add("mq pull", "latency", valueKind, int64(i))
sum += int64(i)
times += 1
}
for _, stat := range ag.dump() {
for k, v := range stat.Values {
if v != float64(sum)/float64(times) {
t.Error("key:", k, "Expected", sum/times, "got", v)
}
}
for k, v := range stat.Counters {
if v != times {
t.Error("key:", k, "Expected", times, "got", v)
}
}
}
if len(ag.stats) != 0 {
t.Error("expected stats map to be clear, got", len(ag.stats))
}
}
type testStat struct {
component string
key string
kind kind
value int64
}
func BenchmarkAggregatorAdd(b *testing.B) {
ag := &Aggregator{
stats: make(map[string]*statHolder, 1000),
}
s := createStatList(1000)
sl := len(s)
b.ResetTimer()
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
e := s[rand.Intn(sl)]
ag.add(e.component, e.key, e.kind, e.value)
}
})
}
func createStatList(n int) []*testStat {
var stats []*testStat
for i := 0; i < n; i++ {
st := testStat{
component: "aggregator_test",
key: fmt.Sprintf("latency.%d", i),
kind: counterKind,
value: 1,
}
if rand.Float32() < 0.5 {
st.key = fmt.Sprintf("test.%d", i)
st.kind = valueKind
st.value = 15999
}
stats = append(stats, &st)
}
return stats
}

View File

@@ -1,31 +0,0 @@
package stats
import (
"time"
"github.com/sirupsen/logrus"
)
type LogReporter struct {
}
func NewLogReporter() *LogReporter {
return (&LogReporter{})
}
func (lr *LogReporter) report(stats []*collectedStat) {
for _, s := range stats {
f := make(logrus.Fields)
for k, v := range s.Counters {
f[k] = v
}
for k, v := range s.Values {
f[k] = v
}
for k, v := range s.Timers {
f[k] = time.Duration(v)
}
logrus.WithFields(f).Info(s.Name)
}
}

View File

@@ -1,26 +0,0 @@
package stats
import (
"runtime"
"time"
)
func StartReportingMemoryAndGC(reporter Statter, d time.Duration) {
ticker := time.Tick(d)
for {
select {
case <-ticker:
var ms runtime.MemStats
runtime.ReadMemStats(&ms)
prefix := "runtime"
reporter.Measure(prefix, "allocated", int64(ms.Alloc), 1.0)
reporter.Measure(prefix, "allocated.heap", int64(ms.HeapAlloc), 1.0)
reporter.Time(prefix, "gc.pause", time.Duration(ms.PauseNs[(ms.NumGC+255)%256]), 1.0)
// GC CPU percentage.
reporter.Measure(prefix, "gc.cpufraction", int64(ms.GCCPUFraction*100), 1.0)
}
}
}

View File

@@ -1,128 +0,0 @@
package stats
import (
"encoding/json"
"fmt"
"io/ioutil"
"net/http"
"os"
"strings"
"github.com/sirupsen/logrus"
)
type NewRelicAgentConfig struct {
Host string `json:"host"`
Version string `json:"version"`
Pid int `json:"pid"`
}
// examples: https://docs.newrelic.com/docs/plugins/plugin-developer-resources/developer-reference/metric-data-plugin-api#examples
type newRelicRequest struct {
Agent *agent `json:"agent"`
Components []*component `json:"components"`
}
type NewRelicReporterConfig struct {
Agent *NewRelicAgentConfig
LicenseKey string `json:"license_key"`
}
type NewRelicReporter struct {
Agent *agent
LicenseKey string
}
func NewNewRelicReporter(version string, licenseKey string) *NewRelicReporter {
r := &NewRelicReporter{}
r.Agent = newNewRelicAgent(version)
r.LicenseKey = licenseKey
return r
}
func (r *NewRelicReporter) report(stats []*collectedStat) {
client := &http.Client{}
req := &newRelicRequest{}
req.Agent = r.Agent
comp := newComponent()
comp.Name = "IronMQ"
comp.Duration = 60
comp.GUID = "io.iron.ironmq"
// TODO - NR has a fixed 3 level heirarchy? and we just use 2?
req.Components = []*component{comp}
// now add metrics
for _, s := range stats {
for k, v := range s.Counters {
comp.Metrics[fmt.Sprintf("Component/%s %s", s.Name, k)] = v
}
for k, v := range s.Values {
comp.Metrics[fmt.Sprintf("Component/%s %s", s.Name, k)] = int64(v)
}
for k, v := range s.Timers {
comp.Metrics[fmt.Sprintf("Component/%s %s", s.Name, k)] = int64(v)
}
}
metricsJson, err := json.Marshal(req)
if err != nil {
logrus.WithError(err).Error("error encoding json for NewRelicReporter")
}
jsonAsString := string(metricsJson)
httpRequest, err := http.NewRequest("POST",
"https://platform-api.newrelic.com/platform/v1/metrics",
strings.NewReader(jsonAsString))
if err != nil {
logrus.WithError(err).Error("error creating New Relic request")
return
}
httpRequest.Header.Set("X-License-Key", r.LicenseKey)
httpRequest.Header.Set("Content-Type", "application/json")
httpRequest.Header.Set("Accept", "application/json")
httpResponse, err := client.Do(httpRequest)
if err != nil {
logrus.WithError(err).Error("error sending http request in NewRelicReporter")
return
}
defer httpResponse.Body.Close()
body, err := ioutil.ReadAll(httpResponse.Body)
if err != nil {
logrus.WithError(err).Error("error reading response body")
} else {
logrus.Debugln("response", "code", httpResponse.Status, "body", string(body))
}
}
type agent struct {
Host string `json:"host"`
Version string `json:"version"`
Pid int `json:"pid"`
}
func newNewRelicAgent(Version string) *agent {
var err error
agent := &agent{
Version: Version,
}
agent.Pid = os.Getpid()
if agent.Host, err = os.Hostname(); err != nil {
logrus.WithError(err).Error("Can not get hostname")
return nil
}
return agent
}
type component struct {
Name string `json:"name"`
GUID string `json:"guid"`
Duration int `json:"duration"`
Metrics map[string]int64 `json:"metrics"`
}
func newComponent() *component {
c := &component{}
c.Metrics = make(map[string]int64)
return c
}

View File

@@ -1,103 +0,0 @@
// +build riemann
package stats
import (
"os"
"time"
"github.com/sirupsen/logrus"
"github.com/amir/raidman"
)
type RiemannClient struct {
client *raidman.Client
attributes map[string]string
}
const (
StateNormal = "normal"
)
func (rc *RiemannClient) Report([]*Stat) {}
func (rc *RiemannClient) Add(s *Stat) {
var events []*raidman.Event
t := time.Now().UnixNano()
for k, v := range rc.attributes {
s.Tags[k] = v
}
for k, v := range s.Counters {
events = append(events, &raidman.Event{
Ttl: 5.0,
Time: t,
State: StateNormal,
Service: s.Name + " " + k,
Metric: v,
Attributes: s.Tags,
})
}
for k, v := range s.Values {
events = append(events, &raidman.Event{
Ttl: 5.0,
Time: t,
State: StateNormal,
Service: s.Name + " " + k,
Metric: v,
Attributes: s.Tags,
})
}
rc.report(events)
}
func (rc *RiemannClient) report(events []*raidman.Event) {
err := rc.client.SendMulti(events)
if err != nil {
logrus.WithError(err).Error("error sending to Riemann")
}
}
func (rc *RiemannClient) heartbeat() {
events := []*raidman.Event{
&raidman.Event{
Ttl: 5.0,
Time: time.Now().UnixNano(),
State: StateNormal,
Service: "heartbeat",
Metric: 1.0,
Attributes: rc.attributes,
},
}
rc.report(events)
}
func newRiemann(config Config) *RiemannClient {
c, err := raidman.Dial("tcp", config.Riemann.RiemannHost)
if err != nil {
logrus.WithError(err).Error("error dialing Riemann")
os.Exit(1)
}
client := &RiemannClient{
client: c,
attributes: map[string]string{},
}
for k, v := range config.Tags {
client.attributes[k] = v
}
// Send out a heartbeat every second
go func(rc *RiemannClient) {
for _ = range time.Tick(1 * time.Second) {
rc.heartbeat()
}
}(client)
return client
}

View File

@@ -1,51 +0,0 @@
package stats
import (
"net/http"
"net/url"
"strconv"
"github.com/sirupsen/logrus"
)
func postStatHat(key, stat string, values url.Values) {
values.Set("stat", stat)
values.Set("ezkey", key)
resp, err := http.PostForm("http://api.stathat.com/ez", values)
if err != nil {
logrus.WithError(err).Error("couldn't post to StatHat")
return
}
if resp.StatusCode != 200 {
logrus.Errorln("bad status posting to StatHat", "status_code", resp.StatusCode)
}
resp.Body.Close()
}
type StatHatReporterConfig struct {
Email string
Prefix string
}
func (shr *StatHatReporterConfig) report(stats []*collectedStat) {
for _, s := range stats {
for k, v := range s.Counters {
n := shr.Prefix + " " + s.Name + " " + k
values := url.Values{}
values.Set("count", strconv.FormatInt(v, 10))
postStatHat(shr.Email, n, values)
}
for k, v := range s.Values {
n := shr.Prefix + " " + s.Name + " " + k
values := url.Values{}
values.Set("value", strconv.FormatFloat(v, 'f', 3, 64))
postStatHat(shr.Email, n, values)
}
for k, v := range s.Timers {
n := shr.Prefix + " " + s.Name + " " + k
values := url.Values{}
values.Set("value", strconv.FormatInt(int64(v), 10))
postStatHat(shr.Email, n, values)
}
}
}

View File

@@ -1,173 +0,0 @@
package stats
import (
"encoding/json"
"errors"
"net/http"
"strings"
"time"
"github.com/sirupsen/logrus"
)
type HTTPSubHandler interface {
HTTPHandler(relativeUrl []string, w http.ResponseWriter, r *http.Request)
}
type Config struct {
Interval float64 `json:"interval"` // seconds
History int // minutes
Log string `json:"log"`
StatHat *StatHatReporterConfig
NewRelic *NewRelicReporterConfig
Statsd *StatsdConfig
GCStats int `json:"gc_stats"`
}
type Statter interface {
Inc(component string, stat string, value int64, rate float32)
Gauge(component string, stat string, value int64, rate float32)
Measure(component string, stat string, value int64, rate float32)
Time(component string, stat string, value time.Duration, rate float32)
NewTimer(component string, stat string, rate float32) *Timer
}
type MultiStatter struct {
statters []Statter
}
func (s *MultiStatter) Inc(component string, stat string, value int64, rate float32) {
for _, st := range s.statters {
st.Inc(component, stat, value, rate)
}
}
func (s *MultiStatter) Gauge(component string, stat string, value int64, rate float32) {
for _, st := range s.statters {
st.Gauge(component, stat, value, rate)
}
}
func (s *MultiStatter) Measure(component string, stat string, value int64, rate float32) {
for _, st := range s.statters {
st.Measure(component, stat, value, rate)
}
}
func (s *MultiStatter) Time(component string, stat string, value time.Duration, rate float32) {
for _, st := range s.statters {
st.Time(component, stat, value, rate)
}
}
func (s *MultiStatter) NewTimer(component string, stat string, rate float32) *Timer {
return newTimer(s, component, stat, rate)
}
var badDecode error = errors.New("bad stats decode")
func New(config Config) Statter {
s := new(MultiStatter)
if config.Interval == 0.0 {
config.Interval = 10.0 // convenience
}
var reporters []reporter
if config.StatHat != nil && config.StatHat.Email != "" {
reporters = append(reporters, config.StatHat)
}
if config.NewRelic != nil && config.NewRelic.LicenseKey != "" {
// NR wants version?
// can get it out of the namespace? roll it here?
reporters = append(reporters, NewNewRelicReporter("1.0", config.NewRelic.LicenseKey))
}
if config.Log != "" {
reporters = append(reporters, NewLogReporter())
}
if len(reporters) > 0 {
ag := newAggregator(reporters)
s.statters = append(s.statters, ag)
go func() {
for range time.Tick(time.Duration(config.Interval * float64(time.Second))) {
ag.report(nil)
}
}()
}
if config.Statsd != nil && config.Statsd.StatsdUdpTarget != "" {
std, err := NewStatsd(config.Statsd)
if err == nil {
s.statters = append(s.statters, std)
} else {
logrus.WithError(err).Error("Couldn't create statsd reporter")
}
}
if len(reporters) == 0 && config.Statsd == nil && config.History == 0 {
return &NilStatter{}
}
if config.GCStats >= 0 {
if config.GCStats == 0 {
config.GCStats = 1
}
go StartReportingMemoryAndGC(s, time.Duration(config.GCStats)*time.Second)
}
return s
}
func HTTPReturnJson(w http.ResponseWriter, result interface{}) {
w.Header().Set("Content-Type", "application/json")
res, err := json.Marshal(result)
if err != nil {
http.Error(w, err.Error(), http.StatusInternalServerError)
} else {
w.Write(res)
}
}
// Convert a string to a stat name by replacing '.' with '_', lowercasing the
// string and trimming it. Doesn't do any validation, so do try this out
// locally before sending stats.
func AsStatField(input string) string {
return strings.Replace(strings.ToLower(strings.TrimSpace(input)), ".", "_", -1)
}
// statsd like API on top of the map manipulation API.
type Timer struct {
statter Statter
component string
stat string
start time.Time
rate float32
measured bool
}
func newTimer(st Statter, component, stat string, rate float32) *Timer {
return &Timer{st, component, stat, time.Now(), rate, false}
}
func (timer *Timer) Measure() {
if timer.measured {
return
}
timer.measured = true
timer.statter.Time(timer.component, timer.stat, time.Since(timer.start), timer.rate)
}
type NilStatter struct{}
func (n *NilStatter) Inc(component string, stat string, value int64, rate float32) {}
func (n *NilStatter) Gauge(component string, stat string, value int64, rate float32) {}
func (n *NilStatter) Measure(component string, stat string, value int64, rate float32) {}
func (n *NilStatter) Time(component string, stat string, value time.Duration, rate float32) {}
func (r *NilStatter) NewTimer(component string, stat string, rate float32) *Timer {
return newTimer(r, component, stat, rate)
}

View File

@@ -1,112 +0,0 @@
package stats
import (
"bytes"
"fmt"
"net"
"strings"
"time"
"github.com/cactus/go-statsd-client/statsd"
)
type StatsdConfig struct {
StatsdUdpTarget string `json:"target" mapstructure:"target"`
Interval int64 `json:"interval"`
Prefix string `json:"prefix"`
}
type keyCreator interface {
// The return value of Key *MUST* never have a '.' at the end.
Key(stat string) string
}
type theStatsdReporter struct {
keyCreator
client statsd.Statter
}
type prefixKeyCreator struct {
parent keyCreator
prefixes []string
}
func (pkc *prefixKeyCreator) Key(stat string) string {
prefix := strings.Join(pkc.prefixes, ".")
if pkc.parent != nil {
prefix = pkc.parent.Key(prefix)
}
if stat == "" {
return prefix
}
if prefix == "" {
return stat
}
return prefix + "." + stat
}
func whoami() string {
a, _ := net.InterfaceAddrs()
for i := range a {
// is a textual representation of an IPv4 address
z, _, err := net.ParseCIDR(a[i].String())
if a[i].Network() == "ip+net" && err == nil && z.To4() != nil {
if !bytes.Equal(z, net.ParseIP("127.0.0.1")) {
return strings.Replace(fmt.Sprintf("%v", z), ".", "_", -1)
}
}
}
return "127_0_0_1" // shrug
}
// The config.Prefix is sent before each message and can be used to set API
// keys. The prefix is used as the key prefix.
// If config is nil, creates a noop reporter.
//
// st, e := NewStatsd(config, "servicename")
// st.Inc("eventname", 1) -> Actually records to key servicename.eventname.
func NewStatsd(config *StatsdConfig) (*theStatsdReporter, error) {
var client statsd.Statter
var err error
if config != nil {
// 512 for now since we are sending to hostedgraphite over the internet.
config.Prefix += "." + whoami()
client, err = statsd.NewBufferedClient(config.StatsdUdpTarget, config.Prefix, time.Duration(config.Interval)*time.Second, 512)
} else {
client, err = statsd.NewNoopClient()
}
if err != nil {
return nil, err
}
return &theStatsdReporter{keyCreator: &prefixKeyCreator{}, client: client}, nil
}
func (sr *theStatsdReporter) Inc(component, stat string, value int64, rate float32) {
sr.client.Inc(sr.keyCreator.Key(component+"."+stat), value, rate)
}
func (sr *theStatsdReporter) Measure(component, stat string, delta int64, rate float32) {
sr.client.Timing(sr.keyCreator.Key(component+"."+stat), delta, rate)
}
func (sr *theStatsdReporter) Time(component, stat string, delta time.Duration, rate float32) {
sr.client.TimingDuration(sr.keyCreator.Key(component+"."+stat), delta, rate)
}
func (sr *theStatsdReporter) Gauge(component, stat string, value int64, rate float32) {
sr.client.Gauge(sr.keyCreator.Key(component+"."+stat), value, rate)
}
func (sr *theStatsdReporter) NewTimer(component string, stat string, rate float32) *Timer {
return newTimer(sr, component, stat, rate)
}
// We need some kind of all-or-nothing sampler where multiple stats can be
// given the same rate and they are either all logged on that run or none of
// them are. The statsd library we use ends up doing its own rate calculation
// which is going to impede doing something like this.

View File

@@ -1,190 +0,0 @@
package common
import (
"bytes"
"errors"
"io"
)
// lineWriter will break apart a stream of data into individual lines.
// Downstream writer will be called for each complete new line. When Flush
// is called, a newline will be appended if there isn't one at the end.
// Not thread-safe
type LineWriter struct {
b *bytes.Buffer
w io.Writer
}
func NewLineWriter(w io.Writer) *LineWriter {
return &LineWriter{
w: w,
b: bytes.NewBuffer(make([]byte, 0, 1024)),
}
}
func (li *LineWriter) Write(p []byte) (int, error) {
n, err := li.b.Write(p)
if err != nil {
return n, err
}
if n != len(p) {
return n, errors.New("short write")
}
for {
b := li.b.Bytes()
i := bytes.IndexByte(b, '\n')
if i < 0 {
break
}
l := b[:i+1]
ns, err := li.w.Write(l)
if err != nil {
return ns, err
}
li.b.Next(len(l))
}
return n, nil
}
func (li *LineWriter) Flush() (int, error) {
b := li.b.Bytes()
if len(b) == 0 {
return 0, nil
}
if b[len(b)-1] != '\n' {
b = append(b, '\n')
}
return li.w.Write(b)
}
// HeadLinesWriter stores upto the first N lines in a buffer that can be
// retrieved via Head().
type HeadLinesWriter struct {
buffer bytes.Buffer
max int
}
func NewHeadLinesWriter(max int) *HeadLinesWriter {
return &HeadLinesWriter{
buffer: bytes.Buffer{},
max: max,
}
}
// Writes start failing once the writer has reached capacity.
// In such cases the return value is the actual count written (may be zero) and io.ErrShortWrite.
func (h *HeadLinesWriter) Write(p []byte) (n int, err error) {
var afterNewLine int
for h.max > 0 && afterNewLine < len(p) {
idx := bytes.IndexByte(p[afterNewLine:], '\n')
if idx == -1 {
h.buffer.Write(p[afterNewLine:])
afterNewLine = len(p)
} else {
h.buffer.Write(p[afterNewLine : afterNewLine+idx+1])
afterNewLine = afterNewLine + idx + 1
h.max--
}
}
if afterNewLine == len(p) {
return afterNewLine, nil
}
return afterNewLine, io.ErrShortWrite
}
// The returned bytes alias the buffer, the same restrictions as
// bytes.Buffer.Bytes() apply.
func (h *HeadLinesWriter) Head() []byte {
return h.buffer.Bytes()
}
// TailLinesWriter stores upto the last N lines in a buffer that can be retrieved
// via Tail(). The truncation is only performed when more bytes are received
// after '\n', so the buffer contents for both these writes are identical.
//
// tail writer that captures last 3 lines.
// 'a\nb\nc\nd\n' -> 'b\nc\nd\n'
// 'a\nb\nc\nd' -> 'b\nc\nd'
type TailLinesWriter struct {
buffer bytes.Buffer
max int
newlineEncountered bool
// Tail is not idempotent without this.
tailCalled bool
}
func NewTailLinesWriter(max int) *TailLinesWriter {
return &TailLinesWriter{
buffer: bytes.Buffer{},
max: max,
}
}
// Write always succeeds! This is because all len(p) bytes are written to the
// buffer before it is truncated.
func (t *TailLinesWriter) Write(p []byte) (n int, err error) {
if t.tailCalled {
return 0, errors.New("Tail() has already been called.")
}
var afterNewLine int
for afterNewLine < len(p) {
// This is at the top of the loop so it does not operate on trailing
// newlines. That is handled by Tail() where we have full knowledge that it
// is indeed the true trailing newline (if any).
if t.newlineEncountered {
if t.max > 0 {
// we still have capacity
t.max--
} else {
// chomp a newline.
t.chompNewline()
}
}
idx := bytes.IndexByte(p[afterNewLine:], '\n')
if idx == -1 {
t.buffer.Write(p[afterNewLine:])
afterNewLine = len(p)
t.newlineEncountered = false
} else {
t.buffer.Write(p[afterNewLine : afterNewLine+idx+1])
afterNewLine = afterNewLine + idx + 1
t.newlineEncountered = true
}
}
return len(p), nil
}
func (t *TailLinesWriter) chompNewline() {
b := t.buffer.Bytes()
idx := bytes.IndexByte(b, '\n')
if idx >= 0 {
t.buffer.Next(idx + 1)
} else {
// pretend a trailing newline exists. In the call in Write() this will
// never be hit.
t.buffer.Truncate(0)
}
}
// The returned bytes alias the buffer, the same restrictions as
// bytes.Buffer.Bytes() apply.
//
// Once Tail() is called, further Write()s error.
func (t *TailLinesWriter) Tail() []byte {
if !t.tailCalled {
t.tailCalled = true
if t.max <= 0 {
t.chompNewline()
}
}
return t.buffer.Bytes()
}

View File

@@ -1,135 +0,0 @@
package common
import (
"bytes"
"fmt"
"io"
"testing"
)
type testSliceWriter struct {
b [][]byte
}
func (tsw *testSliceWriter) Write(p []byte) (n int, err error) {
l := make([]byte, len(p))
copy(l, p)
tsw.b = append(tsw.b, l)
return len(p), nil
}
func TestLineWriter(t *testing.T) {
tsw := &testSliceWriter{}
lw := NewLineWriter(tsw)
lineCount := 7
lw.Write([]byte("0 line\n1 line\n2 line\n\n4 line"))
lw.Write([]byte("+more\n5 line\n"))
lw.Write([]byte("6 line"))
lw.Flush()
if len(tsw.b) != lineCount {
t.Errorf("Expected %v individual rows; got %v", lineCount, len(tsw.b))
}
for x := 0; x < len(tsw.b); x++ {
l := fmt.Sprintf("%v line\n", x)
if x == 3 {
if len(tsw.b[x]) != 1 {
t.Errorf("Expected slice with only newline; got %v", tsw.b[x])
}
continue
} else if x == 4 {
l = "4 line+more\n"
}
if !bytes.Equal(tsw.b[x], []byte(l)) {
t.Errorf("Expected slice %s equal to %s", []byte(l), tsw.b[x])
}
}
}
func TestHeadWriter(t *testing.T) {
data := []byte("the quick\n brown\n fox jumped\n over the\n lazy dog.")
w := NewHeadLinesWriter(3)
_, err := w.Write(data[:4])
if err != nil {
t.Errorf("Expected nil error on small write")
}
if !bytes.Equal(w.Head(), []byte("the ")) {
t.Errorf("Expected 4 bytes in head, got '%s'", w.Head())
}
n, err := w.Write(data[4:16])
if n != len(data[4:16]) || err != nil {
t.Errorf("HeadWriter Write() does not satisfy contract about failing writes.")
}
if !bytes.Equal(w.Head(), []byte("the quick\n brown")) {
t.Errorf("unexpected contents of head, got '%s'", w.Head())
}
n, err = w.Write(data[16:])
if n != (29-16) || err != io.ErrShortWrite {
t.Errorf("HeadWriter Write() does not satisfy contract about failing writes.")
}
if !bytes.Equal(w.Head(), data[:29]) {
t.Errorf("unexpected contents of head, got '%s'", w.Head())
}
}
func testTail(t *testing.T, n int, output []byte, writes ...[]byte) {
w := NewTailLinesWriter(n)
for _, slice := range writes {
written, err := w.Write(slice)
if written != len(slice) || err != nil {
t.Errorf("Tail Write() should always succeed, but failed, input=%s, input length = %d, written=%d, err=%s", slice, len(slice), written, err)
}
}
if !bytes.Equal(w.Tail(), output) {
t.Errorf("Output did not match for tail writer of length %d: Expected '%s', got '%s'", n, output, w.Tail())
}
}
func TestTailWriter(t *testing.T) {
inputs := [][]byte{[]byte("a\nb\n"), []byte("gh"), []byte("\n")}
testTail(t, 2, []byte("b\ngh\n"), inputs...)
}
func TestZeroAndOneTailWriter(t *testing.T) {
// zero line writer, with only single line added to it should return empty buffer.
testTail(t, 0, []byte(""), []byte("Hello World\n"))
testTail(t, 0, []byte(""), []byte("Hello World"))
b1 := []byte("Hello World")
testTail(t, 1, b1, b1)
b1 = []byte("Hello World\n")
testTail(t, 1, b1, b1)
b2 := []byte("Yeah!\n")
testTail(t, 1, b2, b1, b2)
b1 = []byte("Flat write")
b2 = []byte("Yeah!\n")
j := bytes.Join([][]byte{b1, b2}, []byte{})
testTail(t, 1, j, b1, b2)
}
func TestTailWriterTrailing(t *testing.T) {
input1 := []byte("a\nb\nc\nd\ne")
input2 := []byte("a\nb\nc\nd\ne\n")
w1 := NewTailLinesWriter(4)
w1.Write(input1)
w2 := NewTailLinesWriter(4)
w2.Write(input2)
if !bytes.Equal(w1.Tail(), []byte("b\nc\nd\ne")) {
t.Errorf("Tail not working correctly, got '%s'", w1.Tail())
}
t2 := w2.Tail()
if !bytes.Equal(w1.Tail(), t2[:len(t2)-1]) {
t.Errorf("Tailwriter does not transition correctly over trailing newline. '%s', '%s'", w1.Tail(), t2)
}
}

View File

@@ -1,22 +0,0 @@
package runner
import (
"strings"
)
type dockerRegistry struct {
Name string `json:"name"`
Username string `json:"username"`
Password string `json:"password"`
}
type dockerRegistries []dockerRegistry
func (t dockerRegistries) Find(name string) *dockerRegistry {
for _, v := range t {
if strings.HasSuffix(v.Name, name) {
return &v
}
}
return nil
}

View File

@@ -1,76 +0,0 @@
package runner
import (
"bytes"
"io"
"testing"
)
type nopCloser struct {
io.Writer
}
func (n nopCloser) Close() error { return nil }
func TestLimitWriter(t *testing.T) {
var b bytes.Buffer
const max = 5
lw := newLimitWriter(max, nopCloser{&b})
lw.Write([]byte("yo"))
if b.Len() != 2 {
t.Fatal("expected 2 bytes in buffer, got:", b.Len())
}
n, _ := lw.Write([]byte("dawg"))
// can't check b.Len() really since the overage message is written in
if n != 3 {
t.Fatalf("limit writer allowed writing over the limit or n was wrong. n: %d", n)
}
n, err := lw.Write([]byte("yodawg"))
if n != 0 || err == nil {
t.Fatalf("limit writer wrote after limit exceeded, n > 0 or err is nil. n: %d err: %v", n, err)
}
// yes should const this. yes i'm wrong. yes you're wrong. no it doesn't matter.
if !bytes.HasPrefix(b.Bytes(), []byte("yodaw\n-----max")) {
t.Fatal("expected buffer to be 'yodawg', got:", b.String())
}
}
func TestLineWriter(t *testing.T) {
var b bytes.Buffer
lw := newLineWriter(&b)
lw.Write([]byte("yo"))
if b.Len() != 0 {
t.Fatal("expected no bytes to be written, got bytes")
}
lw.Write([]byte("\ndawg"))
if b.Len() != 3 {
t.Fatal("expected 3 bytes to be written in, got:", b.Len())
}
lw.Write([]byte("\ndawgy\ndawg"))
if b.Len() != 14 {
t.Fatal("expected 14 bytes to be written in, got:", b.Len())
}
lw.Close()
if b.Len() != 19 {
t.Fatal("expected 19 bytes to be written in, got:", b.Len())
}
if !bytes.HasSuffix(b.Bytes(), []byte("\n")) {
t.Fatal("line writer close is broked, expected new line")
}
}

View File

@@ -1,19 +0,0 @@
package protocol
import (
"context"
"github.com/fnproject/fn/api/runner/task"
)
// DefaultProtocol is the protocol used by cold-containers
type DefaultProtocol struct {
}
func (p *DefaultProtocol) IsStreamable() bool {
return false
}
func (p *DefaultProtocol) Dispatch(context.Context, *task.Config) error {
return nil
}

View File

@@ -1,74 +0,0 @@
package protocol
import (
"bufio"
"bytes"
"context"
"fmt"
"io"
"net/http"
"net/http/httputil"
"time"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/task"
)
// HTTPProtocol converts stdin/stdout streams into HTTP/1.1 compliant
// communication. It relies on Content-Length to know when to stop reading from
// containers stdout. It also mandates valid HTTP headers back and forth, thus
// returning errors in case of parsing problems.
type HTTPProtocol struct {
in io.Writer
out io.Reader
}
func (p *HTTPProtocol) IsStreamable() bool {
return true
}
func (p *HTTPProtocol) Dispatch(ctx context.Context, cfg *task.Config) error {
var retErr error
done := make(chan struct{})
go func() {
// TODO not okay. plumb content-length from req into cfg..
var body bytes.Buffer
io.Copy(&body, cfg.Stdin)
req, err := http.NewRequest("GET", "/", &body)
if err != nil {
retErr = err
return
}
for k, v := range cfg.Env {
req.Header.Set(k, v)
}
req.Header.Set("Content-Length", fmt.Sprint(body.Len()))
req.Header.Set("Task-ID", cfg.ID)
raw, err := httputil.DumpRequest(req, true)
if err != nil {
retErr = err
return
}
p.in.Write(raw)
res, err := http.ReadResponse(bufio.NewReader(p.out), req)
if err != nil {
retErr = err
return
}
io.Copy(cfg.Stdout, res.Body)
done <- struct{}{}
}()
timeout := time.After(cfg.Timeout)
select {
case <-ctx.Done():
return ctx.Err()
case <-timeout:
return models.ErrRunnerTimeout
case <-done:
return retErr
}
}

View File

@@ -1,338 +0,0 @@
package runner
import (
"bufio"
"context"
"errors"
"fmt"
"io/ioutil"
"os"
"runtime"
"strconv"
"strings"
"sync"
"time"
"github.com/sirupsen/logrus"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/common"
"github.com/fnproject/fn/api/runner/drivers"
"github.com/fnproject/fn/api/runner/drivers/docker"
"github.com/fnproject/fn/api/runner/drivers/mock"
"github.com/fnproject/fn/api/runner/task"
"github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/log"
)
// TODO clean all of this up, the exposed API is huge and incohesive,
// we need 1 thing that runs 1 thing and 1 thing that runs those things;
// right now this is all the things.
type Runner struct {
driver drivers.Driver
taskQueue chan *containerTask
flog FuncLogger
availableMem int64
usedMem int64
usedMemMutex sync.RWMutex
hcmgr htfnmgr
datastore models.Datastore
stats
}
var (
ErrTimeOutNoMemory = errors.New("Task timed out. No available memory.")
ErrFullQueue = errors.New("The runner queue is full")
WaitMemoryTimeout = 10 * time.Second
)
const (
DefaultTimeout = 30 * time.Second
DefaultIdleTimeout = 30 * time.Second
)
func New(ctx context.Context, flog FuncLogger, ds models.Datastore) (*Runner, error) {
// TODO: Is this really required for the container drivers? Can we remove it?
env := common.NewEnvironment(func(e *common.Environment) {})
// TODO: Create drivers.New(runnerConfig)
driver, err := selectDriver("docker", env, &drivers.Config{})
if err != nil {
return nil, err
}
r := &Runner{
driver: driver,
taskQueue: make(chan *containerTask, 100),
flog: flog,
availableMem: getAvailableMemory(),
usedMem: 0,
datastore: ds,
}
go r.queueHandler(ctx)
return r, nil
}
// This routine checks for available memory;
// If there's memory then send signal to the task to proceed.
// If there's not available memory to run the task it waits
// If the task waits for more than X seconds it timeouts
func (r *Runner) queueHandler(ctx context.Context) {
consumeQueue:
for {
select {
case task := <-r.taskQueue:
r.handleTask(task)
case <-ctx.Done():
break consumeQueue
}
}
// consume remainders
for len(r.taskQueue) > 0 {
r.handleTask(<-r.taskQueue)
}
}
func (r *Runner) handleTask(task *containerTask) {
waitStart := time.Now()
var waitTime time.Duration
var timedOut bool
// Loop waiting for available memory
for !r.checkRequiredMem(task.cfg.Memory) {
waitTime = time.Since(waitStart)
if waitTime > WaitMemoryTimeout {
timedOut = true
break
}
time.Sleep(time.Microsecond)
}
if timedOut {
// Send to a signal to this task saying it cannot run
task.canRun <- false
return
}
// Send a signal to this task saying it can run
task.canRun <- true
}
func (r *Runner) hasAsyncAvailableMemory() bool {
r.usedMemMutex.RLock()
defer r.usedMemMutex.RUnlock()
// reserve at least half of the memory for sync
return (r.availableMem/2)-r.usedMem > 0
}
func (r *Runner) checkRequiredMem(req uint64) bool {
r.usedMemMutex.RLock()
defer r.usedMemMutex.RUnlock()
return r.availableMem-r.usedMem-(int64(req)*1024*1024) > 0
}
func (r *Runner) addUsedMem(used int64) {
r.usedMemMutex.Lock()
r.usedMem = r.usedMem + used*1024*1024
if r.usedMem < 0 {
r.usedMem = 0
}
r.usedMemMutex.Unlock()
}
func (r *Runner) checkMemAndUse(req uint64) bool {
r.usedMemMutex.Lock()
defer r.usedMemMutex.Unlock()
used := int64(req) * 1024 * 1024
if r.availableMem-r.usedMem-used < 0 {
return false
}
r.usedMem += used
return true
}
func (r *Runner) awaitSlot(ctask *containerTask) error {
span, _ := opentracing.StartSpanFromContext(ctask.ctx, "wait_mem_slot")
defer span.Finish()
// Check if has enough available memory
// If available, use it
if !r.checkMemAndUse(ctask.cfg.Memory) {
// If not, try add task to the queue
select {
case r.taskQueue <- ctask:
default:
span.LogFields(log.Int("queue full", 1))
// If queue is full, return error
return ErrFullQueue
}
// If task was added to the queue, wait for permission
if ok := <-ctask.canRun; !ok {
span.LogFields(log.Int("memory timeout", 1))
// This task timed out, not available memory
return ErrTimeOutNoMemory
}
}
return nil
}
// run is responsible for running 1 instance of a docker container
func (r *Runner) run(ctx context.Context, cfg *task.Config) (drivers.RunResult, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "run_container")
defer span.Finish()
if cfg.Stdout == nil {
// TODO why? async?
cfg.Stdout = cfg.Stderr
}
ctask := &containerTask{
ctx: ctx,
cfg: cfg,
canRun: make(chan bool),
}
err := r.awaitSlot(ctask)
if err != nil {
return nil, err
}
defer r.addUsedMem(-1 * int64(cfg.Memory))
span, pctx := opentracing.StartSpanFromContext(ctx, "prepare")
cookie, err := r.driver.Prepare(pctx, ctask)
span.Finish()
if err != nil {
return nil, err
}
defer cookie.Close(ctx)
select {
case <-cfg.Ready:
default:
close(cfg.Ready)
}
span, rctx := opentracing.StartSpanFromContext(ctx, "run")
result, err := cookie.Run(rctx)
span.Finish()
if err != nil {
return nil, err
}
span.LogFields(log.String("status", result.Status()))
return result, nil
}
func (r Runner) EnsureImageExists(ctx context.Context, cfg *task.Config) error {
ctask := &containerTask{
cfg: cfg,
}
auth, err := ctask.DockerAuth()
if err != nil {
return err
}
_, err = docker.CheckRegistry(ctx, ctask.Image(), auth)
return err
}
func selectDriver(driver string, env *common.Environment, conf *drivers.Config) (drivers.Driver, error) {
switch driver {
case "docker":
docker := docker.NewDocker(env, *conf)
return docker, nil
case "mock":
return mock.New(), nil
}
return nil, fmt.Errorf("driver %v not found", driver)
}
func getAvailableMemory() int64 {
const tooBig = 322122547200 // #300GB or 0, biggest aws instance is 244GB
var availableMemory uint64 = tooBig
if runtime.GOOS == "linux" {
availableMemory, err := checkCgroup()
if err != nil {
logrus.WithError(err).Error("Error checking for cgroup memory limits, falling back to host memory available..")
}
if availableMemory > tooBig || availableMemory == 0 {
// Then -m flag probably wasn't set, so use max available on system
availableMemory, err = checkProc()
if err != errCantReadMemInfo &&
(availableMemory > tooBig || availableMemory == 0) {
logrus.WithError(err).Fatal("Cannot get the proper information to. You must specify the maximum available memory by passing the -m command with docker run when starting the runner via docker, eg: `docker run -m 2G ...`")
}
}
} else {
// This still lets 10-20 functions execute concurrently assuming a 2GB machine.
availableMemory = 2 * 1024 * 1024 * 1024
}
return int64(availableMemory)
}
func checkCgroup() (uint64, error) {
f, err := os.Open("/sys/fs/cgroup/memory/memory.limit_in_bytes")
if err != nil {
return 0, err
}
defer f.Close()
b, err := ioutil.ReadAll(f)
limBytes := string(b)
limBytes = strings.TrimSpace(limBytes)
if err != nil {
return 0, err
}
return strconv.ParseUint(limBytes, 10, 64)
}
var errCantReadMemInfo = errors.New("Didn't find MemAvailable in /proc/meminfo, kernel is probably < 3.14")
func checkProc() (uint64, error) {
f, err := os.Open("/proc/meminfo")
if err != nil {
return 0, err
}
defer f.Close()
scanner := bufio.NewScanner(f)
for scanner.Scan() {
b := scanner.Text()
if !strings.HasPrefix(b, "MemAvailable") {
continue
}
// expect form:
// MemAvailable: 1234567890 kB
tri := strings.Fields(b)
if len(tri) != 3 {
return 0, fmt.Errorf("MemAvailable line has unexpected format: %v", b)
}
c, err := strconv.ParseUint(tri[1], 10, 64)
if err != nil {
return 0, fmt.Errorf("Could not parse MemAvailable: %v", b)
}
switch tri[2] { // convert units to bytes
case "kB":
c *= 1024
case "MB":
c *= 1024 * 1024
default:
return 0, fmt.Errorf("Unexpected units for MemAvailable in /proc/meminfo, need kB or MB, got: %v", tri[2])
}
return c, nil
}
return 0, errCantReadMemInfo
}

View File

@@ -1,151 +0,0 @@
package runner
import (
"bytes"
"context"
"fmt"
"strings"
"testing"
"time"
"github.com/fnproject/fn/api/datastore"
"github.com/fnproject/fn/api/id"
"github.com/fnproject/fn/api/logs"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/task"
)
func TestRunnerHello(t *testing.T) {
buf := setLogBuffer()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ds := datastore.NewMock()
fnl := logs.NewMock()
fLogger := NewFuncLogger(fnl)
runner, err := New(ctx, fLogger, ds)
if err != nil {
t.Fatalf("Test error during New() - %s", err)
}
for i, test := range []struct {
route *models.Route
payload string
expectedStatus string
expectedOut string
expectedErr string
taskID string
}{
{&models.Route{Image: "fnproject/hello"}, ``, "success", "Hello World!", "", id.New().String()},
{&models.Route{Image: "fnproject/hello"}, `{"name": "test"}`, "success", "Hello test!", "", id.New().String()},
} {
var stdout, stderr bytes.Buffer
cfg := &task.Config{
ID: test.taskID,
Image: test.route.Image,
Timeout: 10 * time.Second,
Memory: 128,
Ready: make(chan struct{}),
Stdin: strings.NewReader(test.payload),
AppName: test.route.AppName,
Stdout: &stdout,
Stderr: nopCloser{&stderr},
}
result, err := runner.run(ctx, cfg)
if err != nil {
t.Log(buf.String())
t.Fatalf("Test %d: error during Run() - %s", i, err)
}
if test.expectedStatus != result.Status() {
t.Log(buf.String())
t.Fatalf("Test %d: expected result status to be `%s` but it was `%s`", i, test.expectedStatus, result.Status())
}
if !bytes.Contains(stdout.Bytes(), []byte(test.expectedOut)) {
t.Log(buf.String())
t.Fatalf("Test %d: expected output log to contain `%s` in `%s`", i, test.expectedOut, stdout.String())
}
if !bytes.Contains(stderr.Bytes(), []byte(test.expectedErr)) {
t.Log(buf.String())
t.Fatalf("Test %d: expected error log to contain `%s` in `%s`", i, test.expectedErr, stderr.String())
}
}
}
func TestRunnerError(t *testing.T) {
buf := setLogBuffer()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ds := datastore.NewMock()
fnl := logs.NewMock()
fLogger := NewFuncLogger(fnl)
runner, err := New(ctx, fLogger, ds)
if err != nil {
t.Fatalf("Test error during New() - %s", err)
}
for i, test := range []struct {
route *models.Route
payload string
expectedStatus string
expectedOut string
expectedErr string
taskID string
}{
{&models.Route{Image: "fnproject/error"}, ``, "error", "", "", id.New().String()},
{&models.Route{Image: "fnproject/error"}, `{"name": "test"}`, "error", "", "", id.New().String()},
} {
var stdout, stderr bytes.Buffer
cfg := &task.Config{
ID: fmt.Sprintf("err-%d-%d", i, time.Now().Unix()),
Image: test.route.Image,
Timeout: 10 * time.Second,
Memory: 128,
Ready: make(chan struct{}),
Stdin: strings.NewReader(test.payload),
Stdout: &stdout,
Stderr: nopCloser{&stderr},
}
result, err := runner.run(ctx, cfg)
if err != nil {
t.Log(buf.String())
t.Fatalf("Test %d: error during Run() - %s", i, err)
}
if test.expectedStatus != result.Status() {
t.Log(buf.String())
t.Fatalf("Test %d: expected result status to be `%s` but it was `%s`", i, test.expectedStatus, result.Status())
}
if !bytes.Contains(stdout.Bytes(), []byte(test.expectedOut)) {
t.Log(buf.String())
t.Fatalf("Test %d: expected output log to contain `%s` in `%s`", i, test.expectedOut, stdout.String())
}
if !bytes.Contains(stderr.Bytes(), []byte(test.expectedErr)) {
t.Log(buf.String())
t.Fatalf("Test %d: expected error log to contain `%s` in `%s`", i, test.expectedErr, stderr.String())
}
}
}
func TestRunnerMemory(t *testing.T) {
// make sure we get MB out of a task.Config when turned into a containerTask
// (so if Config.Memory changes to not be MB we hear about it)
cfg := &task.Config{
Memory: 128,
}
task := &containerTask{cfg: cfg}
const exp = 128 * 1024 * 1024
if task.Memory() != exp {
t.Fatalf("Expected Memory to return %v but got %v", exp, task.Memory())
}
}

View File

@@ -1,105 +0,0 @@
package runner
import (
"context"
"encoding/json"
"io"
"os"
"os/user"
"path/filepath"
"time"
"github.com/docker/cli/cli/config/configfile"
"github.com/fnproject/fn/api/runner/drivers"
"github.com/fnproject/fn/api/runner/protocol"
"github.com/fnproject/fn/api/runner/task"
docker "github.com/fsouza/go-dockerclient"
)
var registries dockerRegistries
func init() {
// TODO this is docker specific. and the docker client is capable of doing this, remove & test
// Attempt to fetch it from an environment variable
regsettings := os.Getenv("DOCKER_AUTH")
if regsettings == "" {
u, err := user.Current()
if err == nil {
var config configfile.ConfigFile
cfile, err := os.Open(filepath.Join(u.HomeDir, ".docker", "config.json"))
if err != nil {
return
}
err = config.LoadFromReader(cfile)
if err != nil {
return
}
var regs []dockerRegistry
for _, auth := range config.AuthConfigs {
regs = append(regs, dockerRegistry{
Username: auth.Username,
Password: auth.Password,
Name: auth.ServerAddress,
})
}
registries = dockerRegistries(regs)
}
} else {
// If we have settings, unmarshal them
json.Unmarshal([]byte(regsettings), &registries)
}
}
// TODO task.Config should implement the interface. this is sad :(
// implements drivers.ContainerTask
type containerTask struct {
ctx context.Context
cfg *task.Config
canRun chan bool
}
func (t *containerTask) EnvVars() map[string]string {
if protocol.IsStreamable(protocol.Protocol(t.cfg.Format)) {
return t.cfg.BaseEnv
}
return t.cfg.Env
}
func (t *containerTask) Labels() map[string]string {
// TODO this seems inaccurate? is this used by anyone (dev or not)?
return map[string]string{"LogName": t.cfg.AppName}
}
func (t *containerTask) Command() string { return "" }
func (t *containerTask) Input() io.Reader { return t.cfg.Stdin }
func (t *containerTask) Id() string { return t.cfg.ID }
func (t *containerTask) Image() string { return t.cfg.Image }
func (t *containerTask) Timeout() time.Duration { return t.cfg.Timeout }
func (t *containerTask) Logger() (io.Writer, io.Writer) { return t.cfg.Stdout, t.cfg.Stderr }
func (t *containerTask) Volumes() [][2]string { return [][2]string{} }
func (t *containerTask) Memory() uint64 { return t.cfg.Memory * 1024 * 1024 } // convert MB
func (t *containerTask) WorkDir() string { return "" }
func (t *containerTask) Close() {}
func (t *containerTask) WriteStat(drivers.Stat) {}
// Implementing the docker.AuthConfiguration interface. Pulling in
// the docker repo password from environment variables
func (t *containerTask) DockerAuth() (docker.AuthConfiguration, error) {
reg, _, _ := drivers.ParseImage(t.Image())
authconfig := docker.AuthConfiguration{}
if customAuth := registries.Find(reg); customAuth != nil {
authconfig = docker.AuthConfiguration{
Password: customAuth.Password,
ServerAddress: customAuth.Name,
Username: customAuth.Username,
}
}
return authconfig, nil
}

View File

@@ -1,85 +0,0 @@
package task
import (
"context"
"io"
"strings"
"time"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/drivers"
"github.com/go-openapi/strfmt"
)
// TODO this whole package should be hanged, drawn & quartered
type Config struct {
ID string
AppName string
Path string
Image string
Timeout time.Duration
IdleTimeout time.Duration
Memory uint64
BaseEnv map[string]string // only app & route config vals [for hot]
Env map[string]string // includes BaseEnv
Format string
ReceivedTime time.Time
// Ready is used to await the first pull
Ready chan struct{}
Stdin io.Reader
Stdout io.Writer
Stderr io.WriteCloser // closer for flushy poo
}
// TODO Task & Config should be merged
func TaskFromConfig(cfg *Config) *models.Task {
return &models.Task{
IDStatus: models.IDStatus{ID: cfg.ID},
AppName: cfg.AppName,
Path: cfg.Path,
Image: cfg.Image,
Timeout: int32(cfg.Timeout.Seconds()),
IdleTimeout: int32(cfg.IdleTimeout.Seconds()),
Memory: cfg.Memory,
BaseEnv: cfg.BaseEnv,
EnvVars: cfg.Env,
// Format: cfg.Format, TODO plumb this
CreatedAt: strfmt.DateTime(time.Now()),
Delay: 0, // TODO not wired to users
// Payload: stdin
Priority: new(int32), // 0, TODO not wired atm to users.
}
}
func ConfigFromTask(t *models.Task) *Config {
return &Config{
ID: t.ID,
AppName: t.AppName,
Path: t.Path,
Image: t.Image,
Timeout: time.Duration(t.Timeout) * time.Second,
IdleTimeout: time.Duration(t.IdleTimeout) * time.Second,
Memory: t.Memory,
BaseEnv: t.BaseEnv,
Env: t.EnvVars,
Stdin: strings.NewReader(t.Payload),
Ready: make(chan struct{}),
}
}
// Request stores the task to be executed, It holds in itself the channel to
// return its response to its caller.
type Request struct {
Ctx context.Context
Config *Config
Response chan Response
}
// Response holds the response metainformation of a Request
type Response struct {
Result drivers.RunResult
Err error
}

View File

@@ -1,425 +0,0 @@
package runner
import (
"context"
"crypto/sha1"
"errors"
"fmt"
"io"
"sort"
"sync"
"time"
"github.com/fnproject/fn/api/id"
"github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/drivers"
"github.com/fnproject/fn/api/runner/protocol"
"github.com/fnproject/fn/api/runner/task"
"github.com/go-openapi/strfmt"
"github.com/opentracing/opentracing-go"
"github.com/sirupsen/logrus"
)
// hot functions - theory of operation
//
// A function is converted into a hot function if its `Format` is either
// a streamable format/protocol. At the very first task request a hot
// container shall be started and run it. Each hot function has an internal
// clock that actually halts the container if it goes idle long enough. In the
// absence of workload, it just stops the whole clockwork.
//
// Internally, the hot function uses a modified Config whose Stdin and Stdout
// are bound to an internal pipe. This internal pipe is fed with incoming tasks
// Stdin and feeds incoming tasks with Stdout.
//
// Each execution is the alternation of feeding hot functions stdin with tasks
// stdin, and reading the answer back from containers stdout. For all `Format`s
// we send embedded into the message metadata to help the container to know when
// to stop reading from its stdin and Functions expect the container to do the
// same. Refer to api/runner/protocol.go for details of these communications.
//
// hot functions implementation relies in two moving parts (drawn below):
// htfnmgr and htfn. Refer to their respective comments for
// details.
// │
// Incoming
// Task
// │
// ┌──────▼────────┐
// ┌┴──────────────┐│
// │ Per Function ││ non-streamable f()
// ┌───────│ Container │├──────┐───────────────┐
// │ │ Manager ├┘ │ │
// │ └───────────────┘ │ │
// │ │ │ │
// ▼ ▼ ▼ ▼
// ┌───────────┐ ┌───────────┐ ┌───────────┐ ┌───────────┐
// │ Hot │ │ Hot │ │ Hot │ │ Cold │
// │ Function │ │ Function │ │ Function │ │ Function │
// └───────────┘ └───────────┘ └───────────┘ └───────────┘
// Timeout
// Terminate
// (internal clock)
// RunTrackedTask is just a wrapper for shared logic for async/sync runners
func (rnr *Runner) RunTrackedTask(newTask *models.Task, ctx context.Context, cfg *task.Config) (drivers.RunResult, error) {
startedAt := strfmt.DateTime(time.Now())
newTask.StartedAt = startedAt
result, err := rnr.RunTask(ctx, cfg)
completedAt := strfmt.DateTime(time.Now())
status := "error"
if result != nil {
status = result.Status()
}
newTask.CompletedAt = completedAt
newTask.Status = status
if err := rnr.datastore.InsertTask(ctx, newTask); err != nil {
// TODO we should just log this error not return it to user? just issue storing task status but task is run
logrus.WithError(err).Error("error inserting task into datastore")
}
return result, err
}
// RunTask will dispatch a task specified by cfg to a hot container, if possible,
// that already exists or will create a new container to run a task and then run it.
// TODO XXX (reed): merge this and RunTrackedTask to reduce surface area...
func (rnr *Runner) RunTask(ctx context.Context, cfg *task.Config) (drivers.RunResult, error) {
rnr.Start() // TODO layering issue ???
defer rnr.Complete()
tresp := make(chan task.Response)
treq := task.Request{Ctx: ctx, Config: cfg, Response: tresp}
tasks := rnr.hcmgr.getPipe(ctx, rnr, cfg)
if tasks == nil {
// TODO get rid of this to use herd stuff
go runTaskReq(rnr, treq)
} else {
tasks <- treq
}
resp := <-treq.Response
if resp.Result == nil && resp.Err == nil {
resp.Err = errors.New("error running task with unknown error")
}
return resp.Result, resp.Err
}
// htfnmgr tracks all hot functions, used to funnel kittens into existing tubes
// XXX (reed): this map grows unbounded, need to add LRU but need to make
// sure that no functions are running when we evict
type htfnmgr struct {
sync.RWMutex
hc map[string]*htfnsvr
}
func (h *htfnmgr) getPipe(ctx context.Context, rnr *Runner, cfg *task.Config) chan<- task.Request {
isStream := protocol.IsStreamable(protocol.Protocol(cfg.Format))
if !isStream {
// TODO stop doing this, to prevent herds
return nil
}
h.RLock()
if h.hc == nil {
h.RUnlock()
h.Lock()
if h.hc == nil {
h.hc = make(map[string]*htfnsvr)
}
h.Unlock()
h.RLock()
}
fn := key(cfg)
svr, ok := h.hc[fn]
h.RUnlock()
if !ok {
h.Lock()
svr, ok = h.hc[fn]
if !ok {
svr = newhtfnsvr(ctx, cfg, rnr)
h.hc[fn] = svr
}
h.Unlock()
}
return svr.tasksin
}
func key(cfg *task.Config) string {
// return a sha1 hash of a (hopefully) unique string of all the config
// values, to make map lookups quicker [than the giant unique string]
hash := sha1.New()
fmt.Fprint(hash, cfg.AppName, "\x00")
fmt.Fprint(hash, cfg.Path, "\x00")
fmt.Fprint(hash, cfg.Image, "\x00")
fmt.Fprint(hash, cfg.Timeout, "\x00")
fmt.Fprint(hash, cfg.IdleTimeout, "\x00")
fmt.Fprint(hash, cfg.Memory, "\x00")
fmt.Fprint(hash, cfg.Format, "\x00")
// we have to sort these before printing, yay. TODO do better
keys := make([]string, 0, len(cfg.BaseEnv))
for k := range cfg.BaseEnv {
keys = append(keys, k)
}
sort.Strings(keys)
for _, k := range keys {
fmt.Fprint(hash, k, "\x00", cfg.BaseEnv[k], "\x00")
}
var buf [sha1.Size]byte
return string(hash.Sum(buf[:0]))
}
// htfnsvr is part of htfnmgr, abstracted apart for simplicity, its only
// purpose is to test for hot functions saturation and try starting as many as
// needed. In case of absence of workload, it will stop trying to start new hot
// containers.
type htfnsvr struct {
cfg *task.Config
rnr *Runner
// TODO sharing with only a channel among hot containers will result in
// inefficient recycling of containers, we need a stack not a queue, so that
// when a lot of hot containers are up and throughput drops they don't all
// find a task every few seconds and stay up for a lot longer than we really
// need them.
tasksin chan task.Request
tasksout chan task.Request
first chan struct{}
once sync.Once // TODO this really needs to happen any time runner count goes to 0
}
func newhtfnsvr(ctx context.Context, cfg *task.Config, rnr *Runner) *htfnsvr {
svr := &htfnsvr{
cfg: cfg,
rnr: rnr,
tasksin: make(chan task.Request),
tasksout: make(chan task.Request, 1),
first: make(chan struct{}, 1),
}
svr.first <- struct{}{} // prime so that 1 thread will start the first container, others will wait
// This pipe will take all incoming tasks and just forward them to the
// started hot functions. The catch here is that it feeds a buffered
// channel from an unbuffered one. And this buffered channel is
// then used to determine the presence of running hot functions.
// If no hot function is available, tasksout will fill up to its
// capacity and pipe() will start them.
go svr.pipe(context.Background()) // XXX (reed): real context for adding consuela
return svr
}
func (svr *htfnsvr) pipe(ctx context.Context) {
for {
select {
case t := <-svr.tasksin:
svr.tasksout <- t
// TODO move checking for ram up here? then we can wait for hot functions to open up instead of always
// trying to make new ones if all hot functions are busy (and if machine is full and all functions are
// hot then most new hot functions are going to time out waiting to get available ram)
// TODO need to add some kind of metering here, we could track average run time and # of runners
select {
case _, ok := <-svr.first: // wait for >= 1 to be up to avoid herd
if ok || len(svr.tasksout) > 0 {
svr.launch(ctx)
}
case <-ctx.Done(): // TODO we should prob watch the task timeout not just the pipe...
return
}
case <-ctx.Done():
return
}
}
}
func (svr *htfnsvr) launch(ctx context.Context) {
hc := newhtfn(
svr.cfg,
svr.tasksout,
svr.rnr,
func() { svr.once.Do(func() { close(svr.first) }) },
)
go hc.serve(ctx)
}
// htfn is one instance of a hot container, which may or may not be running a
// task. If idle long enough, it will stop. It uses route configuration to
// determine which protocol to use.
type htfn struct {
id string
cfg *task.Config
proto protocol.ContainerIO
tasks <-chan task.Request
once func()
// Receiving side of the container.
containerIn io.Reader
containerOut io.Writer
rnr *Runner
}
func newhtfn(cfg *task.Config, tasks <-chan task.Request, rnr *Runner, once func()) *htfn {
stdinr, stdinw := io.Pipe()
stdoutr, stdoutw := io.Pipe()
return &htfn{
id: id.New().String(),
cfg: cfg,
proto: protocol.New(protocol.Protocol(cfg.Format), stdinw, stdoutr),
tasks: tasks,
once: once,
containerIn: stdinr,
containerOut: stdoutw,
rnr: rnr,
}
}
// ghostWriter is a writer who will pass writes to an inner writer
// (that may be changed at will).
type ghostWriter struct {
sync.Mutex
inner io.Writer
}
func (g *ghostWriter) swap(w io.Writer) {
g.Lock()
g.inner = w
g.Unlock()
}
func (g *ghostWriter) Write(b []byte) (int, error) {
// we don't need to serialize writes but swapping g.inner could be a race if unprotected
g.Lock()
w := g.inner
g.Unlock()
return w.Write(b)
}
func (g *ghostWriter) Close() error { return nil }
func (hc *htfn) serve(ctx context.Context) {
span, ctx := opentracing.StartSpanFromContext(ctx, "run_hot_container")
defer span.Finish()
ctx, cancel := context.WithCancel(ctx)
defer cancel()
cfg := *hc.cfg
logger := logrus.WithFields(logrus.Fields{"hot_id": hc.id, "app": cfg.AppName, "route": cfg.Path, "image": cfg.Image, "memory": cfg.Memory, "format": cfg.Format, "idle_timeout": cfg.IdleTimeout})
// TODO go through FuncLogger probably
// if there's no active call, log any errors to stderr (for debugging issues)
bwLog := newLineWriter(&logWriter{ctx: ctx, appName: cfg.AppName, path: cfg.Path, image: cfg.Image, reqID: hc.id})
defer bwLog.Close()
stderr := &ghostWriter{inner: bwLog}
first := true
go func() {
for {
select {
case <-ctx.Done():
case <-cfg.Ready:
// on first execution, wait before starting idle timeout / stopping wait time clock,
// since docker pull / container create need to happen.
// XXX (reed): should we still obey the task timeout? docker image could be 8GB...
}
select {
case <-ctx.Done():
return
case <-time.After(cfg.IdleTimeout):
logger.Info("Canceling inactive hot function")
cancel()
case t := <-hc.tasks:
var span opentracing.Span
if first {
// TODO this doesn't work as intended; beyond me atm, but the spans do come up.
// need a way to add the span from starting container to the first execution, basically.
spanHot := opentracing.SpanFromContext(ctx)
spanTask := opentracing.SpanFromContext(t.Ctx)
span = opentracing.StartSpan("dispatch", opentracing.ChildOf(spanTask.Context()), opentracing.FollowsFrom(spanHot.Context()))
ctx = opentracing.ContextWithSpan(t.Ctx, span)
first = false
} else {
span, ctx = opentracing.StartSpanFromContext(t.Ctx, "dispatch")
}
// swap logs to log to the task logger instead of stderr
tlog := hc.rnr.flog.Writer(ctx, cfg.AppName, cfg.Path, cfg.Image, cfg.ID)
stderr.swap(tlog)
start := time.Now()
err := hc.proto.Dispatch(ctx, t.Config)
status := "success"
if err != nil {
status = "error"
logrus.WithField("ctx", ctx).Info("task failed")
}
span.Finish()
hc.once()
stderr.swap(bwLog) // swap back out before flush
tlog.Close() // write to db/flush
t.Response <- task.Response{
Result: &runResult{start: start, status: status, error: err},
Err: err,
}
}
}
}()
cfg.Timeout = 0 // add a timeout to simulate ab.end. failure.
cfg.Stdin = hc.containerIn
cfg.Stdout = hc.containerOut
cfg.Stderr = stderr
// TODO how to tie a span from the first task into this? yikes
result, err := hc.rnr.run(ctx, &cfg)
if err != nil {
logger.WithError(err).Error("hot function failure detected")
}
logger.WithField("result", result).Info("hot function terminated")
}
// TODO make Default protocol a real thing and get rid of this in favor of Dispatch
func runTaskReq(rnr *Runner, t task.Request) {
// TODO this will not be such a shit storm after the above TODO is TODONE
cfg := t.Config
t.Config.Stderr = rnr.flog.Writer(t.Ctx, cfg.AppName, cfg.Path, cfg.Image, cfg.ID)
defer t.Config.Stderr.Close()
result, err := rnr.run(t.Ctx, t.Config)
select {
case t.Response <- task.Response{result, err}:
close(t.Response)
default:
}
}
type runResult struct {
error
status string
start time.Time
}
func (r *runResult) Error() string {
if r.error == nil {
return ""
}
return r.error.Error()
}
func (r *runResult) Status() string { return r.status }
func (r *runResult) StartTime() time.Time { return r.start }

View File

@@ -4,8 +4,8 @@ import (
"net/http" "net/http"
"github.com/fnproject/fn/api" "github.com/fnproject/fn/api"
"github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/common"
"github.com/gin-gonic/gin" "github.com/gin-gonic/gin"
) )

View File

@@ -29,7 +29,7 @@ func TestAppCreate(t *testing.T) {
buf := setLogBuffer() buf := setLogBuffer()
for i, test := range []struct { for i, test := range []struct {
mock models.Datastore mock models.Datastore
logDB models.FnLog logDB models.LogStore
path string path string
body string body string
expectedCode int expectedCode int
@@ -48,7 +48,7 @@ func TestAppCreate(t *testing.T) {
{datastore.NewMock(), logs.NewMock(), "/v1/apps", `{ "app": { "name": "teste" } }`, http.StatusOK, nil}, {datastore.NewMock(), logs.NewMock(), "/v1/apps", `{ "app": { "name": "teste" } }`, http.StatusOK, nil},
} { } {
rnr, cancel := testRunner(t) rnr, cancel := testRunner(t)
srv := testServer(test.mock, &mqs.Mock{}, test.logDB, rnr, DefaultEnqueue) srv := testServer(test.mock, &mqs.Mock{}, test.logDB, rnr)
router := srv.Router router := srv.Router
body := bytes.NewBuffer([]byte(test.body)) body := bytes.NewBuffer([]byte(test.body))
@@ -78,7 +78,7 @@ func TestAppDelete(t *testing.T) {
for i, test := range []struct { for i, test := range []struct {
ds models.Datastore ds models.Datastore
logDB models.FnLog logDB models.LogStore
path string path string
body string body string
expectedCode int expectedCode int
@@ -88,11 +88,11 @@ func TestAppDelete(t *testing.T) {
{datastore.NewMockInit( {datastore.NewMockInit(
[]*models.App{{ []*models.App{{
Name: "myapp", Name: "myapp",
}}, nil, nil, nil, }}, nil, nil,
), logs.NewMock(), "/v1/apps/myapp", "", http.StatusOK, nil}, ), logs.NewMock(), "/v1/apps/myapp", "", http.StatusOK, nil},
} { } {
rnr, cancel := testRunner(t) rnr, cancel := testRunner(t)
srv := testServer(test.ds, &mqs.Mock{}, test.logDB, rnr, DefaultEnqueue) srv := testServer(test.ds, &mqs.Mock{}, test.logDB, rnr)
_, rec := routerRequest(t, srv.Router, "DELETE", test.path, nil) _, rec := routerRequest(t, srv.Router, "DELETE", test.path, nil)
@@ -122,7 +122,7 @@ func TestAppList(t *testing.T) {
defer cancel() defer cancel()
ds := datastore.NewMock() ds := datastore.NewMock()
fnl := logs.NewMock() fnl := logs.NewMock()
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
for i, test := range []struct { for i, test := range []struct {
path string path string
@@ -159,7 +159,7 @@ func TestAppGet(t *testing.T) {
defer cancel() defer cancel()
ds := datastore.NewMock() ds := datastore.NewMock()
fnl := logs.NewMock() fnl := logs.NewMock()
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
for i, test := range []struct { for i, test := range []struct {
path string path string
@@ -194,7 +194,7 @@ func TestAppUpdate(t *testing.T) {
for i, test := range []struct { for i, test := range []struct {
mock models.Datastore mock models.Datastore
logDB models.FnLog logDB models.LogStore
path string path string
body string body string
expectedCode int expectedCode int
@@ -207,18 +207,18 @@ func TestAppUpdate(t *testing.T) {
{datastore.NewMockInit( {datastore.NewMockInit(
[]*models.App{{ []*models.App{{
Name: "myapp", Name: "myapp",
}}, nil, nil, nil, }}, nil, nil,
), logs.NewMock(), "/v1/apps/myapp", `{ "app": { "config": { "test": "1" } } }`, http.StatusOK, nil}, ), logs.NewMock(), "/v1/apps/myapp", `{ "app": { "config": { "test": "1" } } }`, http.StatusOK, nil},
// Addresses #380 // Addresses #380
{datastore.NewMockInit( {datastore.NewMockInit(
[]*models.App{{ []*models.App{{
Name: "myapp", Name: "myapp",
}}, nil, nil, nil, }}, nil, nil,
), logs.NewMock(), "/v1/apps/myapp", `{ "app": { "name": "othername" } }`, http.StatusConflict, nil}, ), logs.NewMock(), "/v1/apps/myapp", `{ "app": { "name": "othername" } }`, http.StatusConflict, nil},
} { } {
rnr, cancel := testRunner(t) rnr, cancel := testRunner(t)
srv := testServer(test.mock, &mqs.Mock{}, test.logDB, rnr, DefaultEnqueue) srv := testServer(test.mock, &mqs.Mock{}, test.logDB, rnr)
body := bytes.NewBuffer([]byte(test.body)) body := bytes.NewBuffer([]byte(test.body))
_, rec := routerRequest(t, srv.Router, "PATCH", test.path, body) _, rec := routerRequest(t, srv.Router, "PATCH", test.path, body)

View File

@@ -10,8 +10,9 @@ import (
func (s *Server) handleCallGet(c *gin.Context) { func (s *Server) handleCallGet(c *gin.Context) {
ctx := c.Request.Context() ctx := c.Request.Context()
appName := c.MustGet(api.AppName).(string)
callID := c.Param(api.Call) callID := c.Param(api.Call)
callObj, err := s.Datastore.GetTask(ctx, callID) callObj, err := s.Datastore.GetCall(ctx, appName, callID)
if err != nil { if err != nil {
handleErrorResponse(c, err) handleErrorResponse(c, err)
return return

View File

@@ -20,7 +20,7 @@ func (s *Server) handleCallList(c *gin.Context) {
filter := models.CallFilter{AppName: appName, Path: c.Query(api.CRoute)} filter := models.CallFilter{AppName: appName, Path: c.Query(api.CRoute)}
calls, err := s.Datastore.GetTasks(ctx, &filter) calls, err := s.Datastore.GetCalls(ctx, &filter)
if err != nil { if err != nil {
handleErrorResponse(c, err) handleErrorResponse(c, err)
return return

View File

@@ -10,8 +10,9 @@ import (
func (s *Server) handleCallLogGet(c *gin.Context) { func (s *Server) handleCallLogGet(c *gin.Context) {
ctx := c.Request.Context() ctx := c.Request.Context()
appName := c.MustGet(api.AppName).(string)
callID := c.Param(api.Call) callID := c.Param(api.Call)
_, err := s.Datastore.GetTask(ctx, callID) _, err := s.Datastore.GetCall(ctx, appName, callID)
if err != nil { if err != nil {
handleErrorResponse(c, err) handleErrorResponse(c, err)
return return
@@ -29,8 +30,9 @@ func (s *Server) handleCallLogGet(c *gin.Context) {
func (s *Server) handleCallLogDelete(c *gin.Context) { func (s *Server) handleCallLogDelete(c *gin.Context) {
ctx := c.Request.Context() ctx := c.Request.Context()
appName := c.MustGet(api.AppName).(string)
callID := c.Param(api.Call) callID := c.Param(api.Call)
_, err := s.Datastore.GetTask(ctx, callID) _, err := s.Datastore.GetCall(ctx, appName, callID)
if err != nil { if err != nil {
handleErrorResponse(c, err) handleErrorResponse(c, err)
return return

View File

@@ -7,10 +7,10 @@ import (
"net/http" "net/http"
"runtime/debug" "runtime/debug"
"github.com/sirupsen/logrus" "github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner/common"
"github.com/gin-gonic/gin" "github.com/gin-gonic/gin"
"github.com/sirupsen/logrus"
) )
// ErrInternalServerError returned when something exceptional happens. // ErrInternalServerError returned when something exceptional happens.
@@ -28,13 +28,12 @@ func handleErrorResponse(c *gin.Context, err error) {
func HandleErrorResponse(ctx context.Context, w http.ResponseWriter, err error) { func HandleErrorResponse(ctx context.Context, w http.ResponseWriter, err error) {
log := common.Logger(ctx) log := common.Logger(ctx)
var statuscode int var statuscode int
switch e := err.(type) { if e, ok := err.(models.APIError); ok {
case models.APIError:
if e.Code() >= 500 { if e.Code() >= 500 {
log.WithFields(logrus.Fields{"code": e.Code()}).WithError(e).Error("api error") log.WithFields(logrus.Fields{"code": e.Code()}).WithError(e).Error("api error")
} }
statuscode = e.Code() statuscode = e.Code()
default: } else {
log.WithError(err).WithFields(logrus.Fields{"stack": string(debug.Stack())}).Error("internal server error") log.WithError(err).WithFields(logrus.Fields{"stack": string(debug.Stack())}).Error("internal server error")
statuscode = http.StatusInternalServerError statuscode = http.StatusInternalServerError
err = ErrInternalServerError err = ErrInternalServerError

View File

@@ -4,8 +4,7 @@ import (
"context" "context"
"net/http" "net/http"
"github.com/fnproject/fn/api/runner/common" "github.com/fnproject/fn/api/common"
"github.com/gin-gonic/gin" "github.com/gin-gonic/gin"
) )

View File

@@ -45,7 +45,6 @@ func (s *Server) handleRoutesPostPutPatch(c *gin.Context) {
return return
} }
s.cachedelete(resp.Route.AppName, resp.Route.Path)
c.JSON(http.StatusOK, resp) c.JSON(http.StatusOK, resp)
} }

View File

@@ -24,6 +24,5 @@ func (s *Server) handleRouteDelete(c *gin.Context) {
return return
} }
s.cachedelete(appName, routePath)
c.JSON(http.StatusOK, gin.H{"message": "Route deleted"}) c.JSON(http.StatusOK, gin.H{"message": "Route deleted"})
} }

View File

@@ -14,7 +14,7 @@ import (
type routeTestCase struct { type routeTestCase struct {
ds models.Datastore ds models.Datastore
logDB models.FnLog logDB models.LogStore
method string method string
path string path string
body string body string
@@ -24,7 +24,7 @@ type routeTestCase struct {
func (test *routeTestCase) run(t *testing.T, i int, buf *bytes.Buffer) { func (test *routeTestCase) run(t *testing.T, i int, buf *bytes.Buffer) {
rnr, cancel := testRunner(t) rnr, cancel := testRunner(t)
srv := testServer(test.ds, &mqs.Mock{}, test.logDB, rnr, DefaultEnqueue) srv := testServer(test.ds, &mqs.Mock{}, test.logDB, rnr)
body := bytes.NewBuffer([]byte(test.body)) body := bytes.NewBuffer([]byte(test.body))
_, rec := routerRequest(t, srv.Router, test.method, test.path, body) _, rec := routerRequest(t, srv.Router, test.method, test.path, body)
@@ -70,7 +70,7 @@ func TestRouteCreate(t *testing.T) {
AppName: "a", AppName: "a",
Path: "/myroute", Path: "/myroute",
}, },
}, nil, nil, }, nil,
), logs.NewMock(), http.MethodPost, "/v1/apps/a/routes", `{ "route": { "image": "fnproject/hello", "path": "/myroute", "type": "sync" } }`, http.StatusConflict, models.ErrRoutesAlreadyExists}, ), logs.NewMock(), http.MethodPost, "/v1/apps/a/routes", `{ "route": { "image": "fnproject/hello", "path": "/myroute", "type": "sync" } }`, http.StatusConflict, models.ErrRoutesAlreadyExists},
// success // success
@@ -106,22 +106,22 @@ func TestRoutePut(t *testing.T) {
func TestRouteDelete(t *testing.T) { func TestRouteDelete(t *testing.T) {
buf := setLogBuffer() buf := setLogBuffer()
routes := models.Routes{{AppName: "a", Path: "/myroute"}} routes := []*models.Route{{AppName: "a", Path: "/myroute"}}
apps := models.Apps{{Name: "a", Routes: routes, Config: nil}} apps := []*models.App{{Name: "a", Routes: routes, Config: nil}}
for i, test := range []struct { for i, test := range []struct {
ds models.Datastore ds models.Datastore
logDB models.FnLog logDB models.LogStore
path string path string
body string body string
expectedCode int expectedCode int
expectedError error expectedError error
}{ }{
{datastore.NewMock(), logs.NewMock(), "/v1/apps/a/routes/missing", "", http.StatusNotFound, models.ErrRoutesNotFound}, {datastore.NewMock(), logs.NewMock(), "/v1/apps/a/routes/missing", "", http.StatusNotFound, models.ErrRoutesNotFound},
{datastore.NewMockInit(apps, routes, nil, nil), logs.NewMock(), "/v1/apps/a/routes/myroute", "", http.StatusOK, nil}, {datastore.NewMockInit(apps, routes, nil), logs.NewMock(), "/v1/apps/a/routes/myroute", "", http.StatusOK, nil},
} { } {
rnr, cancel := testRunner(t) rnr, cancel := testRunner(t)
srv := testServer(test.ds, &mqs.Mock{}, test.logDB, rnr, DefaultEnqueue) srv := testServer(test.ds, &mqs.Mock{}, test.logDB, rnr)
_, rec := routerRequest(t, srv.Router, "DELETE", test.path, nil) _, rec := routerRequest(t, srv.Router, "DELETE", test.path, nil)
if rec.Code != test.expectedCode { if rec.Code != test.expectedCode {
@@ -152,7 +152,7 @@ func TestRouteList(t *testing.T) {
ds := datastore.NewMock() ds := datastore.NewMock()
fnl := logs.NewMock() fnl := logs.NewMock()
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
for i, test := range []struct { for i, test := range []struct {
path string path string
@@ -191,7 +191,7 @@ func TestRouteGet(t *testing.T) {
ds := datastore.NewMock() ds := datastore.NewMock()
fnl := logs.NewMock() fnl := logs.NewMock()
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
for i, test := range []struct { for i, test := range []struct {
path string path string
@@ -238,7 +238,7 @@ func TestRouteUpdate(t *testing.T) {
AppName: "a", AppName: "a",
Path: "/myroute/do", Path: "/myroute/do",
}, },
}, nil, nil, }, nil,
), logs.NewMock(), http.MethodPatch, "/v1/apps/a/routes/myroute/do", `{ "route": { "image": "fnproject/hello" } }`, http.StatusOK, nil}, ), logs.NewMock(), http.MethodPatch, "/v1/apps/a/routes/myroute/do", `{ "route": { "image": "fnproject/hello" } }`, http.StatusOK, nil},
// Addresses #381 // Addresses #381
@@ -248,7 +248,7 @@ func TestRouteUpdate(t *testing.T) {
AppName: "a", AppName: "a",
Path: "/myroute/do", Path: "/myroute/do",
}, },
}, nil, nil, }, nil,
), logs.NewMock(), http.MethodPatch, "/v1/apps/a/routes/myroute/do", `{ "route": { "path": "/otherpath" } }`, http.StatusConflict, models.ErrRoutesPathImmutable}, ), logs.NewMock(), http.MethodPatch, "/v1/apps/a/routes/myroute/do", `{ "route": { "path": "/otherpath" } }`, http.StatusConflict, models.ErrRoutesPathImmutable},
} { } {
test.run(t, i, buf) test.run(t, i, buf)

View File

@@ -3,23 +3,15 @@ package server
import ( import (
"bytes" "bytes"
"context" "context"
"fmt"
"io"
"io/ioutil" "io/ioutil"
"net/http" "net/http"
"path" "path"
"strings" "strings"
"time"
"github.com/sirupsen/logrus"
"github.com/fnproject/fn/api" "github.com/fnproject/fn/api"
"github.com/fnproject/fn/api/id" "github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/api/models" "github.com/fnproject/fn/api/models"
"github.com/fnproject/fn/api/runner"
"github.com/fnproject/fn/api/runner/common"
"github.com/fnproject/fn/api/runner/task"
"github.com/gin-gonic/gin" "github.com/gin-gonic/gin"
cache "github.com/patrickmn/go-cache"
) )
type runnerResponse struct { type runnerResponse struct {
@@ -27,15 +19,7 @@ type runnerResponse struct {
Error *models.ErrorBody `json:"error,omitempty"` Error *models.ErrorBody `json:"error,omitempty"`
} }
func toEnvName(envtype, name string) string { func (s *Server) handleRequest(c *gin.Context) {
name = strings.ToUpper(strings.Replace(name, "-", "_", -1))
if envtype == "" {
return name
}
return fmt.Sprintf("%s_%s", envtype, name)
}
func (s *Server) handleRequest(c *gin.Context, enqueue models.Enqueue) {
if strings.HasPrefix(c.Request.URL.Path, "/v1") { if strings.HasPrefix(c.Request.URL.Path, "/v1") {
c.Status(http.StatusNotFound) c.Status(http.StatusNotFound)
return return
@@ -43,22 +27,10 @@ func (s *Server) handleRequest(c *gin.Context, enqueue models.Enqueue) {
ctx := c.Request.Context() ctx := c.Request.Context()
reqID := id.New().String() if c.Request.Method == "GET" {
ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": reqID}) // TODO we _could_ check the normal body, this is still weird
// TODO do we need to flush the original body if we do this? (hint: yes)
var err error c.Request.Body = ioutil.NopCloser(strings.NewReader(c.Request.URL.Query().Get("payload")))
var payload io.Reader
if c.Request.Method == "POST" {
payload = c.Request.Body
// Load complete body and close
defer func() {
io.Copy(ioutil.Discard, c.Request.Body)
c.Request.Body.Close()
}()
} else if c.Request.Method == "GET" {
reqPayload := c.Request.URL.Query().Get("payload")
payload = strings.NewReader(reqPayload)
} }
r, routeExists := c.Get(api.Path) r, routeExists := c.Get(api.Path)
@@ -73,228 +45,65 @@ func (s *Server) handleRequest(c *gin.Context, enqueue models.Enqueue) {
s.FireBeforeDispatch(ctx, reqRoute) s.FireBeforeDispatch(ctx, reqRoute)
appName := reqRoute.AppName s.serve(c, reqRoute.AppName, reqRoute.Path)
path := reqRoute.Path
app, err := s.Datastore.GetApp(ctx, appName) s.FireAfterDispatch(ctx, reqRoute)
if err != nil {
handleErrorResponse(c, err)
return
} else if app == nil {
handleErrorResponse(c, models.ErrAppsNotFound)
return
}
log.WithFields(logrus.Fields{"app": appName, "path": path}).Debug("Finding route on datastore")
route, err := s.loadroute(ctx, appName, path)
if err != nil {
handleErrorResponse(c, err)
return
}
if route == nil {
handleErrorResponse(c, models.ErrRoutesNotFound)
return
}
log = log.WithFields(logrus.Fields{"app": appName, "path": route.Path, "image": route.Image})
log.Debug("Got route from datastore")
if s.serve(ctx, c, appName, route, app, path, reqID, payload, enqueue) {
s.FireAfterDispatch(ctx, reqRoute)
return
}
handleErrorResponse(c, models.ErrRoutesNotFound)
} }
func (s *Server) loadroute(ctx context.Context, appName, path string) (*models.Route, error) { // TODO it would be nice if we could make this have nothing to do with the gin.Context but meh
if route, ok := s.cacheget(appName, path); ok { // TODO make async store an *http.Request? would be sexy until we have different api format...
return route, nil func (s *Server) serve(c *gin.Context, appName, path string) {
} // GetCall can mod headers, assign an id, look up the route/app (cached),
key := routeCacheKey(appName, path) // strip params, etc.
resp, err := s.singleflight.do( call, err := s.Agent.GetCall(
key, agent.WithWriter(c.Writer), // XXX (reed): order matters [for now]
func() (interface{}, error) { agent.FromRequest(appName, path, c.Request),
return s.Datastore.GetRoute(ctx, appName, path)
},
) )
if err != nil { if err != nil {
return nil, err handleErrorResponse(c, err)
} return
route := resp.(*models.Route)
s.routeCache.Set(key, route, cache.DefaultExpiration)
return route, nil
}
// TODO: Should remove *gin.Context from these functions, should use only context.Context
func (s *Server) serve(ctx context.Context, c *gin.Context, appName string, route *models.Route, app *models.App, path, reqID string, payload io.Reader, enqueue models.Enqueue) (ok bool) {
ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"app": appName, "route": route.Path, "image": route.Image})
params, match := matchRoute(route.Path, path)
if !match {
return false
} }
var stdout bytes.Buffer // TODO: should limit the size of this, error if gets too big. akin to: https://golang.org/pkg/io/#LimitReader // TODO we could add FireBeforeDispatch right here with Call in hand
if route.Format == "" { if model := call.Model(); model.Type == "async" {
route.Format = "default" // TODO we should push this into GetCall somehow (CallOpt maybe) or maybe agent.Queue(Call) ?
} buf := bytes.NewBuffer(make([]byte, 0, c.Request.ContentLength)) // TODO sync.Pool me
_, err := buf.ReadFrom(c.Request.Body)
// baseVars are the vars on the route & app, not on this specific request [for hot functions]
baseVars := make(map[string]string, len(app.Config)+len(route.Config)+3)
baseVars["FN_FORMAT"] = route.Format
baseVars["APP_NAME"] = appName
baseVars["ROUTE"] = route.Path
baseVars["MEMORY_MB"] = fmt.Sprintf("%d", route.Memory)
// app config
for k, v := range app.Config {
k = toEnvName("", k)
baseVars[k] = v
}
for k, v := range route.Config {
k = toEnvName("", k)
baseVars[k] = v
}
// envVars contains the full set of env vars, per request + base
envVars := make(map[string]string, len(baseVars)+len(params)+len(c.Request.Header)+3)
for k, v := range baseVars {
envVars[k] = v
}
envVars["CALL_ID"] = reqID
envVars["METHOD"] = c.Request.Method
envVars["REQUEST_URL"] = fmt.Sprintf("%v://%v%v", func() string {
if c.Request.TLS == nil {
return "http"
}
return "https"
}(), c.Request.Host, c.Request.URL.String())
// params
for _, param := range params {
envVars[toEnvName("PARAM", param.Key)] = param.Value
}
// headers
for header, value := range c.Request.Header {
envVars[toEnvName("HEADER", header)] = strings.Join(value, ", ")
}
cfg := &task.Config{
AppName: appName,
Path: route.Path,
BaseEnv: baseVars,
Env: envVars,
Format: route.Format,
ID: reqID,
Image: route.Image,
Memory: route.Memory,
Stdin: payload,
Stdout: &stdout,
Timeout: time.Duration(route.Timeout) * time.Second,
IdleTimeout: time.Duration(route.IdleTimeout) * time.Second,
ReceivedTime: time.Now(),
Ready: make(chan struct{}),
}
// ensure valid values
if cfg.Timeout <= 0 {
cfg.Timeout = runner.DefaultTimeout
}
if cfg.IdleTimeout <= 0 {
cfg.IdleTimeout = runner.DefaultIdleTimeout
}
s.Runner.Enqueue()
newTask := task.TaskFromConfig(cfg)
switch route.Type {
case "async":
// TODO we should be able to do hot input to async. plumb protocol stuff
// TODO enqueue should unravel the payload?
// Read payload
pl, err := ioutil.ReadAll(cfg.Stdin)
if err != nil { if err != nil {
handleErrorResponse(c, models.ErrInvalidPayload) handleErrorResponse(c, models.ErrInvalidPayload)
return true return
} }
// Add in payload model.Payload = buf.String()
newTask.Payload = string(pl)
// Push to queue // TODO we should probably add this to the datastore too. consider the plumber!
_, err = enqueue(c, s.MQ, newTask) _, err = s.MQ.Push(c.Request.Context(), model)
if err != nil { if err != nil {
handleErrorResponse(c, err) handleErrorResponse(c, err)
return true return
} }
log.Info("Added new task to queue") c.JSON(http.StatusAccepted, map[string]string{"call_id": model.ID})
c.JSON(http.StatusAccepted, map[string]string{"call_id": cfg.ID}) return
default:
result, err := s.Runner.RunTrackedTask(newTask, ctx, cfg)
if result != nil {
waitTime := result.StartTime().Sub(cfg.ReceivedTime)
c.Header("XXX-FXLB-WAIT", waitTime.String())
}
if err != nil {
c.JSON(http.StatusInternalServerError, runnerResponse{
RequestID: cfg.ID,
Error: &models.ErrorBody{
Message: err.Error(),
},
})
log.WithError(err).Error("Failed to run task")
break
}
for k, v := range route.Headers {
c.Header(k, v[0])
}
// this will help users to track sync execution in a manner of async
// FN_CALL_ID is an equivalent of call_id
c.Header("FN_CALL_ID", newTask.ID)
switch result.Status() {
case "success":
c.Data(http.StatusOK, "", stdout.Bytes())
case "timeout":
c.JSON(http.StatusGatewayTimeout, runnerResponse{
RequestID: cfg.ID,
Error: &models.ErrorBody{
Message: models.ErrRunnerTimeout.Error(),
},
})
default:
c.JSON(http.StatusInternalServerError, runnerResponse{
RequestID: cfg.ID,
Error: &models.ErrorBody{
Message: result.Error(),
},
})
}
} }
return true err = s.Agent.Submit(call)
} if err != nil {
// NOTE if they cancel the request then it will stop the call (kind of cool),
// we could filter that error out here too as right now it yells a little
var fakeHandler = func(http.ResponseWriter, *http.Request, Params) {} if err == context.DeadlineExceeded {
err = models.ErrCallTimeout // 504 w/ friendly note
func matchRoute(baseRoute, route string) (Params, bool) { }
tree := &node{} // NOTE: if the task wrote the headers already then this will fail to write
tree.addRoute(baseRoute, fakeHandler) // a 5xx (and log about it to us) -- that's fine (nice, even!)
handler, p, _ := tree.getValue(route) handleErrorResponse(c, err)
if handler == nil { return
return nil, false
} }
return p, true // TODO plumb FXLB-WAIT somehow (api?)
// TODO we need to watch the response writer and if no bytes written
// then write a 200 at this point?
// c.Data(http.StatusOK)
} }

View File

@@ -5,28 +5,23 @@ import (
"context" "context"
"fmt" "fmt"
"net/http" "net/http"
"sync"
"testing" "testing"
"time"
"github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/api/datastore" "github.com/fnproject/fn/api/datastore"
"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/fnproject/fn/api/runner"
"github.com/gin-gonic/gin" "github.com/gin-gonic/gin"
cache "github.com/patrickmn/go-cache"
) )
func testRouterAsync(ds models.Datastore, mq models.MessageQueue, rnr *runner.Runner, enqueue models.Enqueue) *gin.Engine { func testRouterAsync(ds models.Datastore, mq models.MessageQueue, rnr agent.Agent) *gin.Engine {
ctx := context.Background() ctx := context.Background()
s := &Server{ s := &Server{
Runner: rnr, Agent: rnr,
Router: gin.New(), Router: gin.New(),
Datastore: ds, Datastore: ds,
MQ: mq, MQ: mq,
Enqueue: enqueue,
routeCache: cache.New(60*time.Second, 5*time.Minute),
} }
r := s.Router r := s.Router
@@ -46,7 +41,7 @@ func TestRouteRunnerAsyncExecution(t *testing.T) {
{Type: "async", Path: "/myroute", AppName: "myapp", Image: "fnproject/hello", Config: map[string]string{"test": "true"}}, {Type: "async", Path: "/myroute", AppName: "myapp", Image: "fnproject/hello", Config: map[string]string{"test": "true"}},
{Type: "async", Path: "/myerror", AppName: "myapp", Image: "fnproject/error", Config: map[string]string{"test": "true"}}, {Type: "async", Path: "/myerror", AppName: "myapp", Image: "fnproject/error", Config: map[string]string{"test": "true"}},
{Type: "async", Path: "/myroute/:param", AppName: "myapp", Image: "fnproject/hello", Config: map[string]string{"test": "true"}}, {Type: "async", Path: "/myroute/:param", AppName: "myapp", Image: "fnproject/hello", Config: map[string]string{"test": "true"}},
}, nil, nil, }, nil,
) )
mq := &mqs.Mock{} mq := &mqs.Mock{}
@@ -75,29 +70,10 @@ func TestRouteRunnerAsyncExecution(t *testing.T) {
}, },
} { } {
body := bytes.NewBuffer([]byte(test.body)) body := bytes.NewBuffer([]byte(test.body))
var wg sync.WaitGroup
wg.Add(1)
fmt.Println("About to start router") fmt.Println("About to start router")
rnr, cancel := testRunner(t) rnr, cancel := testRunner(t, ds)
router := testRouterAsync(ds, mq, rnr, func(_ context.Context, _ models.MessageQueue, task *models.Task) (*models.Task, error) { router := testRouterAsync(ds, mq, rnr)
if test.body != task.Payload {
t.Errorf("Test %d: Expected task Payload to be the same as the test body", i)
}
if test.expectedEnv != nil {
for name, value := range test.expectedEnv {
taskName := name
if value != task.EnvVars[taskName] {
t.Errorf("Test %d: Expected header `%s` to be `%s` but was `%s`",
i, name, value, task.EnvVars[taskName])
}
}
}
wg.Done()
return task, nil
})
fmt.Println("makeing requests") fmt.Println("makeing requests")
req, rec := newRouterRequest(t, "POST", test.path, body) req, rec := newRouterRequest(t, "POST", test.path, body)
@@ -112,8 +88,9 @@ func TestRouteRunnerAsyncExecution(t *testing.T) {
t.Errorf("Test %d: Expected status code to be %d but was %d", t.Errorf("Test %d: Expected status code to be %d but was %d",
i, test.expectedCode, rec.Code) i, test.expectedCode, rec.Code)
} }
// TODO can test body and headers in the actual mq message w/ an agent that doesn't dequeue?
// this just makes sure tasks are submitted (ok)...
wg.Wait()
cancel() cancel()
} }
} }

View File

@@ -3,40 +3,45 @@ package server
import ( import (
"bytes" "bytes"
"context" "context"
"errors"
"net/http" "net/http"
"strings" "strings"
"testing" "testing"
"errors" "github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/api/datastore" "github.com/fnproject/fn/api/datastore"
"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/fnproject/fn/api/runner"
) )
func testRunner(t *testing.T) (*runner.Runner, context.CancelFunc) { func testRunner(t *testing.T, args ...interface{}) (agent.Agent, context.CancelFunc) {
ctx, cancel := context.WithCancel(context.Background())
ds := datastore.NewMock() ds := datastore.NewMock()
fnl := logs.NewMock() var mq models.MessageQueue = &mqs.Mock{}
r, err := runner.New(ctx, runner.NewFuncLogger(fnl), ds) for _, a := range args {
if err != nil { switch arg := a.(type) {
t.Fatal("Test: failed to create new runner") case models.Datastore:
ds = arg
case models.MessageQueue:
mq = arg
}
} }
return r, cancel r := agent.New(ds, mq)
return r, func() { r.Close() }
} }
func TestRouteRunnerGet(t *testing.T) { func TestRouteRunnerGet(t *testing.T) {
buf := setLogBuffer() buf := setLogBuffer()
rnr, cancel := testRunner(t)
defer cancel()
ds := datastore.NewMockInit( ds := datastore.NewMockInit(
[]*models.App{ []*models.App{
{Name: "myapp", Config: models.Config{}}, {Name: "myapp", Config: models.Config{}},
}, nil, nil, nil, }, nil, nil,
) )
rnr, cancel := testRunner(t, ds)
defer cancel()
logDB := logs.NewMock() logDB := logs.NewMock()
srv := testServer(ds, &mqs.Mock{}, logDB, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, logDB, rnr)
for i, test := range []struct { for i, test := range []struct {
path string path string
@@ -71,16 +76,17 @@ func TestRouteRunnerGet(t *testing.T) {
func TestRouteRunnerPost(t *testing.T) { func TestRouteRunnerPost(t *testing.T) {
buf := setLogBuffer() buf := setLogBuffer()
rnr, cancel := testRunner(t)
defer cancel()
ds := datastore.NewMockInit( ds := datastore.NewMockInit(
[]*models.App{ []*models.App{
{Name: "myapp", Config: models.Config{}}, {Name: "myapp", Config: models.Config{}},
}, nil, nil, nil, }, nil, nil,
) )
rnr, cancel := testRunner(t, ds)
defer cancel()
fnl := logs.NewMock() fnl := logs.NewMock()
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
for i, test := range []struct { for i, test := range []struct {
path string path string
@@ -117,9 +123,6 @@ func TestRouteRunnerPost(t *testing.T) {
func TestRouteRunnerExecution(t *testing.T) { func TestRouteRunnerExecution(t *testing.T) {
buf := setLogBuffer() buf := setLogBuffer()
rnr, cancelrnr := testRunner(t)
defer cancelrnr()
ds := datastore.NewMockInit( ds := datastore.NewMockInit(
[]*models.App{ []*models.App{
{Name: "myapp", Config: models.Config{}}, {Name: "myapp", Config: models.Config{}},
@@ -128,11 +131,14 @@ func TestRouteRunnerExecution(t *testing.T) {
{Path: "/", AppName: "myapp", Image: "fnproject/hello", Headers: map[string][]string{"X-Function": {"Test"}}}, {Path: "/", AppName: "myapp", Image: "fnproject/hello", Headers: map[string][]string{"X-Function": {"Test"}}},
{Path: "/myroute", AppName: "myapp", Image: "fnproject/hello", Headers: map[string][]string{"X-Function": {"Test"}}}, {Path: "/myroute", AppName: "myapp", Image: "fnproject/hello", Headers: map[string][]string{"X-Function": {"Test"}}},
{Path: "/myerror", AppName: "myapp", Image: "fnproject/error", Headers: map[string][]string{"X-Function": {"Test"}}}, {Path: "/myerror", AppName: "myapp", Image: "fnproject/error", Headers: map[string][]string{"X-Function": {"Test"}}},
}, nil, nil, }, nil,
) )
rnr, cancelrnr := testRunner(t, ds)
defer cancelrnr()
fnl := logs.NewMock() fnl := logs.NewMock()
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
for i, test := range []struct { for i, test := range []struct {
path string path string
@@ -172,26 +178,34 @@ func TestRouteRunnerExecution(t *testing.T) {
} }
} }
// implement models.MQ and models.APIError
type errorMQ struct {
error
code int
}
func (mock *errorMQ) Push(context.Context, *models.Call) (*models.Call, error) { return nil, mock }
func (mock *errorMQ) Reserve(context.Context) (*models.Call, error) { return nil, mock }
func (mock *errorMQ) Delete(context.Context, *models.Call) error { return mock }
func (mock *errorMQ) Code() int { return mock.code }
func TestFailedEnqueue(t *testing.T) { func TestFailedEnqueue(t *testing.T) {
buf := setLogBuffer() buf := setLogBuffer()
rnr, cancelrnr := testRunner(t)
defer cancelrnr()
ds := datastore.NewMockInit( ds := datastore.NewMockInit(
[]*models.App{ []*models.App{
{Name: "myapp", Config: models.Config{}}, {Name: "myapp", Config: models.Config{}},
}, },
[]*models.Route{ []*models.Route{
{Path: "/dummy", AppName: "myapp", Image: "dummy/dummy", Type: "async"}, {Path: "/dummy", AppName: "myapp", Image: "dummy/dummy", Type: "async"},
}, nil, nil, }, nil,
) )
err := errors.New("Unable to push task to queue")
mq := &errorMQ{err, http.StatusInternalServerError}
fnl := logs.NewMock() fnl := logs.NewMock()
rnr, cancelrnr := testRunner(t, ds, mq)
defer cancelrnr()
enqueue := func(ctx context.Context, mq models.MessageQueue, task *models.Task) (*models.Task, error) { srv := testServer(ds, mq, fnl, rnr)
return nil, errors.New("Unable to push task to queue")
}
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, enqueue)
for i, test := range []struct { for i, test := range []struct {
path string path string
body string body string
@@ -215,19 +229,20 @@ func TestRouteRunnerTimeout(t *testing.T) {
t.Skip("doesn't work on old Ubuntu") t.Skip("doesn't work on old Ubuntu")
buf := setLogBuffer() buf := setLogBuffer()
rnr, cancelrnr := testRunner(t)
defer cancelrnr()
ds := datastore.NewMockInit( ds := datastore.NewMockInit(
[]*models.App{ []*models.App{
{Name: "myapp", Config: models.Config{}}, {Name: "myapp", Config: models.Config{}},
}, },
[]*models.Route{ []*models.Route{
{Path: "/sleeper", AppName: "myapp", Image: "fnproject/sleeper", Timeout: 1}, {Path: "/sleeper", AppName: "myapp", Image: "fnproject/sleeper", Timeout: 1},
}, nil, nil, }, nil,
) )
rnr, cancelrnr := testRunner(t, ds)
defer cancelrnr()
fnl := logs.NewMock() fnl := logs.NewMock()
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
for i, test := range []struct { for i, test := range []struct {
path string path string
@@ -261,29 +276,29 @@ func TestRouteRunnerTimeout(t *testing.T) {
} }
} }
func TestMatchRoute(t *testing.T) { //func TestMatchRoute(t *testing.T) {
buf := setLogBuffer() //buf := setLogBuffer()
for i, test := range []struct { //for i, test := range []struct {
baseRoute string //baseRoute string
route string //route string
expectedParams []Param //expectedParams []Param
}{ //}{
{"/myroute/", `/myroute/`, nil}, //{"/myroute/", `/myroute/`, nil},
{"/myroute/:mybigparam", `/myroute/1`, []Param{{"mybigparam", "1"}}}, //{"/myroute/:mybigparam", `/myroute/1`, []Param{{"mybigparam", "1"}}},
{"/:param/*test", `/1/2`, []Param{{"param", "1"}, {"test", "/2"}}}, //{"/:param/*test", `/1/2`, []Param{{"param", "1"}, {"test", "/2"}}},
} { //} {
if params, match := matchRoute(test.baseRoute, test.route); match { //if params, match := matchRoute(test.baseRoute, test.route); match {
if test.expectedParams != nil { //if test.expectedParams != nil {
for j, param := range test.expectedParams { //for j, param := range test.expectedParams {
if params[j].Key != param.Key || params[j].Value != param.Value { //if params[j].Key != param.Key || params[j].Value != param.Value {
t.Log(buf.String()) //t.Log(buf.String())
t.Errorf("Test %d: expected param %d, key = %s, value = %s", i, j, param.Key, param.Value) //t.Errorf("Test %d: expected param %d, key = %s, value = %s", i, j, param.Key, param.Value)
} //}
} //}
} //}
} else { //} else {
t.Log(buf.String()) //t.Log(buf.String())
t.Errorf("Test %d: %s should match %s", i, test.route, test.baseRoute) //t.Errorf("Test %d: %s should match %s", i, test.route, test.baseRoute)
} //}
} //}
} //}

View File

@@ -3,31 +3,26 @@ package server
import ( import (
"bytes" "bytes"
"context" "context"
"encoding/json"
"errors" "errors"
"fmt" "fmt"
"io/ioutil"
"net" "net"
"net/http" "net/http"
"os" "os"
"path" "path"
"time"
"github.com/sirupsen/logrus"
"github.com/ccirello/supervisor"
"github.com/fnproject/fn/api" "github.com/fnproject/fn/api"
"github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/api/common"
"github.com/fnproject/fn/api/datastore" "github.com/fnproject/fn/api/datastore"
"github.com/fnproject/fn/api/id" "github.com/fnproject/fn/api/id"
"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/fnproject/fn/api/runner"
"github.com/fnproject/fn/api/runner/common"
"github.com/gin-gonic/gin" "github.com/gin-gonic/gin"
"github.com/opentracing/opentracing-go" "github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/ext" "github.com/opentracing/opentracing-go/ext"
"github.com/openzipkin/zipkin-go-opentracing" "github.com/openzipkin/zipkin-go-opentracing"
"github.com/patrickmn/go-cache" "github.com/sirupsen/logrus"
"github.com/spf13/viper" "github.com/spf13/viper"
) )
@@ -42,25 +37,17 @@ const (
) )
type Server struct { type Server struct {
Datastore models.Datastore
Runner *runner.Runner
Router *gin.Engine Router *gin.Engine
Agent agent.Agent
Datastore models.Datastore
MQ models.MessageQueue MQ models.MessageQueue
Enqueue models.Enqueue LogDB models.LogStore
LogDB models.FnLog
apiURL string
appListeners []AppListener appListeners []AppListener
middlewares []Middleware middlewares []Middleware
runnerListeners []RunnerListener runnerListeners []RunnerListener
routeCache *cache.Cache
singleflight singleflight // singleflight assists Datastore
} }
const cacheSize = 1024
// NewFromEnv creates a new Functions server based on env vars. // NewFromEnv creates a new Functions server based on env vars.
func NewFromEnv(ctx context.Context) *Server { func NewFromEnv(ctx context.Context) *Server {
ds, err := datastore.New(viper.GetString(EnvDBURL)) ds, err := datastore.New(viper.GetString(EnvDBURL))
@@ -73,7 +60,7 @@ func NewFromEnv(ctx context.Context) *Server {
logrus.WithError(err).Fatal("Error initializing message queue.") logrus.WithError(err).Fatal("Error initializing message queue.")
} }
var logDB models.FnLog = ds var logDB models.LogStore = ds
if ldb := viper.GetString(EnvLOGDBURL); ldb != "" && ldb != viper.GetString(EnvDBURL) { if ldb := viper.GetString(EnvLOGDBURL); ldb != "" && ldb != viper.GetString(EnvDBURL) {
logDB, err = logs.New(viper.GetString(EnvLOGDBURL)) logDB, err = logs.New(viper.GetString(EnvLOGDBURL))
if err != nil { if err != nil {
@@ -81,30 +68,17 @@ func NewFromEnv(ctx context.Context) *Server {
} }
} }
apiURL := viper.GetString(EnvAPIURL) return New(ctx, ds, mq, logDB)
return New(ctx, ds, mq, logDB, apiURL)
} }
// New creates a new Functions server with the passed in datastore, message queue and API URL // New creates a new Functions server with the passed in datastore, message queue and API URL
func New(ctx context.Context, ds models.Datastore, mq models.MessageQueue, logDB models.FnLog, apiURL string, opts ...ServerOption) *Server { func New(ctx context.Context, ds models.Datastore, mq models.MessageQueue, logDB models.LogStore, opts ...ServerOption) *Server {
funcLogger := runner.NewFuncLogger(logDB)
rnr, err := runner.New(ctx, funcLogger, ds)
if err != nil {
logrus.WithError(err).Fatalln("Failed to create a runner")
return nil
}
s := &Server{ s := &Server{
Runner: rnr, Agent: agent.New(ds, mq),
Router: gin.New(), Router: gin.New(),
Datastore: ds, Datastore: ds,
MQ: mq, MQ: mq,
routeCache: cache.New(5*time.Second, 5*time.Minute), LogDB: logDB,
LogDB: logDB,
Enqueue: DefaultEnqueue,
apiURL: apiURL,
} }
setMachineId() setMachineId()
@@ -234,58 +208,8 @@ func loggerWrap(c *gin.Context) {
c.Next() c.Next()
} }
func DefaultEnqueue(ctx context.Context, mq models.MessageQueue, task *models.Task) (*models.Task, error) {
ctx, _ = common.LoggerWithFields(ctx, logrus.Fields{"call_id": task.ID})
return mq.Push(ctx, task)
}
func routeCacheKey(appname, path string) string {
return fmt.Sprintf("%s_%s", appname, path)
}
func (s *Server) cacheget(appname, path string) (*models.Route, bool) {
route, ok := s.routeCache.Get(routeCacheKey(appname, path))
if !ok {
return nil, false
}
return route.(*models.Route), ok
}
func (s *Server) cachedelete(appname, path string) {
s.routeCache.Delete(routeCacheKey(appname, path))
}
func (s *Server) handleRunnerRequest(c *gin.Context) { func (s *Server) handleRunnerRequest(c *gin.Context) {
s.handleRequest(c, s.Enqueue) s.handleRequest(c)
}
func (s *Server) handleTaskRequest(c *gin.Context) {
ctx, _ := common.LoggerWithFields(c, nil)
switch c.Request.Method {
case "GET":
task, err := s.MQ.Reserve(ctx)
if err != nil {
handleErrorResponse(c, err)
return
}
c.JSON(http.StatusOK, task)
case "DELETE":
body, err := ioutil.ReadAll(c.Request.Body)
if err != nil {
handleErrorResponse(c, err)
return
}
var task models.Task
if err = json.Unmarshal(body, &task); err != nil {
handleErrorResponse(c, err)
return
}
if err := s.MQ.Delete(ctx, &task); err != nil {
handleErrorResponse(c, err)
return
}
c.JSON(http.StatusAccepted, task)
}
} }
func extractFields(c *gin.Context) logrus.Fields { func extractFields(c *gin.Context) logrus.Fields {
@@ -305,10 +229,6 @@ func (s *Server) startGears(ctx context.Context) {
// By default it serves on :8080 unless a // By default it serves on :8080 unless a
// PORT environment variable was defined. // PORT environment variable was defined.
listen := fmt.Sprintf(":%d", viper.GetInt(EnvPort)) listen := fmt.Sprintf(":%d", viper.GetInt(EnvPort))
listener, err := net.Listen("tcp", listen)
if err != nil {
logrus.WithError(err).Fatalln("Failed to serve functions API.")
}
const runHeader = ` const runHeader = `
______ ______
@@ -320,29 +240,23 @@ func (s *Server) startGears(ctx context.Context) {
fmt.Println(runHeader) fmt.Println(runHeader)
logrus.Infof("Serving Functions API on address `%s`", listen) logrus.Infof("Serving Functions API on address `%s`", listen)
svr := &supervisor.Supervisor{ server := http.Server{
MaxRestarts: supervisor.AlwaysRestart, Addr: listen,
Log: func(msg interface{}) { Handler: s.Router,
logrus.Debug("supervisor: ", msg) // TODO we should set read/write timeouts
},
} }
svr.AddFunc(func(ctx context.Context) { go func() {
go func() { <-ctx.Done() // listening for signals...
err := http.Serve(listener, s.Router) server.Shutdown(context.Background()) // we can wait
if err != nil { }()
logrus.Fatalf("Error serving API: %v", err)
}
}()
<-ctx.Done()
})
svr.AddFunc(func(ctx context.Context) { err := server.ListenAndServe()
runner.RunAsyncRunner(ctx, s.apiURL, s.Runner, s.Datastore) if err != nil {
}) logrus.WithError(err).Error("error opening server")
}
svr.Serve(ctx) s.Agent.Close() // after we stop taking requests, wait for all tasks to finish
s.Runner.Wait() // wait for tasks to finish (safe shutdown)
} }
func (s *Server) bindHandlers(ctx context.Context) { func (s *Server) bindHandlers(ctx context.Context) {
@@ -380,8 +294,6 @@ func (s *Server) bindHandlers(ctx context.Context) {
} }
} }
engine.DELETE("/tasks", s.handleTaskRequest)
engine.GET("/tasks", s.handleTaskRequest)
engine.Any("/r/:app", s.handleRunnerRequest) engine.Any("/r/:app", s.handleRunnerRequest)
engine.Any("/r/:app/*route", s.handleRunnerRequest) engine.Any("/r/:app/*route", s.handleRunnerRequest)
@@ -397,8 +309,8 @@ type appResponse struct {
} }
type appsResponse struct { type appsResponse struct {
Message string `json:"message"` Message string `json:"message"`
Apps models.Apps `json:"apps"` Apps []*models.App `json:"apps"`
} }
type routeResponse struct { type routeResponse struct {
@@ -407,26 +319,21 @@ type routeResponse struct {
} }
type routesResponse struct { type routesResponse struct {
Message string `json:"message"` Message string `json:"message"`
Routes models.Routes `json:"routes"` Routes []*models.Route `json:"routes"`
}
type tasksResponse struct {
Message string `json:"message"`
Task models.Task `json:"tasksResponse"`
} }
type fnCallResponse struct { type fnCallResponse struct {
Message string `json:"message"` Message string `json:"message"`
Call *models.FnCall `json:"call"` Call *models.Call `json:"call"`
} }
type fnCallsResponse struct { type fnCallsResponse struct {
Message string `json:"message"` Message string `json:"message"`
Calls models.FnCalls `json:"calls"` Calls []*models.Call `json:"calls"`
} }
type fnCallLogResponse struct { type fnCallLogResponse struct {
Message string `json:"message"` Message string `json:"message"`
Log *models.FnCallLog `json:"log"` Log *models.CallLog `json:"log"`
} }

View File

@@ -10,29 +10,25 @@ import (
"net/http/httptest" "net/http/httptest"
"os" "os"
"testing" "testing"
"time"
"github.com/fnproject/fn/api/agent"
"github.com/fnproject/fn/api/datastore" "github.com/fnproject/fn/api/datastore"
"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/fnproject/fn/api/runner"
"github.com/gin-gonic/gin" "github.com/gin-gonic/gin"
cache "github.com/patrickmn/go-cache"
) )
var tmpDatastoreTests = "/tmp/func_test_datastore.db" var tmpDatastoreTests = "/tmp/func_test_datastore.db"
func testServer(ds models.Datastore, mq models.MessageQueue, logDB models.FnLog, rnr *runner.Runner, enqueue models.Enqueue) *Server { func testServer(ds models.Datastore, mq models.MessageQueue, logDB models.LogStore, rnr agent.Agent) *Server {
ctx := context.Background() ctx := context.Background()
s := &Server{ s := &Server{
Runner: rnr, Agent: rnr,
Router: gin.New(), Router: gin.New(),
Datastore: ds, Datastore: ds,
LogDB: logDB, LogDB: logDB,
MQ: mq, MQ: mq,
Enqueue: enqueue,
routeCache: cache.New(60*time.Second, 5*time.Minute),
} }
r := s.Router r := s.Router
@@ -81,7 +77,7 @@ func getErrorResponse(t *testing.T, rec *httptest.ResponseRecorder) models.Error
return errResp return errResp
} }
func prepareDB(ctx context.Context, t *testing.T) (models.Datastore, models.FnLog, func()) { func prepareDB(ctx context.Context, t *testing.T) (models.Datastore, models.LogStore, func()) {
os.Remove(tmpDatastoreTests) os.Remove(tmpDatastoreTests)
ds, err := datastore.New("sqlite3://" + tmpDatastoreTests) ds, err := datastore.New("sqlite3://" + tmpDatastoreTests)
if err != nil { if err != nil {
@@ -99,10 +95,10 @@ func TestFullStack(t *testing.T) {
ds, logDB, close := prepareDB(ctx, t) ds, logDB, close := prepareDB(ctx, t)
defer close() defer close()
rnr, rnrcancel := testRunner(t) rnr, rnrcancel := testRunner(t, ds)
defer rnrcancel() defer rnrcancel()
srv := testServer(ds, &mqs.Mock{}, logDB, rnr, DefaultEnqueue) srv := testServer(ds, &mqs.Mock{}, logDB, rnr)
for _, test := range []struct { for _, test := range []struct {
name string name string
@@ -110,7 +106,7 @@ func TestFullStack(t *testing.T) {
path string path string
body string body string
expectedCode int expectedCode int
expectedCacheSize int expectedCacheSize int // TODO kill me
}{ }{
{"create my app", "POST", "/v1/apps", `{ "app": { "name": "myapp" } }`, http.StatusOK, 0}, {"create my app", "POST", "/v1/apps", `{ "app": { "name": "myapp" } }`, http.StatusOK, 0},
{"list apps", "GET", "/v1/apps", ``, http.StatusOK, 0}, {"list apps", "GET", "/v1/apps", ``, http.StatusOK, 0},
@@ -138,10 +134,5 @@ func TestFullStack(t *testing.T) {
t.Errorf("Test \"%s\": Expected status code to be %d but was %d", t.Errorf("Test \"%s\": Expected status code to be %d but was %d",
test.name, test.expectedCode, rec.Code) test.name, test.expectedCode, rec.Code)
} }
if srv.routeCache.ItemCount() != test.expectedCacheSize {
t.Log(buf.String())
t.Errorf("Test \"%s\": Expected cache size to be %d but was %d",
test.name, test.expectedCacheSize, srv.routeCache.ItemCount())
}
} }
} }

View File

@@ -7,5 +7,5 @@ import (
) )
func (s *Server) handleStats(c *gin.Context) { func (s *Server) handleStats(c *gin.Context) {
c.JSON(http.StatusOK, s.Runner.Stats()) c.JSON(http.StatusOK, s.Agent.Stats())
} }

View File

@@ -44,7 +44,7 @@ building the function.
`type` (optional) allows you to set the type of the route. `sync`, for functions `type` (optional) allows you to set the type of the route. `sync`, for functions
whose response are sent back to the requester; or `async`, for functions that whose response are sent back to the requester; or `async`, for functions that
are started and return a task ID to customer while it executes in background. are started and return a call ID to customer while it executes in background.
Default: `sync`. Default: `sync`.
`memory` (optional) allows you to set a maximum memory threshold for this `memory` (optional) allows you to set a maximum memory threshold for this

View File

@@ -20,7 +20,7 @@ The format is still up for discussion and in order to move forward and remain fl
#### Default I/O Format #### Default I/O Format
The default I/O format is simply the request body itself plus some environment variables. For instance, if someone were to post a JSON body, the unmodified body would be sent in via STDIN. The result comes via STDOUT. When task is done, pipes are closed and the container running the function is terminated. The default I/O format is simply the request body itself plus some environment variables. For instance, if someone were to post a JSON body, the unmodified body would be sent in via STDIN. The result comes via STDOUT. When call is done, pipes are closed and the container running the function is terminated.
Pros: Pros:

View File

@@ -16,7 +16,7 @@ expects to be an integer. Default: `128`.
`type` is the type of the function. Either `sync`, in which the client waits `type` is the type of the function. Either `sync`, in which the client waits
until the request is successfully completed, or `async`, in which the clients until the request is successfully completed, or `async`, in which the clients
dispatches a new request, gets a task ID back and closes the HTTP connection. dispatches a new request, gets a call ID back and closes the HTTP connection.
Default: `sync`. Default: `sync`.
`config` is a map of values passed to the route runtime in the form of `config` is a map of values passed to the route runtime in the form of

View File

@@ -39,7 +39,6 @@ container_ip)
esac esac
go test -v $(go list ./... | grep -v vendor | grep -v examples | grep -v tool | grep -v cli | grep -v tmp/go/src) go test -v $(go list ./... | grep -v vendor | grep -v examples | grep -v tool | grep -v cli | grep -v tmp/go/src)
# go test -v github.com/fnproject/fn/api/runner/drivers/docker
docker rm --force func-postgres-test docker rm --force func-postgres-test
docker rm --force func-mysql-test docker rm --force func-mysql-test

View File

@@ -25,7 +25,7 @@ type TimeoutBody struct {
func CallAsync(t *testing.T, u url.URL, content io.Reader) string { func CallAsync(t *testing.T, u url.URL, content io.Reader) string {
output := &bytes.Buffer{} output := &bytes.Buffer{}
err := CallFN(u.String(), content, output, "POST", []string{}) _, err := CallFN(u.String(), content, output, "POST", []string{})
if err != nil { if err != nil {
t.Errorf("Got unexpected error: %v", err) t.Errorf("Got unexpected error: %v", err)
} }
@@ -67,7 +67,7 @@ func TestRouteExecutions(t *testing.T) {
content := &bytes.Buffer{} content := &bytes.Buffer{}
output := &bytes.Buffer{} output := &bytes.Buffer{}
err := CallFN(u.String(), content, output, "POST", []string{}) _, err := CallFN(u.String(), content, output, "POST", []string{})
if err != nil { if err != nil {
t.Errorf("Got unexpected error: %v", err) t.Errorf("Got unexpected error: %v", err)
} }
@@ -97,7 +97,7 @@ func TestRouteExecutions(t *testing.T) {
Name string Name string
}{Name: "John"}) }{Name: "John"})
output := &bytes.Buffer{} output := &bytes.Buffer{}
err := CallFN(u.String(), content, output, "POST", []string{}) _, err := CallFN(u.String(), content, output, "POST", []string{})
if err != nil { if err != nil {
t.Errorf("Got unexpected error: %v", err) t.Errorf("Got unexpected error: %v", err)
} }
@@ -216,17 +216,14 @@ func TestRouteExecutions(t *testing.T) {
}{Seconds: 31}) }{Seconds: 31})
output := &bytes.Buffer{} output := &bytes.Buffer{}
CallFN(u.String(), content, output, "POST", []string{}) headers, _ := CallFN(u.String(), content, output, "POST", []string{})
if !strings.Contains(output.String(), "Timed out") { if !strings.Contains(output.String(), "Timed out") {
t.Errorf("Must fail because of timeout, but got error message: %v", output.String()) t.Errorf("Must fail because of timeout, but got error message: %v", output.String())
} }
tB := &TimeoutBody{}
json.NewDecoder(output).Decode(tB)
cfg := &call.GetAppsAppCallsCallParams{ cfg := &call.GetAppsAppCallsCallParams{
Call: tB.CallID, Call: headers.Get("FN_CALL_ID"),
App: s.AppName, App: s.AppName,
Context: s.Context, Context: s.Context,
} }

View File

@@ -180,7 +180,7 @@ func EnvAsHeader(req *http.Request, selectedEnv []string) {
} }
} }
func CallFN(u string, content io.Reader, output io.Writer, method string, env []string) error { func CallFN(u string, content io.Reader, output io.Writer, method string, env []string) (http.Header, error) {
if method == "" { if method == "" {
if content == nil { if content == nil {
method = "GET" method = "GET"
@@ -191,7 +191,7 @@ func CallFN(u string, content io.Reader, output io.Writer, method string, env []
req, err := http.NewRequest(method, u, content) req, err := http.NewRequest(method, u, content)
if err != nil { if err != nil {
return fmt.Errorf("error running route: %s", err) return nil, fmt.Errorf("error running route: %s", err)
} }
req.Header.Set("Content-Type", "application/json") req.Header.Set("Content-Type", "application/json")
@@ -202,12 +202,12 @@ func CallFN(u string, content io.Reader, output io.Writer, method string, env []
resp, err := http.DefaultClient.Do(req) resp, err := http.DefaultClient.Do(req)
if err != nil { if err != nil {
return fmt.Errorf("error running route: %s", err) return nil, fmt.Errorf("error running route: %s", err)
} }
io.Copy(output, resp.Body) io.Copy(output, resp.Body)
return nil return resp.Header, nil
} }
func init() { func init() {