mirror of
https://github.com/fnproject/fn.git
synced 2022-10-28 21:29:17 +03:00
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:
committed by
Denis Makogon
parent
1b1b64436f
commit
71a88a991c
@@ -1,2 +0,0 @@
|
||||
vendor/
|
||||
_vendor*/
|
||||
15
Dockerfile
15
Dockerfile
@@ -3,9 +3,18 @@ FROM golang:alpine AS build-env
|
||||
RUN apk --no-cache add build-base git bzr mercurial gcc
|
||||
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
|
||||
RUN go get -u github.com/Masterminds/glide
|
||||
ADD glide.* $D/
|
||||
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!!!!!!!
|
||||
# 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
|
||||
RUN cd $D && go build -o fn-alpine && cp fn-alpine /tmp/
|
||||
|
||||
|
||||
2
Makefile
2
Makefile
@@ -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-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 --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
750
api/agent/agent.go
Normal 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
63
api/agent/async.go
Normal 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
359
api/agent/call.go
Normal 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)
|
||||
}
|
||||
@@ -7,13 +7,13 @@ import (
|
||||
"os"
|
||||
"path"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
"github.com/fnproject/fn/api/runner/drivers"
|
||||
"github.com/fnproject/fn/api/agent/drivers"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fsouza/go-dockerclient"
|
||||
"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
|
||||
@@ -34,7 +34,6 @@ type Auther interface {
|
||||
type runResult struct {
|
||||
error
|
||||
status string
|
||||
start time.Time
|
||||
}
|
||||
|
||||
func (r *runResult) Error() string {
|
||||
@@ -44,33 +43,48 @@ func (r *runResult) Error() string {
|
||||
return r.error.Error()
|
||||
}
|
||||
|
||||
func (r *runResult) Status() string { return r.status }
|
||||
func (r *runResult) UserVisible() bool { return common.IsUserVisibleError(r.error) }
|
||||
func (r *runResult) StartTime() time.Time { return r.start }
|
||||
func (r *runResult) Status() string { return r.status }
|
||||
func (r *runResult) UserVisible() bool { return common.IsUserVisibleError(r.error) }
|
||||
|
||||
type DockerDriver struct {
|
||||
conf drivers.Config
|
||||
docker dockerClient // retries on *docker.Client, restricts ad hoc *docker.Client usage / retries
|
||||
hostname string
|
||||
|
||||
*common.Environment
|
||||
auths map[string]docker.AuthConfiguration
|
||||
}
|
||||
|
||||
// implements drivers.Driver
|
||||
func NewDocker(env *common.Environment, conf drivers.Config) *DockerDriver {
|
||||
func NewDocker(conf drivers.Config) *DockerDriver {
|
||||
hostname, err := os.Hostname()
|
||||
if err != nil {
|
||||
logrus.WithError(err).Fatal("couldn't resolve hostname")
|
||||
}
|
||||
|
||||
return &DockerDriver{
|
||||
conf: conf,
|
||||
docker: newClient(env),
|
||||
hostname: hostname,
|
||||
Environment: env,
|
||||
conf: conf,
|
||||
docker: newClient(),
|
||||
hostname: hostname,
|
||||
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) {
|
||||
ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"stack": "Prepare"})
|
||||
var cmd []string
|
||||
@@ -85,9 +99,8 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
|
||||
envvars = append(envvars, name+"="+val)
|
||||
}
|
||||
|
||||
containerName := newContainerID(task)
|
||||
container := docker.CreateContainerOptions{
|
||||
Name: containerName,
|
||||
Name: task.Id(),
|
||||
Config: &docker.Config{
|
||||
Env: envvars,
|
||||
Cmd: cmd,
|
||||
@@ -96,7 +109,6 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
|
||||
Hostname: drv.hostname,
|
||||
Image: task.Image(),
|
||||
Volumes: map[string]struct{}{},
|
||||
Labels: task.Labels(),
|
||||
OpenStdin: true,
|
||||
AttachStdin: true,
|
||||
StdinOnce: true,
|
||||
@@ -131,7 +143,7 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
|
||||
if err != docker.ErrContainerAlreadyExists {
|
||||
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,
|
||||
"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")
|
||||
|
||||
if ce := containerConfigError(err); ce != nil {
|
||||
@@ -142,7 +154,7 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
|
||||
}
|
||||
|
||||
// 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 {
|
||||
@@ -151,9 +163,11 @@ type cookie struct {
|
||||
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)
|
||||
}
|
||||
|
||||
@@ -174,6 +188,15 @@ func (drv *DockerDriver) ensureImage(ctx context.Context, task drivers.Container
|
||||
// validate creds even if the image is downloaded.
|
||||
|
||||
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 {
|
||||
var err error
|
||||
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.
|
||||
// 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()
|
||||
|
||||
var cancel context.CancelFunc
|
||||
@@ -253,22 +276,34 @@ func (drv *DockerDriver) run(ctx context.Context, container string, task drivers
|
||||
return nil, err
|
||||
}
|
||||
|
||||
start := time.Now()
|
||||
|
||||
err = drv.startTask(ctx, container)
|
||||
if err != nil && ctx.Err() == nil {
|
||||
// if there's just a timeout making the docker calls, drv.wait below will rewrite it to timeout
|
||||
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() {
|
||||
waiter.Close()
|
||||
waiter.Wait() // make sure we gather all logs
|
||||
w.waiter.Close()
|
||||
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{
|
||||
start: start,
|
||||
status: status,
|
||||
error: err,
|
||||
}, 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 {
|
||||
log := common.Logger(ctx)
|
||||
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 {
|
||||
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:
|
||||
return drivers.StatusSuccess, nil
|
||||
case 137: // OOM
|
||||
drv.Inc("docker", "oom", 1, 1)
|
||||
opentracing.SpanFromContext(ctx).LogFields(log.String("docker", "oom"))
|
||||
return drivers.StatusKilled, drivers.ErrOutOfMemory
|
||||
}
|
||||
}
|
||||
@@ -12,11 +12,11 @@ import (
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fsouza/go-dockerclient"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/opentracing/opentracing-go/log"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
const (
|
||||
@@ -42,7 +42,7 @@ type dockerClient interface {
|
||||
}
|
||||
|
||||
// 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
|
||||
// docker, err := docker.NewClient(conf.Docker)
|
||||
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")
|
||||
}
|
||||
|
||||
return &dockerWrap{client, clientNoTimeout, env}
|
||||
return &dockerWrap{client, clientNoTimeout}
|
||||
}
|
||||
|
||||
type dockerWrap struct {
|
||||
docker *docker.Client
|
||||
dockerNoTimeout *docker.Client
|
||||
*common.Environment
|
||||
}
|
||||
|
||||
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++ {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
d.Inc("task", "fail.docker", 1, 1)
|
||||
logger.WithError(ctx.Err()).Warnf("retrying on docker errors timed out, restart docker or rotate this instance?")
|
||||
span.LogFields(log.String("task", "fail.docker"))
|
||||
logger.WithError(ctx.Err()).Warnf("docker call timed out")
|
||||
return ctx.Err()
|
||||
default:
|
||||
}
|
||||
@@ -116,11 +115,11 @@ func (d *dockerWrap) retry(ctx context.Context, f func() error) error {
|
||||
if common.IsTemporary(err) || isDocker50x(err) {
|
||||
logger.WithError(err).Warn("docker temporary error, retrying")
|
||||
b.Sleep()
|
||||
d.Inc("task", "error.docker", 1, 1)
|
||||
span.LogFields(log.String("task", "tmperror.docker"))
|
||||
continue
|
||||
}
|
||||
if err != nil {
|
||||
d.Inc("task", "error.docker", 1, 1)
|
||||
span.LogFields(log.String("task", "error.docker"))
|
||||
}
|
||||
return err
|
||||
}
|
||||
@@ -8,8 +8,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
"github.com/fnproject/fn/api/runner/drivers"
|
||||
"github.com/fnproject/fn/api/agent/drivers"
|
||||
"github.com/fsouza/go-dockerclient"
|
||||
)
|
||||
|
||||
@@ -37,8 +36,7 @@ func (f *taskDockerTest) Close() {}
|
||||
func (f *taskDockerTest) Input() io.Reader { return f.input }
|
||||
|
||||
func TestRunnerDocker(t *testing.T) {
|
||||
env := common.NewEnvironment(func(e *common.Environment) {})
|
||||
dkr := NewDocker(env, drivers.Config{})
|
||||
dkr := NewDocker(drivers.Config{})
|
||||
ctx := context.Background()
|
||||
|
||||
task := &taskDockerTest{"test-docker", nil, nil}
|
||||
@@ -49,7 +47,12 @@ func TestRunnerDocker(t *testing.T) {
|
||||
}
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@@ -60,8 +63,7 @@ func TestRunnerDocker(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestRunnerDockerStdin(t *testing.T) {
|
||||
env := common.NewEnvironment(func(e *common.Environment) {})
|
||||
dkr := NewDocker(env, drivers.Config{})
|
||||
dkr := NewDocker(drivers.Config{})
|
||||
ctx := context.Background()
|
||||
|
||||
input := `{"name": "test"}`
|
||||
@@ -75,7 +77,12 @@ func TestRunnerDockerStdin(t *testing.T) {
|
||||
}
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@@ -19,7 +19,7 @@ import (
|
||||
"github.com/docker/distribution/registry/client/auth"
|
||||
"github.com/docker/distribution/registry/client/auth/challenge"
|
||||
"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"
|
||||
)
|
||||
|
||||
@@ -28,7 +28,15 @@ type Cookie interface {
|
||||
//
|
||||
// Run() MUST monitor the context. task cancellation is indicated by
|
||||
// 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 {
|
||||
@@ -51,11 +59,6 @@ type RunResult interface {
|
||||
// Status should return the current status of the task.
|
||||
// Only valid options are {"error", "success", "timeout", "killed", "cancelled"}.
|
||||
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
|
||||
@@ -73,9 +76,6 @@ type ContainerTask interface {
|
||||
// Input feeds the container with data
|
||||
Input() io.Reader
|
||||
|
||||
// Labels returns container label key-value pairs.
|
||||
Labels() map[string]string
|
||||
|
||||
// The id to assign the container
|
||||
Id() string
|
||||
|
||||
@@ -5,7 +5,7 @@ import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/fnproject/fn/api/runner/drivers"
|
||||
"github.com/fnproject/fn/api/agent/drivers"
|
||||
)
|
||||
|
||||
func New() drivers.Driver {
|
||||
@@ -26,7 +26,7 @@ type cookie struct {
|
||||
|
||||
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++
|
||||
if c.m.count%100 == 0 {
|
||||
return nil, fmt.Errorf("Mocker error! Bad.")
|
||||
@@ -44,5 +44,6 @@ type runResult struct {
|
||||
start time.Time
|
||||
}
|
||||
|
||||
func (r *runResult) Status() string { return r.status }
|
||||
func (r *runResult) StartTime() time.Time { return r.start }
|
||||
func (r *runResult) Wait(context.Context) (drivers.RunResult, error) { return r, nil }
|
||||
func (r *runResult) Status() string { return r.status }
|
||||
func (r *runResult) StartTime() time.Time { return r.start }
|
||||
@@ -1,4 +1,4 @@
|
||||
package runner
|
||||
package agent
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
@@ -8,45 +8,32 @@ import (
|
||||
"io"
|
||||
"sync"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"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 {
|
||||
Writer(ctx context.Context, appName, path, image, reqID string) io.WriteCloser
|
||||
}
|
||||
|
||||
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) }},
|
||||
}
|
||||
}
|
||||
// TODO we can have different types of these func loggers later
|
||||
// TODO move this to a different package
|
||||
|
||||
// DefaultFuncLogger returns a WriteCloser that writes STDERR output from a
|
||||
// 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.
|
||||
type DefaultFuncLogger struct {
|
||||
logDB models.FnLog
|
||||
bufPool *sync.Pool // these are usually small, for buffering lines
|
||||
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)
|
||||
func NewFuncLogger(ctx context.Context, appName, path, image, reqID string, logDB models.LogStore) io.WriteCloser {
|
||||
lbuf := bufPool.Get().(*bytes.Buffer)
|
||||
dbuf := logPool.Get().(*bytes.Buffer)
|
||||
|
||||
close := func() error {
|
||||
// TODO we may want to toss out buffers that grow to grotesque size but meh they will prob get GC'd
|
||||
lbuf.Reset()
|
||||
dbuf.Reset()
|
||||
l.bufPool.Put(lbuf)
|
||||
l.logPool.Put(dbuf)
|
||||
bufPool.Put(lbuf)
|
||||
logPool.Put(dbuf)
|
||||
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
|
||||
limitw := newLimitWriter(MB, &dbWriter{
|
||||
Buffer: dbuf,
|
||||
db: l.logDB,
|
||||
db: logDB,
|
||||
ctx: ctx,
|
||||
reqID: reqID,
|
||||
})
|
||||
@@ -189,7 +176,7 @@ func (li *lineWriter) Close() error {
|
||||
type dbWriter struct {
|
||||
*bytes.Buffer
|
||||
|
||||
db models.FnLog
|
||||
db models.LogStore
|
||||
ctx context.Context
|
||||
reqID string
|
||||
}
|
||||
97
api/agent/mem.go
Normal file
97
api/agent/mem.go
Normal 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
|
||||
}
|
||||
12
api/agent/protocol/default.go
Normal file
12
api/agent/protocol/default.go
Normal 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 }
|
||||
@@ -1,24 +1,32 @@
|
||||
package protocol
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"io"
|
||||
"net/http"
|
||||
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/fnproject/fn/api/runner/task"
|
||||
)
|
||||
|
||||
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.
|
||||
// Internally, a protocol must know when to alternate between stdin and stdout.
|
||||
// It returns any protocol error, if present.
|
||||
type ContainerIO interface {
|
||||
IsStreamable() bool
|
||||
|
||||
// TODO this should take a drivers.ContainerTask?
|
||||
Dispatch(ctx context.Context, t *task.Config) error
|
||||
// Dispatch will handle sending stdin and stdout to a container. Implementers
|
||||
// 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.
|
||||
@@ -53,12 +61,6 @@ func (p Protocol) MarshalJSON() ([]byte, error) {
|
||||
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
|
||||
// stdin/stdout.
|
||||
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:
|
||||
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
|
||||
// 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
144
api/agent/protocol/http.go
Normal 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
|
||||
}
|
||||
@@ -1,14 +1,16 @@
|
||||
package runner
|
||||
package agent
|
||||
|
||||
import "sync"
|
||||
|
||||
// TODO this should expose:
|
||||
// * hot containers active
|
||||
// * memory used / available
|
||||
|
||||
type stats struct {
|
||||
mu sync.Mutex
|
||||
queue uint64
|
||||
running uint64
|
||||
complete uint64
|
||||
|
||||
wait sync.WaitGroup
|
||||
}
|
||||
|
||||
type Stats struct {
|
||||
@@ -24,7 +26,6 @@ func (s *stats) Enqueue() {
|
||||
}
|
||||
|
||||
func (s *stats) Start() {
|
||||
s.wait.Add(1)
|
||||
s.mu.Lock()
|
||||
s.queue--
|
||||
s.running++
|
||||
@@ -32,7 +33,6 @@ func (s *stats) Start() {
|
||||
}
|
||||
|
||||
func (s *stats) Complete() {
|
||||
s.wait.Done()
|
||||
s.mu.Lock()
|
||||
s.running--
|
||||
s.complete++
|
||||
@@ -48,5 +48,3 @@ func (s *stats) Stats() Stats {
|
||||
s.mu.Unlock()
|
||||
return stats
|
||||
}
|
||||
|
||||
func (s *stats) Wait() { s.wait.Wait() }
|
||||
@@ -2,7 +2,10 @@
|
||||
// Use of this source code is governed by a BSD-style license that can be found
|
||||
// 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 (
|
||||
"net/http"
|
||||
@@ -6,7 +6,7 @@ import (
|
||||
"syscall"
|
||||
)
|
||||
|
||||
// Errors that can be directly exposed to task creators/users.
|
||||
// Errors that can be directly exposed to call creators/users.
|
||||
type UserVisibleError interface {
|
||||
UserVisible() bool
|
||||
}
|
||||
@@ -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
|
||||
|
||||
@@ -13,7 +15,7 @@ type call struct {
|
||||
err error
|
||||
}
|
||||
|
||||
type singleflight struct {
|
||||
type SingleFlight struct {
|
||||
mu sync.Mutex // protects m
|
||||
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
|
||||
// time. If a duplicate comes in, the duplicate caller waits for the
|
||||
// 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()
|
||||
if g.m == nil {
|
||||
g.m = make(map[interface{}]*call)
|
||||
@@ -33,47 +33,47 @@ func Test(t *testing.T, ds models.Datastore) {
|
||||
|
||||
ctx := context.Background()
|
||||
|
||||
task := &models.Task{}
|
||||
task.CreatedAt = strfmt.DateTime(time.Now())
|
||||
task.Status = "success"
|
||||
task.StartedAt = strfmt.DateTime(time.Now())
|
||||
task.CompletedAt = strfmt.DateTime(time.Now())
|
||||
task.AppName = testApp.Name
|
||||
task.Path = testRoute.Path
|
||||
call := new(models.Call)
|
||||
call.CreatedAt = strfmt.DateTime(time.Now())
|
||||
call.Status = "success"
|
||||
call.StartedAt = strfmt.DateTime(time.Now())
|
||||
call.CompletedAt = strfmt.DateTime(time.Now())
|
||||
call.AppName = testApp.Name
|
||||
call.Path = testRoute.Path
|
||||
|
||||
t.Run("call-insert", func(t *testing.T) {
|
||||
task.ID = id.New().String()
|
||||
err := ds.InsertTask(ctx, task)
|
||||
call.ID = id.New().String()
|
||||
err := ds.InsertCall(ctx, call)
|
||||
if err != nil {
|
||||
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) {
|
||||
task.ID = id.New().String()
|
||||
ds.InsertTask(ctx, task)
|
||||
newTask, err := ds.GetTask(ctx, task.ID)
|
||||
call.ID = id.New().String()
|
||||
ds.InsertCall(ctx, call)
|
||||
newCall, err := ds.GetCall(ctx, call.AppName, call.ID)
|
||||
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.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) {
|
||||
filter := &models.CallFilter{AppName: task.AppName, Path: task.Path}
|
||||
task.ID = id.New().String()
|
||||
ds.InsertTask(ctx, task)
|
||||
calls, err := ds.GetTasks(ctx, filter)
|
||||
filter := &models.CallFilter{AppName: call.AppName, Path: call.Path}
|
||||
call.ID = id.New().String()
|
||||
ds.InsertCall(ctx, call)
|
||||
calls, err := ds.GetCalls(ctx, filter)
|
||||
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 {
|
||||
t.Log(buf.String())
|
||||
t.Fatalf("Test GetTasks(ctx, filter): unexpected error `%v`", err)
|
||||
t.Fatalf("Test GetCalls(ctx, filter): unexpected error `%v`", err)
|
||||
}
|
||||
})
|
||||
|
||||
|
||||
@@ -82,22 +82,22 @@ func (m *metricds) RemoveRoute(ctx context.Context, appName, routePath string) e
|
||||
return m.ds.RemoveRoute(ctx, appName, routePath)
|
||||
}
|
||||
|
||||
func (m *metricds) InsertTask(ctx context.Context, task *models.Task) error {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_insert_task")
|
||||
func (m *metricds) InsertCall(ctx context.Context, call *models.Call) error {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_insert_call")
|
||||
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) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_task")
|
||||
func (m *metricds) GetCall(ctx context.Context, appName, callID string) (*models.Call, error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_call")
|
||||
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) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_tasks")
|
||||
func (m *metricds) GetCalls(ctx context.Context, filter *models.CallFilter) ([]*models.Call, error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "ds_get_calls")
|
||||
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 {
|
||||
@@ -106,7 +106,7 @@ func (m *metricds) InsertLog(ctx context.Context, callID string, callLog string)
|
||||
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")
|
||||
defer span.Finish()
|
||||
return m.ds.GetLog(ctx, callID)
|
||||
|
||||
@@ -131,11 +131,11 @@ func (v *validator) RemoveRoute(ctx context.Context, appName, routePath string)
|
||||
}
|
||||
|
||||
// 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 == "" {
|
||||
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 {
|
||||
|
||||
@@ -11,31 +11,19 @@ import (
|
||||
)
|
||||
|
||||
type mock struct {
|
||||
Apps models.Apps
|
||||
Routes models.Routes
|
||||
Calls models.FnCalls
|
||||
Apps []*models.App
|
||||
Routes []*models.Route
|
||||
Calls []*models.Call
|
||||
data map[string][]byte
|
||||
|
||||
models.FnLog
|
||||
models.LogStore
|
||||
}
|
||||
|
||||
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 {
|
||||
if apps == nil {
|
||||
apps = models.Apps{}
|
||||
}
|
||||
if routes == nil {
|
||||
routes = models.Routes{}
|
||||
}
|
||||
if calls == nil {
|
||||
calls = models.FnCalls{}
|
||||
}
|
||||
if loggos == nil {
|
||||
loggos = []*models.FnCallLog{}
|
||||
}
|
||||
func NewMockInit(apps []*models.App, routes []*models.Route, calls []*models.Call) models.Datastore {
|
||||
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
|
||||
}
|
||||
|
||||
func (m *mock) InsertTask(ctx context.Context, task *models.Task) error {
|
||||
var call *models.FnCall
|
||||
m.Calls = append(m.Calls, call.FromTask(task))
|
||||
func (m *mock) InsertCall(ctx context.Context, call *models.Call) error {
|
||||
m.Calls = append(m.Calls, call)
|
||||
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 {
|
||||
if t.ID == callID {
|
||||
return t, nil
|
||||
@@ -166,7 +153,7 @@ func (m *mock) GetTask(ctx context.Context, callID string) (*models.FnCall, erro
|
||||
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
|
||||
}
|
||||
|
||||
|
||||
@@ -526,7 +526,7 @@ func (ds *sqlStore) Tx(f func(*sqlx.Tx) error) error {
|
||||
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 (
|
||||
id,
|
||||
created_at,
|
||||
@@ -538,9 +538,9 @@ func (ds *sqlStore) InsertTask(ctx context.Context, task *models.Task) error {
|
||||
)
|
||||
VALUES (?, ?, ?, ?, ?, ?, ?);`)
|
||||
|
||||
_, err := ds.db.Exec(query, task.ID, task.CreatedAt.String(),
|
||||
task.StartedAt.String(), task.CompletedAt.String(),
|
||||
task.Status, task.AppName, task.Path)
|
||||
_, err := ds.db.Exec(query, call.ID, call.CreatedAt.String(),
|
||||
call.StartedAt.String(), call.CompletedAt.String(),
|
||||
call.Status, call.AppName, call.Path)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -548,12 +548,15 @@ func (ds *sqlStore) InsertTask(ctx context.Context, task *models.Task) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ds *sqlStore) GetTask(ctx context.Context, callID string) (*models.FnCall, error) {
|
||||
query := fmt.Sprintf(`%s WHERE id=?`, callSelector)
|
||||
// TODO calls are not fully qualified in this backend currently. need to discuss,
|
||||
// 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)
|
||||
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)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -561,8 +564,8 @@ func (ds *sqlStore) GetTask(ctx context.Context, callID string) (*models.FnCall,
|
||||
return &call, nil
|
||||
}
|
||||
|
||||
func (ds *sqlStore) GetTasks(ctx context.Context, filter *models.CallFilter) (models.FnCalls, error) {
|
||||
res := models.FnCalls{}
|
||||
func (ds *sqlStore) GetCalls(ctx context.Context, filter *models.CallFilter) ([]*models.Call, error) {
|
||||
res := []*models.Call{}
|
||||
query, args := buildFilterCallQuery(filter)
|
||||
query = fmt.Sprintf("%s %s", callSelector, query)
|
||||
query = ds.db.Rebind(query)
|
||||
@@ -573,7 +576,7 @@ func (ds *sqlStore) GetTasks(ctx context.Context, filter *models.CallFilter) (mo
|
||||
defer rows.Close()
|
||||
|
||||
for rows.Next() {
|
||||
var call models.FnCall
|
||||
var call models.Call
|
||||
err := scanCall(rows, &call)
|
||||
if err != nil {
|
||||
continue
|
||||
@@ -592,7 +595,7 @@ func (ds *sqlStore) InsertLog(ctx context.Context, callID, callLog string) error
|
||||
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=?`)
|
||||
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 &models.FnCallLog{
|
||||
return &models.CallLog{
|
||||
CallID: callID,
|
||||
Log: log,
|
||||
}, nil
|
||||
@@ -622,7 +625,7 @@ type RowScanner interface {
|
||||
Scan(dest ...interface{}) error
|
||||
}
|
||||
|
||||
func ScanLog(scanner RowScanner, log *models.FnCallLog) error {
|
||||
func ScanLog(scanner RowScanner, log *models.CallLog) error {
|
||||
return scanner.Scan(
|
||||
&log.CallID,
|
||||
&log.Log,
|
||||
@@ -746,7 +749,7 @@ func buildFilterCallQuery(filter *models.CallFilter) (string, []interface{}) {
|
||||
return b.String(), args
|
||||
}
|
||||
|
||||
func scanCall(scanner RowScanner, call *models.FnCall) error {
|
||||
func scanCall(scanner RowScanner, call *models.Call) error {
|
||||
err := scanner.Scan(
|
||||
&call.ID,
|
||||
&call.CreatedAt,
|
||||
|
||||
@@ -9,7 +9,7 @@ import (
|
||||
"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)
|
||||
if err != nil {
|
||||
logrus.WithError(err).WithFields(logrus.Fields{"url": dbURL}).Fatal("bad DB URL")
|
||||
|
||||
@@ -7,19 +7,19 @@ import (
|
||||
)
|
||||
|
||||
type mock struct {
|
||||
Logs map[string]*models.FnCallLog
|
||||
Logs map[string]*models.CallLog
|
||||
ds models.Datastore
|
||||
}
|
||||
|
||||
func NewMock() models.FnLog {
|
||||
func NewMock() models.LogStore {
|
||||
return NewMockInit(nil)
|
||||
}
|
||||
|
||||
func NewMockInit(logs map[string]*models.FnCallLog) models.FnLog {
|
||||
func NewMockInit(logs map[string]*models.CallLog) models.LogStore {
|
||||
if logs == nil {
|
||||
logs = map[string]*models.FnCallLog{}
|
||||
logs = map[string]*models.CallLog{}
|
||||
}
|
||||
fnl := NewValidator(&mock{logs, nil})
|
||||
fnl := &mock{logs, nil}
|
||||
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 {
|
||||
m.Logs[callID] = &models.FnCallLog{CallID: callID, Log: callLog}
|
||||
m.Logs[callID] = &models.CallLog{CallID: callID, Log: callLog}
|
||||
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]
|
||||
if logEntry == nil {
|
||||
return nil, errors.New("Call log not found")
|
||||
|
||||
@@ -23,68 +23,68 @@ var testRoute = &models.Route{
|
||||
Format: "http",
|
||||
}
|
||||
|
||||
func SetUpTestTask() *models.Task {
|
||||
task := &models.Task{}
|
||||
task.CreatedAt = strfmt.DateTime(time.Now())
|
||||
task.Status = "success"
|
||||
task.StartedAt = strfmt.DateTime(time.Now())
|
||||
task.CompletedAt = strfmt.DateTime(time.Now())
|
||||
task.AppName = testApp.Name
|
||||
task.Path = testRoute.Path
|
||||
return task
|
||||
func SetupTestCall() *models.Call {
|
||||
var call models.Call
|
||||
call.CreatedAt = strfmt.DateTime(time.Now())
|
||||
call.Status = "success"
|
||||
call.StartedAt = strfmt.DateTime(time.Now())
|
||||
call.CompletedAt = strfmt.DateTime(time.Now())
|
||||
call.AppName = testApp.Name
|
||||
call.Path = testRoute.Path
|
||||
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()
|
||||
task := SetUpTestTask()
|
||||
call := SetupTestCall()
|
||||
|
||||
t.Run("call-log-insert", func(t *testing.T) {
|
||||
task.ID = id.New().String()
|
||||
err := ds.InsertTask(ctx, task)
|
||||
call.ID = id.New().String()
|
||||
err := ds.InsertCall(ctx, call)
|
||||
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 {
|
||||
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) {
|
||||
task.ID = id.New().String()
|
||||
err := ds.InsertTask(ctx, task)
|
||||
call.ID = id.New().String()
|
||||
err := ds.InsertCall(ctx, call)
|
||||
logText := "test"
|
||||
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 {
|
||||
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) {
|
||||
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)
|
||||
}
|
||||
})
|
||||
t.Run("call-log-insert-get-delete", func(t *testing.T) {
|
||||
task.ID = id.New().String()
|
||||
err := ds.InsertTask(ctx, task)
|
||||
call.ID = id.New().String()
|
||||
err := ds.InsertCall(ctx, call)
|
||||
logText := "test"
|
||||
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 {
|
||||
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) {
|
||||
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)
|
||||
}
|
||||
err = fnl.DeleteLog(ctx, task.ID)
|
||||
err = fnl.DeleteLog(ctx, call.ID)
|
||||
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)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
@@ -1,9 +1,5 @@
|
||||
package models
|
||||
|
||||
type Apps []*App
|
||||
type Tasks []*Task
|
||||
type FnCalls []*FnCall
|
||||
|
||||
type App struct {
|
||||
Name string `json:"name"`
|
||||
Routes Routes `json:"routes,omitempty"`
|
||||
|
||||
134
api/models/call.go
Normal file
134
api/models/call.go
Normal 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
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -38,11 +38,11 @@ type Datastore interface {
|
||||
GetRoute(ctx context.Context, appName, routePath string) (*Route, error)
|
||||
|
||||
// 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).
|
||||
// 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
|
||||
// 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.
|
||||
RemoveRoute(ctx context.Context, appName, routePath string) error
|
||||
|
||||
// InsertTask inserts a task
|
||||
InsertTask(ctx context.Context, task *Task) error
|
||||
GetTask(ctx context.Context, callID string) (*FnCall, error)
|
||||
GetTasks(ctx context.Context, filter *CallFilter) (FnCalls, error)
|
||||
// InsertCall inserts a call into the datastore, it will error if the call already
|
||||
// exists.
|
||||
InsertCall(ctx context.Context, call *Call) error
|
||||
|
||||
// Implement FnLog methods for convenience
|
||||
FnLog
|
||||
// GetCall returns a call at a certain id and app name.
|
||||
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() *sqlx.DB
|
||||
|
||||
@@ -16,7 +16,7 @@ var (
|
||||
code: http.StatusBadRequest,
|
||||
error: errors.New("Invalid JSON"),
|
||||
}
|
||||
ErrRunnerTimeout = err{
|
||||
ErrCallTimeout = err{
|
||||
code: http.StatusGatewayTimeout,
|
||||
error: errors.New("Timed out"),
|
||||
}
|
||||
@@ -72,9 +72,9 @@ var (
|
||||
code: http.StatusBadRequest,
|
||||
error: errors.New("Missing key"),
|
||||
}
|
||||
ErrDatastoreEmptyTaskID = err{
|
||||
ErrDatastoreEmptyCallID = err{
|
||||
code: http.StatusBadRequest,
|
||||
error: errors.New("Missing task ID"),
|
||||
error: errors.New("Missing call ID"),
|
||||
}
|
||||
ErrInvalidPayload = err{
|
||||
code: http.StatusBadRequest,
|
||||
|
||||
@@ -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
|
||||
}
|
||||
@@ -4,14 +4,18 @@ import (
|
||||
"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
|
||||
// existed.
|
||||
InsertLog(ctx context.Context, callID string, callLog string) error
|
||||
|
||||
// GetLog will return the log at callID, an error will be returned if the log
|
||||
// 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
|
||||
// the log does not exist before removal.
|
||||
|
||||
@@ -3,18 +3,18 @@ package models
|
||||
import "context"
|
||||
|
||||
// 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
|
||||
// timeouts and retries.
|
||||
//
|
||||
// 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
|
||||
// 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.
|
||||
//
|
||||
// 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
|
||||
// priority field. Tasks with higher priority always get pulled off the queue
|
||||
// logical time. Priorities are currently 0-2 and available in the call's
|
||||
// priority field. call with higher priority always get pulled off the queue
|
||||
// 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
|
||||
@@ -22,12 +22,12 @@ import "context"
|
||||
// the head of the queue:
|
||||
// Reserve() leads to A being passed to a consumer, and timeout started.
|
||||
// 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.
|
||||
// A's timeout occurs before the job is finished. At this point the ordering
|
||||
// should be [A, C] and not [C, A].
|
||||
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
|
||||
// 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
|
||||
// = 5 seconds, and the same priority, then they may be available on the
|
||||
// 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
|
||||
// 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
|
||||
// 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
|
||||
// the job does not have an outstanding reservation, error. If a job did not
|
||||
// exist, succeed.
|
||||
Delete(context.Context, *Task) error
|
||||
Delete(context.Context, *Call) error
|
||||
}
|
||||
|
||||
type Enqueue func(context.Context, MessageQueue, *Task) (*Task, error)
|
||||
|
||||
@@ -8,8 +8,8 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
defaultRouteTimeout = 30 // seconds
|
||||
htfnScaleDownTimeout = 30 // seconds
|
||||
DefaultRouteTimeout = 30 // seconds
|
||||
DefaultIdleTimeout = 30 // seconds
|
||||
)
|
||||
|
||||
type Routes []*Route
|
||||
@@ -50,11 +50,11 @@ func (r *Route) SetDefaults() {
|
||||
}
|
||||
|
||||
if r.Timeout == 0 {
|
||||
r.Timeout = defaultRouteTimeout
|
||||
r.Timeout = DefaultRouteTimeout
|
||||
}
|
||||
|
||||
if r.IdleTimeout == 0 {
|
||||
r.IdleTimeout = htfnScaleDownTimeout
|
||||
r.IdleTimeout = DefaultIdleTimeout
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -13,8 +13,8 @@ import (
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/boltdb/bolt"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
)
|
||||
|
||||
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
|
||||
// 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.
|
||||
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)
|
||||
err := mq.db.Update(func(tx *bolt.Tx) error {
|
||||
b := tx.Bucket(delayQueueName)
|
||||
@@ -204,12 +204,12 @@ func (mq *BoltDbMQ) delayTask(job *models.Task) (*models.Task, error) {
|
||||
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})
|
||||
log.Println("Pushed to MQ")
|
||||
|
||||
if job.Delay > 0 {
|
||||
return mq.delayTask(job)
|
||||
return mq.delayCall(job)
|
||||
}
|
||||
|
||||
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:]
|
||||
}
|
||||
|
||||
func (mq *BoltDbMQ) Reserve(ctx context.Context) (*models.Task, error) {
|
||||
func (mq *BoltDbMQ) Reserve(ctx context.Context) (*models.Call, error) {
|
||||
// Start a writable transaction.
|
||||
tx, err := mq.db.Begin(true)
|
||||
if err != nil {
|
||||
@@ -284,7 +284,7 @@ func (mq *BoltDbMQ) Reserve(ctx context.Context) (*models.Task, error) {
|
||||
|
||||
b.Delete(key)
|
||||
|
||||
var job models.Task
|
||||
var job models.Call
|
||||
err = json.Unmarshal([]byte(value), &job)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -324,7 +324,7 @@ func (mq *BoltDbMQ) Reserve(ctx context.Context) (*models.Task, error) {
|
||||
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})
|
||||
defer log.Println("Deleted")
|
||||
|
||||
|
||||
@@ -93,7 +93,7 @@ func NewIronMQ(url *url.URL) *IronMQ {
|
||||
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 {
|
||||
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
|
||||
}
|
||||
|
||||
func (mq *IronMQ) Reserve(ctx context.Context) (*models.Task, error) {
|
||||
var job models.Task
|
||||
func (mq *IronMQ) Reserve(ctx context.Context) (*models.Call, error) {
|
||||
var job models.Call
|
||||
|
||||
var messages []ironmq.Message
|
||||
var err error
|
||||
@@ -148,7 +148,7 @@ func (mq *IronMQ) Reserve(ctx context.Context) (*models.Task, error) {
|
||||
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 {
|
||||
return fmt.Errorf("IronMQ Delete job %s: Bad priority", job.ID)
|
||||
}
|
||||
|
||||
@@ -7,18 +7,18 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
"github.com/google/btree"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type MemoryMQ struct {
|
||||
// WorkQueue A buffered channel that we can send work requests on.
|
||||
PriorityQueues []chan *models.Task
|
||||
PriorityQueues []chan *models.Call
|
||||
Ticker *time.Ticker
|
||||
BTree *btree.BTree
|
||||
Timeouts map[string]*TaskItem
|
||||
Timeouts map[string]*callItem
|
||||
// Protects B-tree and Timeouts
|
||||
// If this becomes a bottleneck, consider separating the two mutexes. The
|
||||
// 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
|
||||
|
||||
func NewMemoryMQ() *MemoryMQ {
|
||||
var queues []chan *models.Task
|
||||
var queues []chan *models.Call
|
||||
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)
|
||||
mq := &MemoryMQ{
|
||||
PriorityQueues: queues,
|
||||
Ticker: ticker,
|
||||
BTree: btree.New(2),
|
||||
Timeouts: make(map[string]*TaskItem, 0),
|
||||
Timeouts: make(map[string]*callItem, 0),
|
||||
}
|
||||
mq.start()
|
||||
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
|
||||
go func() {
|
||||
for range mq.Ticker.C {
|
||||
ji := &TaskItem{
|
||||
ji := &callItem{
|
||||
StartAt: time.Now(),
|
||||
}
|
||||
mq.Mutex.Lock()
|
||||
mq.BTree.AscendLessThan(ji, func(a btree.Item) bool {
|
||||
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
|
||||
_, err := mq.pushForce(ji2.Task)
|
||||
_, err := mq.pushForce(ji2.Call)
|
||||
if err != nil {
|
||||
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.
|
||||
go func() {
|
||||
for range mq.Ticker.C {
|
||||
ji := &TaskItem{
|
||||
ji := &callItem{
|
||||
StartAt: time.Now(),
|
||||
}
|
||||
mq.Mutex.Lock()
|
||||
for _, jobItem := range mq.Timeouts {
|
||||
if jobItem.Less(ji) {
|
||||
delete(mq.Timeouts, jobItem.Task.ID)
|
||||
_, err := mq.pushForce(jobItem.Task)
|
||||
delete(mq.Timeouts, jobItem.Call.ID)
|
||||
_, err := mq.pushForce(jobItem.Call)
|
||||
if err != nil {
|
||||
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
|
||||
type TaskItem struct {
|
||||
Task *models.Task
|
||||
// callItem is for the Btree, implements btree.Item
|
||||
type callItem struct {
|
||||
Call *models.Call
|
||||
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
|
||||
ji2 := than.(*TaskItem)
|
||||
ji2 := than.(*callItem)
|
||||
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.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)
|
||||
if job.Delay > 0 {
|
||||
// then we'll put into short term storage until ready
|
||||
ji := &TaskItem{
|
||||
Task: job,
|
||||
ji := &callItem{
|
||||
Call: job,
|
||||
StartAt: time.Now().Add(time.Second * time.Duration(job.Delay)),
|
||||
}
|
||||
mq.Mutex.Lock()
|
||||
@@ -136,10 +136,10 @@ func (mq *MemoryMQ) Push(ctx context.Context, job *models.Task) (*models.Task, e
|
||||
return mq.pushForce(job)
|
||||
}
|
||||
|
||||
func (mq *MemoryMQ) pushTimeout(job *models.Task) error {
|
||||
func (mq *MemoryMQ) pushTimeout(job *models.Call) error {
|
||||
|
||||
ji := &TaskItem{
|
||||
Task: job,
|
||||
ji := &callItem{
|
||||
Call: job,
|
||||
StartAt: time.Now().Add(time.Minute),
|
||||
}
|
||||
mq.Mutex.Lock()
|
||||
@@ -148,13 +148,13 @@ func (mq *MemoryMQ) pushTimeout(job *models.Task) error {
|
||||
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
|
||||
return job, nil
|
||||
}
|
||||
|
||||
// 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 {
|
||||
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])
|
||||
if job == nil {
|
||||
return nil, nil
|
||||
@@ -178,7 +178,7 @@ func (mq *MemoryMQ) Reserve(ctx context.Context) (*models.Task, error) {
|
||||
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})
|
||||
|
||||
mq.Mutex.Lock()
|
||||
|
||||
@@ -13,14 +13,14 @@ type Mock struct {
|
||||
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
|
||||
}
|
||||
|
||||
func (mock *Mock) Reserve(context.Context) (*models.Task, error) {
|
||||
func (mock *Mock) Reserve(context.Context) (*models.Call, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (mock *Mock) Delete(context.Context, *models.Task) error {
|
||||
func (mock *Mock) Delete(context.Context, *models.Call) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -46,19 +46,19 @@ type metricMQ struct {
|
||||
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")
|
||||
defer span.Finish()
|
||||
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")
|
||||
defer span.Finish()
|
||||
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")
|
||||
defer span.Finish()
|
||||
return m.mq.Delete(ctx, t)
|
||||
|
||||
@@ -9,10 +9,10 @@ import (
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
"github.com/garyburd/redigo/redis"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type RedisMQ struct {
|
||||
@@ -100,7 +100,7 @@ func (mq *RedisMQ) processPendingReservations() {
|
||||
return
|
||||
}
|
||||
|
||||
var job models.Task
|
||||
var job models.Call
|
||||
err = json.Unmarshal(response, &job)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Error("error unmarshaling job json")
|
||||
@@ -114,7 +114,7 @@ func (mq *RedisMQ) processPendingReservations() {
|
||||
redisPush(conn, mq.queueName, &job)
|
||||
}
|
||||
|
||||
func (mq *RedisMQ) processDelayedTasks() {
|
||||
func (mq *RedisMQ) processDelayedCalls() {
|
||||
conn := mq.pool.Get()
|
||||
defer conn.Close()
|
||||
|
||||
@@ -141,7 +141,7 @@ func (mq *RedisMQ) processDelayedTasks() {
|
||||
continue
|
||||
}
|
||||
|
||||
var job models.Task
|
||||
var job models.Call
|
||||
err = json.Unmarshal(buf, &job)
|
||||
if err != nil {
|
||||
logrus.WithError(err).WithFields(logrus.Fields{"buf": buf, "reservationId": resId}).Error("Error unmarshaling job")
|
||||
@@ -164,12 +164,12 @@ func (mq *RedisMQ) start() {
|
||||
go func() {
|
||||
for range mq.ticker.C {
|
||||
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)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -181,7 +181,7 @@ func redisPush(conn redis.Conn, queue string, job *models.Task) (*models.Task, e
|
||||
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)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -209,7 +209,7 @@ func (mq *RedisMQ) delayTask(conn redis.Conn, job *models.Task) (*models.Task, e
|
||||
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})
|
||||
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()
|
||||
|
||||
if job.Delay > 0 {
|
||||
return mq.delayTask(conn, job)
|
||||
return mq.delayCall(conn, 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
|
||||
func (mq *RedisMQ) Reserve(ctx context.Context) (*models.Task, error) {
|
||||
func (mq *RedisMQ) Reserve(ctx context.Context) (*models.Call, error) {
|
||||
|
||||
conn := mq.pool.Get()
|
||||
defer conn.Close()
|
||||
var job models.Task
|
||||
var job models.Call
|
||||
var resp []byte
|
||||
var err error
|
||||
for i := 2; i >= 0; i-- {
|
||||
@@ -286,7 +286,7 @@ func (mq *RedisMQ) Reserve(ctx context.Context) (*models.Task, error) {
|
||||
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})
|
||||
defer log.Println("Deleted")
|
||||
|
||||
|
||||
@@ -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()
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
@@ -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.
|
||||
@@ -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()
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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")
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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())
|
||||
}
|
||||
}
|
||||
@@ -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), ®istries)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// 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
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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 }
|
||||
@@ -4,8 +4,8 @@ import (
|
||||
"net/http"
|
||||
|
||||
"github.com/fnproject/fn/api"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
"github.com/gin-gonic/gin"
|
||||
)
|
||||
|
||||
|
||||
@@ -29,7 +29,7 @@ func TestAppCreate(t *testing.T) {
|
||||
buf := setLogBuffer()
|
||||
for i, test := range []struct {
|
||||
mock models.Datastore
|
||||
logDB models.FnLog
|
||||
logDB models.LogStore
|
||||
path string
|
||||
body string
|
||||
expectedCode int
|
||||
@@ -48,7 +48,7 @@ func TestAppCreate(t *testing.T) {
|
||||
{datastore.NewMock(), logs.NewMock(), "/v1/apps", `{ "app": { "name": "teste" } }`, http.StatusOK, nil},
|
||||
} {
|
||||
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
|
||||
|
||||
body := bytes.NewBuffer([]byte(test.body))
|
||||
@@ -78,7 +78,7 @@ func TestAppDelete(t *testing.T) {
|
||||
|
||||
for i, test := range []struct {
|
||||
ds models.Datastore
|
||||
logDB models.FnLog
|
||||
logDB models.LogStore
|
||||
path string
|
||||
body string
|
||||
expectedCode int
|
||||
@@ -88,11 +88,11 @@ func TestAppDelete(t *testing.T) {
|
||||
{datastore.NewMockInit(
|
||||
[]*models.App{{
|
||||
Name: "myapp",
|
||||
}}, nil, nil, nil,
|
||||
}}, nil, nil,
|
||||
), logs.NewMock(), "/v1/apps/myapp", "", http.StatusOK, nil},
|
||||
} {
|
||||
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)
|
||||
|
||||
@@ -122,7 +122,7 @@ func TestAppList(t *testing.T) {
|
||||
defer cancel()
|
||||
ds := datastore.NewMock()
|
||||
fnl := logs.NewMock()
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
|
||||
|
||||
for i, test := range []struct {
|
||||
path string
|
||||
@@ -159,7 +159,7 @@ func TestAppGet(t *testing.T) {
|
||||
defer cancel()
|
||||
ds := datastore.NewMock()
|
||||
fnl := logs.NewMock()
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
|
||||
|
||||
for i, test := range []struct {
|
||||
path string
|
||||
@@ -194,7 +194,7 @@ func TestAppUpdate(t *testing.T) {
|
||||
|
||||
for i, test := range []struct {
|
||||
mock models.Datastore
|
||||
logDB models.FnLog
|
||||
logDB models.LogStore
|
||||
path string
|
||||
body string
|
||||
expectedCode int
|
||||
@@ -207,18 +207,18 @@ func TestAppUpdate(t *testing.T) {
|
||||
{datastore.NewMockInit(
|
||||
[]*models.App{{
|
||||
Name: "myapp",
|
||||
}}, nil, nil, nil,
|
||||
}}, nil, nil,
|
||||
), logs.NewMock(), "/v1/apps/myapp", `{ "app": { "config": { "test": "1" } } }`, http.StatusOK, nil},
|
||||
|
||||
// Addresses #380
|
||||
{datastore.NewMockInit(
|
||||
[]*models.App{{
|
||||
Name: "myapp",
|
||||
}}, nil, nil, nil,
|
||||
}}, nil, nil,
|
||||
), logs.NewMock(), "/v1/apps/myapp", `{ "app": { "name": "othername" } }`, http.StatusConflict, nil},
|
||||
} {
|
||||
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))
|
||||
_, rec := routerRequest(t, srv.Router, "PATCH", test.path, body)
|
||||
|
||||
@@ -10,8 +10,9 @@ import (
|
||||
func (s *Server) handleCallGet(c *gin.Context) {
|
||||
ctx := c.Request.Context()
|
||||
|
||||
appName := c.MustGet(api.AppName).(string)
|
||||
callID := c.Param(api.Call)
|
||||
callObj, err := s.Datastore.GetTask(ctx, callID)
|
||||
callObj, err := s.Datastore.GetCall(ctx, appName, callID)
|
||||
if err != nil {
|
||||
handleErrorResponse(c, err)
|
||||
return
|
||||
|
||||
@@ -20,7 +20,7 @@ func (s *Server) handleCallList(c *gin.Context) {
|
||||
|
||||
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 {
|
||||
handleErrorResponse(c, err)
|
||||
return
|
||||
|
||||
@@ -10,8 +10,9 @@ import (
|
||||
func (s *Server) handleCallLogGet(c *gin.Context) {
|
||||
ctx := c.Request.Context()
|
||||
|
||||
appName := c.MustGet(api.AppName).(string)
|
||||
callID := c.Param(api.Call)
|
||||
_, err := s.Datastore.GetTask(ctx, callID)
|
||||
_, err := s.Datastore.GetCall(ctx, appName, callID)
|
||||
if err != nil {
|
||||
handleErrorResponse(c, err)
|
||||
return
|
||||
@@ -29,8 +30,9 @@ func (s *Server) handleCallLogGet(c *gin.Context) {
|
||||
func (s *Server) handleCallLogDelete(c *gin.Context) {
|
||||
ctx := c.Request.Context()
|
||||
|
||||
appName := c.MustGet(api.AppName).(string)
|
||||
callID := c.Param(api.Call)
|
||||
_, err := s.Datastore.GetTask(ctx, callID)
|
||||
_, err := s.Datastore.GetCall(ctx, appName, callID)
|
||||
if err != nil {
|
||||
handleErrorResponse(c, err)
|
||||
return
|
||||
|
||||
@@ -7,10 +7,10 @@ import (
|
||||
"net/http"
|
||||
"runtime/debug"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
"github.com/gin-gonic/gin"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// 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) {
|
||||
log := common.Logger(ctx)
|
||||
var statuscode int
|
||||
switch e := err.(type) {
|
||||
case models.APIError:
|
||||
if e, ok := err.(models.APIError); ok {
|
||||
if e.Code() >= 500 {
|
||||
log.WithFields(logrus.Fields{"code": e.Code()}).WithError(e).Error("api error")
|
||||
}
|
||||
statuscode = e.Code()
|
||||
default:
|
||||
} else {
|
||||
log.WithError(err).WithFields(logrus.Fields{"stack": string(debug.Stack())}).Error("internal server error")
|
||||
statuscode = http.StatusInternalServerError
|
||||
err = ErrInternalServerError
|
||||
|
||||
@@ -4,8 +4,7 @@ import (
|
||||
"context"
|
||||
"net/http"
|
||||
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/gin-gonic/gin"
|
||||
)
|
||||
|
||||
|
||||
@@ -45,7 +45,6 @@ func (s *Server) handleRoutesPostPutPatch(c *gin.Context) {
|
||||
return
|
||||
}
|
||||
|
||||
s.cachedelete(resp.Route.AppName, resp.Route.Path)
|
||||
c.JSON(http.StatusOK, resp)
|
||||
}
|
||||
|
||||
|
||||
@@ -24,6 +24,5 @@ func (s *Server) handleRouteDelete(c *gin.Context) {
|
||||
return
|
||||
}
|
||||
|
||||
s.cachedelete(appName, routePath)
|
||||
c.JSON(http.StatusOK, gin.H{"message": "Route deleted"})
|
||||
}
|
||||
|
||||
@@ -14,7 +14,7 @@ import (
|
||||
|
||||
type routeTestCase struct {
|
||||
ds models.Datastore
|
||||
logDB models.FnLog
|
||||
logDB models.LogStore
|
||||
method string
|
||||
path string
|
||||
body string
|
||||
@@ -24,7 +24,7 @@ type routeTestCase struct {
|
||||
|
||||
func (test *routeTestCase) run(t *testing.T, i int, buf *bytes.Buffer) {
|
||||
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))
|
||||
_, rec := routerRequest(t, srv.Router, test.method, test.path, body)
|
||||
@@ -70,7 +70,7 @@ func TestRouteCreate(t *testing.T) {
|
||||
AppName: "a",
|
||||
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},
|
||||
|
||||
// success
|
||||
@@ -106,22 +106,22 @@ func TestRoutePut(t *testing.T) {
|
||||
func TestRouteDelete(t *testing.T) {
|
||||
buf := setLogBuffer()
|
||||
|
||||
routes := models.Routes{{AppName: "a", Path: "/myroute"}}
|
||||
apps := models.Apps{{Name: "a", Routes: routes, Config: nil}}
|
||||
routes := []*models.Route{{AppName: "a", Path: "/myroute"}}
|
||||
apps := []*models.App{{Name: "a", Routes: routes, Config: nil}}
|
||||
|
||||
for i, test := range []struct {
|
||||
ds models.Datastore
|
||||
logDB models.FnLog
|
||||
logDB models.LogStore
|
||||
path string
|
||||
body string
|
||||
expectedCode int
|
||||
expectedError error
|
||||
}{
|
||||
{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)
|
||||
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)
|
||||
|
||||
if rec.Code != test.expectedCode {
|
||||
@@ -152,7 +152,7 @@ func TestRouteList(t *testing.T) {
|
||||
ds := datastore.NewMock()
|
||||
fnl := logs.NewMock()
|
||||
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
|
||||
|
||||
for i, test := range []struct {
|
||||
path string
|
||||
@@ -191,7 +191,7 @@ func TestRouteGet(t *testing.T) {
|
||||
ds := datastore.NewMock()
|
||||
fnl := logs.NewMock()
|
||||
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
|
||||
|
||||
for i, test := range []struct {
|
||||
path string
|
||||
@@ -238,7 +238,7 @@ func TestRouteUpdate(t *testing.T) {
|
||||
AppName: "a",
|
||||
Path: "/myroute/do",
|
||||
},
|
||||
}, nil, nil,
|
||||
}, nil,
|
||||
), logs.NewMock(), http.MethodPatch, "/v1/apps/a/routes/myroute/do", `{ "route": { "image": "fnproject/hello" } }`, http.StatusOK, nil},
|
||||
|
||||
// Addresses #381
|
||||
@@ -248,7 +248,7 @@ func TestRouteUpdate(t *testing.T) {
|
||||
AppName: "a",
|
||||
Path: "/myroute/do",
|
||||
},
|
||||
}, nil, nil,
|
||||
}, nil,
|
||||
), logs.NewMock(), http.MethodPatch, "/v1/apps/a/routes/myroute/do", `{ "route": { "path": "/otherpath" } }`, http.StatusConflict, models.ErrRoutesPathImmutable},
|
||||
} {
|
||||
test.run(t, i, buf)
|
||||
|
||||
@@ -3,23 +3,15 @@ package server
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"path"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"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/runner"
|
||||
"github.com/fnproject/fn/api/runner/common"
|
||||
"github.com/fnproject/fn/api/runner/task"
|
||||
"github.com/gin-gonic/gin"
|
||||
cache "github.com/patrickmn/go-cache"
|
||||
)
|
||||
|
||||
type runnerResponse struct {
|
||||
@@ -27,15 +19,7 @@ type runnerResponse struct {
|
||||
Error *models.ErrorBody `json:"error,omitempty"`
|
||||
}
|
||||
|
||||
func toEnvName(envtype, name string) string {
|
||||
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) {
|
||||
func (s *Server) handleRequest(c *gin.Context) {
|
||||
if strings.HasPrefix(c.Request.URL.Path, "/v1") {
|
||||
c.Status(http.StatusNotFound)
|
||||
return
|
||||
@@ -43,22 +27,10 @@ func (s *Server) handleRequest(c *gin.Context, enqueue models.Enqueue) {
|
||||
|
||||
ctx := c.Request.Context()
|
||||
|
||||
reqID := id.New().String()
|
||||
ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"call_id": reqID})
|
||||
|
||||
var err error
|
||||
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)
|
||||
if c.Request.Method == "GET" {
|
||||
// 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)
|
||||
c.Request.Body = ioutil.NopCloser(strings.NewReader(c.Request.URL.Query().Get("payload")))
|
||||
}
|
||||
|
||||
r, routeExists := c.Get(api.Path)
|
||||
@@ -73,228 +45,65 @@ func (s *Server) handleRequest(c *gin.Context, enqueue models.Enqueue) {
|
||||
|
||||
s.FireBeforeDispatch(ctx, reqRoute)
|
||||
|
||||
appName := reqRoute.AppName
|
||||
path := reqRoute.Path
|
||||
s.serve(c, reqRoute.AppName, reqRoute.Path)
|
||||
|
||||
app, err := s.Datastore.GetApp(ctx, appName)
|
||||
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)
|
||||
s.FireAfterDispatch(ctx, reqRoute)
|
||||
}
|
||||
|
||||
func (s *Server) loadroute(ctx context.Context, appName, path string) (*models.Route, error) {
|
||||
if route, ok := s.cacheget(appName, path); ok {
|
||||
return route, nil
|
||||
}
|
||||
key := routeCacheKey(appName, path)
|
||||
resp, err := s.singleflight.do(
|
||||
key,
|
||||
func() (interface{}, error) {
|
||||
return s.Datastore.GetRoute(ctx, appName, path)
|
||||
},
|
||||
// TODO it would be nice if we could make this have nothing to do with the gin.Context but meh
|
||||
// TODO make async store an *http.Request? would be sexy until we have different api format...
|
||||
func (s *Server) serve(c *gin.Context, appName, path string) {
|
||||
// GetCall can mod headers, assign an id, look up the route/app (cached),
|
||||
// strip params, etc.
|
||||
call, err := s.Agent.GetCall(
|
||||
agent.WithWriter(c.Writer), // XXX (reed): order matters [for now]
|
||||
agent.FromRequest(appName, path, c.Request),
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
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
|
||||
handleErrorResponse(c, err)
|
||||
return
|
||||
}
|
||||
|
||||
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 == "" {
|
||||
route.Format = "default"
|
||||
}
|
||||
|
||||
// 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 model := call.Model(); model.Type == "async" {
|
||||
// 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)
|
||||
if err != nil {
|
||||
handleErrorResponse(c, models.ErrInvalidPayload)
|
||||
return true
|
||||
return
|
||||
}
|
||||
// Add in payload
|
||||
newTask.Payload = string(pl)
|
||||
model.Payload = buf.String()
|
||||
|
||||
// Push to queue
|
||||
_, err = enqueue(c, s.MQ, newTask)
|
||||
// TODO we should probably add this to the datastore too. consider the plumber!
|
||||
_, err = s.MQ.Push(c.Request.Context(), model)
|
||||
if err != nil {
|
||||
handleErrorResponse(c, err)
|
||||
return true
|
||||
return
|
||||
}
|
||||
|
||||
log.Info("Added new task to queue")
|
||||
c.JSON(http.StatusAccepted, map[string]string{"call_id": cfg.ID})
|
||||
|
||||
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(),
|
||||
},
|
||||
})
|
||||
}
|
||||
c.JSON(http.StatusAccepted, map[string]string{"call_id": model.ID})
|
||||
return
|
||||
}
|
||||
|
||||
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) {}
|
||||
|
||||
func matchRoute(baseRoute, route string) (Params, bool) {
|
||||
tree := &node{}
|
||||
tree.addRoute(baseRoute, fakeHandler)
|
||||
handler, p, _ := tree.getValue(route)
|
||||
if handler == nil {
|
||||
return nil, false
|
||||
if err == context.DeadlineExceeded {
|
||||
err = models.ErrCallTimeout // 504 w/ friendly note
|
||||
}
|
||||
// NOTE: if the task wrote the headers already then this will fail to write
|
||||
// a 5xx (and log about it to us) -- that's fine (nice, even!)
|
||||
handleErrorResponse(c, err)
|
||||
return
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
|
||||
@@ -5,28 +5,23 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net/http"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/fnproject/fn/api/agent"
|
||||
"github.com/fnproject/fn/api/datastore"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/fnproject/fn/api/mqs"
|
||||
"github.com/fnproject/fn/api/runner"
|
||||
"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()
|
||||
|
||||
s := &Server{
|
||||
Runner: rnr,
|
||||
Router: gin.New(),
|
||||
Datastore: ds,
|
||||
MQ: mq,
|
||||
Enqueue: enqueue,
|
||||
routeCache: cache.New(60*time.Second, 5*time.Minute),
|
||||
Agent: rnr,
|
||||
Router: gin.New(),
|
||||
Datastore: ds,
|
||||
MQ: mq,
|
||||
}
|
||||
|
||||
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: "/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"}},
|
||||
}, nil, nil,
|
||||
}, nil,
|
||||
)
|
||||
mq := &mqs.Mock{}
|
||||
|
||||
@@ -75,29 +70,10 @@ func TestRouteRunnerAsyncExecution(t *testing.T) {
|
||||
},
|
||||
} {
|
||||
body := bytes.NewBuffer([]byte(test.body))
|
||||
var wg sync.WaitGroup
|
||||
|
||||
wg.Add(1)
|
||||
fmt.Println("About to start router")
|
||||
rnr, cancel := testRunner(t)
|
||||
router := testRouterAsync(ds, mq, rnr, func(_ context.Context, _ models.MessageQueue, task *models.Task) (*models.Task, error) {
|
||||
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
|
||||
})
|
||||
rnr, cancel := testRunner(t, ds)
|
||||
router := testRouterAsync(ds, mq, rnr)
|
||||
|
||||
fmt.Println("makeing requests")
|
||||
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",
|
||||
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()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -3,40 +3,45 @@ package server
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"errors"
|
||||
"net/http"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"errors"
|
||||
"github.com/fnproject/fn/api/agent"
|
||||
"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"
|
||||
)
|
||||
|
||||
func testRunner(t *testing.T) (*runner.Runner, context.CancelFunc) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
func testRunner(t *testing.T, args ...interface{}) (agent.Agent, context.CancelFunc) {
|
||||
ds := datastore.NewMock()
|
||||
fnl := logs.NewMock()
|
||||
r, err := runner.New(ctx, runner.NewFuncLogger(fnl), ds)
|
||||
if err != nil {
|
||||
t.Fatal("Test: failed to create new runner")
|
||||
var mq models.MessageQueue = &mqs.Mock{}
|
||||
for _, a := range args {
|
||||
switch arg := a.(type) {
|
||||
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) {
|
||||
buf := setLogBuffer()
|
||||
rnr, cancel := testRunner(t)
|
||||
defer cancel()
|
||||
ds := datastore.NewMockInit(
|
||||
[]*models.App{
|
||||
{Name: "myapp", Config: models.Config{}},
|
||||
}, nil, nil, nil,
|
||||
}, nil, nil,
|
||||
)
|
||||
|
||||
rnr, cancel := testRunner(t, ds)
|
||||
defer cancel()
|
||||
logDB := logs.NewMock()
|
||||
srv := testServer(ds, &mqs.Mock{}, logDB, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, logDB, rnr)
|
||||
|
||||
for i, test := range []struct {
|
||||
path string
|
||||
@@ -71,16 +76,17 @@ func TestRouteRunnerGet(t *testing.T) {
|
||||
func TestRouteRunnerPost(t *testing.T) {
|
||||
buf := setLogBuffer()
|
||||
|
||||
rnr, cancel := testRunner(t)
|
||||
defer cancel()
|
||||
|
||||
ds := datastore.NewMockInit(
|
||||
[]*models.App{
|
||||
{Name: "myapp", Config: models.Config{}},
|
||||
}, nil, nil, nil,
|
||||
}, nil, nil,
|
||||
)
|
||||
|
||||
rnr, cancel := testRunner(t, ds)
|
||||
defer cancel()
|
||||
|
||||
fnl := logs.NewMock()
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
|
||||
|
||||
for i, test := range []struct {
|
||||
path string
|
||||
@@ -117,9 +123,6 @@ func TestRouteRunnerPost(t *testing.T) {
|
||||
func TestRouteRunnerExecution(t *testing.T) {
|
||||
buf := setLogBuffer()
|
||||
|
||||
rnr, cancelrnr := testRunner(t)
|
||||
defer cancelrnr()
|
||||
|
||||
ds := datastore.NewMockInit(
|
||||
[]*models.App{
|
||||
{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: "/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"}}},
|
||||
}, nil, nil,
|
||||
}, nil,
|
||||
)
|
||||
|
||||
rnr, cancelrnr := testRunner(t, ds)
|
||||
defer cancelrnr()
|
||||
|
||||
fnl := logs.NewMock()
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
|
||||
|
||||
for i, test := range []struct {
|
||||
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) {
|
||||
buf := setLogBuffer()
|
||||
rnr, cancelrnr := testRunner(t)
|
||||
defer cancelrnr()
|
||||
|
||||
ds := datastore.NewMockInit(
|
||||
[]*models.App{
|
||||
{Name: "myapp", Config: models.Config{}},
|
||||
},
|
||||
[]*models.Route{
|
||||
{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()
|
||||
rnr, cancelrnr := testRunner(t, ds, mq)
|
||||
defer cancelrnr()
|
||||
|
||||
enqueue := func(ctx context.Context, mq models.MessageQueue, task *models.Task) (*models.Task, error) {
|
||||
return nil, errors.New("Unable to push task to queue")
|
||||
}
|
||||
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, enqueue)
|
||||
srv := testServer(ds, mq, fnl, rnr)
|
||||
for i, test := range []struct {
|
||||
path string
|
||||
body string
|
||||
@@ -215,19 +229,20 @@ func TestRouteRunnerTimeout(t *testing.T) {
|
||||
t.Skip("doesn't work on old Ubuntu")
|
||||
buf := setLogBuffer()
|
||||
|
||||
rnr, cancelrnr := testRunner(t)
|
||||
defer cancelrnr()
|
||||
|
||||
ds := datastore.NewMockInit(
|
||||
[]*models.App{
|
||||
{Name: "myapp", Config: models.Config{}},
|
||||
},
|
||||
[]*models.Route{
|
||||
{Path: "/sleeper", AppName: "myapp", Image: "fnproject/sleeper", Timeout: 1},
|
||||
}, nil, nil,
|
||||
}, nil,
|
||||
)
|
||||
|
||||
rnr, cancelrnr := testRunner(t, ds)
|
||||
defer cancelrnr()
|
||||
|
||||
fnl := logs.NewMock()
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, fnl, rnr)
|
||||
|
||||
for i, test := range []struct {
|
||||
path string
|
||||
@@ -261,29 +276,29 @@ func TestRouteRunnerTimeout(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestMatchRoute(t *testing.T) {
|
||||
buf := setLogBuffer()
|
||||
for i, test := range []struct {
|
||||
baseRoute string
|
||||
route string
|
||||
expectedParams []Param
|
||||
}{
|
||||
{"/myroute/", `/myroute/`, nil},
|
||||
{"/myroute/:mybigparam", `/myroute/1`, []Param{{"mybigparam", "1"}}},
|
||||
{"/:param/*test", `/1/2`, []Param{{"param", "1"}, {"test", "/2"}}},
|
||||
} {
|
||||
if params, match := matchRoute(test.baseRoute, test.route); match {
|
||||
if test.expectedParams != nil {
|
||||
for j, param := range test.expectedParams {
|
||||
if params[j].Key != param.Key || params[j].Value != param.Value {
|
||||
t.Log(buf.String())
|
||||
t.Errorf("Test %d: expected param %d, key = %s, value = %s", i, j, param.Key, param.Value)
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
t.Log(buf.String())
|
||||
t.Errorf("Test %d: %s should match %s", i, test.route, test.baseRoute)
|
||||
}
|
||||
}
|
||||
}
|
||||
//func TestMatchRoute(t *testing.T) {
|
||||
//buf := setLogBuffer()
|
||||
//for i, test := range []struct {
|
||||
//baseRoute string
|
||||
//route string
|
||||
//expectedParams []Param
|
||||
//}{
|
||||
//{"/myroute/", `/myroute/`, nil},
|
||||
//{"/myroute/:mybigparam", `/myroute/1`, []Param{{"mybigparam", "1"}}},
|
||||
//{"/:param/*test", `/1/2`, []Param{{"param", "1"}, {"test", "/2"}}},
|
||||
//} {
|
||||
//if params, match := matchRoute(test.baseRoute, test.route); match {
|
||||
//if test.expectedParams != nil {
|
||||
//for j, param := range test.expectedParams {
|
||||
//if params[j].Key != param.Key || params[j].Value != param.Value {
|
||||
//t.Log(buf.String())
|
||||
//t.Errorf("Test %d: expected param %d, key = %s, value = %s", i, j, param.Key, param.Value)
|
||||
//}
|
||||
//}
|
||||
//}
|
||||
//} else {
|
||||
//t.Log(buf.String())
|
||||
//t.Errorf("Test %d: %s should match %s", i, test.route, test.baseRoute)
|
||||
//}
|
||||
//}
|
||||
//}
|
||||
|
||||
@@ -3,31 +3,26 @@ package server
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"net/http"
|
||||
"os"
|
||||
"path"
|
||||
"time"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/ccirello/supervisor"
|
||||
"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/id"
|
||||
"github.com/fnproject/fn/api/logs"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"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/opentracing/opentracing-go"
|
||||
"github.com/opentracing/opentracing-go/ext"
|
||||
"github.com/openzipkin/zipkin-go-opentracing"
|
||||
"github.com/patrickmn/go-cache"
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/spf13/viper"
|
||||
)
|
||||
|
||||
@@ -42,25 +37,17 @@ const (
|
||||
)
|
||||
|
||||
type Server struct {
|
||||
Datastore models.Datastore
|
||||
Runner *runner.Runner
|
||||
Router *gin.Engine
|
||||
Agent agent.Agent
|
||||
Datastore models.Datastore
|
||||
MQ models.MessageQueue
|
||||
Enqueue models.Enqueue
|
||||
LogDB models.FnLog
|
||||
|
||||
apiURL string
|
||||
LogDB models.LogStore
|
||||
|
||||
appListeners []AppListener
|
||||
middlewares []Middleware
|
||||
runnerListeners []RunnerListener
|
||||
|
||||
routeCache *cache.Cache
|
||||
singleflight singleflight // singleflight assists Datastore
|
||||
}
|
||||
|
||||
const cacheSize = 1024
|
||||
|
||||
// NewFromEnv creates a new Functions server based on env vars.
|
||||
func NewFromEnv(ctx context.Context) *Server {
|
||||
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.")
|
||||
}
|
||||
|
||||
var logDB models.FnLog = ds
|
||||
var logDB models.LogStore = ds
|
||||
if ldb := viper.GetString(EnvLOGDBURL); ldb != "" && ldb != viper.GetString(EnvDBURL) {
|
||||
logDB, err = logs.New(viper.GetString(EnvLOGDBURL))
|
||||
if err != nil {
|
||||
@@ -81,30 +68,17 @@ func NewFromEnv(ctx context.Context) *Server {
|
||||
}
|
||||
}
|
||||
|
||||
apiURL := viper.GetString(EnvAPIURL)
|
||||
|
||||
return New(ctx, ds, mq, logDB, apiURL)
|
||||
return New(ctx, ds, mq, logDB)
|
||||
}
|
||||
|
||||
// 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 {
|
||||
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
|
||||
}
|
||||
|
||||
func New(ctx context.Context, ds models.Datastore, mq models.MessageQueue, logDB models.LogStore, opts ...ServerOption) *Server {
|
||||
s := &Server{
|
||||
Runner: rnr,
|
||||
Router: gin.New(),
|
||||
Datastore: ds,
|
||||
MQ: mq,
|
||||
routeCache: cache.New(5*time.Second, 5*time.Minute),
|
||||
LogDB: logDB,
|
||||
Enqueue: DefaultEnqueue,
|
||||
apiURL: apiURL,
|
||||
Agent: agent.New(ds, mq),
|
||||
Router: gin.New(),
|
||||
Datastore: ds,
|
||||
MQ: mq,
|
||||
LogDB: logDB,
|
||||
}
|
||||
|
||||
setMachineId()
|
||||
@@ -234,58 +208,8 @@ func loggerWrap(c *gin.Context) {
|
||||
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) {
|
||||
s.handleRequest(c, s.Enqueue)
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
s.handleRequest(c)
|
||||
}
|
||||
|
||||
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
|
||||
// PORT environment variable was defined.
|
||||
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 = `
|
||||
______
|
||||
@@ -320,29 +240,23 @@ func (s *Server) startGears(ctx context.Context) {
|
||||
fmt.Println(runHeader)
|
||||
logrus.Infof("Serving Functions API on address `%s`", listen)
|
||||
|
||||
svr := &supervisor.Supervisor{
|
||||
MaxRestarts: supervisor.AlwaysRestart,
|
||||
Log: func(msg interface{}) {
|
||||
logrus.Debug("supervisor: ", msg)
|
||||
},
|
||||
server := http.Server{
|
||||
Addr: listen,
|
||||
Handler: s.Router,
|
||||
// TODO we should set read/write timeouts
|
||||
}
|
||||
|
||||
svr.AddFunc(func(ctx context.Context) {
|
||||
go func() {
|
||||
err := http.Serve(listener, s.Router)
|
||||
if err != nil {
|
||||
logrus.Fatalf("Error serving API: %v", err)
|
||||
}
|
||||
}()
|
||||
<-ctx.Done()
|
||||
})
|
||||
go func() {
|
||||
<-ctx.Done() // listening for signals...
|
||||
server.Shutdown(context.Background()) // we can wait
|
||||
}()
|
||||
|
||||
svr.AddFunc(func(ctx context.Context) {
|
||||
runner.RunAsyncRunner(ctx, s.apiURL, s.Runner, s.Datastore)
|
||||
})
|
||||
err := server.ListenAndServe()
|
||||
if err != nil {
|
||||
logrus.WithError(err).Error("error opening server")
|
||||
}
|
||||
|
||||
svr.Serve(ctx)
|
||||
s.Runner.Wait() // wait for tasks to finish (safe shutdown)
|
||||
s.Agent.Close() // after we stop taking requests, wait for all tasks to finish
|
||||
}
|
||||
|
||||
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/*route", s.handleRunnerRequest)
|
||||
|
||||
@@ -397,8 +309,8 @@ type appResponse struct {
|
||||
}
|
||||
|
||||
type appsResponse struct {
|
||||
Message string `json:"message"`
|
||||
Apps models.Apps `json:"apps"`
|
||||
Message string `json:"message"`
|
||||
Apps []*models.App `json:"apps"`
|
||||
}
|
||||
|
||||
type routeResponse struct {
|
||||
@@ -407,26 +319,21 @@ type routeResponse struct {
|
||||
}
|
||||
|
||||
type routesResponse struct {
|
||||
Message string `json:"message"`
|
||||
Routes models.Routes `json:"routes"`
|
||||
}
|
||||
|
||||
type tasksResponse struct {
|
||||
Message string `json:"message"`
|
||||
Task models.Task `json:"tasksResponse"`
|
||||
Message string `json:"message"`
|
||||
Routes []*models.Route `json:"routes"`
|
||||
}
|
||||
|
||||
type fnCallResponse struct {
|
||||
Message string `json:"message"`
|
||||
Call *models.FnCall `json:"call"`
|
||||
Message string `json:"message"`
|
||||
Call *models.Call `json:"call"`
|
||||
}
|
||||
|
||||
type fnCallsResponse struct {
|
||||
Message string `json:"message"`
|
||||
Calls models.FnCalls `json:"calls"`
|
||||
Calls []*models.Call `json:"calls"`
|
||||
}
|
||||
|
||||
type fnCallLogResponse struct {
|
||||
Message string `json:"message"`
|
||||
Log *models.FnCallLog `json:"log"`
|
||||
Message string `json:"message"`
|
||||
Log *models.CallLog `json:"log"`
|
||||
}
|
||||
|
||||
@@ -10,29 +10,25 @@ import (
|
||||
"net/http/httptest"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/fnproject/fn/api/agent"
|
||||
"github.com/fnproject/fn/api/datastore"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/fnproject/fn/api/mqs"
|
||||
"github.com/fnproject/fn/api/runner"
|
||||
"github.com/gin-gonic/gin"
|
||||
cache "github.com/patrickmn/go-cache"
|
||||
)
|
||||
|
||||
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()
|
||||
|
||||
s := &Server{
|
||||
Runner: rnr,
|
||||
Router: gin.New(),
|
||||
Datastore: ds,
|
||||
LogDB: logDB,
|
||||
MQ: mq,
|
||||
Enqueue: enqueue,
|
||||
routeCache: cache.New(60*time.Second, 5*time.Minute),
|
||||
Agent: rnr,
|
||||
Router: gin.New(),
|
||||
Datastore: ds,
|
||||
LogDB: logDB,
|
||||
MQ: mq,
|
||||
}
|
||||
|
||||
r := s.Router
|
||||
@@ -81,7 +77,7 @@ func getErrorResponse(t *testing.T, rec *httptest.ResponseRecorder) models.Error
|
||||
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)
|
||||
ds, err := datastore.New("sqlite3://" + tmpDatastoreTests)
|
||||
if err != nil {
|
||||
@@ -99,10 +95,10 @@ func TestFullStack(t *testing.T) {
|
||||
ds, logDB, close := prepareDB(ctx, t)
|
||||
defer close()
|
||||
|
||||
rnr, rnrcancel := testRunner(t)
|
||||
rnr, rnrcancel := testRunner(t, ds)
|
||||
defer rnrcancel()
|
||||
|
||||
srv := testServer(ds, &mqs.Mock{}, logDB, rnr, DefaultEnqueue)
|
||||
srv := testServer(ds, &mqs.Mock{}, logDB, rnr)
|
||||
|
||||
for _, test := range []struct {
|
||||
name string
|
||||
@@ -110,7 +106,7 @@ func TestFullStack(t *testing.T) {
|
||||
path string
|
||||
body string
|
||||
expectedCode int
|
||||
expectedCacheSize int
|
||||
expectedCacheSize int // TODO kill me
|
||||
}{
|
||||
{"create my app", "POST", "/v1/apps", `{ "app": { "name": "myapp" } }`, 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",
|
||||
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())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -7,5 +7,5 @@ import (
|
||||
)
|
||||
|
||||
func (s *Server) handleStats(c *gin.Context) {
|
||||
c.JSON(http.StatusOK, s.Runner.Stats())
|
||||
c.JSON(http.StatusOK, s.Agent.Stats())
|
||||
}
|
||||
|
||||
@@ -44,7 +44,7 @@ building the function.
|
||||
|
||||
`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
|
||||
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`.
|
||||
|
||||
`memory` (optional) allows you to set a maximum memory threshold for this
|
||||
|
||||
@@ -20,7 +20,7 @@ The format is still up for discussion and in order to move forward and remain fl
|
||||
|
||||
#### 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:
|
||||
|
||||
|
||||
@@ -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
|
||||
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`.
|
||||
|
||||
`config` is a map of values passed to the route runtime in the form of
|
||||
@@ -87,4 +87,4 @@ curl -H "Content-Type: application/json" -X POST -d '{
|
||||
"config": {"APPLOG": "stdout"}
|
||||
}
|
||||
}' http://localhost:8080/v1/apps/myapp/routes/hello
|
||||
```
|
||||
```
|
||||
|
||||
1
test.sh
1
test.sh
@@ -39,7 +39,6 @@ container_ip)
|
||||
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 github.com/fnproject/fn/api/runner/drivers/docker
|
||||
docker rm --force func-postgres-test
|
||||
docker rm --force func-mysql-test
|
||||
|
||||
|
||||
@@ -25,7 +25,7 @@ type TimeoutBody struct {
|
||||
|
||||
func CallAsync(t *testing.T, u url.URL, content io.Reader) string {
|
||||
output := &bytes.Buffer{}
|
||||
err := CallFN(u.String(), content, output, "POST", []string{})
|
||||
_, err := CallFN(u.String(), content, output, "POST", []string{})
|
||||
if err != nil {
|
||||
t.Errorf("Got unexpected error: %v", err)
|
||||
}
|
||||
@@ -67,7 +67,7 @@ func TestRouteExecutions(t *testing.T) {
|
||||
|
||||
content := &bytes.Buffer{}
|
||||
output := &bytes.Buffer{}
|
||||
err := CallFN(u.String(), content, output, "POST", []string{})
|
||||
_, err := CallFN(u.String(), content, output, "POST", []string{})
|
||||
if err != nil {
|
||||
t.Errorf("Got unexpected error: %v", err)
|
||||
}
|
||||
@@ -97,7 +97,7 @@ func TestRouteExecutions(t *testing.T) {
|
||||
Name string
|
||||
}{Name: "John"})
|
||||
output := &bytes.Buffer{}
|
||||
err := CallFN(u.String(), content, output, "POST", []string{})
|
||||
_, err := CallFN(u.String(), content, output, "POST", []string{})
|
||||
if err != nil {
|
||||
t.Errorf("Got unexpected error: %v", err)
|
||||
}
|
||||
@@ -216,17 +216,14 @@ func TestRouteExecutions(t *testing.T) {
|
||||
}{Seconds: 31})
|
||||
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") {
|
||||
t.Errorf("Must fail because of timeout, but got error message: %v", output.String())
|
||||
}
|
||||
tB := &TimeoutBody{}
|
||||
|
||||
json.NewDecoder(output).Decode(tB)
|
||||
|
||||
cfg := &call.GetAppsAppCallsCallParams{
|
||||
Call: tB.CallID,
|
||||
Call: headers.Get("FN_CALL_ID"),
|
||||
App: s.AppName,
|
||||
Context: s.Context,
|
||||
}
|
||||
|
||||
@@ -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 content == nil {
|
||||
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)
|
||||
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")
|
||||
@@ -202,12 +202,12 @@ func CallFN(u string, content io.Reader, output io.Writer, method string, env []
|
||||
|
||||
resp, err := http.DefaultClient.Do(req)
|
||||
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)
|
||||
|
||||
return nil
|
||||
return resp.Header, nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
|
||||
Reference in New Issue
Block a user