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
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
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user