mirror of
https://github.com/fnproject/fn.git
synced 2022-10-28 21:29:17 +03:00
fn: pre-fork pool for namespace/network speedup (#874)
* fn: pre-fork pool experimental implementation
This commit is contained in:
@@ -129,7 +129,10 @@ func createAgent(da DataAccess, withDocker bool) Agent {
|
||||
var driver drivers.Driver
|
||||
if withDocker {
|
||||
driver = docker.NewDocker(drivers.Config{
|
||||
ServerVersion: cfg.MinDockerVersion,
|
||||
ServerVersion: cfg.MinDockerVersion,
|
||||
PreForkPoolSize: cfg.PreForkPoolSize,
|
||||
PreForkImage: cfg.PreForkImage,
|
||||
PreForkCmd: cfg.PreForkCmd,
|
||||
})
|
||||
} else {
|
||||
driver = mock.New()
|
||||
@@ -154,12 +157,16 @@ func (a *agent) Enqueue(ctx context.Context, call *models.Call) error {
|
||||
}
|
||||
|
||||
func (a *agent) Close() error {
|
||||
var err error
|
||||
a.shutonce.Do(func() {
|
||||
if a.driver != nil {
|
||||
err = a.driver.Close()
|
||||
}
|
||||
close(a.shutdown)
|
||||
})
|
||||
|
||||
a.wg.Wait()
|
||||
return nil
|
||||
return err
|
||||
}
|
||||
|
||||
func (a *agent) Submit(callI Call) error {
|
||||
|
||||
@@ -20,6 +20,9 @@ type AgentConfig struct {
|
||||
MaxTotalCPU uint64 `json:"max_total_cpu_mcpus"`
|
||||
MaxTotalMemory uint64 `json:"max_total_memory_bytes"`
|
||||
MaxFsSize uint64 `json:"max_fs_size_mb"`
|
||||
PreForkPoolSize uint64 `json:"pre_fork_pool_size"`
|
||||
PreForkImage string `json:"pre_fork_image"`
|
||||
PreForkCmd string `json:"pre_fork_pool_cmd"`
|
||||
}
|
||||
|
||||
const (
|
||||
@@ -33,6 +36,9 @@ const (
|
||||
EnvMaxTotalCPU = "FN_MAX_TOTAL_CPU_MCPUS"
|
||||
EnvMaxTotalMemory = "FN_MAX_TOTAL_MEMORY_BYTES"
|
||||
EnvMaxFsSize = "FN_MAX_FS_SIZE_MB"
|
||||
EnvPreForkPoolSize = "FN_EXPERIMENTAL_PREFORK_POOL_SIZE"
|
||||
EnvPreForkImage = "FN_EXPERIMENTAL_PREFORK_IMAGE"
|
||||
EnvPreForkCmd = "FN_EXPERIMENTAL_PREFORK_CMD"
|
||||
|
||||
MaxDisabledMsecs = time.Duration(math.MaxInt64)
|
||||
)
|
||||
@@ -56,11 +62,15 @@ func NewAgentConfig() (*AgentConfig, error) {
|
||||
err = setEnvUint(err, EnvMaxTotalCPU, &cfg.MaxTotalCPU)
|
||||
err = setEnvUint(err, EnvMaxTotalMemory, &cfg.MaxTotalMemory)
|
||||
err = setEnvUint(err, EnvMaxFsSize, &cfg.MaxFsSize)
|
||||
err = setEnvUint(err, EnvPreForkPoolSize, &cfg.PreForkPoolSize)
|
||||
|
||||
if err != nil {
|
||||
return cfg, err
|
||||
}
|
||||
|
||||
cfg.PreForkImage = os.Getenv(EnvPreForkImage)
|
||||
cfg.PreForkCmd = os.Getenv(EnvPreForkCmd)
|
||||
|
||||
if cfg.EjectIdle == time.Duration(0) {
|
||||
return cfg, fmt.Errorf("error %s cannot be zero", EnvEjectIdle)
|
||||
}
|
||||
|
||||
@@ -52,6 +52,7 @@ type DockerDriver struct {
|
||||
docker dockerClient // retries on *docker.Client, restricts ad hoc *docker.Client usage / retries
|
||||
hostname string
|
||||
auths map[string]docker.AuthConfiguration
|
||||
pool DockerPool
|
||||
}
|
||||
|
||||
// implements drivers.Driver
|
||||
@@ -75,6 +76,10 @@ func NewDocker(conf drivers.Config) *DockerDriver {
|
||||
}
|
||||
}
|
||||
|
||||
if conf.PreForkPoolSize != 0 {
|
||||
driver.pool = NewDockerPool(conf, driver)
|
||||
}
|
||||
|
||||
return driver
|
||||
}
|
||||
|
||||
@@ -118,6 +123,14 @@ func registryFromEnv() map[string]docker.AuthConfiguration {
|
||||
return auths.Configs
|
||||
}
|
||||
|
||||
func (drv *DockerDriver) Close() error {
|
||||
var err error
|
||||
if drv.pool != nil {
|
||||
err = drv.pool.Close()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
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
|
||||
@@ -141,7 +154,6 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
|
||||
MemorySwap: int64(task.Memory()), // disables swap
|
||||
KernelMemory: int64(task.Memory()),
|
||||
CPUShares: drv.conf.CPUShares,
|
||||
Hostname: drv.hostname,
|
||||
Image: task.Image(),
|
||||
Volumes: map[string]struct{}{},
|
||||
OpenStdin: true,
|
||||
@@ -159,6 +171,20 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
|
||||
Context: ctx,
|
||||
}
|
||||
|
||||
poolId := ""
|
||||
if drv.pool != nil {
|
||||
id, err := drv.pool.AllocPoolId()
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not fetch pre fork pool container")
|
||||
} else {
|
||||
poolId = id
|
||||
container.HostConfig.NetworkMode = fmt.Sprintf("container:%s", id)
|
||||
}
|
||||
} else {
|
||||
// hostname and container NetworkMode is not compatible.
|
||||
container.Config.Hostname = drv.hostname
|
||||
}
|
||||
|
||||
// Translate milli cpus into CPUQuota & CPUPeriod (see Linux cGroups CFS cgroup v1 documentation)
|
||||
// eg: task.CPUQuota() of 8000 means CPUQuota of 8 * 100000 usecs in 100000 usec period,
|
||||
// which is approx 8 CPUS in CFS world.
|
||||
@@ -210,16 +236,20 @@ func (drv *DockerDriver) Prepare(ctx context.Context, task drivers.ContainerTask
|
||||
}
|
||||
|
||||
// discard removal error
|
||||
return &cookie{id: task.Id(), task: task, drv: drv}, nil
|
||||
return &cookie{id: task.Id(), task: task, drv: drv, poolId: poolId}, nil
|
||||
}
|
||||
|
||||
type cookie struct {
|
||||
id string
|
||||
task drivers.ContainerTask
|
||||
drv *DockerDriver
|
||||
id string
|
||||
poolId string
|
||||
task drivers.ContainerTask
|
||||
drv *DockerDriver
|
||||
}
|
||||
|
||||
func (c *cookie) Close(ctx context.Context) error {
|
||||
if c.poolId != "" && c.drv.pool != nil {
|
||||
defer c.drv.pool.FreePoolId(c.poolId)
|
||||
}
|
||||
return c.drv.removeContainer(ctx, c.id)
|
||||
}
|
||||
|
||||
|
||||
@@ -275,7 +275,7 @@ func filterNoSuchContainer(ctx context.Context, err error) error {
|
||||
_, containerNotFound := err.(*docker.NoSuchContainer)
|
||||
dockerErr, ok := err.(*docker.Error)
|
||||
if containerNotFound || (ok && dockerErr.Status == 404) {
|
||||
log.WithError(err).Error("filtering error")
|
||||
log.WithError(err).Info("filtering error")
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
|
||||
297
api/agent/drivers/docker/docker_pool.go
Normal file
297
api/agent/drivers/docker/docker_pool.go
Normal file
@@ -0,0 +1,297 @@
|
||||
package docker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"runtime"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/fnproject/fn/api/agent/drivers"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/id"
|
||||
|
||||
"github.com/fsouza/go-dockerclient"
|
||||
"github.com/sirupsen/logrus"
|
||||
"golang.org/x/time/rate"
|
||||
)
|
||||
|
||||
// Prefork Pool is used in namespace optimizations to avoid creating and
|
||||
// tearing down namespaces with every function container run. Instead, we reuse
|
||||
// already available namespaces from these running container instances. These
|
||||
// containers are not designed to run anything but placeholders for namespaces
|
||||
// such as a minimal busybox container with 'tail -f /dev/null' which blocks
|
||||
// forever. In other words, every function container is paired with a pool buddy
|
||||
// where pool buddy provides already creates namespaces. These are currently
|
||||
// network and user namespaces, but perhaps can be extended to also use pid and ipc.
|
||||
// (see docker.go Prepare() on how this is currently being used.)
|
||||
// Currently the pool is a set size and it does not grow on demand.
|
||||
|
||||
var (
|
||||
ErrorPoolEmpty = errors.New("docker pre fork pool empty")
|
||||
)
|
||||
|
||||
const (
|
||||
LimitPerSec = 10
|
||||
LimitBurst = 20
|
||||
|
||||
DefaultImage = "busybox"
|
||||
DefaultCmd = "tail -f /dev/null"
|
||||
|
||||
ShutdownTimeout = time.Duration(1) * time.Second
|
||||
)
|
||||
|
||||
type poolTask struct {
|
||||
id string
|
||||
image string
|
||||
cmd string
|
||||
}
|
||||
|
||||
func (c *poolTask) Id() string { return c.id }
|
||||
func (c *poolTask) Command() string { return c.cmd }
|
||||
func (c *poolTask) Input() io.Reader { return nil }
|
||||
func (c *poolTask) Logger() (io.Writer, io.Writer) { return nil, nil }
|
||||
func (c *poolTask) Volumes() [][2]string { return nil }
|
||||
func (c *poolTask) WorkDir() string { return "" }
|
||||
func (c *poolTask) Close() {}
|
||||
func (c *poolTask) Image() string { return c.image }
|
||||
func (c *poolTask) Timeout() time.Duration { return 0 }
|
||||
func (c *poolTask) EnvVars() map[string]string { return nil }
|
||||
func (c *poolTask) Memory() uint64 { return 0 }
|
||||
func (c *poolTask) CPUs() uint64 { return 0 }
|
||||
func (c *poolTask) FsSize() uint64 { return 0 }
|
||||
func (c *poolTask) WriteStat(ctx context.Context, stat drivers.Stat) {}
|
||||
|
||||
type dockerPool struct {
|
||||
lock sync.Mutex
|
||||
inuse map[string]struct{}
|
||||
free []string
|
||||
limiter *rate.Limiter
|
||||
cancel func()
|
||||
wg sync.WaitGroup // TODO rename
|
||||
}
|
||||
|
||||
type DockerPoolStats struct {
|
||||
inuse int
|
||||
free int
|
||||
}
|
||||
|
||||
type DockerPool interface {
|
||||
// fetch a pre-allocated free id from the pool
|
||||
// may return too busy error
|
||||
AllocPoolId() (string, error)
|
||||
|
||||
// Release the id back to the pool
|
||||
FreePoolId(id string)
|
||||
|
||||
// stop and terminate the pool
|
||||
Close() error
|
||||
|
||||
// returns inuse versus free
|
||||
Usage() DockerPoolStats
|
||||
}
|
||||
|
||||
func NewDockerPool(conf drivers.Config, driver *DockerDriver) DockerPool {
|
||||
|
||||
// Docker pool is an optimization & feature only for Linux
|
||||
if runtime.GOOS != "linux" || conf.PreForkPoolSize == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
log := common.Logger(ctx)
|
||||
log.Error("WARNING: Experimental Prefork Docker Pool Enabled")
|
||||
|
||||
pool := &dockerPool{
|
||||
inuse: make(map[string]struct{}, conf.PreForkPoolSize),
|
||||
free: make([]string, 0, conf.PreForkPoolSize),
|
||||
limiter: rate.NewLimiter(LimitPerSec, LimitBurst),
|
||||
cancel: cancel,
|
||||
}
|
||||
|
||||
for i := uint64(0); i < conf.PreForkPoolSize; i++ {
|
||||
|
||||
task := &poolTask{
|
||||
id: fmt.Sprintf("%d_prefork_%s", i, id.New().String()),
|
||||
image: DefaultImage,
|
||||
cmd: DefaultCmd,
|
||||
}
|
||||
|
||||
if conf.PreForkImage != "" {
|
||||
task.image = conf.PreForkImage
|
||||
}
|
||||
if conf.PreForkCmd != "" {
|
||||
task.cmd = conf.PreForkCmd
|
||||
}
|
||||
|
||||
pool.wg.Add(1)
|
||||
go pool.nannyContainer(ctx, driver, task)
|
||||
}
|
||||
|
||||
return pool
|
||||
}
|
||||
|
||||
func (pool *dockerPool) Close() error {
|
||||
pool.cancel()
|
||||
pool.wg.Wait()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (pool *dockerPool) nannyContainer(ctx context.Context, driver *DockerDriver, task *poolTask) {
|
||||
defer pool.wg.Done()
|
||||
|
||||
log := common.Logger(ctx).WithFields(logrus.Fields{"name": task.Id()})
|
||||
|
||||
containerOpts := docker.CreateContainerOptions{
|
||||
Name: task.Id(),
|
||||
Config: &docker.Config{
|
||||
Cmd: strings.Fields(task.Command()),
|
||||
Hostname: task.Id(),
|
||||
Image: task.Image(),
|
||||
Volumes: map[string]struct{}{},
|
||||
OpenStdin: false,
|
||||
AttachStdout: false,
|
||||
AttachStderr: false,
|
||||
AttachStdin: false,
|
||||
StdinOnce: false,
|
||||
},
|
||||
HostConfig: &docker.HostConfig{
|
||||
LogConfig: docker.LogConfig{
|
||||
Type: "none",
|
||||
},
|
||||
AutoRemove: true,
|
||||
},
|
||||
Context: ctx,
|
||||
}
|
||||
|
||||
removeOpts := docker.RemoveContainerOptions{
|
||||
ID: task.Id(),
|
||||
Force: true,
|
||||
RemoveVolumes: true,
|
||||
Context: ctx,
|
||||
}
|
||||
|
||||
// We spin forever, keeping the pool resident and running at all times.
|
||||
for {
|
||||
err := pool.limiter.Wait(ctx)
|
||||
if err != nil {
|
||||
// should not really happen unless ctx has a deadline or burst is 0.
|
||||
log.WithError(err).Info("prefork pool rate limiter failed")
|
||||
break
|
||||
}
|
||||
|
||||
// Let's try to clean up any left overs
|
||||
err = driver.docker.RemoveContainer(removeOpts)
|
||||
if err != nil {
|
||||
log.WithError(err).Info("prefork pool container remove failed (this is probably OK)")
|
||||
}
|
||||
|
||||
err = driver.ensureImage(ctx, task)
|
||||
if err != nil {
|
||||
log.WithError(err).Info("prefork pool image pull failed")
|
||||
continue
|
||||
}
|
||||
|
||||
_, err = driver.docker.CreateContainer(containerOpts)
|
||||
if err != nil {
|
||||
log.WithError(err).Info("prefork pool container create failed")
|
||||
continue
|
||||
}
|
||||
|
||||
err = driver.docker.StartContainerWithContext(task.Id(), nil, ctx)
|
||||
if err != nil {
|
||||
log.WithError(err).Info("prefork pool container start failed")
|
||||
continue
|
||||
}
|
||||
|
||||
log.Debug("prefork pool container ready")
|
||||
|
||||
// IMPORTANT: container is now up and running. Register it to make it
|
||||
// available for function containers.
|
||||
pool.register(task.Id())
|
||||
|
||||
// We are optimistic here where provided image and command really blocks
|
||||
// and runs forever.
|
||||
exitCode, err := driver.docker.WaitContainerWithContext(task.Id(), ctx)
|
||||
|
||||
// IMPORTANT: We have exited. This window is potentially very destructive, as any new
|
||||
// function containers created during this window will fail. We must immediately
|
||||
// proceed to unregister ourself to avoid further issues.
|
||||
pool.unregister(task.Id())
|
||||
|
||||
log.WithError(err).Infof("prefork pool container exited exit_code=%d", exitCode)
|
||||
}
|
||||
|
||||
// final exit cleanup
|
||||
ctx, cancel := context.WithTimeout(context.Background(), ShutdownTimeout)
|
||||
defer cancel()
|
||||
removeOpts.Context = ctx
|
||||
driver.docker.RemoveContainer(removeOpts)
|
||||
}
|
||||
|
||||
func (pool *dockerPool) register(id string) {
|
||||
pool.lock.Lock()
|
||||
pool.free = append(pool.free, id)
|
||||
pool.lock.Unlock()
|
||||
}
|
||||
|
||||
func (pool *dockerPool) unregister(id string) {
|
||||
pool.lock.Lock()
|
||||
|
||||
_, ok := pool.inuse[id]
|
||||
if ok {
|
||||
delete(pool.inuse, id)
|
||||
} else {
|
||||
for i := 0; i < len(pool.free); i += 1 {
|
||||
if pool.free[i] == id {
|
||||
pool.free = append(pool.free[:i], pool.free[i+1:]...)
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pool.lock.Unlock()
|
||||
}
|
||||
|
||||
func (pool *dockerPool) AllocPoolId() (string, error) {
|
||||
pool.lock.Lock()
|
||||
defer pool.lock.Unlock()
|
||||
|
||||
// We currently do not grow the pool if we run out of pre-forked containers
|
||||
if len(pool.free) == 0 {
|
||||
return "", ErrorPoolEmpty
|
||||
}
|
||||
|
||||
id := pool.free[len(pool.free)-1]
|
||||
pool.free = pool.free[:len(pool.free)-1]
|
||||
pool.inuse[id] = struct{}{}
|
||||
|
||||
return id, nil
|
||||
}
|
||||
|
||||
func (pool *dockerPool) FreePoolId(id string) {
|
||||
pool.lock.Lock()
|
||||
|
||||
_, ok := pool.inuse[id]
|
||||
if ok {
|
||||
delete(pool.inuse, id)
|
||||
pool.free = append(pool.free, id)
|
||||
}
|
||||
|
||||
pool.lock.Unlock()
|
||||
}
|
||||
|
||||
func (pool *dockerPool) Usage() DockerPoolStats {
|
||||
var stats DockerPoolStats
|
||||
pool.lock.Lock()
|
||||
|
||||
stats.inuse = len(pool.inuse)
|
||||
stats.free = len(pool.free)
|
||||
|
||||
pool.lock.Unlock()
|
||||
return stats
|
||||
}
|
||||
131
api/agent/drivers/docker/docker_pool_test.go
Normal file
131
api/agent/drivers/docker/docker_pool_test.go
Normal file
@@ -0,0 +1,131 @@
|
||||
package docker
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/fnproject/fn/api/agent/drivers"
|
||||
)
|
||||
|
||||
func TestRunnerDockerPool(t *testing.T) {
|
||||
|
||||
cfg := &drivers.Config{}
|
||||
|
||||
// shouldn't spin up a pool since cfg is empty
|
||||
drv := NewDocker(*cfg)
|
||||
|
||||
cfg.PreForkPoolSize = 2
|
||||
pool := NewDockerPool(*cfg, drv)
|
||||
|
||||
// primitive wait here
|
||||
i := 0
|
||||
for ; i < 10; i++ {
|
||||
stats := pool.Usage()
|
||||
if stats.free == 2 {
|
||||
break
|
||||
}
|
||||
|
||||
<-time.After(time.Duration(500) * time.Millisecond)
|
||||
}
|
||||
if i == 10 {
|
||||
t.Fatalf("pool initialize timeout stats=%+v", pool.Usage())
|
||||
}
|
||||
|
||||
id1, err := pool.AllocPoolId()
|
||||
if err != nil {
|
||||
t.Fatalf("pool AllocPoolId id1 err=%s", err.Error())
|
||||
}
|
||||
t.Logf("pool AllocPoolId id1=%s", id1)
|
||||
|
||||
id2, err := pool.AllocPoolId()
|
||||
if err != nil {
|
||||
t.Fatalf("pool AllocPoolId id2 err=%s", err.Error())
|
||||
}
|
||||
t.Logf("pool AllocPoolId id2=%s", id2)
|
||||
|
||||
id3, err := pool.AllocPoolId()
|
||||
if err == nil {
|
||||
t.Fatalf("pool AllocPoolId id3 should be err, but got id=%s", id3)
|
||||
}
|
||||
t.Logf("pool AllocPoolId id3=%s", id3)
|
||||
|
||||
pool.FreePoolId("nonsense")
|
||||
|
||||
id4, err := pool.AllocPoolId()
|
||||
if err == nil {
|
||||
t.Fatalf("pool AllocPoolId id4 should be err, but got id=%s", id3)
|
||||
}
|
||||
t.Logf("pool AllocPoolId id4=%s", id4)
|
||||
|
||||
pool.FreePoolId(id1)
|
||||
|
||||
id5, err := pool.AllocPoolId()
|
||||
if err != nil {
|
||||
t.Fatalf("pool AllocPoolId id5 err=%s", err.Error())
|
||||
}
|
||||
t.Logf("pool AllocPoolId id5=%s", id5)
|
||||
if id5 != id1 {
|
||||
t.Fatalf("pool AllocPoolId id5 != id1 (%s != %s)", id5, id1)
|
||||
}
|
||||
|
||||
err = pool.Close()
|
||||
if err != nil {
|
||||
t.Fatalf("pool close err=%s", err.Error())
|
||||
}
|
||||
|
||||
err = drv.Close()
|
||||
if err != nil {
|
||||
t.Fatalf("drv close err=%s", err.Error())
|
||||
}
|
||||
|
||||
stats := pool.Usage()
|
||||
if stats.free != 0 && stats.inuse != 0 {
|
||||
t.Fatalf("pool shutdown timeout stats=%+v", stats)
|
||||
}
|
||||
}
|
||||
|
||||
func TestRunnerDockerPoolFaulty(t *testing.T) {
|
||||
|
||||
cfg := &drivers.Config{}
|
||||
|
||||
// shouldn't spin up a pool since cfg is empty
|
||||
drv := NewDocker(*cfg)
|
||||
|
||||
cfg.PreForkPoolSize = 2
|
||||
cfg.PreForkCmd = "sleep 0"
|
||||
|
||||
pool := NewDockerPool(*cfg, drv)
|
||||
|
||||
<-time.After(time.Duration(500) * time.Millisecond)
|
||||
|
||||
// Not much to see if pre-fork has exited, but let's close
|
||||
// and wait at least to make sure we don't crash and burn.
|
||||
id1, err := pool.AllocPoolId()
|
||||
t.Logf("pool AllocPoolId id=%s err=%v", id1, err)
|
||||
if id1 != "" {
|
||||
pool.FreePoolId(id1)
|
||||
}
|
||||
|
||||
<-time.After(time.Duration(500) * time.Millisecond)
|
||||
|
||||
id2, err := pool.AllocPoolId()
|
||||
t.Logf("pool AllocPoolId id=%s err=%v", id2, err)
|
||||
if id2 != "" {
|
||||
pool.FreePoolId(id2)
|
||||
}
|
||||
|
||||
err = pool.Close()
|
||||
if err != nil {
|
||||
t.Fatalf("pool close err=%s", err.Error())
|
||||
}
|
||||
|
||||
err = drv.Close()
|
||||
if err != nil {
|
||||
t.Fatalf("drv close err=%s", err.Error())
|
||||
}
|
||||
|
||||
stats := pool.Usage()
|
||||
if stats.free != 0 && stats.inuse != 0 {
|
||||
t.Fatalf("pool shutdown timeout stats=%+v", stats)
|
||||
}
|
||||
}
|
||||
@@ -60,6 +60,9 @@ type Driver interface {
|
||||
//
|
||||
// The returned cookie should respect the task's timeout when it is run.
|
||||
Prepare(ctx context.Context, task ContainerTask) (Cookie, error)
|
||||
|
||||
// close & shutdown the driver
|
||||
Close() error
|
||||
}
|
||||
|
||||
// RunResult indicates only the final state of the task.
|
||||
@@ -189,8 +192,11 @@ const (
|
||||
type Config struct {
|
||||
Docker string `json:"docker"`
|
||||
// TODO CPUShares should likely be on a per container basis
|
||||
CPUShares int64 `json:"cpu_shares"`
|
||||
ServerVersion string `json:"server_version"`
|
||||
CPUShares int64 `json:"cpu_shares"`
|
||||
ServerVersion string `json:"server_version"`
|
||||
PreForkPoolSize uint64 `json:"pre_fork_pool_size"`
|
||||
PreForkImage string `json:"pre_fork_image"`
|
||||
PreForkCmd string `json:"pre_fork_cmd"`
|
||||
}
|
||||
|
||||
func average(samples []Stat) (Stat, bool) {
|
||||
|
||||
@@ -20,6 +20,10 @@ func (m *Mocker) Prepare(context.Context, drivers.ContainerTask) (drivers.Cookie
|
||||
return &cookie{m}, nil
|
||||
}
|
||||
|
||||
func (m *Mocker) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
type cookie struct {
|
||||
m *Mocker
|
||||
}
|
||||
|
||||
@@ -388,7 +388,7 @@ func (a *resourceTracker) initializeMemory(cfg *AgentConfig) {
|
||||
}
|
||||
|
||||
// clamp the available memory by head room (for docker, ourselves, other processes)
|
||||
headRoom, err := getMemoryHeadRoom(availMemory)
|
||||
headRoom, err := getMemoryHeadRoom(availMemory, cfg)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Fatal("Out of memory")
|
||||
}
|
||||
@@ -441,11 +441,19 @@ func (a *resourceTracker) initializeMemory(cfg *AgentConfig) {
|
||||
}
|
||||
|
||||
// headroom estimation in order not to consume entire RAM if possible
|
||||
func getMemoryHeadRoom(usableMemory uint64) (uint64, error) {
|
||||
func getMemoryHeadRoom(usableMemory uint64, cfg *AgentConfig) (uint64, error) {
|
||||
|
||||
// get %10 of the RAM
|
||||
headRoom := uint64(usableMemory / 10)
|
||||
|
||||
// TODO: improve this pre-fork calculation, we should fetch/query this
|
||||
// instead of estimate below.
|
||||
// if pre-fork pool is enabled, add 1 MB per pool-item
|
||||
if cfg != nil && cfg.PreForkPoolSize != 0 {
|
||||
headRoom += Mem1MB * cfg.PreForkPoolSize
|
||||
}
|
||||
|
||||
// TODO: improve these calculations.
|
||||
// clamp this with 256MB min -- 5GB max
|
||||
maxHeadRoom := uint64(5 * Mem1GB)
|
||||
minHeadRoom := uint64(256 * Mem1MB)
|
||||
|
||||
Reference in New Issue
Block a user