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
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)
|
||||
}
|
||||
1
api/agent/drivers/README.md
Normal file
1
api/agent/drivers/README.md
Normal file
@@ -0,0 +1 @@
|
||||
This package is intended as a general purpose container abstraction library. With the same code, you can run on Docker, Rkt, etc.
|
||||
448
api/agent/drivers/docker/docker.go
Normal file
448
api/agent/drivers/docker/docker.go
Normal file
@@ -0,0 +1,448 @@
|
||||
package docker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"path"
|
||||
"strings"
|
||||
|
||||
"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
|
||||
// like to use not-necessarily-public docker images for any or all task
|
||||
// invocations.
|
||||
type Auther interface {
|
||||
// DockerAuth should return docker auth credentials that will authenticate
|
||||
// against a docker registry for a given drivers.ContainerTask.Image(). An
|
||||
// error may be returned which will cause the task not to be run, this can be
|
||||
// useful for an implementer to do things like testing auth configurations
|
||||
// before returning them; e.g. if the implementer would like to impose
|
||||
// certain restrictions on images or if credentials must be acquired right
|
||||
// before runtime and there's an error doing so. If these credentials don't
|
||||
// work, the docker pull will fail and the task will be set to error status.
|
||||
DockerAuth() (docker.AuthConfiguration, error)
|
||||
}
|
||||
|
||||
type runResult struct {
|
||||
error
|
||||
status string
|
||||
}
|
||||
|
||||
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) 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
|
||||
auths map[string]docker.AuthConfiguration
|
||||
}
|
||||
|
||||
// implements drivers.Driver
|
||||
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(),
|
||||
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
|
||||
if task.Command() != "" {
|
||||
// NOTE: this is hyper-sensitive and may not be correct like this even, but it passes old tests
|
||||
cmd = strings.Fields(task.Command())
|
||||
log.WithFields(logrus.Fields{"call_id": task.Id(), "cmd": cmd, "len": len(cmd)}).Debug("docker command")
|
||||
}
|
||||
|
||||
envvars := make([]string, 0, len(task.EnvVars()))
|
||||
for name, val := range task.EnvVars() {
|
||||
envvars = append(envvars, name+"="+val)
|
||||
}
|
||||
|
||||
container := docker.CreateContainerOptions{
|
||||
Name: task.Id(),
|
||||
Config: &docker.Config{
|
||||
Env: envvars,
|
||||
Cmd: cmd,
|
||||
Memory: int64(task.Memory()),
|
||||
CPUShares: drv.conf.CPUShares,
|
||||
Hostname: drv.hostname,
|
||||
Image: task.Image(),
|
||||
Volumes: map[string]struct{}{},
|
||||
OpenStdin: true,
|
||||
AttachStdin: true,
|
||||
StdinOnce: true,
|
||||
},
|
||||
HostConfig: &docker.HostConfig{},
|
||||
Context: ctx,
|
||||
}
|
||||
|
||||
volumes := task.Volumes()
|
||||
for _, mapping := range volumes {
|
||||
hostDir := mapping[0]
|
||||
containerDir := mapping[1]
|
||||
container.Config.Volumes[containerDir] = struct{}{}
|
||||
mapn := fmt.Sprintf("%s:%s", hostDir, containerDir)
|
||||
container.HostConfig.Binds = append(container.HostConfig.Binds, mapn)
|
||||
log.WithFields(logrus.Fields{"volumes": mapn, "call_id": task.Id()}).Debug("setting volumes")
|
||||
}
|
||||
|
||||
if wd := task.WorkDir(); wd != "" {
|
||||
log.WithFields(logrus.Fields{"wd": wd, "call_id": task.Id()}).Debug("setting work dir")
|
||||
container.Config.WorkingDir = wd
|
||||
}
|
||||
|
||||
err := drv.ensureImage(ctx, task)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
_, err = drv.docker.CreateContainer(container)
|
||||
if err != nil {
|
||||
// since we retry under the hood, if the container gets created and retry fails, we can just ignore error
|
||||
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": container.Name,
|
||||
}).WithError(err).Error("Could not create container")
|
||||
|
||||
if ce := containerConfigError(err); ce != nil {
|
||||
return nil, common.UserError(fmt.Errorf("Failed to create container from task configuration '%s'", ce))
|
||||
}
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// discard removal error
|
||||
return &cookie{id: task.Id(), task: task, drv: drv}, nil
|
||||
}
|
||||
|
||||
type cookie struct {
|
||||
id string
|
||||
task drivers.ContainerTask
|
||||
drv *DockerDriver
|
||||
}
|
||||
|
||||
func (c *cookie) Close(ctx context.Context) error {
|
||||
return c.drv.removeContainer(ctx, c.id)
|
||||
}
|
||||
|
||||
func (c *cookie) Run(ctx context.Context) (drivers.WaitResult, error) {
|
||||
return c.drv.run(ctx, c.id, c.task)
|
||||
}
|
||||
|
||||
func (drv *DockerDriver) removeContainer(ctx context.Context, container string) error {
|
||||
err := drv.docker.RemoveContainer(docker.RemoveContainerOptions{
|
||||
ID: container, Force: true, RemoveVolumes: true, Context: ctx})
|
||||
|
||||
if err != nil {
|
||||
logrus.WithError(err).WithFields(logrus.Fields{"container": container}).Error("error removing container")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (drv *DockerDriver) ensureImage(ctx context.Context, task drivers.ContainerTask) error {
|
||||
reg, _, _ := drivers.ParseImage(task.Image())
|
||||
|
||||
// ask for docker creds before looking for image, as the tasker may need to
|
||||
// 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")
|
||||
config, err = task.DockerAuth()
|
||||
span.Finish()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
if reg != "" {
|
||||
config.ServerAddress = reg
|
||||
}
|
||||
|
||||
// see if we already have it, if not, pull it
|
||||
_, err := drv.docker.InspectImage(ctx, task.Image())
|
||||
if err == docker.ErrNoSuchImage {
|
||||
err = drv.pullImage(ctx, task, config)
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
func (drv *DockerDriver) pullImage(ctx context.Context, task drivers.ContainerTask, config docker.AuthConfiguration) error {
|
||||
log := common.Logger(ctx)
|
||||
reg, repo, tag := drivers.ParseImage(task.Image())
|
||||
globalRepo := path.Join(reg, repo)
|
||||
if reg != "" {
|
||||
config.ServerAddress = reg
|
||||
}
|
||||
|
||||
var err error
|
||||
config.ServerAddress, err = registryURL(config.ServerAddress)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
log.WithFields(logrus.Fields{"registry": config.ServerAddress, "username": config.Username, "image": task.Image()}).Info("Pulling image")
|
||||
|
||||
err = drv.docker.PullImage(docker.PullImageOptions{Repository: globalRepo, Tag: tag, Context: ctx}, config)
|
||||
if err != nil {
|
||||
log.WithFields(logrus.Fields{"registry": config.ServerAddress, "username": config.Username, "image": task.Image()}).WithError(err).Error("Failed to pull image")
|
||||
|
||||
// TODO need to inspect for hub or network errors and pick.
|
||||
return common.UserError(fmt.Errorf("Failed to pull image '%s': %s", task.Image(), err))
|
||||
|
||||
// TODO what about a case where credentials were good, then credentials
|
||||
// were invalidated -- do we need to keep the credential cache docker
|
||||
// driver side and after pull for this case alone?
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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.WaitResult, error) {
|
||||
timeout := task.Timeout()
|
||||
|
||||
var cancel context.CancelFunc
|
||||
if timeout <= 0 {
|
||||
ctx, cancel = context.WithCancel(ctx)
|
||||
} else {
|
||||
ctx, cancel = context.WithTimeout(ctx, timeout)
|
||||
}
|
||||
defer cancel() // do this so that after Run exits, collect stops
|
||||
go drv.collectStats(ctx, container, task)
|
||||
|
||||
mwOut, mwErr := task.Logger()
|
||||
|
||||
waiter, err := drv.docker.AttachToContainerNonBlocking(ctx, docker.AttachToContainerOptions{
|
||||
Container: container, OutputStream: mwOut, ErrorStream: mwErr,
|
||||
Stream: true, Logs: true, Stdout: true, Stderr: true,
|
||||
Stdin: true, InputStream: task.Input()})
|
||||
if err != nil && ctx.Err() == nil {
|
||||
// ignore if ctx has errored, rewrite status lay below
|
||||
return nil, err
|
||||
}
|
||||
|
||||
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() {
|
||||
w.waiter.Close()
|
||||
w.waiter.Wait() // make sure we gather all logs
|
||||
}()
|
||||
|
||||
status, err := w.drv.wait(ctx, w.container)
|
||||
return &runResult{
|
||||
status: status,
|
||||
error: err,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (drv *DockerDriver) collectStats(ctx context.Context, container string, task drivers.ContainerTask) {
|
||||
log := common.Logger(ctx)
|
||||
done := make(chan bool)
|
||||
defer close(done)
|
||||
dstats := make(chan *docker.Stats, 1)
|
||||
go func() {
|
||||
// NOTE: docker automatically streams every 1s. we can skip or avg samples if we'd like but
|
||||
// the memory overhead is < 1MB for 3600 stat points so this seems fine, seems better to stream
|
||||
// (internal docker api streams) than open/close stream for 1 sample over and over.
|
||||
// must be called in goroutine, docker.Stats() blocks
|
||||
err := drv.docker.Stats(docker.StatsOptions{
|
||||
ID: container,
|
||||
Stats: dstats,
|
||||
Stream: true,
|
||||
Done: done, // A flag that enables stopping the stats operation
|
||||
})
|
||||
|
||||
if err != nil && err != io.ErrClosedPipe {
|
||||
log.WithError(err).WithFields(logrus.Fields{"container": container, "call_id": task.Id()}).Error("error streaming docker stats for task")
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case ds, ok := <-dstats:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
task.WriteStat(cherryPick(ds))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func cherryPick(ds *docker.Stats) drivers.Stat {
|
||||
// TODO cpu % is as a % of the whole system... cpu is weird since we're sharing it
|
||||
// across a bunch of containers and it scales based on how many we're sharing with,
|
||||
// do we want users to see as a % of system?
|
||||
systemDelta := float64(ds.CPUStats.SystemCPUUsage - ds.PreCPUStats.SystemCPUUsage)
|
||||
cores := float64(len(ds.CPUStats.CPUUsage.PercpuUsage))
|
||||
var cpuUser, cpuKernel, cpuTotal float64
|
||||
if systemDelta > 0 {
|
||||
// TODO we could leave these in docker format and let hud/viz tools do this instead of us... like net is, could do same for mem, too. thoughts?
|
||||
cpuUser = (float64(ds.CPUStats.CPUUsage.UsageInUsermode-ds.PreCPUStats.CPUUsage.UsageInUsermode) / systemDelta) * cores * 100.0
|
||||
cpuKernel = (float64(ds.CPUStats.CPUUsage.UsageInKernelmode-ds.PreCPUStats.CPUUsage.UsageInKernelmode) / systemDelta) * cores * 100.0
|
||||
cpuTotal = (float64(ds.CPUStats.CPUUsage.TotalUsage-ds.PreCPUStats.CPUUsage.TotalUsage) / systemDelta) * cores * 100.0
|
||||
}
|
||||
|
||||
var rx, tx float64
|
||||
for _, v := range ds.Networks {
|
||||
rx += float64(v.RxBytes)
|
||||
tx += float64(v.TxBytes)
|
||||
}
|
||||
|
||||
var blkRead, blkWrite uint64
|
||||
for _, bioEntry := range ds.BlkioStats.IOServiceBytesRecursive {
|
||||
switch strings.ToLower(bioEntry.Op) {
|
||||
case "read":
|
||||
blkRead = blkRead + bioEntry.Value
|
||||
case "write":
|
||||
blkWrite = blkWrite + bioEntry.Value
|
||||
}
|
||||
}
|
||||
|
||||
return drivers.Stat{
|
||||
Timestamp: ds.Read,
|
||||
Metrics: map[string]uint64{
|
||||
// source: https://godoc.org/github.com/fsouza/go-dockerclient#Stats
|
||||
// ex (for future expansion): {"read":"2016-08-03T18:08:05Z","pids_stats":{},"network":{},"networks":{"eth0":{"rx_bytes":508,"tx_packets":6,"rx_packets":6,"tx_bytes":508}},"memory_stats":{"stats":{"cache":16384,"pgpgout":281,"rss":8826880,"pgpgin":2440,"total_rss":8826880,"hierarchical_memory_limit":536870912,"total_pgfault":3809,"active_anon":8843264,"total_active_anon":8843264,"total_pgpgout":281,"total_cache":16384,"pgfault":3809,"total_pgpgin":2440},"max_usage":8953856,"usage":8953856,"limit":536870912},"blkio_stats":{"io_service_bytes_recursive":[{"major":202,"op":"Read"},{"major":202,"op":"Write"},{"major":202,"op":"Sync"},{"major":202,"op":"Async"},{"major":202,"op":"Total"}],"io_serviced_recursive":[{"major":202,"op":"Read"},{"major":202,"op":"Write"},{"major":202,"op":"Sync"},{"major":202,"op":"Async"},{"major":202,"op":"Total"}]},"cpu_stats":{"cpu_usage":{"percpu_usage":[47641874],"usage_in_usermode":30000000,"total_usage":47641874},"system_cpu_usage":8880800500000000,"throttling_data":{}},"precpu_stats":{"cpu_usage":{"percpu_usage":[44946186],"usage_in_usermode":30000000,"total_usage":44946186},"system_cpu_usage":8880799510000000,"throttling_data":{}}}
|
||||
// mostly stolen values from docker stats cli api...
|
||||
|
||||
// net
|
||||
"net_rx": uint64(rx),
|
||||
"net_tx": uint64(tx),
|
||||
// mem
|
||||
"mem_limit": ds.MemoryStats.Limit,
|
||||
"mem_usage": ds.MemoryStats.Usage,
|
||||
// i/o
|
||||
"disk_read": blkRead,
|
||||
"disk_write": blkWrite,
|
||||
// cpu
|
||||
"cpu_user": uint64(cpuUser),
|
||||
"cpu_total": uint64(cpuTotal),
|
||||
"cpu_kernel": uint64(cpuKernel),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (drv *DockerDriver) startTask(ctx context.Context, container string) error {
|
||||
log := common.Logger(ctx)
|
||||
log.WithFields(logrus.Fields{"container": container}).Debug("Starting container execution")
|
||||
err := drv.docker.StartContainerWithContext(container, nil, ctx)
|
||||
if err != nil {
|
||||
dockerErr, ok := err.(*docker.Error)
|
||||
_, containerAlreadyRunning := err.(*docker.ContainerAlreadyRunning)
|
||||
if containerAlreadyRunning || (ok && dockerErr.Status == 304) {
|
||||
// 304=container already started -- so we can ignore error
|
||||
} else {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (drv *DockerDriver) wait(ctx context.Context, container string) (status string, err error) {
|
||||
// wait retries internally until ctx is up, so we can ignore the error and
|
||||
// just say it was a timeout if we have [fatal] errors talking to docker, etc.
|
||||
// a more prevalent case is calling wait & container already finished, so again ignore err.
|
||||
exitCode, _ := drv.docker.WaitContainerWithContext(container, ctx)
|
||||
|
||||
// check the context first, if it's done then exitCode is invalid iff zero
|
||||
// (can't know 100% without inspecting, but that's expensive and this is a good guess)
|
||||
// if exitCode is non-zero, we prefer that since it proves termination.
|
||||
if exitCode == 0 {
|
||||
select {
|
||||
case <-ctx.Done(): // check if task was canceled or timed out
|
||||
switch ctx.Err() {
|
||||
case context.DeadlineExceeded:
|
||||
return drivers.StatusTimeout, nil
|
||||
case context.Canceled:
|
||||
return drivers.StatusCancelled, nil
|
||||
}
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
switch exitCode {
|
||||
default:
|
||||
return drivers.StatusError, common.UserError(fmt.Errorf("container exit code %d", exitCode))
|
||||
case 0:
|
||||
return drivers.StatusSuccess, nil
|
||||
case 137: // OOM
|
||||
opentracing.SpanFromContext(ctx).LogFields(log.String("docker", "oom"))
|
||||
return drivers.StatusKilled, drivers.ErrOutOfMemory
|
||||
}
|
||||
}
|
||||
291
api/agent/drivers/docker/docker_client.go
Normal file
291
api/agent/drivers/docker/docker_client.go
Normal file
@@ -0,0 +1,291 @@
|
||||
// +build go1.7
|
||||
|
||||
package docker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/http"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"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 (
|
||||
retryTimeout = 10 * time.Minute
|
||||
)
|
||||
|
||||
// wrap docker client calls so we can retry 500s, kind of sucks but fsouza doesn't
|
||||
// bake in retries we can use internally, could contribute it at some point, would
|
||||
// be much more convenient if we didn't have to do this, but it's better than ad hoc retries.
|
||||
// also adds timeouts to many operations, varying by operation
|
||||
// TODO could generate this, maybe not worth it, may not change often
|
||||
type dockerClient interface {
|
||||
// Each of these are github.com/fsouza/go-dockerclient methods
|
||||
|
||||
AttachToContainerNonBlocking(ctx context.Context, opts docker.AttachToContainerOptions) (docker.CloseWaiter, error)
|
||||
WaitContainerWithContext(id string, ctx context.Context) (int, error)
|
||||
StartContainerWithContext(id string, hostConfig *docker.HostConfig, ctx context.Context) error
|
||||
CreateContainer(opts docker.CreateContainerOptions) (*docker.Container, error)
|
||||
RemoveContainer(opts docker.RemoveContainerOptions) error
|
||||
PullImage(opts docker.PullImageOptions, auth docker.AuthConfiguration) error
|
||||
InspectImage(ctx context.Context, name string) (*docker.Image, error)
|
||||
Stats(opts docker.StatsOptions) error
|
||||
}
|
||||
|
||||
// TODO: switch to github.com/docker/engine-api
|
||||
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()
|
||||
if err != nil {
|
||||
logrus.WithError(err).Fatal("couldn't create docker client")
|
||||
}
|
||||
|
||||
t := &http.Transport{
|
||||
Dial: (&net.Dialer{
|
||||
Timeout: 10 * time.Second,
|
||||
KeepAlive: 1 * time.Minute,
|
||||
}).Dial,
|
||||
TLSClientConfig: &tls.Config{
|
||||
ClientSessionCache: tls.NewLRUClientSessionCache(8192),
|
||||
},
|
||||
TLSHandshakeTimeout: 10 * time.Second,
|
||||
MaxIdleConnsPerHost: 512,
|
||||
Proxy: http.ProxyFromEnvironment,
|
||||
MaxIdleConns: 512,
|
||||
IdleConnTimeout: 90 * time.Second,
|
||||
ExpectContinueTimeout: 1 * time.Second,
|
||||
}
|
||||
|
||||
client.HTTPClient = &http.Client{Transport: t}
|
||||
|
||||
if err := client.Ping(); err != nil {
|
||||
logrus.WithError(err).Fatal("couldn't connect to docker daemon")
|
||||
}
|
||||
|
||||
client.SetTimeout(120 * time.Second)
|
||||
|
||||
// get 2 clients, one with a small timeout, one with no timeout to use contexts
|
||||
|
||||
clientNoTimeout, err := docker.NewClientFromEnv()
|
||||
if err != nil {
|
||||
logrus.WithError(err).Fatal("couldn't create other docker client")
|
||||
}
|
||||
|
||||
clientNoTimeout.HTTPClient = &http.Client{Transport: t}
|
||||
|
||||
if err := clientNoTimeout.Ping(); err != nil {
|
||||
logrus.WithError(err).Fatal("couldn't connect to other docker daemon")
|
||||
}
|
||||
|
||||
return &dockerWrap{client, clientNoTimeout}
|
||||
}
|
||||
|
||||
type dockerWrap struct {
|
||||
docker *docker.Client
|
||||
dockerNoTimeout *docker.Client
|
||||
}
|
||||
|
||||
func (d *dockerWrap) retry(ctx context.Context, f func() error) error {
|
||||
var i int
|
||||
span := opentracing.SpanFromContext(ctx)
|
||||
defer func() { span.LogFields(log.Int("docker_call_retries", i)) }()
|
||||
|
||||
logger := common.Logger(ctx)
|
||||
var b common.Backoff
|
||||
for ; ; i++ {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
span.LogFields(log.String("task", "fail.docker"))
|
||||
logger.WithError(ctx.Err()).Warnf("docker call timed out")
|
||||
return ctx.Err()
|
||||
default:
|
||||
}
|
||||
|
||||
err := filter(ctx, f())
|
||||
if common.IsTemporary(err) || isDocker50x(err) {
|
||||
logger.WithError(err).Warn("docker temporary error, retrying")
|
||||
b.Sleep()
|
||||
span.LogFields(log.String("task", "tmperror.docker"))
|
||||
continue
|
||||
}
|
||||
if err != nil {
|
||||
span.LogFields(log.String("task", "error.docker"))
|
||||
}
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
func isDocker50x(err error) bool {
|
||||
derr, ok := err.(*docker.Error)
|
||||
return ok && derr.Status >= 500
|
||||
}
|
||||
|
||||
func containerConfigError(err error) error {
|
||||
derr, ok := err.(*docker.Error)
|
||||
if ok && derr.Status == 400 {
|
||||
// derr.Message is a JSON response from docker, which has a "message" field we want to extract if possible.
|
||||
var v struct {
|
||||
Msg string `json:"message"`
|
||||
}
|
||||
|
||||
err := json.Unmarshal([]byte(derr.Message), &v)
|
||||
if err != nil {
|
||||
// If message was not valid JSON, the raw body is still better than nothing.
|
||||
return fmt.Errorf("%s", derr.Message)
|
||||
}
|
||||
return fmt.Errorf("%s", v.Msg)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
type temporary struct {
|
||||
error
|
||||
}
|
||||
|
||||
func (t *temporary) Temporary() bool { return true }
|
||||
|
||||
func temp(err error) error {
|
||||
return &temporary{err}
|
||||
}
|
||||
|
||||
// some 500s are totally cool
|
||||
func filter(ctx context.Context, err error) error {
|
||||
log := common.Logger(ctx)
|
||||
// "API error (500): {\"message\":\"service endpoint with name task-57d722ecdecb9e7be16aff17 already exists\"}\n" -> ok since container exists
|
||||
switch {
|
||||
default:
|
||||
return err
|
||||
case err == nil:
|
||||
return err
|
||||
case strings.Contains(err.Error(), "service endpoint with name"):
|
||||
}
|
||||
log.WithError(err).Warn("filtering error")
|
||||
return nil
|
||||
}
|
||||
|
||||
func filterNoSuchContainer(ctx context.Context, err error) error {
|
||||
log := common.Logger(ctx)
|
||||
if err == nil {
|
||||
return nil
|
||||
}
|
||||
_, containerNotFound := err.(*docker.NoSuchContainer)
|
||||
dockerErr, ok := err.(*docker.Error)
|
||||
if containerNotFound || (ok && dockerErr.Status == 404) {
|
||||
log.WithError(err).Error("filtering error")
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (d *dockerWrap) AttachToContainerNonBlocking(ctx context.Context, opts docker.AttachToContainerOptions) (w docker.CloseWaiter, err error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "docker_attach_container")
|
||||
defer span.Finish()
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, retryTimeout)
|
||||
defer cancel()
|
||||
err = d.retry(ctx, func() error {
|
||||
w, err = d.docker.AttachToContainerNonBlocking(opts)
|
||||
if err != nil {
|
||||
// always retry if attach errors, task is running, we want logs!
|
||||
err = temp(err)
|
||||
}
|
||||
return err
|
||||
})
|
||||
return w, err
|
||||
}
|
||||
|
||||
func (d *dockerWrap) WaitContainerWithContext(id string, ctx context.Context) (code int, err error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "docker_wait_container")
|
||||
defer span.Finish()
|
||||
err = d.retry(ctx, func() error {
|
||||
code, err = d.dockerNoTimeout.WaitContainerWithContext(id, ctx)
|
||||
return err
|
||||
})
|
||||
return code, filterNoSuchContainer(ctx, err)
|
||||
}
|
||||
|
||||
func (d *dockerWrap) StartContainerWithContext(id string, hostConfig *docker.HostConfig, ctx context.Context) (err error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "docker_start_container")
|
||||
defer span.Finish()
|
||||
err = d.retry(ctx, func() error {
|
||||
err = d.dockerNoTimeout.StartContainerWithContext(id, hostConfig, ctx)
|
||||
if _, ok := err.(*docker.NoSuchContainer); ok {
|
||||
// for some reason create will sometimes return successfully then say no such container here. wtf. so just retry like normal
|
||||
return temp(err)
|
||||
}
|
||||
return err
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
||||
func (d *dockerWrap) CreateContainer(opts docker.CreateContainerOptions) (c *docker.Container, err error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(opts.Context, "docker_create_container")
|
||||
defer span.Finish()
|
||||
err = d.retry(ctx, func() error {
|
||||
c, err = d.dockerNoTimeout.CreateContainer(opts)
|
||||
return err
|
||||
})
|
||||
return c, err
|
||||
}
|
||||
|
||||
func (d *dockerWrap) PullImage(opts docker.PullImageOptions, auth docker.AuthConfiguration) (err error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(opts.Context, "docker_pull_image")
|
||||
defer span.Finish()
|
||||
err = d.retry(ctx, func() error {
|
||||
err = d.dockerNoTimeout.PullImage(opts, auth)
|
||||
return err
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
||||
func (d *dockerWrap) RemoveContainer(opts docker.RemoveContainerOptions) (err error) {
|
||||
// extract the span, but do not keep the context, since the enclosing context
|
||||
// may be timed out, and we still want to remove the container. TODO in caller? who cares?
|
||||
span, _ := opentracing.StartSpanFromContext(opts.Context, "docker_remove_container")
|
||||
defer span.Finish()
|
||||
ctx := opentracing.ContextWithSpan(context.Background(), span)
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, retryTimeout)
|
||||
defer cancel()
|
||||
err = d.retry(ctx, func() error {
|
||||
err = d.docker.RemoveContainer(opts)
|
||||
return err
|
||||
})
|
||||
return filterNoSuchContainer(ctx, err)
|
||||
}
|
||||
|
||||
func (d *dockerWrap) InspectImage(ctx context.Context, name string) (i *docker.Image, err error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "docker_inspect_image")
|
||||
defer span.Finish()
|
||||
ctx, cancel := context.WithTimeout(ctx, retryTimeout)
|
||||
defer cancel()
|
||||
err = d.retry(ctx, func() error {
|
||||
i, err = d.docker.InspectImage(name)
|
||||
return err
|
||||
})
|
||||
return i, err
|
||||
}
|
||||
|
||||
func (d *dockerWrap) Stats(opts docker.StatsOptions) (err error) {
|
||||
// we can't retry this one this way since the callee closes the
|
||||
// stats chan, need a fancier retry mechanism where we can swap out
|
||||
// channels, but stats isn't crucial so... be lazy for now
|
||||
return d.docker.Stats(opts)
|
||||
|
||||
//err = d.retry(func() error {
|
||||
//err = d.docker.Stats(opts)
|
||||
//return err
|
||||
//})
|
||||
//return err
|
||||
}
|
||||
117
api/agent/drivers/docker/docker_test.go
Normal file
117
api/agent/drivers/docker/docker_test.go
Normal file
@@ -0,0 +1,117 @@
|
||||
package docker
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"io"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/fnproject/fn/api/agent/drivers"
|
||||
"github.com/fsouza/go-dockerclient"
|
||||
)
|
||||
|
||||
type taskDockerTest struct {
|
||||
id string
|
||||
input io.Reader
|
||||
output io.Writer
|
||||
}
|
||||
|
||||
func (f *taskDockerTest) Command() string { return "" }
|
||||
func (f *taskDockerTest) EnvVars() map[string]string {
|
||||
return map[string]string{}
|
||||
}
|
||||
func (f *taskDockerTest) Labels() map[string]string { return nil }
|
||||
func (f *taskDockerTest) Id() string { return f.id }
|
||||
func (f *taskDockerTest) Group() string { return "" }
|
||||
func (f *taskDockerTest) Image() string { return "fnproject/hello" }
|
||||
func (f *taskDockerTest) Timeout() time.Duration { return 30 * time.Second }
|
||||
func (f *taskDockerTest) Logger() (stdout, stderr io.Writer) { return f.output, nil }
|
||||
func (f *taskDockerTest) WriteStat(drivers.Stat) { /* TODO */ }
|
||||
func (f *taskDockerTest) Volumes() [][2]string { return [][2]string{} }
|
||||
func (f *taskDockerTest) Memory() uint64 { return 256 * 1024 * 1024 }
|
||||
func (f *taskDockerTest) WorkDir() string { return "" }
|
||||
func (f *taskDockerTest) Close() {}
|
||||
func (f *taskDockerTest) Input() io.Reader { return f.input }
|
||||
|
||||
func TestRunnerDocker(t *testing.T) {
|
||||
dkr := NewDocker(drivers.Config{})
|
||||
ctx := context.Background()
|
||||
|
||||
task := &taskDockerTest{"test-docker", nil, nil}
|
||||
|
||||
cookie, err := dkr.Prepare(ctx, task)
|
||||
if err != nil {
|
||||
t.Fatal("Couldn't prepare task test")
|
||||
}
|
||||
defer cookie.Close(ctx)
|
||||
|
||||
waiter, err := cookie.Run(ctx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
result, err := waiter.Wait(ctx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
if result.Status() != "success" {
|
||||
t.Fatal("Test should successfully run the image")
|
||||
}
|
||||
}
|
||||
|
||||
func TestRunnerDockerStdin(t *testing.T) {
|
||||
dkr := NewDocker(drivers.Config{})
|
||||
ctx := context.Background()
|
||||
|
||||
input := `{"name": "test"}`
|
||||
var output bytes.Buffer
|
||||
|
||||
task := &taskDockerTest{"test-docker-stdin", bytes.NewBufferString(input), &output}
|
||||
|
||||
cookie, err := dkr.Prepare(ctx, task)
|
||||
if err != nil {
|
||||
t.Fatal("Couldn't prepare task test")
|
||||
}
|
||||
defer cookie.Close(ctx)
|
||||
|
||||
waiter, err := cookie.Run(ctx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
result, err := waiter.Wait(ctx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
if result.Status() != "success" {
|
||||
t.Error("Test should successfully run the image")
|
||||
}
|
||||
|
||||
expect := "Hello test!"
|
||||
got := output.String()
|
||||
if !strings.Contains(got, expect) {
|
||||
t.Errorf("Test expected output to contain '%s', got '%s'", expect, got)
|
||||
}
|
||||
}
|
||||
|
||||
func TestRegistry(t *testing.T) {
|
||||
image := "fnproject/hello"
|
||||
|
||||
sizer, err := CheckRegistry(context.Background(), image, docker.AuthConfiguration{})
|
||||
if err != nil {
|
||||
t.Fatal("expected registry check not to fail, got:", err)
|
||||
}
|
||||
|
||||
size, err := sizer.Size()
|
||||
if err != nil {
|
||||
t.Fatal("expected sizer not to fail, got:", err)
|
||||
}
|
||||
|
||||
if size <= 0 {
|
||||
t.Fatalf("expected positive size for image that exists, got size:", size)
|
||||
}
|
||||
}
|
||||
208
api/agent/drivers/docker/registry.go
Normal file
208
api/agent/drivers/docker/registry.go
Normal file
@@ -0,0 +1,208 @@
|
||||
package docker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/docker/distribution"
|
||||
"github.com/docker/distribution/manifest/schema1"
|
||||
"github.com/docker/distribution/manifest/schema2"
|
||||
"github.com/docker/distribution/reference"
|
||||
registry "github.com/docker/distribution/registry/client"
|
||||
"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/agent/drivers"
|
||||
docker "github.com/fsouza/go-dockerclient"
|
||||
)
|
||||
|
||||
var (
|
||||
// we need these imported so that they can be unmarshaled properly (yes, docker is mean)
|
||||
_ = schema1.SchemaVersion
|
||||
_ = schema2.SchemaVersion
|
||||
|
||||
registryTransport = &http.Transport{
|
||||
Dial: (&net.Dialer{
|
||||
Timeout: 10 * time.Second,
|
||||
KeepAlive: 2 * time.Minute,
|
||||
}).Dial,
|
||||
TLSClientConfig: &tls.Config{
|
||||
ClientSessionCache: tls.NewLRUClientSessionCache(8192),
|
||||
},
|
||||
TLSHandshakeTimeout: 10 * time.Second,
|
||||
MaxIdleConnsPerHost: 32, // TODO tune; we will likely be making lots of requests to same place
|
||||
Proxy: http.ProxyFromEnvironment,
|
||||
IdleConnTimeout: 90 * time.Second,
|
||||
MaxIdleConns: 512,
|
||||
ExpectContinueTimeout: 1 * time.Second,
|
||||
}
|
||||
)
|
||||
|
||||
const hubURL = "https://registry.hub.docker.com"
|
||||
|
||||
// CheckRegistry will return a sizer, which can be used to check the size of an
|
||||
// image if the returned error is nil. If the error returned is nil, then
|
||||
// authentication against the given credentials was successful, if the
|
||||
// configuration or image do not specify a config.ServerAddress,
|
||||
// https://hub.docker.com will be tried. CheckRegistry is a package level
|
||||
// method since rkt can also use docker images, we may be interested in using
|
||||
// rkt w/o a docker driver configured; also, we don't have to tote around a
|
||||
// driver in any tasker that may be interested in registry information (2/2
|
||||
// cases thus far).
|
||||
func CheckRegistry(ctx context.Context, image string, config docker.AuthConfiguration) (Sizer, error) {
|
||||
regURL, repoName, tag := drivers.ParseImage(image)
|
||||
|
||||
repoNamed, err := reference.WithName(repoName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if regURL == "" {
|
||||
// image address overrides credential address
|
||||
regURL = config.ServerAddress
|
||||
}
|
||||
|
||||
regURL, err = registryURL(regURL)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
cm := challenge.NewSimpleManager()
|
||||
|
||||
creds := newCreds(config.Username, config.Password)
|
||||
tran := transport.NewTransport(registryTransport,
|
||||
auth.NewAuthorizer(cm,
|
||||
auth.NewTokenHandler(registryTransport,
|
||||
creds,
|
||||
repoNamed.Name(),
|
||||
"pull",
|
||||
),
|
||||
auth.NewBasicHandler(creds),
|
||||
),
|
||||
)
|
||||
|
||||
tran = &retryWrap{cm, tran}
|
||||
|
||||
repo, err := registry.NewRepository(ctx, repoNamed, regURL, tran)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
manis, err := repo.Manifests(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
mani, err := manis.Get(context.TODO(), "", distribution.WithTag(tag))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
blobs := repo.Blobs(ctx)
|
||||
|
||||
// most registries aren't that great, and won't provide a size for the top
|
||||
// level digest, so we need to sum up all the layers. let this be optional
|
||||
// with the sizer, since tag is good enough to check existence / auth.
|
||||
|
||||
return &sizer{mani, blobs}, nil
|
||||
}
|
||||
|
||||
type retryWrap struct {
|
||||
cm challenge.Manager
|
||||
tran http.RoundTripper
|
||||
}
|
||||
|
||||
func (d *retryWrap) RoundTrip(req *http.Request) (*http.Response, error) {
|
||||
resp, err := d.tran.RoundTrip(req)
|
||||
|
||||
// if it's not authed, we have to add this to the challenge manager,
|
||||
// and then retry it (it will get authed and the challenge then accepted).
|
||||
// why the docker distribution transport doesn't do this for you is
|
||||
// a real testament to what sadists those docker people are.
|
||||
if resp.StatusCode == http.StatusUnauthorized {
|
||||
pingPath := req.URL.Path
|
||||
if v2Root := strings.Index(req.URL.Path, "/v2/"); v2Root != -1 {
|
||||
pingPath = pingPath[:v2Root+4]
|
||||
} else if v1Root := strings.Index(req.URL.Path, "/v1/"); v1Root != -1 {
|
||||
pingPath = pingPath[:v1Root] + "/v2/"
|
||||
}
|
||||
|
||||
// seriously, we have to rewrite this to the ping path,
|
||||
// since looking up challenges strips to this path. YUP. GLHF.
|
||||
ogURL := req.URL.Path
|
||||
resp.Request.URL.Path = pingPath
|
||||
|
||||
d.cm.AddResponse(resp)
|
||||
|
||||
io.Copy(ioutil.Discard, resp.Body)
|
||||
resp.Body.Close()
|
||||
|
||||
// put the original URL path back and try again now...
|
||||
req.URL.Path = ogURL
|
||||
resp, err = d.tran.RoundTrip(req)
|
||||
}
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func newCreds(user, pass string) *creds {
|
||||
return &creds{m: make(map[string]string), user: user, pass: pass}
|
||||
}
|
||||
|
||||
// implement auth.CredentialStore
|
||||
type creds struct {
|
||||
m map[string]string
|
||||
user, pass string
|
||||
}
|
||||
|
||||
func (c *creds) Basic(u *url.URL) (string, string) { return c.user, c.pass }
|
||||
func (c *creds) RefreshToken(u *url.URL, service string) string { return c.m[service] }
|
||||
func (c *creds) SetRefreshToken(u *url.URL, service, token string) { c.m[service] = token }
|
||||
|
||||
// Sizer returns size information. This interface is liable to contain more
|
||||
// than a size at some point, change as needed.
|
||||
type Sizer interface {
|
||||
Size() (int64, error)
|
||||
}
|
||||
|
||||
type sizer struct {
|
||||
mani distribution.Manifest
|
||||
blobs distribution.BlobStore
|
||||
}
|
||||
|
||||
func (s *sizer) Size() (int64, error) {
|
||||
var sum int64
|
||||
for _, r := range s.mani.References() {
|
||||
desc, err := s.blobs.Stat(context.TODO(), r.Digest)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
sum += desc.Size
|
||||
}
|
||||
return sum, nil
|
||||
}
|
||||
|
||||
func registryURL(addr string) (string, error) {
|
||||
if addr == "" || strings.Contains(addr, "hub.docker.com") || strings.Contains(addr, "index.docker.io") {
|
||||
return hubURL, nil
|
||||
}
|
||||
|
||||
uri, err := url.Parse(addr)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
if uri.Scheme == "" {
|
||||
uri.Scheme = "https"
|
||||
}
|
||||
uri.Path = strings.TrimSuffix(uri.Path, "/")
|
||||
uri.Path = strings.TrimPrefix(uri.Path, "/v2")
|
||||
uri.Path = strings.TrimPrefix(uri.Path, "/v1") // just try this, if it fails it fails, not supporting v1
|
||||
return uri.String(), nil
|
||||
}
|
||||
269
api/agent/drivers/driver.go
Normal file
269
api/agent/drivers/driver.go
Normal file
@@ -0,0 +1,269 @@
|
||||
// Interface for all container drivers
|
||||
|
||||
package drivers
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"io"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
// A DriverCookie identifies a unique request to run a task.
|
||||
//
|
||||
// Clients should always call Close() on a DriverCookie after they are done
|
||||
// with it.
|
||||
type Cookie interface {
|
||||
// Close should clean up any resources the cookie was using, or was going to use.
|
||||
Close(ctx context.Context) error
|
||||
|
||||
// Run should execute task on the implementation.
|
||||
// RunResult captures the result of task execution. This means if task
|
||||
// execution fails due to a problem in the task, Run() MUST return a valid
|
||||
// RunResult and nil as the error. The RunResult's Error() and Status()
|
||||
// should be used to indicate failure.
|
||||
// If the implementation itself suffers problems (lost of network, out of
|
||||
// disk etc.), a nil RunResult and an error message is preferred.
|
||||
//
|
||||
// Run() MUST monitor the context. task cancellation is indicated by
|
||||
// cancelling the context.
|
||||
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 {
|
||||
// Prepare can be used in order to do any preparation that a specific driver
|
||||
// may need to do before running the task, and can be useful to put
|
||||
// preparation that the task can recover from into (i.e. if pulling an image
|
||||
// fails because a registry is down, the task doesn't need to be failed). It
|
||||
// returns a cookie that can be used to execute the task.
|
||||
// Callers should Close the cookie regardless of whether they run it.
|
||||
//
|
||||
// The returned cookie should respect the task's timeout when it is run.
|
||||
Prepare(ctx context.Context, task ContainerTask) (Cookie, error)
|
||||
}
|
||||
|
||||
// RunResult indicates only the final state of the task.
|
||||
type RunResult interface {
|
||||
// Error is an actionable/checkable error from the container.
|
||||
error
|
||||
|
||||
// Status should return the current status of the task.
|
||||
// Only valid options are {"error", "success", "timeout", "killed", "cancelled"}.
|
||||
Status() string
|
||||
}
|
||||
|
||||
// The ContainerTask interface guides task execution across a wide variety of
|
||||
// container oriented runtimes.
|
||||
// This interface is unstable.
|
||||
//
|
||||
// FIXME: This interface is large, and it is currently a little Docker specific.
|
||||
type ContainerTask interface {
|
||||
// Command returns the command to run within the container.
|
||||
Command() string
|
||||
|
||||
// EnvVars returns environment variable key-value pairs.
|
||||
EnvVars() map[string]string
|
||||
|
||||
// Input feeds the container with data
|
||||
Input() io.Reader
|
||||
|
||||
// The id to assign the container
|
||||
Id() string
|
||||
|
||||
// Image returns the runtime specific image to run.
|
||||
Image() string
|
||||
|
||||
// Timeout specifies the maximum time a task is allowed to run. Return 0 to let it run forever.
|
||||
Timeout() time.Duration
|
||||
|
||||
// Driver will write output log from task execution to these writers. Must be
|
||||
// non-nil. Use io.Discard if log is irrelevant.
|
||||
Logger() (stdout, stderr io.Writer)
|
||||
|
||||
// WriteStat writes a single Stat, implementation need not be thread safe.
|
||||
WriteStat(Stat)
|
||||
|
||||
// Volumes returns an array of 2-element tuples indicating storage volume mounts.
|
||||
// The first element is the path on the host, and the second element is the
|
||||
// path in the container.
|
||||
Volumes() [][2]string
|
||||
|
||||
// Memory determines the max amount of RAM given to the container to use.
|
||||
// 0 is unlimited.
|
||||
Memory() uint64
|
||||
|
||||
// WorkDir returns the working directory to use for the task. Empty string
|
||||
// leaves it unset.
|
||||
WorkDir() string
|
||||
|
||||
// Close is used to perform cleanup after task execution.
|
||||
// Close should be safe to call multiple times.
|
||||
Close()
|
||||
}
|
||||
|
||||
// Stat is a bucket of stats from a driver at a point in time for a certain task.
|
||||
type Stat struct {
|
||||
Timestamp time.Time
|
||||
Metrics map[string]uint64
|
||||
}
|
||||
|
||||
// Set of acceptable errors coming from container engines to TaskRunner
|
||||
var (
|
||||
// ErrOutOfMemory for OOM in container engine
|
||||
ErrOutOfMemory = userError(errors.New("out of memory error"))
|
||||
)
|
||||
|
||||
// TODO agent.UserError should be elsewhere
|
||||
func userError(err error) error { return &ue{err} }
|
||||
|
||||
type ue struct {
|
||||
error
|
||||
}
|
||||
|
||||
func (u *ue) UserVisible() bool { return true }
|
||||
|
||||
// TODO: ensure some type is applied to these statuses.
|
||||
const (
|
||||
// task statuses
|
||||
StatusRunning = "running"
|
||||
StatusSuccess = "success"
|
||||
StatusError = "error"
|
||||
StatusTimeout = "timeout"
|
||||
StatusKilled = "killed"
|
||||
StatusCancelled = "cancelled"
|
||||
)
|
||||
|
||||
type Config struct {
|
||||
Docker string `json:"docker"`
|
||||
// TODO CPUShares should likely be on a per container basis
|
||||
CPUShares int64 `json:"cpu_shares"`
|
||||
}
|
||||
|
||||
// for tests
|
||||
func DefaultConfig() Config {
|
||||
return Config{
|
||||
Docker: "unix:///var/run/docker.sock",
|
||||
CPUShares: 0,
|
||||
}
|
||||
}
|
||||
|
||||
func average(samples []Stat) (Stat, bool) {
|
||||
l := len(samples)
|
||||
if l == 0 {
|
||||
return Stat{}, false
|
||||
} else if l == 1 {
|
||||
return samples[0], true
|
||||
}
|
||||
|
||||
s := Stat{
|
||||
Metrics: samples[0].Metrics, // Recycle Metrics map from first sample
|
||||
}
|
||||
t := samples[0].Timestamp.UnixNano() / int64(l)
|
||||
for _, sample := range samples[1:] {
|
||||
t += sample.Timestamp.UnixNano() / int64(l)
|
||||
for k, v := range sample.Metrics {
|
||||
s.Metrics[k] += v
|
||||
}
|
||||
}
|
||||
|
||||
s.Timestamp = time.Unix(0, t)
|
||||
for k, v := range s.Metrics {
|
||||
s.Metrics[k] = v / uint64(l)
|
||||
}
|
||||
return s, true
|
||||
}
|
||||
|
||||
// Decimate will down sample to a max number of points in a given sample by
|
||||
// averaging samples together. i.e. max=240, if we have 240 samples, return
|
||||
// them all, if we have 480 samples, every 2 samples average them (and time
|
||||
// distance), and return 240 samples. This is relatively naive and if len(in) >
|
||||
// max, <= max points will be returned, not necessarily max: length(out) =
|
||||
// ceil(length(in)/max) -- feel free to fix this, setting a relatively high max
|
||||
// will allow good enough granularity at higher lengths, i.e. for max of 1 hour
|
||||
// tasks, sampling every 1s, decimate will return 15s samples if max=240.
|
||||
// Large gaps in time between samples (a factor > (last-start)/max) will result
|
||||
// in a shorter list being returned to account for lost samples.
|
||||
// Decimate will modify the input list for efficiency, it is not copy safe.
|
||||
// Input must be sorted by timestamp or this will fail gloriously.
|
||||
func Decimate(maxSamples int, stats []Stat) []Stat {
|
||||
if len(stats) <= maxSamples {
|
||||
return stats
|
||||
} else if maxSamples <= 0 { // protect from nefarious input
|
||||
return nil
|
||||
}
|
||||
|
||||
start := stats[0].Timestamp
|
||||
window := stats[len(stats)-1].Timestamp.Sub(start) / time.Duration(maxSamples)
|
||||
|
||||
nextEntry, current := 0, start // nextEntry is the index tracking next Stats record location
|
||||
for x := 0; x < len(stats); {
|
||||
isLastEntry := nextEntry == maxSamples-1 // Last bin is larger than others to handle imprecision
|
||||
|
||||
var samples []Stat
|
||||
for offset := 0; x+offset < len(stats); offset++ { // Iterate through samples until out of window
|
||||
if !isLastEntry && stats[x+offset].Timestamp.After(current.Add(window)) {
|
||||
break
|
||||
}
|
||||
samples = stats[x : x+offset+1]
|
||||
}
|
||||
|
||||
x += len(samples) // Skip # of samples for next window
|
||||
if entry, ok := average(samples); ok { // Only record Stat if 1+ samples exist
|
||||
stats[nextEntry] = entry
|
||||
nextEntry++
|
||||
}
|
||||
|
||||
current = current.Add(window)
|
||||
}
|
||||
return stats[:nextEntry] // Return slice of []Stats that was modified with averages
|
||||
}
|
||||
|
||||
// https://github.com/fsouza/go-dockerclient/blob/master/misc.go#L166
|
||||
func parseRepositoryTag(repoTag string) (repository string, tag string) {
|
||||
parts := strings.SplitN(repoTag, "@", 2)
|
||||
repoTag = parts[0]
|
||||
n := strings.LastIndex(repoTag, ":")
|
||||
if n < 0 {
|
||||
return repoTag, ""
|
||||
}
|
||||
if tag := repoTag[n+1:]; !strings.Contains(tag, "/") {
|
||||
return repoTag[:n], tag
|
||||
}
|
||||
return repoTag, ""
|
||||
}
|
||||
|
||||
func ParseImage(image string) (registry, repo, tag string) {
|
||||
repo, tag = parseRepositoryTag(image)
|
||||
// Officially sanctioned at https://github.com/moby/moby/blob/master/registry/session.go#L319 to deal with "Official Repositories".
|
||||
// Without this, token auth fails.
|
||||
// Registries must exist at root (https://github.com/moby/moby/issues/7067#issuecomment-54302847)
|
||||
// This cannot support the `library/` shortcut for private registries.
|
||||
parts := strings.Split(repo, "/")
|
||||
switch len(parts) {
|
||||
case 1:
|
||||
repo = "library/" + repo
|
||||
case 2:
|
||||
if strings.Contains(repo, ".") {
|
||||
registry = parts[0]
|
||||
repo = parts[1]
|
||||
}
|
||||
case 3:
|
||||
registry = parts[0]
|
||||
repo = parts[1] + "/" + parts[2]
|
||||
}
|
||||
|
||||
if tag == "" {
|
||||
tag = "latest"
|
||||
}
|
||||
|
||||
return registry, repo, tag
|
||||
}
|
||||
114
api/agent/drivers/driver_test.go
Normal file
114
api/agent/drivers/driver_test.go
Normal file
@@ -0,0 +1,114 @@
|
||||
package drivers
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestAverage(t *testing.T) {
|
||||
start := time.Date(2016, 8, 11, 0, 0, 0, 0, time.UTC)
|
||||
stats := make([]Stat, 10)
|
||||
for i := 0; i < len(stats); i++ {
|
||||
stats[i] = Stat{
|
||||
Timestamp: start.Add(time.Duration(i) * time.Minute),
|
||||
Metrics: map[string]uint64{"x": uint64(i)},
|
||||
}
|
||||
}
|
||||
|
||||
res, ok := average(stats)
|
||||
if !ok {
|
||||
t.Error("Expected good record")
|
||||
}
|
||||
|
||||
expectedV := uint64(4)
|
||||
if v, ok := res.Metrics["x"]; !ok || v != expectedV {
|
||||
t.Error("Actual average didn't match expected", "actual", v, "expected", expectedV)
|
||||
}
|
||||
|
||||
expectedT := time.Unix(1470873870, 0)
|
||||
if res.Timestamp != expectedT {
|
||||
t.Error("Actual average didn't match expected", "actual", res.Timestamp, "expected", expectedT)
|
||||
}
|
||||
}
|
||||
|
||||
func TestDecimate(t *testing.T) {
|
||||
start := time.Now()
|
||||
stats := make([]Stat, 480)
|
||||
for i := range stats {
|
||||
stats[i] = Stat{
|
||||
Timestamp: start.Add(time.Duration(i) * time.Second),
|
||||
Metrics: map[string]uint64{"x": uint64(i)},
|
||||
}
|
||||
// t.Log(stats[i])
|
||||
}
|
||||
|
||||
stats = Decimate(240, stats)
|
||||
if len(stats) != 240 {
|
||||
t.Error("decimate function bad", len(stats))
|
||||
}
|
||||
|
||||
//for i := range stats {
|
||||
//t.Log(stats[i])
|
||||
//}
|
||||
|
||||
stats = make([]Stat, 700)
|
||||
for i := range stats {
|
||||
stats[i] = Stat{
|
||||
Timestamp: start.Add(time.Duration(i) * time.Second),
|
||||
Metrics: map[string]uint64{"x": uint64(i)},
|
||||
}
|
||||
}
|
||||
stats = Decimate(240, stats)
|
||||
if len(stats) != 240 {
|
||||
t.Error("decimate function bad", len(stats))
|
||||
}
|
||||
|
||||
stats = make([]Stat, 300)
|
||||
for i := range stats {
|
||||
stats[i] = Stat{
|
||||
Timestamp: start.Add(time.Duration(i) * time.Second),
|
||||
Metrics: map[string]uint64{"x": uint64(i)},
|
||||
}
|
||||
}
|
||||
stats = Decimate(240, stats)
|
||||
if len(stats) != 240 {
|
||||
t.Error("decimate function bad", len(stats))
|
||||
}
|
||||
|
||||
stats = make([]Stat, 300)
|
||||
for i := range stats {
|
||||
if i == 150 {
|
||||
// leave 1 large gap
|
||||
start = start.Add(20 * time.Minute)
|
||||
}
|
||||
stats[i] = Stat{
|
||||
Timestamp: start.Add(time.Duration(i) * time.Second),
|
||||
Metrics: map[string]uint64{"x": uint64(i)},
|
||||
}
|
||||
}
|
||||
stats = Decimate(240, stats)
|
||||
if len(stats) != 49 {
|
||||
t.Error("decimate function bad", len(stats))
|
||||
}
|
||||
}
|
||||
|
||||
func TestParseImage(t *testing.T) {
|
||||
cases := map[string][]string{
|
||||
"fnproject/hello": {"", "fnproject/hello", "latest"},
|
||||
"fnproject/hello:v1": {"", "fnproject/hello", "v1"},
|
||||
"my.registry/hello": {"my.registry", "hello", "latest"},
|
||||
"my.registry/hello:v1": {"my.registry", "hello", "v1"},
|
||||
"mongo": {"", "library/mongo", "latest"},
|
||||
"mongo:v1": {"", "library/mongo", "v1"},
|
||||
"quay.com/fnproject/hello": {"quay.com", "fnproject/hello", "latest"},
|
||||
"quay.com:8080/fnproject/hello:v2": {"quay.com:8080", "fnproject/hello", "v2"},
|
||||
"localhost.localdomain:5000/samalba/hipache:latest": {"localhost.localdomain:5000", "samalba/hipache", "latest"},
|
||||
}
|
||||
|
||||
for in, out := range cases {
|
||||
reg, repo, tag := ParseImage(in)
|
||||
if reg != out[0] || repo != out[1] || tag != out[2] {
|
||||
t.Errorf("Test input %q wasn't parsed as expected. Expected %q, got %q", in, out, []string{reg, repo, tag})
|
||||
}
|
||||
}
|
||||
}
|
||||
49
api/agent/drivers/mock/mocker.go
Normal file
49
api/agent/drivers/mock/mocker.go
Normal file
@@ -0,0 +1,49 @@
|
||||
package mock
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/fnproject/fn/api/agent/drivers"
|
||||
)
|
||||
|
||||
func New() drivers.Driver {
|
||||
return &Mocker{}
|
||||
}
|
||||
|
||||
type Mocker struct {
|
||||
count int
|
||||
}
|
||||
|
||||
func (m *Mocker) Prepare(context.Context, drivers.ContainerTask) (drivers.Cookie, error) {
|
||||
return &cookie{m}, nil
|
||||
}
|
||||
|
||||
type cookie struct {
|
||||
m *Mocker
|
||||
}
|
||||
|
||||
func (c *cookie) Close(context.Context) error { return nil }
|
||||
|
||||
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.")
|
||||
}
|
||||
return &runResult{
|
||||
error: nil,
|
||||
status: "success",
|
||||
start: time.Now(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
type runResult struct {
|
||||
error
|
||||
status string
|
||||
start time.Time
|
||||
}
|
||||
|
||||
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 }
|
||||
217
api/agent/func_logger.go
Normal file
217
api/agent/func_logger.go
Normal file
@@ -0,0 +1,217 @@
|
||||
package agent
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"sync"
|
||||
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
var (
|
||||
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.
|
||||
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()
|
||||
bufPool.Put(lbuf)
|
||||
logPool.Put(dbuf)
|
||||
return nil
|
||||
}
|
||||
|
||||
// we don't need to limit the log writer, but we do need it to dispense lines
|
||||
linew := newLineWriterWithBuffer(lbuf, &logWriter{
|
||||
ctx: ctx,
|
||||
appName: appName,
|
||||
path: path,
|
||||
image: image,
|
||||
reqID: reqID,
|
||||
})
|
||||
|
||||
const MB = 1 * 1024 * 1024 // pick a number any number.. TODO configurable ?
|
||||
|
||||
// we don't need to log per line to db, but we do need to limit it
|
||||
limitw := newLimitWriter(MB, &dbWriter{
|
||||
Buffer: dbuf,
|
||||
db: logDB,
|
||||
ctx: ctx,
|
||||
reqID: reqID,
|
||||
})
|
||||
|
||||
// TODO / NOTE: we want linew to be first becauase limitw may error if limit
|
||||
// is reached but we still want to log. we should probably ignore hitting the
|
||||
// limit error since we really just want to not write too much to db and
|
||||
// that's handled as is. put buffers back last to avoid misuse, if there's
|
||||
// an error they won't get put back and that's really okay too.
|
||||
return multiWriteCloser{linew, limitw, &fCloser{close}}
|
||||
}
|
||||
|
||||
// implements passthrough Write & arbitrary func close to have a seat at the cool kids lunch table
|
||||
type fCloser struct {
|
||||
close func() error
|
||||
}
|
||||
|
||||
func (f *fCloser) Write(b []byte) (int, error) { return len(b), nil }
|
||||
func (f *fCloser) Close() error { return f.close() }
|
||||
|
||||
// multiWriteCloser returns the first write or close that returns a non-nil
|
||||
// err, if no non-nil err is returned, then the returned bytes written will be
|
||||
// from the last call to write.
|
||||
type multiWriteCloser []io.WriteCloser
|
||||
|
||||
func (m multiWriteCloser) Write(b []byte) (n int, err error) {
|
||||
for _, mw := range m {
|
||||
n, err = mw.Write(b)
|
||||
if err != nil {
|
||||
return n, err
|
||||
}
|
||||
}
|
||||
return n, err
|
||||
}
|
||||
|
||||
func (m multiWriteCloser) Close() (err error) {
|
||||
for _, mw := range m {
|
||||
err = mw.Close()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// logWriter will log (to real stderr) every call to Write as a line. it should
|
||||
// be wrapped with a lineWriter so that the output makes sense.
|
||||
type logWriter struct {
|
||||
ctx context.Context
|
||||
appName string
|
||||
path string
|
||||
image string
|
||||
reqID string
|
||||
}
|
||||
|
||||
func (l *logWriter) Write(b []byte) (int, error) {
|
||||
log := common.Logger(l.ctx)
|
||||
log = log.WithFields(logrus.Fields{"user_log": true, "app_name": l.appName, "path": l.path, "image": l.image, "call_id": l.reqID})
|
||||
log.Println(string(b))
|
||||
return len(b), nil
|
||||
}
|
||||
|
||||
// lineWriter buffers all calls to Write and will call Write
|
||||
// on the underlying writer once per new line. Close must
|
||||
// be called to ensure that the buffer is flushed, and a newline
|
||||
// will be appended in Close if none is present.
|
||||
type lineWriter struct {
|
||||
b *bytes.Buffer
|
||||
w io.Writer
|
||||
}
|
||||
|
||||
func newLineWriter(w io.Writer) io.WriteCloser {
|
||||
return &lineWriter{b: new(bytes.Buffer), w: w}
|
||||
}
|
||||
|
||||
func newLineWriterWithBuffer(b *bytes.Buffer, w io.Writer) io.WriteCloser {
|
||||
return &lineWriter{b: b, w: w}
|
||||
}
|
||||
|
||||
func (li *lineWriter) Write(ogb []byte) (int, error) {
|
||||
li.b.Write(ogb) // bytes.Buffer is guaranteed, read it!
|
||||
|
||||
for {
|
||||
b := li.b.Bytes()
|
||||
i := bytes.IndexByte(b, '\n')
|
||||
if i < 0 {
|
||||
break // no more newlines in buffer
|
||||
}
|
||||
|
||||
// write in this line and advance buffer past it
|
||||
l := b[:i+1]
|
||||
ns, err := li.w.Write(l)
|
||||
if err != nil {
|
||||
return ns, err
|
||||
}
|
||||
li.b.Next(len(l))
|
||||
}
|
||||
|
||||
// technically we wrote all the bytes, so make things appear normal
|
||||
return len(ogb), nil
|
||||
}
|
||||
|
||||
func (li *lineWriter) Close() error {
|
||||
// flush the remaining bytes in the buffer to underlying writer, adding a
|
||||
// newline if needed
|
||||
b := li.b.Bytes()
|
||||
if len(b) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
if b[len(b)-1] != '\n' {
|
||||
b = append(b, '\n')
|
||||
}
|
||||
_, err := li.w.Write(b)
|
||||
return err
|
||||
}
|
||||
|
||||
// dbWriter accumulates all calls to Write into an in memory buffer
|
||||
// and writes them to the database when Close is called, returning
|
||||
// any error from Close. it should be wrapped in a limitWriter to
|
||||
// prevent blowing out the buffer and bloating the db.
|
||||
type dbWriter struct {
|
||||
*bytes.Buffer
|
||||
|
||||
db models.LogStore
|
||||
ctx context.Context
|
||||
reqID string
|
||||
}
|
||||
|
||||
func (w *dbWriter) Close() error {
|
||||
return w.db.InsertLog(w.ctx, w.reqID, w.String())
|
||||
}
|
||||
|
||||
func (w *dbWriter) Write(b []byte) (int, error) {
|
||||
return w.Buffer.Write(b)
|
||||
}
|
||||
|
||||
// overrides Write, keeps Close
|
||||
type limitWriter struct {
|
||||
n, max int
|
||||
io.WriteCloser
|
||||
}
|
||||
|
||||
func newLimitWriter(max int, w io.WriteCloser) io.WriteCloser {
|
||||
return &limitWriter{max: max, WriteCloser: w}
|
||||
}
|
||||
|
||||
func (l *limitWriter) Write(b []byte) (int, error) {
|
||||
if l.n >= l.max {
|
||||
return 0, errors.New("max log size exceeded, truncating log")
|
||||
}
|
||||
if l.n+len(b) >= l.max {
|
||||
// cut off to prevent gigantic line attack
|
||||
b = b[:l.max-l.n]
|
||||
}
|
||||
n, err := l.WriteCloser.Write(b)
|
||||
l.n += n
|
||||
if l.n >= l.max {
|
||||
// write in truncation message to log once
|
||||
l.WriteCloser.Write([]byte(fmt.Sprintf("\n-----max log size %d bytes exceeded, truncating log-----\n")))
|
||||
}
|
||||
return n, err
|
||||
}
|
||||
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 }
|
||||
78
api/agent/protocol/factory.go
Normal file
78
api/agent/protocol/factory.go
Normal file
@@ -0,0 +1,78 @@
|
||||
package protocol
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"io"
|
||||
"net/http"
|
||||
|
||||
"github.com/fnproject/fn/api/models"
|
||||
)
|
||||
|
||||
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
|
||||
|
||||
// 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.
|
||||
type Protocol string
|
||||
|
||||
// hot function protocols
|
||||
const (
|
||||
Default Protocol = models.FormatDefault
|
||||
HTTP Protocol = models.FormatHTTP
|
||||
Empty Protocol = ""
|
||||
)
|
||||
|
||||
func (p *Protocol) UnmarshalJSON(b []byte) error {
|
||||
switch Protocol(b) {
|
||||
case Empty, Default:
|
||||
*p = Default
|
||||
case HTTP:
|
||||
*p = HTTP
|
||||
default:
|
||||
return errInvalidProtocol
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p Protocol) MarshalJSON() ([]byte, error) {
|
||||
switch p {
|
||||
case Empty, Default:
|
||||
return []byte(Default), nil
|
||||
case HTTP:
|
||||
return []byte(HTTP), nil
|
||||
}
|
||||
return nil, 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 {
|
||||
switch p {
|
||||
case HTTP:
|
||||
return &HTTPProtocol{in, out}
|
||||
case Default, Empty:
|
||||
return &DefaultProtocol{}
|
||||
}
|
||||
return &errorProto{errInvalidProtocol}
|
||||
}
|
||||
|
||||
// IsStreamable says whether the given protocol can be used for streaming into
|
||||
// hot functions.
|
||||
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
|
||||
}
|
||||
50
api/agent/stats.go
Normal file
50
api/agent/stats.go
Normal file
@@ -0,0 +1,50 @@
|
||||
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
|
||||
}
|
||||
|
||||
type Stats struct {
|
||||
Queue uint64
|
||||
Running uint64
|
||||
Complete uint64
|
||||
}
|
||||
|
||||
func (s *stats) Enqueue() {
|
||||
s.mu.Lock()
|
||||
s.queue++
|
||||
s.mu.Unlock()
|
||||
}
|
||||
|
||||
func (s *stats) Start() {
|
||||
s.mu.Lock()
|
||||
s.queue--
|
||||
s.running++
|
||||
s.mu.Unlock()
|
||||
}
|
||||
|
||||
func (s *stats) Complete() {
|
||||
s.mu.Lock()
|
||||
s.running--
|
||||
s.complete++
|
||||
s.mu.Unlock()
|
||||
}
|
||||
|
||||
func (s *stats) Stats() Stats {
|
||||
var stats Stats
|
||||
s.mu.Lock()
|
||||
stats.Running = s.running
|
||||
stats.Complete = s.complete
|
||||
stats.Queue = s.queue
|
||||
s.mu.Unlock()
|
||||
return stats
|
||||
}
|
||||
660
api/agent/tree.go
Normal file
660
api/agent/tree.go
Normal file
@@ -0,0 +1,660 @@
|
||||
// Copyright 2013 Julien Schmidt. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style license that can be found
|
||||
// in the LICENSE file.
|
||||
|
||||
// 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"
|
||||
"strings"
|
||||
"unicode"
|
||||
"unicode/utf8"
|
||||
)
|
||||
|
||||
type Handle func(http.ResponseWriter, *http.Request, Params)
|
||||
type Param struct {
|
||||
Key string
|
||||
Value string
|
||||
}
|
||||
type Params []Param
|
||||
|
||||
func min(a, b int) int {
|
||||
if a <= b {
|
||||
return a
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
func countParams(path string) uint8 {
|
||||
var n uint
|
||||
for i := 0; i < len(path); i++ {
|
||||
if path[i] != ':' && path[i] != '*' {
|
||||
continue
|
||||
}
|
||||
n++
|
||||
}
|
||||
if n >= 255 {
|
||||
return 255
|
||||
}
|
||||
return uint8(n)
|
||||
}
|
||||
|
||||
type nodeType uint8
|
||||
|
||||
const (
|
||||
static nodeType = iota // default
|
||||
root
|
||||
param
|
||||
catchAll
|
||||
)
|
||||
|
||||
type node struct {
|
||||
path string
|
||||
wildChild bool
|
||||
nType nodeType
|
||||
maxParams uint8
|
||||
indices string
|
||||
children []*node
|
||||
handle Handle
|
||||
priority uint32
|
||||
}
|
||||
|
||||
// increments priority of the given child and reorders if necessary
|
||||
func (n *node) incrementChildPrio(pos int) int {
|
||||
n.children[pos].priority++
|
||||
prio := n.children[pos].priority
|
||||
|
||||
// adjust position (move to front)
|
||||
newPos := pos
|
||||
for newPos > 0 && n.children[newPos-1].priority < prio {
|
||||
// swap node positions
|
||||
tmpN := n.children[newPos-1]
|
||||
n.children[newPos-1] = n.children[newPos]
|
||||
n.children[newPos] = tmpN
|
||||
|
||||
newPos--
|
||||
}
|
||||
|
||||
// build new index char string
|
||||
if newPos != pos {
|
||||
n.indices = n.indices[:newPos] + // unchanged prefix, might be empty
|
||||
n.indices[pos:pos+1] + // the index char we move
|
||||
n.indices[newPos:pos] + n.indices[pos+1:] // rest without char at 'pos'
|
||||
}
|
||||
|
||||
return newPos
|
||||
}
|
||||
|
||||
// addRoute adds a node with the given handle to the path.
|
||||
// Not concurrency-safe!
|
||||
func (n *node) addRoute(path string, handle Handle) {
|
||||
fullPath := path
|
||||
n.priority++
|
||||
numParams := countParams(path)
|
||||
|
||||
// non-empty tree
|
||||
if len(n.path) > 0 || len(n.children) > 0 {
|
||||
walk:
|
||||
for {
|
||||
// Update maxParams of the current node
|
||||
if numParams > n.maxParams {
|
||||
n.maxParams = numParams
|
||||
}
|
||||
|
||||
// Find the longest common prefix.
|
||||
// This also implies that the common prefix contains no ':' or '*'
|
||||
// since the existing key can't contain those chars.
|
||||
i := 0
|
||||
max := min(len(path), len(n.path))
|
||||
for i < max && path[i] == n.path[i] {
|
||||
i++
|
||||
}
|
||||
|
||||
// Split edge
|
||||
if i < len(n.path) {
|
||||
child := node{
|
||||
path: n.path[i:],
|
||||
wildChild: n.wildChild,
|
||||
nType: static,
|
||||
indices: n.indices,
|
||||
children: n.children,
|
||||
handle: n.handle,
|
||||
priority: n.priority - 1,
|
||||
}
|
||||
|
||||
// Update maxParams (max of all children)
|
||||
for i := range child.children {
|
||||
if child.children[i].maxParams > child.maxParams {
|
||||
child.maxParams = child.children[i].maxParams
|
||||
}
|
||||
}
|
||||
|
||||
n.children = []*node{&child}
|
||||
// []byte for proper unicode char conversion, see #65
|
||||
n.indices = string([]byte{n.path[i]})
|
||||
n.path = path[:i]
|
||||
n.handle = nil
|
||||
n.wildChild = false
|
||||
}
|
||||
|
||||
// Make new node a child of this node
|
||||
if i < len(path) {
|
||||
path = path[i:]
|
||||
|
||||
if n.wildChild {
|
||||
n = n.children[0]
|
||||
n.priority++
|
||||
|
||||
// Update maxParams of the child node
|
||||
if numParams > n.maxParams {
|
||||
n.maxParams = numParams
|
||||
}
|
||||
numParams--
|
||||
|
||||
// Check if the wildcard matches
|
||||
if len(path) >= len(n.path) && n.path == path[:len(n.path)] {
|
||||
// check for longer wildcard, e.g. :name and :names
|
||||
if len(n.path) >= len(path) || path[len(n.path)] == '/' {
|
||||
continue walk
|
||||
}
|
||||
}
|
||||
|
||||
panic("path segment '" + path +
|
||||
"' conflicts with existing wildcard '" + n.path +
|
||||
"' in path '" + fullPath + "'")
|
||||
}
|
||||
|
||||
c := path[0]
|
||||
|
||||
// slash after param
|
||||
if n.nType == param && c == '/' && len(n.children) == 1 {
|
||||
n = n.children[0]
|
||||
n.priority++
|
||||
continue walk
|
||||
}
|
||||
|
||||
// Check if a child with the next path byte exists
|
||||
for i := 0; i < len(n.indices); i++ {
|
||||
if c == n.indices[i] {
|
||||
i = n.incrementChildPrio(i)
|
||||
n = n.children[i]
|
||||
continue walk
|
||||
}
|
||||
}
|
||||
|
||||
// Otherwise insert it
|
||||
if c != ':' && c != '*' {
|
||||
// []byte for proper unicode char conversion, see #65
|
||||
n.indices += string([]byte{c})
|
||||
child := &node{
|
||||
maxParams: numParams,
|
||||
}
|
||||
n.children = append(n.children, child)
|
||||
n.incrementChildPrio(len(n.indices) - 1)
|
||||
n = child
|
||||
}
|
||||
n.insertChild(numParams, path, fullPath, handle)
|
||||
return
|
||||
|
||||
} else if i == len(path) { // Make node a (in-path) leaf
|
||||
if n.handle != nil {
|
||||
panic("a handle is already registered for path '" + fullPath + "'")
|
||||
}
|
||||
n.handle = handle
|
||||
}
|
||||
return
|
||||
}
|
||||
} else { // Empty tree
|
||||
n.insertChild(numParams, path, fullPath, handle)
|
||||
n.nType = root
|
||||
}
|
||||
}
|
||||
|
||||
func (n *node) insertChild(numParams uint8, path, fullPath string, handle Handle) {
|
||||
var offset int // already handled bytes of the path
|
||||
|
||||
// find prefix until first wildcard (beginning with ':'' or '*'')
|
||||
for i, max := 0, len(path); numParams > 0; i++ {
|
||||
c := path[i]
|
||||
if c != ':' && c != '*' {
|
||||
continue
|
||||
}
|
||||
|
||||
// find wildcard end (either '/' or path end)
|
||||
end := i + 1
|
||||
for end < max && path[end] != '/' {
|
||||
switch path[end] {
|
||||
// the wildcard name must not contain ':' and '*'
|
||||
case ':', '*':
|
||||
panic("only one wildcard per path segment is allowed, has: '" +
|
||||
path[i:] + "' in path '" + fullPath + "'")
|
||||
default:
|
||||
end++
|
||||
}
|
||||
}
|
||||
|
||||
// check if this Node existing children which would be
|
||||
// unreachable if we insert the wildcard here
|
||||
if len(n.children) > 0 {
|
||||
panic("wildcard route '" + path[i:end] +
|
||||
"' conflicts with existing children in path '" + fullPath + "'")
|
||||
}
|
||||
|
||||
// check if the wildcard has a name
|
||||
if end-i < 2 {
|
||||
panic("wildcards must be named with a non-empty name in path '" + fullPath + "'")
|
||||
}
|
||||
|
||||
if c == ':' { // param
|
||||
// split path at the beginning of the wildcard
|
||||
if i > 0 {
|
||||
n.path = path[offset:i]
|
||||
offset = i
|
||||
}
|
||||
|
||||
child := &node{
|
||||
nType: param,
|
||||
maxParams: numParams,
|
||||
}
|
||||
n.children = []*node{child}
|
||||
n.wildChild = true
|
||||
n = child
|
||||
n.priority++
|
||||
numParams--
|
||||
|
||||
// if the path doesn't end with the wildcard, then there
|
||||
// will be another non-wildcard subpath starting with '/'
|
||||
if end < max {
|
||||
n.path = path[offset:end]
|
||||
offset = end
|
||||
|
||||
child := &node{
|
||||
maxParams: numParams,
|
||||
priority: 1,
|
||||
}
|
||||
n.children = []*node{child}
|
||||
n = child
|
||||
}
|
||||
|
||||
} else { // catchAll
|
||||
if end != max || numParams > 1 {
|
||||
panic("catch-all routes are only allowed at the end of the path in path '" + fullPath + "'")
|
||||
}
|
||||
|
||||
if len(n.path) > 0 && n.path[len(n.path)-1] == '/' {
|
||||
panic("catch-all conflicts with existing handle for the path segment root in path '" + fullPath + "'")
|
||||
}
|
||||
|
||||
// currently fixed width 1 for '/'
|
||||
i--
|
||||
if path[i] != '/' {
|
||||
panic("no / before catch-all in path '" + fullPath + "'")
|
||||
}
|
||||
|
||||
n.path = path[offset:i]
|
||||
|
||||
// first node: catchAll node with empty path
|
||||
child := &node{
|
||||
wildChild: true,
|
||||
nType: catchAll,
|
||||
maxParams: 1,
|
||||
}
|
||||
n.children = []*node{child}
|
||||
n.indices = string(path[i])
|
||||
n = child
|
||||
n.priority++
|
||||
|
||||
// second node: node holding the variable
|
||||
child = &node{
|
||||
path: path[i:],
|
||||
nType: catchAll,
|
||||
maxParams: 1,
|
||||
handle: handle,
|
||||
priority: 1,
|
||||
}
|
||||
n.children = []*node{child}
|
||||
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// insert remaining path part and handle to the leaf
|
||||
n.path = path[offset:]
|
||||
n.handle = handle
|
||||
}
|
||||
|
||||
// Returns the handle registered with the given path (key). The values of
|
||||
// wildcards are saved to a map.
|
||||
// If no handle can be found, a TSR (trailing slash redirect) recommendation is
|
||||
// made if a handle exists with an extra (without the) trailing slash for the
|
||||
// given path.
|
||||
func (n *node) getValue(path string) (handle Handle, p Params, tsr bool) {
|
||||
walk: // outer loop for walking the tree
|
||||
for {
|
||||
if len(path) > len(n.path) {
|
||||
if path[:len(n.path)] == n.path {
|
||||
path = path[len(n.path):]
|
||||
// If this node does not have a wildcard (param or catchAll)
|
||||
// child, we can just look up the next child node and continue
|
||||
// to walk down the tree
|
||||
if !n.wildChild {
|
||||
c := path[0]
|
||||
for i := 0; i < len(n.indices); i++ {
|
||||
if c == n.indices[i] {
|
||||
n = n.children[i]
|
||||
continue walk
|
||||
}
|
||||
}
|
||||
|
||||
// Nothing found.
|
||||
// We can recommend to redirect to the same URL without a
|
||||
// trailing slash if a leaf exists for that path.
|
||||
tsr = (path == "/" && n.handle != nil)
|
||||
return
|
||||
|
||||
}
|
||||
|
||||
// handle wildcard child
|
||||
n = n.children[0]
|
||||
switch n.nType {
|
||||
case param:
|
||||
// find param end (either '/' or path end)
|
||||
end := 0
|
||||
for end < len(path) && path[end] != '/' {
|
||||
end++
|
||||
}
|
||||
|
||||
// save param value
|
||||
if p == nil {
|
||||
// lazy allocation
|
||||
p = make(Params, 0, n.maxParams)
|
||||
}
|
||||
i := len(p)
|
||||
p = p[:i+1] // expand slice within preallocated capacity
|
||||
p[i].Key = n.path[1:]
|
||||
p[i].Value = path[:end]
|
||||
|
||||
// we need to go deeper!
|
||||
if end < len(path) {
|
||||
if len(n.children) > 0 {
|
||||
path = path[end:]
|
||||
n = n.children[0]
|
||||
continue walk
|
||||
}
|
||||
|
||||
// ... but we can't
|
||||
tsr = (len(path) == end+1)
|
||||
return
|
||||
}
|
||||
|
||||
if handle = n.handle; handle != nil {
|
||||
return
|
||||
} else if len(n.children) == 1 {
|
||||
// No handle found. Check if a handle for this path + a
|
||||
// trailing slash exists for TSR recommendation
|
||||
n = n.children[0]
|
||||
tsr = (n.path == "/" && n.handle != nil)
|
||||
}
|
||||
|
||||
return
|
||||
|
||||
case catchAll:
|
||||
// save param value
|
||||
if p == nil {
|
||||
// lazy allocation
|
||||
p = make(Params, 0, n.maxParams)
|
||||
}
|
||||
i := len(p)
|
||||
p = p[:i+1] // expand slice within preallocated capacity
|
||||
p[i].Key = n.path[2:]
|
||||
p[i].Value = path
|
||||
|
||||
handle = n.handle
|
||||
return
|
||||
|
||||
default:
|
||||
panic("invalid node type")
|
||||
}
|
||||
}
|
||||
} else if path == n.path {
|
||||
// We should have reached the node containing the handle.
|
||||
// Check if this node has a handle registered.
|
||||
if handle = n.handle; handle != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if path == "/" && n.wildChild && n.nType != root {
|
||||
tsr = true
|
||||
return
|
||||
}
|
||||
|
||||
// No handle found. Check if a handle for this path + a
|
||||
// trailing slash exists for trailing slash recommendation
|
||||
for i := 0; i < len(n.indices); i++ {
|
||||
if n.indices[i] == '/' {
|
||||
n = n.children[i]
|
||||
tsr = (len(n.path) == 1 && n.handle != nil) ||
|
||||
(n.nType == catchAll && n.children[0].handle != nil)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// Nothing found. We can recommend to redirect to the same URL with an
|
||||
// extra trailing slash if a leaf exists for that path
|
||||
tsr = (path == "/") ||
|
||||
(len(n.path) == len(path)+1 && n.path[len(path)] == '/' &&
|
||||
path == n.path[:len(n.path)-1] && n.handle != nil)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// Makes a case-insensitive lookup of the given path and tries to find a handler.
|
||||
// It can optionally also fix trailing slashes.
|
||||
// It returns the case-corrected path and a bool indicating whether the lookup
|
||||
// was successful.
|
||||
func (n *node) findCaseInsensitivePath(path string, fixTrailingSlash bool) (ciPath []byte, found bool) {
|
||||
return n.findCaseInsensitivePathRec(
|
||||
path,
|
||||
strings.ToLower(path),
|
||||
make([]byte, 0, len(path)+1), // preallocate enough memory for new path
|
||||
[4]byte{}, // empty rune buffer
|
||||
fixTrailingSlash,
|
||||
)
|
||||
}
|
||||
|
||||
// shift bytes in array by n bytes left
|
||||
func shiftNRuneBytes(rb [4]byte, n int) [4]byte {
|
||||
switch n {
|
||||
case 0:
|
||||
return rb
|
||||
case 1:
|
||||
return [4]byte{rb[1], rb[2], rb[3], 0}
|
||||
case 2:
|
||||
return [4]byte{rb[2], rb[3]}
|
||||
case 3:
|
||||
return [4]byte{rb[3]}
|
||||
default:
|
||||
return [4]byte{}
|
||||
}
|
||||
}
|
||||
|
||||
// recursive case-insensitive lookup function used by n.findCaseInsensitivePath
|
||||
func (n *node) findCaseInsensitivePathRec(path, loPath string, ciPath []byte, rb [4]byte, fixTrailingSlash bool) ([]byte, bool) {
|
||||
loNPath := strings.ToLower(n.path)
|
||||
|
||||
walk: // outer loop for walking the tree
|
||||
for len(loPath) >= len(loNPath) && (len(loNPath) == 0 || loPath[1:len(loNPath)] == loNPath[1:]) {
|
||||
// add common path to result
|
||||
ciPath = append(ciPath, n.path...)
|
||||
|
||||
if path = path[len(n.path):]; len(path) > 0 {
|
||||
loOld := loPath
|
||||
loPath = loPath[len(loNPath):]
|
||||
|
||||
// If this node does not have a wildcard (param or catchAll) child,
|
||||
// we can just look up the next child node and continue to walk down
|
||||
// the tree
|
||||
if !n.wildChild {
|
||||
// skip rune bytes already processed
|
||||
rb = shiftNRuneBytes(rb, len(loNPath))
|
||||
|
||||
if rb[0] != 0 {
|
||||
// old rune not finished
|
||||
for i := 0; i < len(n.indices); i++ {
|
||||
if n.indices[i] == rb[0] {
|
||||
// continue with child node
|
||||
n = n.children[i]
|
||||
loNPath = strings.ToLower(n.path)
|
||||
continue walk
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// process a new rune
|
||||
var rv rune
|
||||
|
||||
// find rune start
|
||||
// runes are up to 4 byte long,
|
||||
// -4 would definitely be another rune
|
||||
var off int
|
||||
for max := min(len(loNPath), 3); off < max; off++ {
|
||||
if i := len(loNPath) - off; utf8.RuneStart(loOld[i]) {
|
||||
// read rune from cached lowercase path
|
||||
rv, _ = utf8.DecodeRuneInString(loOld[i:])
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// calculate lowercase bytes of current rune
|
||||
utf8.EncodeRune(rb[:], rv)
|
||||
// skipp already processed bytes
|
||||
rb = shiftNRuneBytes(rb, off)
|
||||
|
||||
for i := 0; i < len(n.indices); i++ {
|
||||
// lowercase matches
|
||||
if n.indices[i] == rb[0] {
|
||||
// must use a recursive approach since both the
|
||||
// uppercase byte and the lowercase byte might exist
|
||||
// as an index
|
||||
if out, found := n.children[i].findCaseInsensitivePathRec(
|
||||
path, loPath, ciPath, rb, fixTrailingSlash,
|
||||
); found {
|
||||
return out, true
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// same for uppercase rune, if it differs
|
||||
if up := unicode.ToUpper(rv); up != rv {
|
||||
utf8.EncodeRune(rb[:], up)
|
||||
rb = shiftNRuneBytes(rb, off)
|
||||
|
||||
for i := 0; i < len(n.indices); i++ {
|
||||
// uppercase matches
|
||||
if n.indices[i] == rb[0] {
|
||||
// continue with child node
|
||||
n = n.children[i]
|
||||
loNPath = strings.ToLower(n.path)
|
||||
continue walk
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Nothing found. We can recommend to redirect to the same URL
|
||||
// without a trailing slash if a leaf exists for that path
|
||||
return ciPath, (fixTrailingSlash && path == "/" && n.handle != nil)
|
||||
}
|
||||
|
||||
n = n.children[0]
|
||||
switch n.nType {
|
||||
case param:
|
||||
// find param end (either '/' or path end)
|
||||
k := 0
|
||||
for k < len(path) && path[k] != '/' {
|
||||
k++
|
||||
}
|
||||
|
||||
// add param value to case insensitive path
|
||||
ciPath = append(ciPath, path[:k]...)
|
||||
|
||||
// we need to go deeper!
|
||||
if k < len(path) {
|
||||
if len(n.children) > 0 {
|
||||
// continue with child node
|
||||
n = n.children[0]
|
||||
loNPath = strings.ToLower(n.path)
|
||||
loPath = loPath[k:]
|
||||
path = path[k:]
|
||||
continue
|
||||
}
|
||||
|
||||
// ... but we can't
|
||||
if fixTrailingSlash && len(path) == k+1 {
|
||||
return ciPath, true
|
||||
}
|
||||
return ciPath, false
|
||||
}
|
||||
|
||||
if n.handle != nil {
|
||||
return ciPath, true
|
||||
} else if fixTrailingSlash && len(n.children) == 1 {
|
||||
// No handle found. Check if a handle for this path + a
|
||||
// trailing slash exists
|
||||
n = n.children[0]
|
||||
if n.path == "/" && n.handle != nil {
|
||||
return append(ciPath, '/'), true
|
||||
}
|
||||
}
|
||||
return ciPath, false
|
||||
|
||||
case catchAll:
|
||||
return append(ciPath, path...), true
|
||||
|
||||
default:
|
||||
panic("invalid node type")
|
||||
}
|
||||
} else {
|
||||
// We should have reached the node containing the handle.
|
||||
// Check if this node has a handle registered.
|
||||
if n.handle != nil {
|
||||
return ciPath, true
|
||||
}
|
||||
|
||||
// No handle found.
|
||||
// Try to fix the path by adding a trailing slash
|
||||
if fixTrailingSlash {
|
||||
for i := 0; i < len(n.indices); i++ {
|
||||
if n.indices[i] == '/' {
|
||||
n = n.children[i]
|
||||
if (len(n.path) == 1 && n.handle != nil) ||
|
||||
(n.nType == catchAll && n.children[0].handle != nil) {
|
||||
return append(ciPath, '/'), true
|
||||
}
|
||||
return ciPath, false
|
||||
}
|
||||
}
|
||||
}
|
||||
return ciPath, false
|
||||
}
|
||||
}
|
||||
|
||||
// Nothing found.
|
||||
// Try to fix the path by adding / removing a trailing slash
|
||||
if fixTrailingSlash {
|
||||
if path == "/" {
|
||||
return ciPath, true
|
||||
}
|
||||
if len(loPath)+1 == len(loNPath) && loNPath[len(loPath)] == '/' &&
|
||||
loPath[1:] == loNPath[1:len(loPath)] && n.handle != nil {
|
||||
return append(ciPath, n.path...), true
|
||||
}
|
||||
}
|
||||
return ciPath, false
|
||||
}
|
||||
Reference in New Issue
Block a user