executor v2 (#4656)
* client/executor: refactor client to remove interpolation * executor: POC libcontainer based executor * vendor: use hashicorp libcontainer fork * vendor: add libcontainer/nsenter dep * executor: updated executor interface to simplify operations * executor: implement logging pipe * logmon: new logmon plugin to manage task logs * driver/executor: use logmon for log management * executor: fix tests and windows build * executor: fix logging key names * executor: fix test failures * executor: add config field to toggle between using libcontainer and standard executors * logmon: use discover utility to discover nomad executable * executor: only call libcontainer-shim on main in linux * logmon: use seperate path configs for stdout/stderr fifos * executor: windows fixes * executor: created reusable pid stats collection utility that can be used in an executor * executor: update fifo.Open calls * executor: fix build * remove executor from docker driver * executor: Shutdown func to kill and cleanup executor and its children * executor: move linux specific universal executor funcs to seperate file * move logmon initialization to a task runner hook * client: doc fixes and renaming from code review * taskrunner: use shared config struct for logmon fifo fields * taskrunner: logmon only needs to be started once per task
This commit is contained in:
parent
e6e2930a00
commit
5dee1141d1
|
@ -0,0 +1,109 @@
|
|||
package taskrunner
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
plugin "github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/nomad/client/allocrunnerv2/interfaces"
|
||||
"github.com/hashicorp/nomad/client/logmon"
|
||||
"github.com/hashicorp/nomad/helper/uuid"
|
||||
)
|
||||
|
||||
// logmonHook launches logmon and manages task logging
|
||||
type logmonHook struct {
|
||||
// logmon is the handle to the log monitor process for the task.
|
||||
logmon logmon.LogMon
|
||||
logmonPluginClient *plugin.Client
|
||||
|
||||
config *logmonHookConfig
|
||||
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
type logmonHookConfig struct {
|
||||
logDir string
|
||||
stdoutFifo string
|
||||
stderrFifo string
|
||||
}
|
||||
|
||||
func newLogMonHook(cfg *logmonHookConfig, logger hclog.Logger) *logmonHook {
|
||||
hook := &logmonHook{
|
||||
config: cfg,
|
||||
logger: logger,
|
||||
}
|
||||
|
||||
return hook
|
||||
}
|
||||
|
||||
func newLogMonHookConfig(taskName, logDir string) *logmonHookConfig {
|
||||
cfg := &logmonHookConfig{
|
||||
logDir: logDir,
|
||||
}
|
||||
if runtime.GOOS == "windows" {
|
||||
id := uuid.Generate()[:8]
|
||||
cfg.stdoutFifo = fmt.Sprintf("//./pipe/%s-%s.stdout", taskName, id)
|
||||
cfg.stderrFifo = fmt.Sprintf("//./pipe/%s-%s.stderr", taskName, id)
|
||||
} else {
|
||||
cfg.stdoutFifo = filepath.Join(logDir, fmt.Sprintf(".%s.stdout.fifo", taskName))
|
||||
cfg.stderrFifo = filepath.Join(logDir, fmt.Sprintf(".%s.stderr.fifo", taskName))
|
||||
}
|
||||
return cfg
|
||||
}
|
||||
|
||||
func (*logmonHook) Name() string {
|
||||
return "logmon"
|
||||
}
|
||||
|
||||
func (h *logmonHook) launchLogMon() error {
|
||||
l, c, err := logmon.LaunchLogMon(h.logger)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
h.logmon = l
|
||||
h.logmonPluginClient = c
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *logmonHook) Prestart(ctx context.Context,
|
||||
req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
|
||||
|
||||
// Launch logmon instance for the task.
|
||||
if err := h.launchLogMon(); err != nil {
|
||||
h.logger.Error("failed to launch logmon process", "error", err)
|
||||
return err
|
||||
}
|
||||
|
||||
err := h.logmon.Start(&logmon.LogConfig{
|
||||
LogDir: h.config.logDir,
|
||||
StdoutLogFile: fmt.Sprintf("%s.stdout", req.Task.Name),
|
||||
StderrLogFile: fmt.Sprintf("%s.stderr", req.Task.Name),
|
||||
StdoutFifo: h.config.stdoutFifo,
|
||||
StderrFifo: h.config.stderrFifo,
|
||||
MaxFiles: req.Task.LogConfig.MaxFiles,
|
||||
MaxFileSizeMB: req.Task.LogConfig.MaxFileSizeMB,
|
||||
})
|
||||
if err != nil {
|
||||
h.logger.Error("failed to start logmon", "error", err)
|
||||
return err
|
||||
}
|
||||
|
||||
resp.Done = true
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *logmonHook) Stop(context.Context, *interfaces.TaskStopRequest, *interfaces.TaskStopResponse) error {
|
||||
|
||||
if h.logmon != nil {
|
||||
h.logmon.Stop()
|
||||
}
|
||||
if h.logmonPluginClient != nil {
|
||||
h.logmonPluginClient.Kill()
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,7 @@
|
|||
package taskrunner
|
||||
|
||||
import "github.com/hashicorp/nomad/client/allocrunnerv2/interfaces"
|
||||
|
||||
// Statically assert the logmon hook implements the expected interfaces
|
||||
var _ interfaces.TaskPrestartHook = (*logmonHook)(nil)
|
||||
var _ interfaces.TaskStopHook = (*logmonHook)(nil)
|
|
@ -131,6 +131,10 @@ type TaskRunner struct {
|
|||
// will have these tags, and optionally more.
|
||||
baseLabels []metrics.Label
|
||||
|
||||
// logmonHookConfig is used to get the paths to the stdout and stderr fifos
|
||||
// to be passed to the driver for task logging
|
||||
logmonHookConfig *logmonHookConfig
|
||||
|
||||
// resourceUsage is written via UpdateStats and read via
|
||||
// LatestResourceUsage. May be nil at all times.
|
||||
resourceUsage *cstructs.TaskResourceUsage
|
||||
|
@ -394,6 +398,9 @@ func (tr *TaskRunner) runDriver() error {
|
|||
// Create a new context for Start since the environment may have been updated.
|
||||
ctx = driver.NewExecContext(tr.taskDir, tr.envBuilder.Build())
|
||||
|
||||
ctx.StdoutFifo = tr.logmonHookConfig.stdoutFifo
|
||||
ctx.StderrFifo = tr.logmonHookConfig.stderrFifo
|
||||
|
||||
// Start the job
|
||||
sresp, err := tr.driver.Start(ctx, tr.task)
|
||||
if err != nil {
|
||||
|
|
|
@ -16,11 +16,14 @@ func (tr *TaskRunner) initHooks() {
|
|||
hookLogger := tr.logger.Named("task_hook")
|
||||
task := tr.Task()
|
||||
|
||||
tr.logmonHookConfig = newLogMonHookConfig(task.Name, tr.taskDir.LogDir)
|
||||
|
||||
// Create the task directory hook. This is run first to ensure the
|
||||
// directoy path exists for other hooks.
|
||||
tr.runnerHooks = []interfaces.TaskHook{
|
||||
newValidateHook(tr.clientConfig, hookLogger),
|
||||
newTaskDirHook(tr, hookLogger),
|
||||
newLogMonHook(tr.logmonHookConfig, hookLogger),
|
||||
newArtifactHook(tr, hookLogger),
|
||||
newShutdownDelayHook(task.ShutdownDelay, hookLogger),
|
||||
newStatsHook(tr, tr.clientConfig.StatsCollectionInterval, hookLogger),
|
||||
|
|
|
@ -25,10 +25,8 @@ import (
|
|||
"github.com/docker/docker/registry"
|
||||
|
||||
"github.com/hashicorp/go-multierror"
|
||||
plugin "github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/nomad/client/allocdir"
|
||||
"github.com/hashicorp/nomad/client/driver/env"
|
||||
"github.com/hashicorp/nomad/client/driver/executor"
|
||||
dstructs "github.com/hashicorp/nomad/client/driver/structs"
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
"github.com/hashicorp/nomad/helper"
|
||||
|
@ -477,12 +475,9 @@ type dockerPID struct {
|
|||
ContainerID string
|
||||
KillTimeout time.Duration
|
||||
MaxKillTimeout time.Duration
|
||||
PluginConfig *PluginReattachConfig
|
||||
}
|
||||
|
||||
type DockerHandle struct {
|
||||
pluginClient *plugin.Client
|
||||
executor executor.Executor
|
||||
client *docker.Client
|
||||
waitClient *docker.Client
|
||||
logger *log.Logger
|
||||
|
@ -824,50 +819,10 @@ func (d *DockerDriver) Prestart(ctx *ExecContext, task *structs.Task) (*Prestart
|
|||
}
|
||||
|
||||
func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
|
||||
pluginLogFile := filepath.Join(ctx.TaskDir.Dir, "executor.out")
|
||||
executorConfig := &dstructs.ExecutorConfig{
|
||||
LogFile: pluginLogFile,
|
||||
LogLevel: d.config.LogLevel,
|
||||
}
|
||||
|
||||
exec, pluginClient, err := createExecutor(d.config.LogOutput, d.config, executorConfig)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
executorCtx := &executor.ExecutorContext{
|
||||
TaskEnv: ctx.TaskEnv,
|
||||
Task: task,
|
||||
Driver: "docker",
|
||||
LogDir: ctx.TaskDir.LogDir,
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
PortLowerBound: d.config.ClientMinPort,
|
||||
PortUpperBound: d.config.ClientMaxPort,
|
||||
}
|
||||
if err := exec.SetContext(executorCtx); err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, fmt.Errorf("failed to set executor context: %v", err)
|
||||
}
|
||||
|
||||
// The user hasn't specified any logging options so launch our own syslog
|
||||
// server if possible.
|
||||
syslogAddr := ""
|
||||
if len(d.driverConfig.Logging) == 0 {
|
||||
if runtime.GOOS == "darwin" {
|
||||
d.logger.Printf("[DEBUG] driver.docker: disabling syslog driver as Docker for Mac workaround")
|
||||
} else {
|
||||
ss, err := exec.LaunchSyslogServer()
|
||||
if err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, fmt.Errorf("failed to start syslog collector: %v", err)
|
||||
}
|
||||
syslogAddr = ss.Addr
|
||||
}
|
||||
}
|
||||
|
||||
config, err := d.createContainerConfig(ctx, task, d.driverConfig, syslogAddr)
|
||||
// TODO: implement alternative to launching a syslog server in the executor
|
||||
config, err := d.createContainerConfig(ctx, task, d.driverConfig, "")
|
||||
if err != nil {
|
||||
d.logger.Printf("[ERR] driver.docker: failed to create container configuration for image %q (%q): %v", d.driverConfig.ImageName, d.imageID, err)
|
||||
pluginClient.Kill()
|
||||
return nil, fmt.Errorf("Failed to create container configuration for image %q (%q): %v", d.driverConfig.ImageName, d.imageID, err)
|
||||
}
|
||||
|
||||
|
@ -875,7 +830,6 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
|
|||
if err != nil {
|
||||
wrapped := fmt.Sprintf("Failed to create container: %v", err)
|
||||
d.logger.Printf("[ERR] driver.docker: %s", wrapped)
|
||||
pluginClient.Kill()
|
||||
return nil, structs.WrapRecoverable(wrapped, err)
|
||||
}
|
||||
|
||||
|
@ -888,7 +842,6 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
|
|||
// Start the container
|
||||
if err := d.startContainer(container); err != nil {
|
||||
d.logger.Printf("[ERR] driver.docker: failed to start container %s: %s", container.ID, err)
|
||||
pluginClient.Kill()
|
||||
return nil, structs.NewRecoverableError(fmt.Errorf("Failed to start container %s: %s", container.ID, err), structs.IsRecoverable(err))
|
||||
}
|
||||
|
||||
|
@ -899,7 +852,6 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
|
|||
if err != nil {
|
||||
err = fmt.Errorf("failed to inspect started container %s: %s", container.ID, err)
|
||||
d.logger.Printf("[ERR] driver.docker: %v", err)
|
||||
pluginClient.Kill()
|
||||
return nil, structs.NewRecoverableError(err, true)
|
||||
}
|
||||
container = runningContainer
|
||||
|
@ -923,8 +875,6 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
|
|||
h := &DockerHandle{
|
||||
client: client,
|
||||
waitClient: waitClient,
|
||||
executor: exec,
|
||||
pluginClient: pluginClient,
|
||||
logger: d.logger,
|
||||
jobName: d.DriverContext.jobName,
|
||||
taskGroupName: d.DriverContext.taskGroupName,
|
||||
|
@ -938,7 +888,7 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespon
|
|||
doneCh: make(chan bool),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
removeContainerOnExit: d.config.ReadBoolDefault(dockerCleanupContainerConfigOption, dockerCleanupContainerConfigDefault),
|
||||
net: net,
|
||||
net: net,
|
||||
}
|
||||
go h.collectStats()
|
||||
go h.run()
|
||||
|
@ -1752,9 +1702,6 @@ func (d *DockerDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, er
|
|||
}
|
||||
d.logger.Printf("[INFO] driver.docker: re-attaching to docker process: %s", pid.ContainerID)
|
||||
d.logger.Printf("[DEBUG] driver.docker: re-attached to handle: %s", handleID)
|
||||
pluginConfig := &plugin.ClientConfig{
|
||||
Reattach: pid.PluginConfig.PluginConfig(),
|
||||
}
|
||||
|
||||
client, waitClient, err := d.dockerClients()
|
||||
if err != nil {
|
||||
|
@ -1780,18 +1727,6 @@ func (d *DockerDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, er
|
|||
if !found {
|
||||
return nil, fmt.Errorf("Failed to find container %s", pid.ContainerID)
|
||||
}
|
||||
exec, pluginClient, err := createExecutorWithConfig(pluginConfig, d.config.LogOutput)
|
||||
if err != nil {
|
||||
d.logger.Printf("[INFO] driver.docker: couldn't re-attach to the plugin process: %v", err)
|
||||
d.logger.Printf("[DEBUG] driver.docker: stopping container %q", pid.ContainerID)
|
||||
if e := client.StopContainer(pid.ContainerID, uint(pid.KillTimeout.Seconds())); e != nil {
|
||||
d.logger.Printf("[DEBUG] driver.docker: couldn't stop container: %v", e)
|
||||
}
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ver, _ := exec.Version()
|
||||
d.logger.Printf("[DEBUG] driver.docker: version of executor: %v", ver.Version)
|
||||
|
||||
// Increment the reference count since we successfully attached to this
|
||||
// container
|
||||
|
@ -1802,8 +1737,6 @@ func (d *DockerDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, er
|
|||
h := &DockerHandle{
|
||||
client: client,
|
||||
waitClient: waitClient,
|
||||
executor: exec,
|
||||
pluginClient: pluginClient,
|
||||
logger: d.logger,
|
||||
jobName: d.DriverContext.jobName,
|
||||
taskGroupName: d.DriverContext.taskGroupName,
|
||||
|
@ -1832,7 +1765,6 @@ func (h *DockerHandle) ID() string {
|
|||
ImageID: h.ImageID,
|
||||
KillTimeout: h.killTimeout,
|
||||
MaxKillTimeout: h.maxKillTimeout,
|
||||
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
|
||||
}
|
||||
data, err := json.Marshal(pid)
|
||||
if err != nil {
|
||||
|
@ -1856,9 +1788,6 @@ func (h *DockerHandle) Network() *cstructs.DriverNetwork {
|
|||
func (h *DockerHandle) Update(task *structs.Task) error {
|
||||
// Store the updated kill timeout.
|
||||
h.killTimeout = GetKillTimeout(task.KillTimeout, h.maxKillTimeout)
|
||||
if err := h.executor.UpdateTask(task); err != nil {
|
||||
h.logger.Printf("[DEBUG] driver.docker: failed to update log config: %v", err)
|
||||
}
|
||||
|
||||
// Update is not possible
|
||||
return nil
|
||||
|
@ -1923,10 +1852,8 @@ func (h *DockerHandle) Signal(s os.Signal) error {
|
|||
// Kill is used to terminate the task. This uses `docker stop -t killTimeout`
|
||||
func (h *DockerHandle) Kill() error {
|
||||
// Stop the container
|
||||
err := h.waitClient.StopContainer(h.containerID, uint(h.killTimeout.Seconds()))
|
||||
err := h.client.StopContainer(h.containerID, uint(h.killTimeout.Seconds()))
|
||||
if err != nil {
|
||||
h.executor.Exit()
|
||||
h.pluginClient.Kill()
|
||||
|
||||
// Container has already been removed.
|
||||
if strings.Contains(err.Error(), NoSuchContainerError) {
|
||||
|
@ -1986,10 +1913,6 @@ func (h *DockerHandle) run() {
|
|||
close(h.doneCh)
|
||||
|
||||
// Shutdown the syslog collector
|
||||
if err := h.executor.Exit(); err != nil {
|
||||
h.logger.Printf("[ERR] driver.docker: failed to kill the syslog collector: %v", err)
|
||||
}
|
||||
h.pluginClient.Kill()
|
||||
|
||||
// Stop the container just incase the docker daemon's wait returned
|
||||
// incorrectly
|
||||
|
|
|
@ -58,7 +58,7 @@ func TestDockerDriver_PidsLimit(t *testing.T) {
|
|||
task.Config["args"] = []string{"-c", "sleep 2 & sleep 2"}
|
||||
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the image into the task's directory
|
||||
|
|
|
@ -117,7 +117,7 @@ func dockerSetupWithClient(t *testing.T, task *structs.Task, client *docker.Clie
|
|||
if presp != nil && presp.CreatedResources != nil {
|
||||
driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
|
||||
}
|
||||
tctx.AllocDir.Destroy()
|
||||
tctx.Destroy()
|
||||
t.Fatalf("error in prestart: %v", err)
|
||||
}
|
||||
// Update the exec ctx with the driver network env vars
|
||||
|
@ -126,13 +126,13 @@ func dockerSetupWithClient(t *testing.T, task *structs.Task, client *docker.Clie
|
|||
sresp, err := driver.Start(tctx.ExecCtx, task)
|
||||
if err != nil {
|
||||
driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
|
||||
tctx.AllocDir.Destroy()
|
||||
tctx.Destroy()
|
||||
t.Fatalf("Failed to start driver: %s\nStack\n%s", err, debug.Stack())
|
||||
}
|
||||
|
||||
if sresp.Handle == nil {
|
||||
driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
|
||||
tctx.AllocDir.Destroy()
|
||||
tctx.Destroy()
|
||||
t.Fatalf("handle is nil\nStack\n%s", debug.Stack())
|
||||
}
|
||||
|
||||
|
@ -142,7 +142,7 @@ func dockerSetupWithClient(t *testing.T, task *structs.Task, client *docker.Clie
|
|||
cleanup := func() {
|
||||
driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
|
||||
sresp.Handle.Kill()
|
||||
tctx.AllocDir.Destroy()
|
||||
tctx.Destroy()
|
||||
}
|
||||
|
||||
return client, sresp.Handle.(*DockerHandle), cleanup
|
||||
|
@ -169,7 +169,7 @@ func TestDockerDriver_Fingerprint(t *testing.T) {
|
|||
|
||||
ctx := testDockerDriverContexts(t, &structs.Task{Name: "foo", Driver: "docker", Resources: basicResources})
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
node := &structs.Node{
|
||||
Attributes: make(map[string]string),
|
||||
|
@ -319,7 +319,7 @@ func TestDockerDriver_StartOpen_Wait(t *testing.T) {
|
|||
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
|
||||
|
@ -423,7 +423,7 @@ func TestDockerDriver_Start_StoppedContainer(t *testing.T) {
|
|||
}
|
||||
|
||||
tctx := testDockerDriverContexts(t, task)
|
||||
defer tctx.AllocDir.Destroy()
|
||||
defer tctx.Destroy()
|
||||
|
||||
copyImage(t, tctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
client := newTestDockerClient(t)
|
||||
|
@ -495,7 +495,7 @@ func TestDockerDriver_Start_LoadImage(t *testing.T) {
|
|||
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the image into the task's directory
|
||||
|
@ -563,7 +563,7 @@ func TestDockerDriver_Start_BadPull_Recoverable(t *testing.T) {
|
|||
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
|
||||
_, err := d.Prestart(ctx.ExecCtx, task)
|
||||
|
@ -616,7 +616,7 @@ func TestDockerDriver_Start_Wait_AllocDir(t *testing.T) {
|
|||
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
|
||||
|
@ -769,7 +769,7 @@ func TestDockerDriver_StartN(t *testing.T) {
|
|||
for idx, task := range taskList {
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
|
||||
|
@ -834,7 +834,7 @@ func TestDockerDriver_StartNVersions(t *testing.T) {
|
|||
for idx, task := range taskList {
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox_musl.tar")
|
||||
|
@ -1063,7 +1063,7 @@ func TestDockerDriver_Sysctl_Ulimit_Errors(t *testing.T) {
|
|||
ctx := testDockerDriverContexts(t, task)
|
||||
driver := NewDockerDriver(ctx.DriverCtx)
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
|
||||
_, err := driver.Prestart(ctx.ExecCtx, task)
|
||||
assert.NotNil(t, err, "Expected non nil error")
|
||||
|
@ -1118,7 +1118,7 @@ func TestDockerDriver_ForcePull_IsInvalidConfig(t *testing.T) {
|
|||
task.Config["force_pull"] = "nothing"
|
||||
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
driver := NewDockerDriver(ctx.DriverCtx)
|
||||
|
||||
|
@ -1279,7 +1279,7 @@ func TestDockerDriver_Capabilities(t *testing.T) {
|
|||
|
||||
driver := NewDockerDriver(tctx.DriverCtx)
|
||||
copyImage(t, tctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
defer tctx.AllocDir.Destroy()
|
||||
defer tctx.Destroy()
|
||||
|
||||
presp, err := driver.Prestart(tctx.ExecCtx, task)
|
||||
defer driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
|
||||
|
@ -1571,7 +1571,7 @@ func TestDockerDriver_User(t *testing.T) {
|
|||
ctx := testDockerDriverContexts(t, task)
|
||||
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
|
||||
driver := NewDockerDriver(ctx.DriverCtx)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
|
||||
_, err := driver.Prestart(ctx.ExecCtx, task)
|
||||
|
@ -1731,7 +1731,7 @@ func setupDockerVolumes(t *testing.T, cfg *config.Config, hostpath string) (*str
|
|||
}
|
||||
|
||||
// Build alloc and task directory structure
|
||||
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(cfg.AllocDir, uuid.Generate()))
|
||||
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(cfg.AllocDir, uuid.Generate()))
|
||||
if err := allocDir.Build(); err != nil {
|
||||
t.Fatalf("failed to build alloc dir: %v", err)
|
||||
}
|
||||
|
@ -2055,7 +2055,7 @@ func TestDockerDriver_Mounts(t *testing.T) {
|
|||
ctx := testDockerDriverContexts(t, task)
|
||||
driver := NewDockerDriver(ctx.DriverCtx)
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
|
||||
_, err := driver.Prestart(ctx.ExecCtx, task)
|
||||
if err == nil && c.Error != "" {
|
||||
|
@ -2089,7 +2089,7 @@ func TestDockerDriver_Cleanup(t *testing.T) {
|
|||
},
|
||||
}
|
||||
tctx := testDockerDriverContexts(t, task)
|
||||
defer tctx.AllocDir.Destroy()
|
||||
defer tctx.Destroy()
|
||||
|
||||
// Run Prestart
|
||||
driver := NewDockerDriver(tctx.DriverCtx).(*DockerDriver)
|
||||
|
@ -2277,7 +2277,7 @@ func TestDockerDriver_Devices_IsInvalidConfig(t *testing.T) {
|
|||
ctx := testDockerDriverContexts(t, task)
|
||||
driver := NewDockerDriver(ctx.DriverCtx)
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
|
||||
if _, err := driver.Prestart(ctx.ExecCtx, task); err == nil || err.Error() != tc.err.Error() {
|
||||
t.Fatalf("error expected in prestart, got %v, expected %v", err, tc.err)
|
||||
|
@ -2382,7 +2382,7 @@ func TestDockerDriver_Kill(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
copyImage(t, ctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
|
||||
|
@ -2502,7 +2502,7 @@ func TestDockerDriver_AdvertiseIPv6Address(t *testing.T) {
|
|||
tctx := testDockerDriverContexts(t, task)
|
||||
driver := NewDockerDriver(tctx.DriverCtx)
|
||||
copyImage(t, tctx.ExecCtx.TaskDir, "busybox.tar")
|
||||
defer tctx.AllocDir.Destroy()
|
||||
defer tctx.Destroy()
|
||||
|
||||
presp, err := driver.Prestart(tctx.ExecCtx, task)
|
||||
defer driver.Cleanup(tctx.ExecCtx, presp.CreatedResources)
|
||||
|
|
|
@ -43,7 +43,7 @@ func TestDockerDriver_Signal(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDockerDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewDockerDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the image into the task's directory
|
||||
|
|
|
@ -339,6 +339,12 @@ type ExecContext struct {
|
|||
|
||||
// TaskEnv contains the task's environment variables.
|
||||
TaskEnv *env.TaskEnv
|
||||
|
||||
// StdoutFifo is the path to the named pipe to write stdout to
|
||||
StdoutFifo string
|
||||
|
||||
// StderrFifo is the path to the named pipe to write stderr to
|
||||
StderrFifo string
|
||||
}
|
||||
|
||||
// NewExecContext is used to create a new execution context
|
||||
|
|
|
@ -1,20 +1,25 @@
|
|||
package driver
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"math/rand"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"reflect"
|
||||
"runtime"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
plugin "github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/nomad/client/allocdir"
|
||||
"github.com/hashicorp/nomad/client/config"
|
||||
"github.com/hashicorp/nomad/client/driver/env"
|
||||
"github.com/hashicorp/nomad/client/logmon"
|
||||
"github.com/hashicorp/nomad/helper/testlog"
|
||||
"github.com/hashicorp/nomad/helper/testtask"
|
||||
"github.com/hashicorp/nomad/helper/uuid"
|
||||
"github.com/hashicorp/nomad/nomad/mock"
|
||||
"github.com/hashicorp/nomad/nomad/structs"
|
||||
)
|
||||
|
@ -95,26 +100,36 @@ func testConfig(t *testing.T) *config.Config {
|
|||
conf.MaxKillTimeout = 10 * time.Second
|
||||
conf.Region = "global"
|
||||
conf.Node = mock.Node()
|
||||
conf.LogLevel = "DEBUG"
|
||||
conf.LogOutput = testlog.NewWriter(t)
|
||||
return conf
|
||||
}
|
||||
|
||||
type testContext struct {
|
||||
AllocDir *allocdir.AllocDir
|
||||
DriverCtx *DriverContext
|
||||
ExecCtx *ExecContext
|
||||
EnvBuilder *env.Builder
|
||||
AllocDir *allocdir.AllocDir
|
||||
DriverCtx *DriverContext
|
||||
ExecCtx *ExecContext
|
||||
EnvBuilder *env.Builder
|
||||
logmon logmon.LogMon
|
||||
logmonPlugin *plugin.Client
|
||||
}
|
||||
|
||||
func (ctx *testContext) Destroy() {
|
||||
ctx.AllocDir.Destroy()
|
||||
ctx.logmon.Stop()
|
||||
ctx.logmonPlugin.Kill()
|
||||
}
|
||||
|
||||
// testDriverContext sets up an alloc dir, task dir, DriverContext, and ExecContext.
|
||||
//
|
||||
// It is up to the caller to call AllocDir.Destroy to cleanup.
|
||||
// It is up to the caller to call Destroy to cleanup.
|
||||
func testDriverContexts(t *testing.T, task *structs.Task) *testContext {
|
||||
cfg := testConfig(t)
|
||||
cfg.Node = mock.Node()
|
||||
alloc := mock.Alloc()
|
||||
alloc.NodeID = cfg.Node.ID
|
||||
|
||||
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(cfg.AllocDir, alloc.ID))
|
||||
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(cfg.AllocDir, alloc.ID))
|
||||
if err := allocDir.Build(); err != nil {
|
||||
t.Fatalf("AllocDir.Build() failed: %v", err)
|
||||
}
|
||||
|
@ -139,12 +154,45 @@ func testDriverContexts(t *testing.T, task *structs.Task) *testContext {
|
|||
execCtx := NewExecContext(td, eb.Build())
|
||||
|
||||
logger := testlog.Logger(t)
|
||||
hcLogger := testlog.HCLogger(t)
|
||||
emitter := func(m string, args ...interface{}) {
|
||||
logger.Printf("[EVENT] "+m, args...)
|
||||
hcLogger.Info(fmt.Sprintf("[EVENT] "+m, args...))
|
||||
}
|
||||
driverCtx := NewDriverContext(alloc.Job.Name, alloc.TaskGroup, task.Name, alloc.ID, cfg, cfg.Node, logger, emitter)
|
||||
l, c, err := logmon.LaunchLogMon(hcLogger)
|
||||
if err != nil {
|
||||
allocDir.Destroy()
|
||||
t.Fatalf("LaunchLogMon() failed: %v", err)
|
||||
}
|
||||
|
||||
return &testContext{allocDir, driverCtx, execCtx, eb}
|
||||
var stdoutFifo, stderrFifo string
|
||||
if runtime.GOOS == "windows" {
|
||||
id := uuid.Generate()[:8]
|
||||
stdoutFifo = fmt.Sprintf("//./pipe/%s.stdout.%s", id, task.Name)
|
||||
stderrFifo = fmt.Sprintf("//./pipe/%s.stderr.%s", id, task.Name)
|
||||
} else {
|
||||
stdoutFifo = filepath.Join(td.LogDir, fmt.Sprintf("%s.stdout", task.Name))
|
||||
stderrFifo = filepath.Join(td.LogDir, fmt.Sprintf("%s.stderr", task.Name))
|
||||
}
|
||||
|
||||
err = l.Start(&logmon.LogConfig{
|
||||
LogDir: td.LogDir,
|
||||
StdoutLogFile: fmt.Sprintf("%s.stdout", task.Name),
|
||||
StderrLogFile: fmt.Sprintf("%s.stderr", task.Name),
|
||||
StdoutFifo: stdoutFifo,
|
||||
StderrFifo: stderrFifo,
|
||||
MaxFiles: 10,
|
||||
MaxFileSizeMB: 10,
|
||||
})
|
||||
if err != nil {
|
||||
allocDir.Destroy()
|
||||
t.Fatalf("LogMon.Start() failed: %v", err)
|
||||
}
|
||||
|
||||
execCtx.StdoutFifo = stdoutFifo
|
||||
execCtx.StderrFifo = stderrFifo
|
||||
|
||||
return &testContext{allocDir, driverCtx, execCtx, eb, l, c}
|
||||
}
|
||||
|
||||
// setupTaskEnv creates a test env for GetTaskEnv testing. Returns task dir,
|
||||
|
@ -179,7 +227,7 @@ func setupTaskEnv(t *testing.T, driver string) (*allocdir.TaskDir, map[string]st
|
|||
alloc.Name = "Bar"
|
||||
alloc.AllocatedResources.Tasks["web"].Networks[0].DynamicPorts[0].Value = 2000
|
||||
conf := testConfig(t)
|
||||
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(conf.AllocDir, alloc.ID))
|
||||
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(conf.AllocDir, alloc.ID))
|
||||
taskDir := allocDir.NewTaskDir(task.Name)
|
||||
eb := env.NewBuilder(conf.Node, alloc, task, conf.Region)
|
||||
tmpDriver, err := NewDriver(driver, NewEmptyDriverContext())
|
||||
|
|
|
@ -37,17 +37,17 @@ type ExecDriverConfig struct {
|
|||
|
||||
// execHandle is returned from Start/Open as a handle to the PID
|
||||
type execHandle struct {
|
||||
pluginClient *plugin.Client
|
||||
executor executor.Executor
|
||||
isolationConfig *dstructs.IsolationConfig
|
||||
userPid int
|
||||
taskDir *allocdir.TaskDir
|
||||
killTimeout time.Duration
|
||||
maxKillTimeout time.Duration
|
||||
logger *log.Logger
|
||||
waitCh chan *dstructs.WaitResult
|
||||
doneCh chan struct{}
|
||||
version string
|
||||
pluginClient *plugin.Client
|
||||
executor executor.Executor
|
||||
userPid int
|
||||
taskShutdownSignal string
|
||||
taskDir *allocdir.TaskDir
|
||||
killTimeout time.Duration
|
||||
maxKillTimeout time.Duration
|
||||
logger *log.Logger
|
||||
waitCh chan *dstructs.WaitResult
|
||||
doneCh chan struct{}
|
||||
version string
|
||||
}
|
||||
|
||||
// NewExecDriver is used to create a new exec driver
|
||||
|
@ -110,26 +110,16 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
|
||||
pluginLogFile := filepath.Join(ctx.TaskDir.Dir, "executor.out")
|
||||
executorConfig := &dstructs.ExecutorConfig{
|
||||
LogFile: pluginLogFile,
|
||||
LogLevel: d.config.LogLevel,
|
||||
LogFile: pluginLogFile,
|
||||
LogLevel: d.config.LogLevel,
|
||||
FSIsolation: true,
|
||||
}
|
||||
exec, pluginClient, err := createExecutor(d.config.LogOutput, d.config, executorConfig)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
executorCtx := &executor.ExecutorContext{
|
||||
TaskEnv: ctx.TaskEnv,
|
||||
Driver: "exec",
|
||||
LogDir: ctx.TaskDir.LogDir,
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
Task: task,
|
||||
}
|
||||
if err := exec.SetContext(executorCtx); err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, fmt.Errorf("failed to set executor context: %v", err)
|
||||
}
|
||||
|
||||
taskKillSignal, err := getTaskKillSignal(task.KillSignal)
|
||||
_, err = getTaskKillSignal(task.KillSignal)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -137,13 +127,21 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
execCmd := &executor.ExecCommand{
|
||||
Cmd: command,
|
||||
Args: driverConfig.Args,
|
||||
TaskKillSignal: taskKillSignal,
|
||||
FSIsolation: true,
|
||||
ResourceLimits: true,
|
||||
User: getExecutorUser(task),
|
||||
Resources: &executor.Resources{
|
||||
CPU: task.Resources.CPU,
|
||||
MemoryMB: task.Resources.MemoryMB,
|
||||
IOPS: task.Resources.IOPS,
|
||||
DiskMB: task.Resources.DiskMB,
|
||||
},
|
||||
Env: ctx.TaskEnv.List(),
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
StdoutPath: ctx.StdoutFifo,
|
||||
StderrPath: ctx.StderrFifo,
|
||||
}
|
||||
|
||||
ps, err := exec.LaunchCmd(execCmd)
|
||||
ps, err := exec.Launch(execCmd)
|
||||
if err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, err
|
||||
|
@ -154,17 +152,17 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
// Return a driver handle
|
||||
maxKill := d.DriverContext.config.MaxKillTimeout
|
||||
h := &execHandle{
|
||||
pluginClient: pluginClient,
|
||||
userPid: ps.Pid,
|
||||
executor: exec,
|
||||
isolationConfig: ps.IsolationConfig,
|
||||
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
|
||||
maxKillTimeout: maxKill,
|
||||
logger: d.logger,
|
||||
version: d.config.Version.VersionNumber(),
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
taskDir: ctx.TaskDir,
|
||||
pluginClient: pluginClient,
|
||||
userPid: ps.Pid,
|
||||
taskShutdownSignal: task.KillSignal,
|
||||
executor: exec,
|
||||
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
|
||||
maxKillTimeout: maxKill,
|
||||
logger: d.logger,
|
||||
version: d.config.Version.VersionNumber(),
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
taskDir: ctx.TaskDir,
|
||||
}
|
||||
go h.run()
|
||||
return &StartResponse{Handle: h}, nil
|
||||
|
@ -173,12 +171,11 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
func (d *ExecDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }
|
||||
|
||||
type execId struct {
|
||||
Version string
|
||||
KillTimeout time.Duration
|
||||
MaxKillTimeout time.Duration
|
||||
UserPid int
|
||||
IsolationConfig *dstructs.IsolationConfig
|
||||
PluginConfig *PluginReattachConfig
|
||||
Version string
|
||||
KillTimeout time.Duration
|
||||
MaxKillTimeout time.Duration
|
||||
UserPid int
|
||||
PluginConfig *PluginReattachConfig
|
||||
}
|
||||
|
||||
func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error) {
|
||||
|
@ -198,12 +195,6 @@ func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
|
|||
if e := destroyPlugin(id.PluginConfig.Pid, id.UserPid); e != nil {
|
||||
merrs.Errors = append(merrs.Errors, fmt.Errorf("error destroying plugin and userpid: %v", e))
|
||||
}
|
||||
if id.IsolationConfig != nil {
|
||||
ePid := pluginConfig.Reattach.Pid
|
||||
if e := executor.ClientCleanup(id.IsolationConfig, ePid); e != nil {
|
||||
merrs.Errors = append(merrs.Errors, fmt.Errorf("destroying cgroup failed: %v", e))
|
||||
}
|
||||
}
|
||||
return nil, fmt.Errorf("error connecting to plugin: %v", merrs.ErrorOrNil())
|
||||
}
|
||||
|
||||
|
@ -211,17 +202,16 @@ func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
|
|||
d.logger.Printf("[DEBUG] driver.exec : version of executor: %v", ver.Version)
|
||||
// Return a driver handle
|
||||
h := &execHandle{
|
||||
pluginClient: client,
|
||||
executor: exec,
|
||||
userPid: id.UserPid,
|
||||
isolationConfig: id.IsolationConfig,
|
||||
logger: d.logger,
|
||||
version: id.Version,
|
||||
killTimeout: id.KillTimeout,
|
||||
maxKillTimeout: id.MaxKillTimeout,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
taskDir: ctx.TaskDir,
|
||||
pluginClient: client,
|
||||
executor: exec,
|
||||
userPid: id.UserPid,
|
||||
logger: d.logger,
|
||||
version: id.Version,
|
||||
killTimeout: id.KillTimeout,
|
||||
maxKillTimeout: id.MaxKillTimeout,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
taskDir: ctx.TaskDir,
|
||||
}
|
||||
go h.run()
|
||||
return h, nil
|
||||
|
@ -229,12 +219,11 @@ func (d *ExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
|
|||
|
||||
func (h *execHandle) ID() string {
|
||||
id := execId{
|
||||
Version: h.version,
|
||||
KillTimeout: h.killTimeout,
|
||||
MaxKillTimeout: h.maxKillTimeout,
|
||||
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
|
||||
UserPid: h.userPid,
|
||||
IsolationConfig: h.isolationConfig,
|
||||
Version: h.version,
|
||||
KillTimeout: h.killTimeout,
|
||||
MaxKillTimeout: h.maxKillTimeout,
|
||||
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
|
||||
UserPid: h.userPid,
|
||||
}
|
||||
|
||||
data, err := json.Marshal(id)
|
||||
|
@ -251,7 +240,12 @@ func (h *execHandle) WaitCh() chan *dstructs.WaitResult {
|
|||
func (h *execHandle) Update(task *structs.Task) error {
|
||||
// Store the updated kill timeout.
|
||||
h.killTimeout = GetKillTimeout(task.KillTimeout, h.maxKillTimeout)
|
||||
h.executor.UpdateTask(task)
|
||||
h.executor.UpdateResources(&executor.Resources{
|
||||
CPU: task.Resources.CPU,
|
||||
MemoryMB: task.Resources.MemoryMB,
|
||||
IOPS: task.Resources.IOPS,
|
||||
DiskMB: task.Resources.DiskMB,
|
||||
})
|
||||
|
||||
// Update is not possible
|
||||
return nil
|
||||
|
@ -275,11 +269,11 @@ func (d *execHandle) Network() *cstructs.DriverNetwork {
|
|||
}
|
||||
|
||||
func (h *execHandle) Kill() error {
|
||||
if err := h.executor.ShutDown(); err != nil {
|
||||
if err := h.executor.Shutdown(h.taskShutdownSignal, h.killTimeout); err != nil {
|
||||
if h.pluginClient.Exited() {
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("executor Shutdown failed: %v", err)
|
||||
return fmt.Errorf("executor Kill failed: %v", err)
|
||||
}
|
||||
|
||||
select {
|
||||
|
@ -288,8 +282,8 @@ func (h *execHandle) Kill() error {
|
|||
if h.pluginClient.Exited() {
|
||||
break
|
||||
}
|
||||
if err := h.executor.Exit(); err != nil {
|
||||
return fmt.Errorf("executor Exit failed: %v", err)
|
||||
if err := h.executor.Shutdown(h.taskShutdownSignal, h.killTimeout); err != nil {
|
||||
return fmt.Errorf("executor Destroy failed: %v", err)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
|
@ -303,22 +297,8 @@ func (h *execHandle) run() {
|
|||
ps, werr := h.executor.Wait()
|
||||
close(h.doneCh)
|
||||
|
||||
// If the exitcode is 0 and we had an error that means the plugin didn't
|
||||
// connect and doesn't know the state of the user process so we are killing
|
||||
// the user process so that when we create a new executor on restarting the
|
||||
// new user process doesn't have collisions with resources that the older
|
||||
// user pid might be holding onto.
|
||||
if ps.ExitCode == 0 && werr != nil {
|
||||
if h.isolationConfig != nil {
|
||||
ePid := h.pluginClient.ReattachConfig().Pid
|
||||
if e := executor.ClientCleanup(h.isolationConfig, ePid); e != nil {
|
||||
h.logger.Printf("[ERR] driver.exec: destroying resource container failed: %v", e)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Exit the executor
|
||||
if err := h.executor.Exit(); err != nil {
|
||||
// Destroy the executor
|
||||
if err := h.executor.Shutdown(h.taskShutdownSignal, 0); err != nil {
|
||||
h.logger.Printf("[ERR] driver.exec: error destroying executor: %v", err)
|
||||
}
|
||||
h.pluginClient.Kill()
|
||||
|
|
|
@ -56,7 +56,7 @@ func TestExecDriver_Fingerprint(t *testing.T) {
|
|||
Resources: structs.DefaultResources(),
|
||||
}
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
node := &structs.Node{
|
||||
Attributes: map[string]string{
|
||||
|
@ -100,7 +100,7 @@ func TestExecDriver_StartOpen_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -144,7 +144,7 @@ func TestExecDriver_Start_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -198,7 +198,7 @@ func TestExecDriver_Start_Wait_AllocDir(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -252,7 +252,7 @@ func TestExecDriver_Start_Kill_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -304,7 +304,7 @@ func TestExecDriverUser(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -343,7 +343,7 @@ func TestExecDriver_HandlerExec(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
//defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -397,7 +397,7 @@ func TestExecDriver_HandlerExec(t *testing.T) {
|
|||
if line == "" {
|
||||
continue
|
||||
}
|
||||
if !strings.Contains(line, ":/nomad/") {
|
||||
if !strings.Contains(line, ":/nomad/") && !strings.Contains(line, ":name=") {
|
||||
t.Errorf("Not a member of the alloc's cgroup: expected=...:/nomad/... -- found=%q", line)
|
||||
continue
|
||||
}
|
||||
|
@ -420,6 +420,7 @@ func TestExecDriver_HandlerExec(t *testing.T) {
|
|||
}
|
||||
|
||||
if err := handle.Kill(); err != nil {
|
||||
t.Logf("Check allocdir: %x", ctx.AllocDir.AllocDir)
|
||||
t.Fatalf("error killing exec handle: %v", err)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ func TestExecDriver_KillUserPid_OnPluginReconnectFailure(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -111,7 +111,7 @@ func TestExecDriver_Signal(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewExecDriver(ctx.DriverCtx)
|
||||
|
||||
testFile := filepath.Join(ctx.ExecCtx.TaskDir.Dir, "test.sh")
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1,46 +1,15 @@
|
|||
// +build darwin dragonfly freebsd netbsd openbsd solaris windows
|
||||
// +build !linux
|
||||
|
||||
package executor
|
||||
|
||||
import (
|
||||
"os"
|
||||
import hclog "github.com/hashicorp/go-hclog"
|
||||
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
"github.com/mitchellh/go-ps"
|
||||
)
|
||||
|
||||
func (e *UniversalExecutor) configureChroot() error {
|
||||
return nil
|
||||
func NewExecutorWithIsolation(logger hclog.Logger) Executor {
|
||||
logger = logger.Named("executor")
|
||||
logger.Error("isolation executor is not supported on this platform, using default")
|
||||
return NewExecutor(logger)
|
||||
}
|
||||
|
||||
func (e *UniversalExecutor) removeChrootMounts() error {
|
||||
return nil
|
||||
}
|
||||
func (e *UniversalExecutor) configureResourceContainer(_ int) error { return nil }
|
||||
|
||||
func (e *UniversalExecutor) runAs(userid string) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *UniversalExecutor) applyLimits(pid int) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *UniversalExecutor) configureIsolation() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *UniversalExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
|
||||
pidStats, err := e.pidStats()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return e.aggregatedResourceUsage(pidStats), nil
|
||||
}
|
||||
|
||||
func (e *UniversalExecutor) getAllPids() (map[int]*nomadPid, error) {
|
||||
allProcesses, err := ps.Processes()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return e.scanPids(os.Getpid(), allProcesses)
|
||||
}
|
||||
func (e *UniversalExecutor) runAs(_ string) error { return nil }
|
||||
|
|
|
@ -1,189 +1,333 @@
|
|||
// +build linux
|
||||
|
||||
package executor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"os/user"
|
||||
"os/exec"
|
||||
"path"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
"strings"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/go-multierror"
|
||||
"github.com/mitchellh/go-ps"
|
||||
cgroupFs "github.com/opencontainers/runc/libcontainer/cgroups/fs"
|
||||
cgroupConfig "github.com/opencontainers/runc/libcontainer/configs"
|
||||
|
||||
"github.com/armon/circbuf"
|
||||
"github.com/hashicorp/consul-template/signals"
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
multierror "github.com/hashicorp/go-multierror"
|
||||
dstructs "github.com/hashicorp/nomad/client/driver/structs"
|
||||
"github.com/hashicorp/nomad/client/stats"
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
"github.com/hashicorp/nomad/helper/discover"
|
||||
shelpers "github.com/hashicorp/nomad/helper/stats"
|
||||
"github.com/hashicorp/nomad/helper/uuid"
|
||||
"github.com/hashicorp/nomad/nomad/structs"
|
||||
"github.com/opencontainers/runc/libcontainer"
|
||||
"github.com/opencontainers/runc/libcontainer/cgroups"
|
||||
cgroupFs "github.com/opencontainers/runc/libcontainer/cgroups/fs"
|
||||
lconfigs "github.com/opencontainers/runc/libcontainer/configs"
|
||||
|
||||
"github.com/syndtr/gocapability/capability"
|
||||
)
|
||||
|
||||
const (
|
||||
defaultCgroupParent = "nomad"
|
||||
)
|
||||
|
||||
var (
|
||||
// The statistics the executor exposes when using cgroups
|
||||
ExecutorCgroupMeasuredMemStats = []string{"RSS", "Cache", "Swap", "Max Usage", "Kernel Usage", "Kernel Max Usage"}
|
||||
ExecutorCgroupMeasuredCpuStats = []string{"System Mode", "User Mode", "Throttled Periods", "Throttled Time", "Percent"}
|
||||
|
||||
// allCaps is all linux capabilities which is used to configure libcontainer
|
||||
allCaps []string
|
||||
)
|
||||
|
||||
// configureIsolation configures chroot and creates cgroups
|
||||
func (e *UniversalExecutor) configureIsolation() error {
|
||||
if e.command.FSIsolation {
|
||||
if err := e.configureChroot(); err != nil {
|
||||
return err
|
||||
}
|
||||
// initialize the allCaps var with all capabilities available on the system
|
||||
func init() {
|
||||
last := capability.CAP_LAST_CAP
|
||||
// workaround for RHEL6 which has no /proc/sys/kernel/cap_last_cap
|
||||
if last == capability.Cap(63) {
|
||||
last = capability.CAP_BLOCK_SUSPEND
|
||||
}
|
||||
|
||||
if e.command.ResourceLimits || e.command.BasicProcessCgroup {
|
||||
if err := e.configureCgroups(e.ctx.Task.Resources); err != nil {
|
||||
return fmt.Errorf("error creating cgroups: %v", err)
|
||||
for _, cap := range capability.List() {
|
||||
if cap > last {
|
||||
continue
|
||||
}
|
||||
allCaps = append(allCaps, fmt.Sprintf("CAP_%s", strings.ToUpper(cap.String())))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// applyLimits puts a process in a pre-configured cgroup
|
||||
func (e *UniversalExecutor) applyLimits(pid int) error {
|
||||
if !(e.command.ResourceLimits || e.command.BasicProcessCgroup) {
|
||||
return nil
|
||||
}
|
||||
// LibcontainerExecutor implements an Executor with the runc/libcontainer api
|
||||
type LibcontainerExecutor struct {
|
||||
id string
|
||||
command *ExecCommand
|
||||
|
||||
// Entering the process in the cgroup
|
||||
manager := getCgroupManager(e.resConCtx.groups, nil)
|
||||
if err := manager.Apply(pid); err != nil {
|
||||
e.logger.Printf("[ERR] executor: error applying pid to cgroup: %v", err)
|
||||
return err
|
||||
}
|
||||
logger hclog.Logger
|
||||
|
||||
e.resConCtx.cgPaths = manager.GetPaths()
|
||||
totalCpuStats *stats.CpuStats
|
||||
userCpuStats *stats.CpuStats
|
||||
systemCpuStats *stats.CpuStats
|
||||
pidCollector *pidCollector
|
||||
|
||||
// Don't enter all the cgroups since we will inherit resources limits. Only
|
||||
// use devices (required by libcontainer) and freezer. Freezer allows us to
|
||||
// capture all pids and stop any fork/execs from happening while we are
|
||||
// cleaning up.
|
||||
if !e.command.ResourceLimits {
|
||||
// Move the executor into the global cgroup so that the task specific
|
||||
// cgroup can be destroyed.
|
||||
nilGroup := &cgroupConfig.Cgroup{}
|
||||
nilGroup.Path = "/"
|
||||
nilGroup.Resources = e.resConCtx.groups.Resources
|
||||
nilManager := getCgroupManager(nilGroup, nil)
|
||||
err := nilManager.Apply(pid)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to remove executor pid %d: %v", pid, err)
|
||||
}
|
||||
|
||||
// Grab the freezer and devices cgroup paths. We do this from the old
|
||||
// manager after the executor pid has been applied since there is no
|
||||
// other way to determine what the proper cgroup paths would be.
|
||||
freezer := &cgroupFs.FreezerGroup{}
|
||||
devices := &cgroupFs.DevicesGroup{}
|
||||
freezerName, devicesName := freezer.Name(), devices.Name()
|
||||
newPath := map[string]string{
|
||||
freezerName: e.resConCtx.cgPaths[freezerName],
|
||||
devicesName: e.resConCtx.cgPaths[devicesName],
|
||||
}
|
||||
|
||||
// Clear the cgroups paths so that everything is properly cleaned except
|
||||
// the groups we want our process to stay in. This will delete the
|
||||
// directories from disk.
|
||||
manager.Cgroups.Paths = nil
|
||||
delete(manager.Paths, freezerName)
|
||||
delete(manager.Paths, devicesName)
|
||||
if err := manager.Destroy(); err != nil {
|
||||
e.logger.Printf("[ERR] executor: failed to destroy original: %v", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Update our context such that the new cgroup manager only is tracking
|
||||
// the paths we care about now.
|
||||
e.resConCtx.cgPaths = newPath
|
||||
e.resConCtx.groups.Paths = newPath
|
||||
|
||||
// Apply just the freezer and devices now
|
||||
manager = getCgroupManager(e.resConCtx.groups, e.resConCtx.cgPaths)
|
||||
if err := manager.Apply(pid); err != nil {
|
||||
e.logger.Printf("[ERR] executor: error applying pid to cgroup subset %v: %v", e.resConCtx.cgPaths, err)
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
cgConfig := cgroupConfig.Config{Cgroups: e.resConCtx.groups}
|
||||
if err := manager.Set(&cgConfig); err != nil {
|
||||
e.logger.Printf("[ERR] executor: error setting cgroup config: %v", err)
|
||||
if er := DestroyCgroup(e.resConCtx.groups, e.resConCtx.cgPaths, os.Getpid()); er != nil {
|
||||
e.logger.Printf("[ERR] executor: error destroying cgroup: %v", er)
|
||||
}
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
container libcontainer.Container
|
||||
userProc *libcontainer.Process
|
||||
userProcExited chan interface{}
|
||||
exitState *ProcessState
|
||||
}
|
||||
|
||||
// configureCgroups converts a Nomad Resources specification into the equivalent
|
||||
// cgroup configuration. It returns an error if the resources are invalid.
|
||||
func (e *UniversalExecutor) configureCgroups(resources *structs.Resources) error {
|
||||
e.resConCtx.groups = &cgroupConfig.Cgroup{}
|
||||
e.resConCtx.groups.Resources = &cgroupConfig.Resources{}
|
||||
cgroupName := uuid.Generate()
|
||||
e.resConCtx.groups.Path = filepath.Join("/nomad", cgroupName)
|
||||
|
||||
// Allow access to /dev/
|
||||
allowAllDevs := true
|
||||
e.resConCtx.groups.Resources.AllowAllDevices = &allowAllDevs
|
||||
|
||||
// Use a cgroup but don't apply limits
|
||||
if !e.command.ResourceLimits {
|
||||
return nil
|
||||
func NewExecutorWithIsolation(logger hclog.Logger) Executor {
|
||||
logger = logger.Named("isolated_executor")
|
||||
if err := shelpers.Init(); err != nil {
|
||||
logger.Error("unable to initialize stats", "error", err)
|
||||
}
|
||||
|
||||
if resources.MemoryMB > 0 {
|
||||
// Total amount of memory allowed to consume
|
||||
e.resConCtx.groups.Resources.Memory = int64(resources.MemoryMB * 1024 * 1024)
|
||||
// Disable swap to avoid issues on the machine
|
||||
var memSwappiness uint64 = 0
|
||||
e.resConCtx.groups.Resources.MemorySwappiness = &memSwappiness
|
||||
return &LibcontainerExecutor{
|
||||
id: strings.Replace(uuid.Generate(), "-", "_", 0),
|
||||
logger: logger,
|
||||
totalCpuStats: stats.NewCpuStats(),
|
||||
userCpuStats: stats.NewCpuStats(),
|
||||
systemCpuStats: stats.NewCpuStats(),
|
||||
pidCollector: newPidCollector(logger),
|
||||
}
|
||||
|
||||
if resources.CPU < 2 {
|
||||
return fmt.Errorf("resources.CPU must be equal to or greater than 2: %v", resources.CPU)
|
||||
}
|
||||
|
||||
// Set the relative CPU shares for this cgroup.
|
||||
e.resConCtx.groups.Resources.CpuShares = uint64(resources.CPU)
|
||||
|
||||
if resources.IOPS != 0 {
|
||||
// Validate it is in an acceptable range.
|
||||
if resources.IOPS < 10 || resources.IOPS > 1000 {
|
||||
return fmt.Errorf("resources.IOPS must be between 10 and 1000: %d", resources.IOPS)
|
||||
}
|
||||
|
||||
e.resConCtx.groups.Resources.BlkioWeight = uint16(resources.IOPS)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stats reports the resource utilization of the cgroup. If there is no resource
|
||||
// isolation we aggregate the resource utilization of all the pids launched by
|
||||
// the executor.
|
||||
func (e *UniversalExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
|
||||
// If we don't use full resource limits fallback to normal collection. It is
|
||||
// not enough to be in the Cgroup since you must be in the memory, cpu, and
|
||||
// cpuacct cgroup to gather the correct statistics.
|
||||
if !e.command.ResourceLimits {
|
||||
pidStats, err := e.pidStats()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return e.aggregatedResourceUsage(pidStats), nil
|
||||
// Launch creates a new container in libcontainer and starts a new process with it
|
||||
func (l *LibcontainerExecutor) Launch(command *ExecCommand) (*ProcessState, error) {
|
||||
l.logger.Info("launching command", "command", command.Cmd, "args", strings.Join(command.Args, " "))
|
||||
// Find the nomad executable to launch the executor process with
|
||||
bin, err := discover.NomadExecutable()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("unable to find the nomad binary: %v", err)
|
||||
}
|
||||
ts := time.Now()
|
||||
manager := getCgroupManager(e.resConCtx.groups, e.resConCtx.cgPaths)
|
||||
stats, err := manager.GetStats()
|
||||
|
||||
l.command = command
|
||||
|
||||
// Move to the root cgroup until process is started
|
||||
subsystems, err := cgroups.GetAllSubsystems()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := JoinRootCgroup(subsystems); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// create a new factory which will store the container state in the allocDir
|
||||
factory, err := libcontainer.New(
|
||||
path.Join(command.TaskDir, "../alloc/container"),
|
||||
libcontainer.Cgroupfs,
|
||||
libcontainer.InitArgs(bin, "libcontainer-shim"),
|
||||
)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to create factory: %v", err)
|
||||
}
|
||||
|
||||
// A container groups processes under the same isolation enforcement
|
||||
container, err := factory.Create(l.id, newLibcontainerConfig(command))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to create container(%s): %v", l.id, err)
|
||||
}
|
||||
l.container = container
|
||||
|
||||
combined := append([]string{command.Cmd}, command.Args...)
|
||||
stdout, err := command.Stdout()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
stderr, err := command.Stderr()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// the task process will be started by the container
|
||||
process := &libcontainer.Process{
|
||||
Args: combined,
|
||||
Env: command.Env,
|
||||
Stdout: stdout,
|
||||
Stderr: stderr,
|
||||
Init: true,
|
||||
}
|
||||
|
||||
if command.User != "" {
|
||||
process.User = command.User
|
||||
}
|
||||
l.userProc = process
|
||||
|
||||
l.totalCpuStats = stats.NewCpuStats()
|
||||
l.userCpuStats = stats.NewCpuStats()
|
||||
l.systemCpuStats = stats.NewCpuStats()
|
||||
|
||||
// Starts the task
|
||||
if err := container.Run(process); err != nil {
|
||||
container.Destroy()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
pid, err := process.Pid()
|
||||
if err != nil {
|
||||
container.Destroy()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Join process cgroups
|
||||
containerState, err := container.State()
|
||||
if err != nil {
|
||||
l.logger.Error("error entering user process cgroups", "executor_pid", os.Getpid(), "error", err)
|
||||
}
|
||||
if err := cgroups.EnterPid(containerState.CgroupPaths, os.Getpid()); err != nil {
|
||||
l.logger.Error("error entering user process cgroups", "executor_pid", os.Getpid(), "error", err)
|
||||
}
|
||||
|
||||
// start a goroutine to wait on the process to complete, so Wait calls can
|
||||
// be multiplexed
|
||||
l.userProcExited = make(chan interface{})
|
||||
go l.pidCollector.collectPids(l.userProcExited, l.getAllPids)
|
||||
go l.wait()
|
||||
|
||||
return &ProcessState{
|
||||
Pid: pid,
|
||||
ExitCode: -1,
|
||||
Time: time.Now(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (l *LibcontainerExecutor) getAllPids() (map[int]*nomadPid, error) {
|
||||
pids, err := l.container.Processes()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
nPids := make(map[int]*nomadPid)
|
||||
for _, pid := range pids {
|
||||
nPids[pid] = &nomadPid{
|
||||
pid: pid,
|
||||
cpuStatsTotal: stats.NewCpuStats(),
|
||||
cpuStatsUser: stats.NewCpuStats(),
|
||||
cpuStatsSys: stats.NewCpuStats(),
|
||||
}
|
||||
}
|
||||
return nPids, nil
|
||||
}
|
||||
|
||||
// Wait waits until a process has exited and returns it's exitcode and errors
|
||||
func (l *LibcontainerExecutor) Wait() (*ProcessState, error) {
|
||||
<-l.userProcExited
|
||||
return l.exitState, nil
|
||||
}
|
||||
|
||||
func (l *LibcontainerExecutor) wait() {
|
||||
defer close(l.userProcExited)
|
||||
|
||||
ps, err := l.userProc.Wait()
|
||||
if err != nil {
|
||||
// If the process has exited before we called wait an error is returned
|
||||
// the process state is embedded in the error
|
||||
if exitErr, ok := err.(*exec.ExitError); ok {
|
||||
ps = exitErr.ProcessState
|
||||
} else {
|
||||
l.logger.Error("failed to call wait on user process", "error", err)
|
||||
l.exitState = &ProcessState{Pid: 0, ExitCode: 0, Time: time.Now()}
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
l.command.Close()
|
||||
|
||||
exitCode := 1
|
||||
var signal int
|
||||
if status, ok := ps.Sys().(syscall.WaitStatus); ok {
|
||||
exitCode = status.ExitStatus()
|
||||
if status.Signaled() {
|
||||
const exitSignalBase = 128
|
||||
signal = int(status.Signal())
|
||||
exitCode = exitSignalBase + signal
|
||||
}
|
||||
}
|
||||
|
||||
l.exitState = &ProcessState{
|
||||
Pid: ps.Pid(),
|
||||
ExitCode: exitCode,
|
||||
Signal: signal,
|
||||
Time: time.Now(),
|
||||
}
|
||||
}
|
||||
|
||||
// Shutdown stops all processes started and cleans up any resources
|
||||
// created (such as mountpoints, devices, etc).
|
||||
func (l *LibcontainerExecutor) Shutdown(signal string, grace time.Duration) error {
|
||||
if l.container == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// move executor to root cgroup
|
||||
subsystems, err := cgroups.GetAllSubsystems()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := JoinRootCgroup(subsystems); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
status, err := l.container.Status()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
defer l.container.Destroy()
|
||||
|
||||
if status == libcontainer.Stopped {
|
||||
return nil
|
||||
}
|
||||
|
||||
if grace > 0 {
|
||||
if signal == "" {
|
||||
signal = "SIGINT"
|
||||
}
|
||||
|
||||
sig, ok := signals.SignalLookup[signal]
|
||||
if !ok {
|
||||
return fmt.Errorf("error unknown signal given for shutdown: %s", signal)
|
||||
}
|
||||
|
||||
err = l.container.Signal(sig, false)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
select {
|
||||
case <-l.userProcExited:
|
||||
return nil
|
||||
case <-time.After(grace):
|
||||
return l.container.Signal(os.Kill, false)
|
||||
}
|
||||
} else {
|
||||
return l.container.Signal(os.Kill, false)
|
||||
}
|
||||
}
|
||||
|
||||
// UpdateResources updates the resource isolation with new values to be enforced
|
||||
func (l *LibcontainerExecutor) UpdateResources(resources *Resources) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Version returns the api version of the executor
|
||||
func (l *LibcontainerExecutor) Version() (*ExecutorVersion, error) {
|
||||
return &ExecutorVersion{Version: ExecutorVersionLatest}, nil
|
||||
}
|
||||
|
||||
// Stats returns the resource statistics for processes managed by the executor
|
||||
func (l *LibcontainerExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
|
||||
lstats, err := l.container.Stats()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
pidStats, err := l.pidCollector.pidStats()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ts := time.Now()
|
||||
stats := lstats.CgroupStats
|
||||
|
||||
// Memory Related Stats
|
||||
swap := stats.MemoryStats.SwapUsage
|
||||
|
@ -205,14 +349,14 @@ func (e *UniversalExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
|
|||
userModeTime := float64(stats.CpuStats.CpuUsage.UsageInUsermode)
|
||||
kernelModeTime := float64(stats.CpuStats.CpuUsage.UsageInKernelmode)
|
||||
|
||||
totalPercent := e.totalCpuStats.Percent(totalProcessCPUUsage)
|
||||
totalPercent := l.totalCpuStats.Percent(totalProcessCPUUsage)
|
||||
cs := &cstructs.CpuStats{
|
||||
SystemMode: e.systemCpuStats.Percent(kernelModeTime),
|
||||
UserMode: e.userCpuStats.Percent(userModeTime),
|
||||
SystemMode: l.systemCpuStats.Percent(kernelModeTime),
|
||||
UserMode: l.userCpuStats.Percent(userModeTime),
|
||||
Percent: totalPercent,
|
||||
ThrottledPeriods: stats.CpuStats.ThrottlingData.ThrottledPeriods,
|
||||
ThrottledTime: stats.CpuStats.ThrottlingData.ThrottledTime,
|
||||
TotalTicks: e.systemCpuStats.TicksConsumed(totalPercent),
|
||||
TotalTicks: l.systemCpuStats.TicksConsumed(totalPercent),
|
||||
Measured: ExecutorCgroupMeasuredCpuStats,
|
||||
}
|
||||
taskResUsage := cstructs.TaskResourceUsage{
|
||||
|
@ -221,181 +365,246 @@ func (e *UniversalExecutor) Stats() (*cstructs.TaskResourceUsage, error) {
|
|||
CpuStats: cs,
|
||||
},
|
||||
Timestamp: ts.UTC().UnixNano(),
|
||||
Pids: pidStats,
|
||||
}
|
||||
if pidStats, err := e.pidStats(); err == nil {
|
||||
taskResUsage.Pids = pidStats
|
||||
}
|
||||
|
||||
return &taskResUsage, nil
|
||||
}
|
||||
|
||||
// runAs takes a user id as a string and looks up the user, and sets the command
|
||||
// to execute as that user.
|
||||
func (e *UniversalExecutor) runAs(userid string) error {
|
||||
u, err := user.Lookup(userid)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Failed to identify user %v: %v", userid, err)
|
||||
}
|
||||
|
||||
// Get the groups the user is a part of
|
||||
gidStrings, err := u.GroupIds()
|
||||
if err != nil {
|
||||
return fmt.Errorf("Unable to lookup user's group membership: %v", err)
|
||||
}
|
||||
|
||||
gids := make([]uint32, len(gidStrings))
|
||||
for _, gidString := range gidStrings {
|
||||
u, err := strconv.Atoi(gidString)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Unable to convert user's group to int %s: %v", gidString, err)
|
||||
}
|
||||
|
||||
gids = append(gids, uint32(u))
|
||||
}
|
||||
|
||||
// Convert the uid and gid
|
||||
uid, err := strconv.ParseUint(u.Uid, 10, 32)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Unable to convert userid to uint32: %s", err)
|
||||
}
|
||||
gid, err := strconv.ParseUint(u.Gid, 10, 32)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Unable to convert groupid to uint32: %s", err)
|
||||
}
|
||||
|
||||
// Set the command to run as that user and group.
|
||||
if e.cmd.SysProcAttr == nil {
|
||||
e.cmd.SysProcAttr = &syscall.SysProcAttr{}
|
||||
}
|
||||
if e.cmd.SysProcAttr.Credential == nil {
|
||||
e.cmd.SysProcAttr.Credential = &syscall.Credential{}
|
||||
}
|
||||
e.cmd.SysProcAttr.Credential.Uid = uint32(uid)
|
||||
e.cmd.SysProcAttr.Credential.Gid = uint32(gid)
|
||||
e.cmd.SysProcAttr.Credential.Groups = gids
|
||||
|
||||
e.logger.Printf("[DEBUG] executor: running as user:group %d:%d with group membership in %v", uid, gid, gids)
|
||||
|
||||
return nil
|
||||
// Signal sends a signal to the process managed by the executor
|
||||
func (l *LibcontainerExecutor) Signal(s os.Signal) error {
|
||||
return l.userProc.Signal(s)
|
||||
}
|
||||
|
||||
// configureChroot configures a chroot
|
||||
func (e *UniversalExecutor) configureChroot() error {
|
||||
if e.cmd.SysProcAttr == nil {
|
||||
e.cmd.SysProcAttr = &syscall.SysProcAttr{}
|
||||
// Exec starts an additional process inside the container
|
||||
func (l *LibcontainerExecutor) Exec(deadline time.Time, cmd string, args []string) ([]byte, int, error) {
|
||||
combined := append([]string{cmd}, args...)
|
||||
// Capture output
|
||||
buf, _ := circbuf.NewBuffer(int64(dstructs.CheckBufSize))
|
||||
|
||||
process := &libcontainer.Process{
|
||||
Args: combined,
|
||||
Env: l.command.Env,
|
||||
Stdout: buf,
|
||||
Stderr: buf,
|
||||
}
|
||||
e.cmd.SysProcAttr.Chroot = e.ctx.TaskDir
|
||||
e.cmd.Dir = "/"
|
||||
|
||||
e.fsIsolationEnforced = true
|
||||
return nil
|
||||
}
|
||||
err := l.container.Run(process)
|
||||
if err != nil {
|
||||
return nil, 0, err
|
||||
}
|
||||
|
||||
// getAllPids returns the pids of all the processes spun up by the executor. We
|
||||
// use the libcontainer apis to get the pids when the user is using cgroup
|
||||
// isolation and we scan the entire process table if the user is not using any
|
||||
// isolation
|
||||
func (e *UniversalExecutor) getAllPids() (map[int]*nomadPid, error) {
|
||||
if e.command.ResourceLimits || e.command.BasicProcessCgroup {
|
||||
manager := getCgroupManager(e.resConCtx.groups, e.resConCtx.cgPaths)
|
||||
pids, err := manager.GetAllPids()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
np := make(map[int]*nomadPid, len(pids))
|
||||
for _, pid := range pids {
|
||||
np[pid] = &nomadPid{
|
||||
pid: pid,
|
||||
cpuStatsTotal: stats.NewCpuStats(),
|
||||
cpuStatsSys: stats.NewCpuStats(),
|
||||
cpuStatsUser: stats.NewCpuStats(),
|
||||
waitCh := make(chan *waitResult)
|
||||
defer close(waitCh)
|
||||
go l.handleExecWait(waitCh, process)
|
||||
|
||||
select {
|
||||
case result := <-waitCh:
|
||||
ps := result.ps
|
||||
if result.err != nil {
|
||||
if exitErr, ok := result.err.(*exec.ExitError); ok {
|
||||
ps = exitErr.ProcessState
|
||||
} else {
|
||||
return nil, 0, result.err
|
||||
}
|
||||
}
|
||||
return np, nil
|
||||
var exitCode int
|
||||
if status, ok := ps.Sys().(syscall.WaitStatus); ok {
|
||||
exitCode = status.ExitStatus()
|
||||
}
|
||||
return buf.Bytes(), exitCode, nil
|
||||
|
||||
case <-time.After(time.Until(deadline)):
|
||||
process.Signal(os.Kill)
|
||||
return nil, 0, context.DeadlineExceeded
|
||||
}
|
||||
allProcesses, err := ps.Processes()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return e.scanPids(os.Getpid(), allProcesses)
|
||||
|
||||
}
|
||||
|
||||
// destroyCgroup kills all processes in the cgroup and removes the cgroup
|
||||
// configuration from the host. This function is idempotent.
|
||||
func DestroyCgroup(groups *cgroupConfig.Cgroup, cgPaths map[string]string, executorPid int) error {
|
||||
mErrs := new(multierror.Error)
|
||||
if groups == nil {
|
||||
return fmt.Errorf("Can't destroy: cgroup configuration empty")
|
||||
type waitResult struct {
|
||||
ps *os.ProcessState
|
||||
err error
|
||||
}
|
||||
|
||||
func (l *LibcontainerExecutor) handleExecWait(ch chan *waitResult, process *libcontainer.Process) {
|
||||
ps, err := process.Wait()
|
||||
ch <- &waitResult{ps, err}
|
||||
}
|
||||
|
||||
func configureCapabilities(cfg *lconfigs.Config, command *ExecCommand) {
|
||||
// TODO: allow better control of these
|
||||
cfg.Capabilities = &lconfigs.Capabilities{
|
||||
Bounding: allCaps,
|
||||
Permitted: allCaps,
|
||||
Inheritable: allCaps,
|
||||
Ambient: allCaps,
|
||||
Effective: allCaps,
|
||||
}
|
||||
|
||||
// Move the executor into the global cgroup so that the task specific
|
||||
// cgroup can be destroyed.
|
||||
nilGroup := &cgroupConfig.Cgroup{}
|
||||
nilGroup.Path = "/"
|
||||
nilGroup.Resources = groups.Resources
|
||||
nilManager := getCgroupManager(nilGroup, nil)
|
||||
err := nilManager.Apply(executorPid)
|
||||
if err != nil && !strings.Contains(err.Error(), "no such process") {
|
||||
return fmt.Errorf("failed to remove executor pid %d: %v", executorPid, err)
|
||||
}
|
||||
|
||||
func configureIsolation(cfg *lconfigs.Config, command *ExecCommand) {
|
||||
defaultMountFlags := syscall.MS_NOEXEC | syscall.MS_NOSUID | syscall.MS_NODEV
|
||||
|
||||
// set the new root directory for the container
|
||||
cfg.Rootfs = command.TaskDir
|
||||
|
||||
// launch with mount namespace
|
||||
cfg.Namespaces = lconfigs.Namespaces{
|
||||
{Type: lconfigs.NEWNS},
|
||||
}
|
||||
|
||||
// Freeze the Cgroup so that it can not continue to fork/exec.
|
||||
manager := getCgroupManager(groups, cgPaths)
|
||||
err = manager.Freeze(cgroupConfig.Frozen)
|
||||
if err != nil && !strings.Contains(err.Error(), "no such file or directory") {
|
||||
return fmt.Errorf("failed to freeze cgroup: %v", err)
|
||||
// paths to mask using a bind mount to /dev/null to prevent reading
|
||||
cfg.MaskPaths = []string{
|
||||
"/proc/kcore",
|
||||
"/sys/firmware",
|
||||
}
|
||||
|
||||
var procs []*os.Process
|
||||
pids, err := manager.GetAllPids()
|
||||
if err != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("error getting pids: %v", err))
|
||||
// paths that should be remounted as readonly inside the container
|
||||
cfg.ReadonlyPaths = []string{
|
||||
"/proc/sys", "/proc/sysrq-trigger", "/proc/irq", "/proc/bus",
|
||||
}
|
||||
|
||||
// Unfreeze the cgroup.
|
||||
err = manager.Freeze(cgroupConfig.Thawed)
|
||||
if err != nil && !strings.Contains(err.Error(), "no such file or directory") {
|
||||
multierror.Append(mErrs, fmt.Errorf("failed to unfreeze cgroup: %v", err))
|
||||
cfg.Devices = lconfigs.DefaultAutoCreatedDevices
|
||||
cfg.Mounts = []*lconfigs.Mount{
|
||||
{
|
||||
Source: "tmpfs",
|
||||
Destination: "/dev",
|
||||
Device: "tmpfs",
|
||||
Flags: syscall.MS_NOSUID | syscall.MS_STRICTATIME,
|
||||
Data: "mode=755",
|
||||
},
|
||||
{
|
||||
Source: "devpts",
|
||||
Destination: "/dev/pts",
|
||||
Device: "devpts",
|
||||
Flags: syscall.MS_NOSUID | syscall.MS_NOEXEC,
|
||||
Data: "newinstance,ptmxmode=0666,mode=0620,gid=5",
|
||||
},
|
||||
{
|
||||
Device: "tmpfs",
|
||||
Source: "shm",
|
||||
Destination: "/dev/shm",
|
||||
Data: "mode=1777,size=65536k",
|
||||
Flags: defaultMountFlags,
|
||||
},
|
||||
{
|
||||
Source: "mqueue",
|
||||
Destination: "/dev/mqueue",
|
||||
Device: "mqueue",
|
||||
Flags: defaultMountFlags,
|
||||
},
|
||||
{
|
||||
Source: "sysfs",
|
||||
Destination: "/sys",
|
||||
Device: "sysfs",
|
||||
Flags: defaultMountFlags | syscall.MS_RDONLY,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func configureCgroups(cfg *lconfigs.Config, command *ExecCommand) error {
|
||||
|
||||
// If resources are not limited then manually create cgroups needed
|
||||
if !command.ResourceLimits {
|
||||
return configureBasicCgroups(cfg)
|
||||
}
|
||||
|
||||
id := uuid.Generate()
|
||||
cfg.Cgroups.Path = filepath.Join(defaultCgroupParent, id)
|
||||
if command.Resources.MemoryMB > 0 {
|
||||
// Total amount of memory allowed to consume
|
||||
cfg.Cgroups.Resources.Memory = int64(command.Resources.MemoryMB * 1024 * 1024)
|
||||
// Disable swap to avoid issues on the machine
|
||||
var memSwappiness uint64 = 0
|
||||
cfg.Cgroups.Resources.MemorySwappiness = &memSwappiness
|
||||
}
|
||||
|
||||
if command.Resources.CPU < 2 {
|
||||
return fmt.Errorf("resources.CPU must be equal to or greater than 2: %v", command.Resources.CPU)
|
||||
}
|
||||
|
||||
// Set the relative CPU shares for this cgroup.
|
||||
cfg.Cgroups.Resources.CpuShares = uint64(command.Resources.CPU)
|
||||
|
||||
if command.Resources.IOPS != 0 {
|
||||
// Validate it is in an acceptable range.
|
||||
if command.Resources.IOPS < 10 || command.Resources.IOPS > 1000 {
|
||||
return fmt.Errorf("resources.IOPS must be between 10 and 1000: %d", command.Resources.IOPS)
|
||||
}
|
||||
return mErrs.ErrorOrNil()
|
||||
|
||||
cfg.Cgroups.Resources.BlkioWeight = uint16(command.Resources.IOPS)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func configureBasicCgroups(cfg *lconfigs.Config) error {
|
||||
id := uuid.Generate()
|
||||
|
||||
// Manually create freezer cgroup
|
||||
cfg.Cgroups.Paths = map[string]string{}
|
||||
root, err := cgroups.FindCgroupMountpointDir()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Kill the processes in the cgroup
|
||||
for _, pid := range pids {
|
||||
proc, err := os.FindProcess(pid)
|
||||
if _, err := os.Stat(root); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
freezer := cgroupFs.FreezerGroup{}
|
||||
subsystem := freezer.Name()
|
||||
path, err := cgroups.FindCgroupMountpoint(subsystem)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to find %s cgroup mountpoint: %v", subsystem, err)
|
||||
}
|
||||
// Sometimes subsystems can be mounted together as 'cpu,cpuacct'.
|
||||
path = filepath.Join(root, filepath.Base(path), defaultCgroupParent, id)
|
||||
|
||||
if err = os.MkdirAll(path, 0755); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
cfg.Cgroups.Paths[subsystem] = path
|
||||
return nil
|
||||
}
|
||||
|
||||
func newLibcontainerConfig(command *ExecCommand) *lconfigs.Config {
|
||||
cfg := &lconfigs.Config{
|
||||
Cgroups: &lconfigs.Cgroup{
|
||||
Resources: &lconfigs.Resources{
|
||||
AllowAllDevices: nil,
|
||||
MemorySwappiness: nil,
|
||||
AllowedDevices: lconfigs.DefaultAllowedDevices,
|
||||
},
|
||||
},
|
||||
Version: "1.0.0",
|
||||
}
|
||||
|
||||
configureCapabilities(cfg, command)
|
||||
configureIsolation(cfg, command)
|
||||
configureCgroups(cfg, command)
|
||||
return cfg
|
||||
}
|
||||
|
||||
// JoinRootCgroup moves the current process to the cgroups of the init process
|
||||
func JoinRootCgroup(subsystems []string) error {
|
||||
mErrs := new(multierror.Error)
|
||||
paths := map[string]string{}
|
||||
for _, s := range subsystems {
|
||||
mnt, _, err := cgroups.FindCgroupMountpointAndRoot(s)
|
||||
if err != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("error finding process %v: %v", pid, err))
|
||||
multierror.Append(mErrs, fmt.Errorf("error getting cgroup path for subsystem: %s", s))
|
||||
continue
|
||||
}
|
||||
|
||||
procs = append(procs, proc)
|
||||
if e := proc.Kill(); e != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("error killing process %v: %v", pid, e))
|
||||
}
|
||||
paths[s] = mnt
|
||||
}
|
||||
|
||||
// Unfreeze the cgroug so we can wait.
|
||||
err = manager.Freeze(cgroupConfig.Thawed)
|
||||
if err != nil && !strings.Contains(err.Error(), "no such file or directory") {
|
||||
multierror.Append(mErrs, fmt.Errorf("failed to unfreeze cgroup: %v", err))
|
||||
err := cgroups.EnterPid(paths, os.Getpid())
|
||||
if err != nil {
|
||||
multierror.Append(mErrs, err)
|
||||
}
|
||||
|
||||
// Wait on the killed processes to ensure they are cleaned up.
|
||||
for _, proc := range procs {
|
||||
// Don't capture the error because we expect this to fail for
|
||||
// processes we didn't fork.
|
||||
proc.Wait()
|
||||
}
|
||||
|
||||
// Clear the cgroups paths so that everything is properly cleaned
|
||||
manager.Cgroups.Paths = nil
|
||||
|
||||
// Remove the cgroup.
|
||||
if err := manager.Destroy(); err != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("failed to delete the cgroup directories: %v", err))
|
||||
}
|
||||
return mErrs.ErrorOrNil()
|
||||
}
|
||||
|
||||
// getCgroupManager returns the correct libcontainer cgroup manager.
|
||||
func getCgroupManager(groups *cgroupConfig.Cgroup, paths map[string]string) *cgroupFs.Manager {
|
||||
return &cgroupFs.Manager{Cgroups: groups, Paths: paths}
|
||||
}
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package executor
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
|
@ -9,20 +10,30 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/nomad/client/allocdir"
|
||||
"github.com/hashicorp/nomad/client/driver/env"
|
||||
dstructs "github.com/hashicorp/nomad/client/driver/structs"
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
"github.com/hashicorp/nomad/client/testutil"
|
||||
"github.com/hashicorp/nomad/helper/testlog"
|
||||
"github.com/hashicorp/nomad/nomad/mock"
|
||||
tu "github.com/hashicorp/nomad/testutil"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func init() {
|
||||
executorFactories["LibcontainerExecutor"] = libcontainerFactory
|
||||
}
|
||||
|
||||
func libcontainerFactory(l hclog.Logger) Executor {
|
||||
return NewExecutorWithIsolation(l)
|
||||
}
|
||||
|
||||
// testExecutorContextWithChroot returns an ExecutorContext and AllocDir with
|
||||
// chroot. Use testExecutorContext if you don't need a chroot.
|
||||
//
|
||||
// The caller is responsible for calling AllocDir.Destroy() to cleanup.
|
||||
func testExecutorContextWithChroot(t *testing.T) (*ExecutorContext, *allocdir.AllocDir) {
|
||||
func testExecutorCommandWithChroot(t *testing.T) (*ExecCommand, *allocdir.AllocDir) {
|
||||
chrootEnv := map[string]string{
|
||||
"/etc/ld.so.cache": "/etc/ld.so.cache",
|
||||
"/etc/ld.so.conf": "/etc/ld.so.conf",
|
||||
|
@ -41,7 +52,7 @@ func testExecutorContextWithChroot(t *testing.T) (*ExecutorContext, *allocdir.Al
|
|||
task := alloc.Job.TaskGroups[0].Tasks[0]
|
||||
taskEnv := env.NewBuilder(mock.Node(), alloc, task, "global").Build()
|
||||
|
||||
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(os.TempDir(), alloc.ID))
|
||||
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(os.TempDir(), alloc.ID))
|
||||
if err := allocDir.Build(); err != nil {
|
||||
t.Fatalf("AllocDir.Build() failed: %v", err)
|
||||
}
|
||||
|
@ -50,69 +61,69 @@ func testExecutorContextWithChroot(t *testing.T) (*ExecutorContext, *allocdir.Al
|
|||
t.Fatalf("allocDir.NewTaskDir(%q) failed: %v", task.Name, err)
|
||||
}
|
||||
td := allocDir.TaskDirs[task.Name]
|
||||
ctx := &ExecutorContext{
|
||||
TaskEnv: taskEnv,
|
||||
Task: task,
|
||||
cmd := &ExecCommand{
|
||||
Env: taskEnv.List(),
|
||||
TaskDir: td.Dir,
|
||||
LogDir: td.LogDir,
|
||||
Resources: &Resources{
|
||||
CPU: task.Resources.CPU,
|
||||
MemoryMB: task.Resources.MemoryMB,
|
||||
IOPS: task.Resources.IOPS,
|
||||
DiskMB: task.Resources.DiskMB,
|
||||
},
|
||||
}
|
||||
return ctx, allocDir
|
||||
configureTLogging(cmd)
|
||||
|
||||
return cmd, allocDir
|
||||
}
|
||||
|
||||
func TestExecutor_IsolationAndConstraints(t *testing.T) {
|
||||
t.Parallel()
|
||||
require := require.New(t)
|
||||
testutil.ExecCompatible(t)
|
||||
|
||||
execCmd := ExecCommand{Cmd: "/bin/ls", Args: []string{"-F", "/", "/etc/"}}
|
||||
ctx, allocDir := testExecutorContextWithChroot(t)
|
||||
execCmd, allocDir := testExecutorCommandWithChroot(t)
|
||||
execCmd.Cmd = "/bin/ls"
|
||||
execCmd.Args = []string{"-F", "/", "/etc/"}
|
||||
defer allocDir.Destroy()
|
||||
|
||||
execCmd.FSIsolation = true
|
||||
execCmd.ResourceLimits = true
|
||||
execCmd.User = dstructs.DefaultUnprivilegedUser
|
||||
|
||||
executor := NewExecutor(testlog.Logger(t))
|
||||
executor := libcontainerFactory(testlog.HCLogger(t))
|
||||
defer executor.Shutdown("SIGKILL", 0)
|
||||
|
||||
if err := executor.SetContext(ctx); err != nil {
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
ps, err := executor.Launch(execCmd)
|
||||
require.NoError(err)
|
||||
require.NotZero(ps.Pid)
|
||||
|
||||
ps, err := executor.LaunchCmd(&execCmd)
|
||||
if err != nil {
|
||||
t.Fatalf("error in launching command: %v", err)
|
||||
}
|
||||
if ps.Pid == 0 {
|
||||
t.Fatalf("expected process to start and have non zero pid")
|
||||
}
|
||||
state, err := executor.Wait()
|
||||
if err != nil {
|
||||
t.Fatalf("error in waiting for command: %v", err)
|
||||
}
|
||||
if state.ExitCode != 0 {
|
||||
t.Errorf("exited with non-zero code: %v", state.ExitCode)
|
||||
}
|
||||
require.NoError(err)
|
||||
require.Zero(state.ExitCode)
|
||||
|
||||
// Check if the resource constraints were applied
|
||||
memLimits := filepath.Join(ps.IsolationConfig.CgroupPaths["memory"], "memory.limit_in_bytes")
|
||||
data, err := ioutil.ReadFile(memLimits)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
expectedMemLim := strconv.Itoa(ctx.Task.Resources.MemoryMB * 1024 * 1024)
|
||||
actualMemLim := strings.TrimSpace(string(data))
|
||||
if actualMemLim != expectedMemLim {
|
||||
t.Fatalf("actual mem limit: %v, expected: %v", string(data), expectedMemLim)
|
||||
}
|
||||
if lexec, ok := executor.(*LibcontainerExecutor); ok {
|
||||
state, err := lexec.container.State()
|
||||
require.NoError(err)
|
||||
|
||||
if err := executor.Exit(); err != nil {
|
||||
t.Fatalf("error: %v", err)
|
||||
}
|
||||
memLimits := filepath.Join(state.CgroupPaths["memory"], "memory.limit_in_bytes")
|
||||
data, err := ioutil.ReadFile(memLimits)
|
||||
require.NoError(err)
|
||||
|
||||
// Check if Nomad has actually removed the cgroups
|
||||
if _, err := os.Stat(memLimits); err == nil {
|
||||
t.Fatalf("file %v hasn't been removed", memLimits)
|
||||
}
|
||||
expectedMemLim := strconv.Itoa(execCmd.Resources.MemoryMB * 1024 * 1024)
|
||||
actualMemLim := strings.TrimSpace(string(data))
|
||||
require.Equal(actualMemLim, expectedMemLim)
|
||||
require.NoError(executor.Shutdown("", 0))
|
||||
executor.Wait()
|
||||
|
||||
// Check if Nomad has actually removed the cgroups
|
||||
tu.WaitForResult(func() (bool, error) {
|
||||
_, err = os.Stat(memLimits)
|
||||
if err == nil {
|
||||
return false, fmt.Errorf("expected an error from os.Stat %s", memLimits)
|
||||
}
|
||||
return true, nil
|
||||
}, func(err error) { t.Error(err) })
|
||||
|
||||
}
|
||||
expected := `/:
|
||||
alloc/
|
||||
bin/
|
||||
|
@ -123,6 +134,7 @@ lib64/
|
|||
local/
|
||||
proc/
|
||||
secrets/
|
||||
sys/
|
||||
tmp/
|
||||
usr/
|
||||
|
||||
|
@ -130,66 +142,43 @@ usr/
|
|||
ld.so.cache
|
||||
ld.so.conf
|
||||
ld.so.conf.d/`
|
||||
file := filepath.Join(ctx.LogDir, "web.stdout.0")
|
||||
output, err := ioutil.ReadFile(file)
|
||||
if err != nil {
|
||||
t.Fatalf("Couldn't read file %v", file)
|
||||
}
|
||||
|
||||
act := strings.TrimSpace(string(output))
|
||||
if act != expected {
|
||||
t.Errorf("Command output incorrectly: want %v; got %v", expected, act)
|
||||
}
|
||||
tu.WaitForResult(func() (bool, error) {
|
||||
output := execCmd.stdout.(*bufferCloser).String()
|
||||
act := strings.TrimSpace(string(output))
|
||||
if act != expected {
|
||||
return false, fmt.Errorf("Command output incorrectly: want %v; got %v", expected, act)
|
||||
}
|
||||
return true, nil
|
||||
}, func(err error) { t.Error(err) })
|
||||
}
|
||||
|
||||
func TestExecutor_ClientCleanup(t *testing.T) {
|
||||
t.Parallel()
|
||||
testutil.ExecCompatible(t)
|
||||
require := require.New(t)
|
||||
|
||||
ctx, allocDir := testExecutorContextWithChroot(t)
|
||||
ctx.Task.LogConfig.MaxFiles = 1
|
||||
ctx.Task.LogConfig.MaxFileSizeMB = 300
|
||||
execCmd, allocDir := testExecutorCommandWithChroot(t)
|
||||
defer allocDir.Destroy()
|
||||
|
||||
executor := NewExecutor(testlog.Logger(t))
|
||||
|
||||
if err := executor.SetContext(ctx); err != nil {
|
||||
t.Fatalf("Unexpected error")
|
||||
}
|
||||
executor := libcontainerFactory(testlog.HCLogger(t))
|
||||
defer executor.Shutdown("", 0)
|
||||
|
||||
// Need to run a command which will produce continuous output but not
|
||||
// too quickly to ensure executor.Exit() stops the process.
|
||||
execCmd := ExecCommand{Cmd: "/bin/bash", Args: []string{"-c", "while true; do /bin/echo X; /bin/sleep 1; done"}}
|
||||
execCmd.FSIsolation = true
|
||||
execCmd.Cmd = "/bin/bash"
|
||||
execCmd.Args = []string{"-c", "while true; do /bin/echo X; /bin/sleep 1; done"}
|
||||
execCmd.ResourceLimits = true
|
||||
execCmd.User = "nobody"
|
||||
|
||||
ps, err := executor.LaunchCmd(&execCmd)
|
||||
if err != nil {
|
||||
t.Fatalf("error in launching command: %v", err)
|
||||
}
|
||||
if ps.Pid == 0 {
|
||||
t.Fatalf("expected process to start and have non zero pid")
|
||||
}
|
||||
ps, err := executor.Launch(execCmd)
|
||||
require.NoError(err)
|
||||
require.NotZero(ps.Pid)
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
if err := executor.Exit(); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
require.NoError(executor.Shutdown("SIGINT", 100*time.Millisecond))
|
||||
executor.Wait()
|
||||
|
||||
file := filepath.Join(ctx.LogDir, "web.stdout.0")
|
||||
finfo, err := os.Stat(file)
|
||||
if err != nil {
|
||||
t.Fatalf("error stating stdout file: %v", err)
|
||||
}
|
||||
if finfo.Size() == 0 {
|
||||
t.Fatal("Nothing in stdout; expected at least one byte.")
|
||||
}
|
||||
output := execCmd.stdout.(*bufferCloser).String()
|
||||
require.NotZero(len(output))
|
||||
time.Sleep(2 * time.Second)
|
||||
finfo1, err := os.Stat(file)
|
||||
if err != nil {
|
||||
t.Fatalf("error stating stdout file: %v", err)
|
||||
}
|
||||
if finfo.Size() != finfo1.Size() {
|
||||
t.Fatalf("Expected size: %v, actual: %v", finfo.Size(), finfo1.Size())
|
||||
}
|
||||
output1 := execCmd.stdout.(*bufferCloser).String()
|
||||
require.Equal(len(output), len(output1))
|
||||
}
|
||||
|
|
|
@ -1,6 +1,8 @@
|
|||
package executor
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
|
@ -9,24 +11,35 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
tu "github.com/hashicorp/nomad/testutil"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/nomad/client/allocdir"
|
||||
"github.com/hashicorp/nomad/client/driver/env"
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
"github.com/hashicorp/nomad/helper/testlog"
|
||||
"github.com/hashicorp/nomad/nomad/mock"
|
||||
tu "github.com/hashicorp/nomad/testutil"
|
||||
"github.com/mitchellh/go-ps"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
var executorFactories = map[string]func(hclog.Logger) Executor{}
|
||||
var universalFactory = func(l hclog.Logger) Executor {
|
||||
return NewExecutor(l)
|
||||
}
|
||||
|
||||
func init() {
|
||||
executorFactories["UniversalExecutor"] = universalFactory
|
||||
}
|
||||
|
||||
// testExecutorContext returns an ExecutorContext and AllocDir.
|
||||
//
|
||||
// The caller is responsible for calling AllocDir.Destroy() to cleanup.
|
||||
func testExecutorContext(t *testing.T) (*ExecutorContext, *allocdir.AllocDir) {
|
||||
func testExecutorCommand(t *testing.T) (*ExecCommand, *allocdir.AllocDir) {
|
||||
alloc := mock.Alloc()
|
||||
task := alloc.Job.TaskGroups[0].Tasks[0]
|
||||
taskEnv := env.NewBuilder(mock.Node(), alloc, task, "global").Build()
|
||||
|
||||
allocDir := allocdir.NewAllocDir(testlog.Logger(t), filepath.Join(os.TempDir(), alloc.ID))
|
||||
allocDir := allocdir.NewAllocDir(testlog.HCLogger(t), filepath.Join(os.TempDir(), alloc.ID))
|
||||
if err := allocDir.Build(); err != nil {
|
||||
t.Fatalf("AllocDir.Build() failed: %v", err)
|
||||
}
|
||||
|
@ -35,184 +48,176 @@ func testExecutorContext(t *testing.T) (*ExecutorContext, *allocdir.AllocDir) {
|
|||
t.Fatalf("allocDir.NewTaskDir(%q) failed: %v", task.Name, err)
|
||||
}
|
||||
td := allocDir.TaskDirs[task.Name]
|
||||
ctx := &ExecutorContext{
|
||||
TaskEnv: taskEnv,
|
||||
Task: task,
|
||||
cmd := &ExecCommand{
|
||||
Env: taskEnv.List(),
|
||||
TaskDir: td.Dir,
|
||||
LogDir: td.LogDir,
|
||||
Resources: &Resources{
|
||||
CPU: task.Resources.CPU,
|
||||
MemoryMB: task.Resources.MemoryMB,
|
||||
IOPS: task.Resources.IOPS,
|
||||
DiskMB: task.Resources.DiskMB,
|
||||
},
|
||||
}
|
||||
return ctx, allocDir
|
||||
configureTLogging(cmd)
|
||||
return cmd, allocDir
|
||||
}
|
||||
|
||||
func TestExecutor_Start_Invalid(t *testing.T) {
|
||||
t.Parallel()
|
||||
type bufferCloser struct {
|
||||
bytes.Buffer
|
||||
}
|
||||
|
||||
func (_ *bufferCloser) Close() error { return nil }
|
||||
|
||||
func configureTLogging(cmd *ExecCommand) (stdout bufferCloser, stderr bufferCloser) {
|
||||
cmd.stdout = &stdout
|
||||
cmd.stderr = &stderr
|
||||
return
|
||||
}
|
||||
|
||||
func TestExecutor_Start_Invalid(pt *testing.T) {
|
||||
pt.Parallel()
|
||||
invalid := "/bin/foobar"
|
||||
execCmd := ExecCommand{Cmd: invalid, Args: []string{"1"}}
|
||||
ctx, allocDir := testExecutorContext(t)
|
||||
defer allocDir.Destroy()
|
||||
executor := NewExecutor(testlog.Logger(t))
|
||||
for name, factory := range executorFactories {
|
||||
pt.Run(name, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
require := require.New(t)
|
||||
execCmd, allocDir := testExecutorCommand(t)
|
||||
execCmd.Cmd = invalid
|
||||
execCmd.Args = []string{"1"}
|
||||
defer allocDir.Destroy()
|
||||
executor := factory(testlog.HCLogger(t))
|
||||
defer executor.Shutdown("", 0)
|
||||
|
||||
if err := executor.SetContext(ctx); err != nil {
|
||||
t.Fatalf("Unexpected error")
|
||||
}
|
||||
|
||||
if _, err := executor.LaunchCmd(&execCmd); err == nil {
|
||||
t.Fatalf("Expected error")
|
||||
_, err := executor.Launch(execCmd)
|
||||
require.Error(err)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestExecutor_Start_Wait_Failure_Code(t *testing.T) {
|
||||
t.Parallel()
|
||||
execCmd := ExecCommand{Cmd: "/bin/date", Args: []string{"fail"}}
|
||||
ctx, allocDir := testExecutorContext(t)
|
||||
defer allocDir.Destroy()
|
||||
executor := NewExecutor(testlog.Logger(t))
|
||||
func TestExecutor_Start_Wait_Failure_Code(pt *testing.T) {
|
||||
pt.Parallel()
|
||||
for name, factory := range executorFactories {
|
||||
pt.Run(name, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
require := require.New(t)
|
||||
execCmd, allocDir := testExecutorCommand(t)
|
||||
execCmd.Cmd = "/bin/date"
|
||||
execCmd.Args = []string{"fail"}
|
||||
defer allocDir.Destroy()
|
||||
executor := factory(testlog.HCLogger(t))
|
||||
defer executor.Shutdown("", 0)
|
||||
|
||||
if err := executor.SetContext(ctx); err != nil {
|
||||
t.Fatalf("Unexpected error")
|
||||
}
|
||||
|
||||
ps, err := executor.LaunchCmd(&execCmd)
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error")
|
||||
}
|
||||
|
||||
if ps.Pid == 0 {
|
||||
t.Fatalf("expected process to start and have non zero pid")
|
||||
}
|
||||
ps, _ = executor.Wait()
|
||||
if ps.ExitCode < 1 {
|
||||
t.Fatalf("expected exit code to be non zero, actual: %v", ps.ExitCode)
|
||||
}
|
||||
if err := executor.Exit(); err != nil {
|
||||
t.Fatalf("error: %v", err)
|
||||
ps, err := executor.Launch(execCmd)
|
||||
require.NoError(err)
|
||||
require.NotZero(ps.Pid)
|
||||
ps, _ = executor.Wait()
|
||||
require.NotZero(ps.ExitCode, "expected exit code to be non zero")
|
||||
require.NoError(executor.Shutdown("SIGINT", 100*time.Millisecond))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestExecutor_Start_Wait(t *testing.T) {
|
||||
t.Parallel()
|
||||
execCmd := ExecCommand{Cmd: "/bin/echo", Args: []string{"hello world"}}
|
||||
ctx, allocDir := testExecutorContext(t)
|
||||
defer allocDir.Destroy()
|
||||
executor := NewExecutor(testlog.Logger(t))
|
||||
func TestExecutor_Start_Wait(pt *testing.T) {
|
||||
pt.Parallel()
|
||||
for name, factory := range executorFactories {
|
||||
pt.Run(name, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
require := require.New(t)
|
||||
execCmd, allocDir := testExecutorCommand(t)
|
||||
execCmd.Cmd = "/bin/echo"
|
||||
execCmd.Args = []string{"hello world"}
|
||||
defer allocDir.Destroy()
|
||||
executor := factory(testlog.HCLogger(t))
|
||||
defer executor.Shutdown("", 0)
|
||||
|
||||
if err := executor.SetContext(ctx); err != nil {
|
||||
t.Fatalf("Unexpected error")
|
||||
}
|
||||
ps, err := executor.Launch(execCmd)
|
||||
require.NoError(err)
|
||||
require.NotZero(ps.Pid)
|
||||
|
||||
ps, err := executor.LaunchCmd(&execCmd)
|
||||
if err != nil {
|
||||
t.Fatalf("error in launching command: %v", err)
|
||||
}
|
||||
if ps.Pid == 0 {
|
||||
t.Fatalf("expected process to start and have non zero pid")
|
||||
}
|
||||
ps, err = executor.Wait()
|
||||
if err != nil {
|
||||
t.Fatalf("error in waiting for command: %v", err)
|
||||
}
|
||||
if err := executor.Exit(); err != nil {
|
||||
t.Fatalf("error: %v", err)
|
||||
}
|
||||
ps, err = executor.Wait()
|
||||
require.NoError(err)
|
||||
require.NoError(executor.Shutdown("SIGINT", 100*time.Millisecond))
|
||||
|
||||
expected := "hello world"
|
||||
file := filepath.Join(ctx.LogDir, "web.stdout.0")
|
||||
output, err := ioutil.ReadFile(file)
|
||||
if err != nil {
|
||||
t.Fatalf("Couldn't read file %v", file)
|
||||
}
|
||||
|
||||
act := strings.TrimSpace(string(output))
|
||||
if act != expected {
|
||||
t.Fatalf("Command output incorrectly: want %v; got %v", expected, act)
|
||||
expected := "hello world"
|
||||
tu.WaitForResult(func() (bool, error) {
|
||||
output := execCmd.stdout.(*bufferCloser).String()
|
||||
act := strings.TrimSpace(string(output))
|
||||
if expected != act {
|
||||
return false, fmt.Errorf("expected: '%s' actual: '%s'", expected, act)
|
||||
}
|
||||
return true, nil
|
||||
}, func(err error) {
|
||||
require.NoError(err)
|
||||
})
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestExecutor_WaitExitSignal(t *testing.T) {
|
||||
t.Parallel()
|
||||
execCmd := ExecCommand{Cmd: "/bin/sleep", Args: []string{"10000"}}
|
||||
ctx, allocDir := testExecutorContext(t)
|
||||
defer allocDir.Destroy()
|
||||
executor := NewExecutor(testlog.Logger(t))
|
||||
func TestExecutor_WaitExitSignal(pt *testing.T) {
|
||||
pt.Parallel()
|
||||
for name, factory := range executorFactories {
|
||||
pt.Run(name, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
require := require.New(t)
|
||||
execCmd, allocDir := testExecutorCommand(t)
|
||||
execCmd.Cmd = "/bin/sleep"
|
||||
execCmd.Args = []string{"10000"}
|
||||
defer allocDir.Destroy()
|
||||
executor := factory(testlog.HCLogger(t))
|
||||
defer executor.Shutdown("", 0)
|
||||
|
||||
if err := executor.SetContext(ctx); err != nil {
|
||||
t.Fatalf("Unexpected error")
|
||||
}
|
||||
ps, err := executor.Launch(execCmd)
|
||||
require.NoError(err)
|
||||
|
||||
ps, err := executor.LaunchCmd(&execCmd)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
go func() {
|
||||
time.Sleep(2 * time.Second)
|
||||
_, err := executor.Stats()
|
||||
require.NoError(err)
|
||||
//require.NotEmpty(ru.Pids)
|
||||
proc, err := os.FindProcess(ps.Pid)
|
||||
require.NoError(err)
|
||||
err = proc.Signal(syscall.SIGKILL)
|
||||
require.NoError(err)
|
||||
}()
|
||||
|
||||
go func() {
|
||||
time.Sleep(2 * time.Second)
|
||||
ru, err := executor.Stats()
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
if len(ru.Pids) == 0 {
|
||||
t.Fatalf("expected pids")
|
||||
}
|
||||
proc, err := os.FindProcess(ps.Pid)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
if err := proc.Signal(syscall.SIGKILL); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
}()
|
||||
|
||||
ps, err = executor.Wait()
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
if ps.Signal != int(syscall.SIGKILL) {
|
||||
t.Fatalf("expected signal: %v, actual: %v", int(syscall.SIGKILL), ps.Signal)
|
||||
ps, err = executor.Wait()
|
||||
require.NoError(err)
|
||||
require.Equal(ps.Signal, int(syscall.SIGKILL))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestExecutor_Start_Kill(t *testing.T) {
|
||||
t.Parallel()
|
||||
execCmd := ExecCommand{Cmd: "/bin/sleep", Args: []string{"10 && hello world"}}
|
||||
ctx, allocDir := testExecutorContext(t)
|
||||
defer allocDir.Destroy()
|
||||
executor := NewExecutor(testlog.Logger(t))
|
||||
func TestExecutor_Start_Kill(pt *testing.T) {
|
||||
pt.Parallel()
|
||||
for name, factory := range executorFactories {
|
||||
pt.Run(name, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
require := require.New(t)
|
||||
execCmd, allocDir := testExecutorCommand(t)
|
||||
execCmd.Cmd = "/bin/sleep"
|
||||
execCmd.Args = []string{"10 && hello world"}
|
||||
defer allocDir.Destroy()
|
||||
executor := factory(testlog.HCLogger(t))
|
||||
defer executor.Shutdown("", 0)
|
||||
|
||||
if err := executor.SetContext(ctx); err != nil {
|
||||
t.Fatalf("Unexpected error")
|
||||
}
|
||||
ps, err := executor.Launch(execCmd)
|
||||
require.NoError(err)
|
||||
require.NotZero(ps.Pid)
|
||||
|
||||
ps, err := executor.LaunchCmd(&execCmd)
|
||||
if err != nil {
|
||||
t.Fatalf("error in launching command: %v", err)
|
||||
}
|
||||
if ps.Pid == 0 {
|
||||
t.Fatalf("expected process to start and have non zero pid")
|
||||
}
|
||||
ps, err = executor.Wait()
|
||||
if err != nil {
|
||||
t.Fatalf("error in waiting for command: %v", err)
|
||||
}
|
||||
if err := executor.Exit(); err != nil {
|
||||
t.Fatalf("error: %v", err)
|
||||
}
|
||||
require.NoError(executor.Shutdown("SIGINT", 100*time.Millisecond))
|
||||
|
||||
file := filepath.Join(ctx.LogDir, "web.stdout.0")
|
||||
time.Sleep(time.Duration(tu.TestMultiplier()*2) * time.Second)
|
||||
|
||||
output, err := ioutil.ReadFile(file)
|
||||
if err != nil {
|
||||
t.Fatalf("Couldn't read file %v", file)
|
||||
}
|
||||
|
||||
expected := ""
|
||||
act := strings.TrimSpace(string(output))
|
||||
if act != expected {
|
||||
t.Fatalf("Command output incorrectly: want %v; got %v", expected, act)
|
||||
time.Sleep(time.Duration(tu.TestMultiplier()*2) * time.Second)
|
||||
output := execCmd.stdout.(*bufferCloser).String()
|
||||
expected := ""
|
||||
act := strings.TrimSpace(string(output))
|
||||
if act != expected {
|
||||
t.Fatalf("Command output incorrectly: want %v; got %v", expected, act)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestExecutor_MakeExecutable(t *testing.T) {
|
||||
func TestUniversalExecutor_MakeExecutable(t *testing.T) {
|
||||
t.Parallel()
|
||||
// Create a temp file
|
||||
f, err := ioutil.TempFile("", "")
|
||||
|
@ -226,9 +231,9 @@ func TestExecutor_MakeExecutable(t *testing.T) {
|
|||
f.Chmod(os.FileMode(0610))
|
||||
|
||||
// Make a fake executor
|
||||
executor := NewExecutor(testlog.Logger(t))
|
||||
executor := NewExecutor(testlog.HCLogger(t)).(*UniversalExecutor)
|
||||
|
||||
err = executor.(*UniversalExecutor).makeExecutable(f.Name())
|
||||
err = executor.makeExecutable(f.Name())
|
||||
if err != nil {
|
||||
t.Fatalf("makeExecutable() failed: %v", err)
|
||||
}
|
||||
|
@ -245,44 +250,3 @@ func TestExecutor_MakeExecutable(t *testing.T) {
|
|||
t.Fatalf("expected permissions %v; got %v", exp, act)
|
||||
}
|
||||
}
|
||||
|
||||
func TestScanPids(t *testing.T) {
|
||||
t.Parallel()
|
||||
p1 := NewFakeProcess(2, 5)
|
||||
p2 := NewFakeProcess(10, 2)
|
||||
p3 := NewFakeProcess(15, 6)
|
||||
p4 := NewFakeProcess(3, 10)
|
||||
p5 := NewFakeProcess(20, 18)
|
||||
|
||||
// Make a fake executor
|
||||
executor := NewExecutor(testlog.Logger(t)).(*UniversalExecutor)
|
||||
|
||||
nomadPids, err := executor.scanPids(5, []ps.Process{p1, p2, p3, p4, p5})
|
||||
if err != nil {
|
||||
t.Fatalf("error: %v", err)
|
||||
}
|
||||
if len(nomadPids) != 4 {
|
||||
t.Fatalf("expected: 4, actual: %v", len(nomadPids))
|
||||
}
|
||||
}
|
||||
|
||||
type FakeProcess struct {
|
||||
pid int
|
||||
ppid int
|
||||
}
|
||||
|
||||
func (f FakeProcess) Pid() int {
|
||||
return f.pid
|
||||
}
|
||||
|
||||
func (f FakeProcess) PPid() int {
|
||||
return f.ppid
|
||||
}
|
||||
|
||||
func (f FakeProcess) Executable() string {
|
||||
return "fake"
|
||||
}
|
||||
|
||||
func NewFakeProcess(pid int, ppid int) ps.Process {
|
||||
return FakeProcess{pid: pid, ppid: ppid}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,156 @@
|
|||
package executor
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"os/user"
|
||||
"strconv"
|
||||
"syscall"
|
||||
|
||||
multierror "github.com/hashicorp/go-multierror"
|
||||
"github.com/hashicorp/nomad/helper"
|
||||
"github.com/opencontainers/runc/libcontainer/cgroups"
|
||||
cgroupFs "github.com/opencontainers/runc/libcontainer/cgroups/fs"
|
||||
lconfigs "github.com/opencontainers/runc/libcontainer/configs"
|
||||
)
|
||||
|
||||
// runAs takes a user id as a string and looks up the user, and sets the command
|
||||
// to execute as that user.
|
||||
func (e *UniversalExecutor) runAs(userid string) error {
|
||||
u, err := user.Lookup(userid)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Failed to identify user %v: %v", userid, err)
|
||||
}
|
||||
|
||||
// Get the groups the user is a part of
|
||||
gidStrings, err := u.GroupIds()
|
||||
if err != nil {
|
||||
return fmt.Errorf("Unable to lookup user's group membership: %v", err)
|
||||
}
|
||||
|
||||
gids := make([]uint32, len(gidStrings))
|
||||
for _, gidString := range gidStrings {
|
||||
u, err := strconv.Atoi(gidString)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Unable to convert user's group to int %s: %v", gidString, err)
|
||||
}
|
||||
|
||||
gids = append(gids, uint32(u))
|
||||
}
|
||||
|
||||
// Convert the uid and gid
|
||||
uid, err := strconv.ParseUint(u.Uid, 10, 32)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Unable to convert userid to uint32: %s", err)
|
||||
}
|
||||
gid, err := strconv.ParseUint(u.Gid, 10, 32)
|
||||
if err != nil {
|
||||
return fmt.Errorf("Unable to convert groupid to uint32: %s", err)
|
||||
}
|
||||
|
||||
// Set the command to run as that user and group.
|
||||
if e.childCmd.SysProcAttr == nil {
|
||||
e.childCmd.SysProcAttr = &syscall.SysProcAttr{}
|
||||
}
|
||||
if e.childCmd.SysProcAttr.Credential == nil {
|
||||
e.childCmd.SysProcAttr.Credential = &syscall.Credential{}
|
||||
}
|
||||
e.childCmd.SysProcAttr.Credential.Uid = uint32(uid)
|
||||
e.childCmd.SysProcAttr.Credential.Gid = uint32(gid)
|
||||
e.childCmd.SysProcAttr.Credential.Groups = gids
|
||||
|
||||
e.logger.Debug("setting process user", "user", uid, "group", gid, "additional_groups", gids)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// configureResourceContainer configured the cgroups to be used to track pids
|
||||
// created by the executor
|
||||
func (e *UniversalExecutor) configureResourceContainer(pid int) error {
|
||||
cfg := &lconfigs.Config{
|
||||
Cgroups: &lconfigs.Cgroup{
|
||||
Resources: &lconfigs.Resources{
|
||||
AllowAllDevices: helper.BoolToPtr(true),
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
configureBasicCgroups(cfg)
|
||||
e.resConCtx.groups = cfg.Cgroups
|
||||
return cgroups.EnterPid(cfg.Cgroups.Paths, pid)
|
||||
}
|
||||
|
||||
// DestroyCgroup kills all processes in the cgroup and removes the cgroup
|
||||
// configuration from the host. This function is idempotent.
|
||||
func DestroyCgroup(groups *lconfigs.Cgroup, executorPid int) error {
|
||||
mErrs := new(multierror.Error)
|
||||
if groups == nil {
|
||||
return fmt.Errorf("Can't destroy: cgroup configuration empty")
|
||||
}
|
||||
|
||||
// Move the executor into the global cgroup so that the task specific
|
||||
// cgroup can be destroyed.
|
||||
path, err := cgroups.GetInitCgroupPath("freezer")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := cgroups.EnterPid(map[string]string{"freezer": path}, executorPid); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Freeze the Cgroup so that it can not continue to fork/exec.
|
||||
groups.Resources.Freezer = lconfigs.Frozen
|
||||
freezer := cgroupFs.FreezerGroup{}
|
||||
if err := freezer.Set(groups.Paths[freezer.Name()], groups); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var procs []*os.Process
|
||||
pids, err := cgroups.GetAllPids(groups.Paths[freezer.Name()])
|
||||
if err != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("error getting pids: %v", err))
|
||||
|
||||
// Unfreeze the cgroup.
|
||||
groups.Resources.Freezer = lconfigs.Thawed
|
||||
freezer := cgroupFs.FreezerGroup{}
|
||||
if err := freezer.Set(groups.Paths[freezer.Name()], groups); err != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("failed to unfreeze cgroup: %v", err))
|
||||
return mErrs.ErrorOrNil()
|
||||
}
|
||||
}
|
||||
|
||||
// Kill the processes in the cgroup
|
||||
for _, pid := range pids {
|
||||
proc, err := os.FindProcess(pid)
|
||||
if err != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("error finding process %v: %v", pid, err))
|
||||
continue
|
||||
}
|
||||
|
||||
procs = append(procs, proc)
|
||||
if e := proc.Kill(); e != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("error killing process %v: %v", pid, e))
|
||||
}
|
||||
}
|
||||
|
||||
// Unfreeze the cgroug so we can wait.
|
||||
groups.Resources.Freezer = lconfigs.Thawed
|
||||
if err := freezer.Set(groups.Paths[freezer.Name()], groups); err != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("failed to unfreeze cgroup: %v", err))
|
||||
return mErrs.ErrorOrNil()
|
||||
}
|
||||
|
||||
// Wait on the killed processes to ensure they are cleaned up.
|
||||
for _, proc := range procs {
|
||||
// Don't capture the error because we expect this to fail for
|
||||
// processes we didn't fork.
|
||||
proc.Wait()
|
||||
}
|
||||
|
||||
// Remove the cgroup.
|
||||
if err := cgroups.RemovePaths(groups.Paths); err != nil {
|
||||
multierror.Append(mErrs, fmt.Errorf("failed to delete the cgroup directories: %v", err))
|
||||
}
|
||||
return mErrs.ErrorOrNil()
|
||||
}
|
|
@ -10,10 +10,10 @@ import (
|
|||
|
||||
// configure new process group for child process
|
||||
func (e *UniversalExecutor) setNewProcessGroup() error {
|
||||
if e.cmd.SysProcAttr == nil {
|
||||
e.cmd.SysProcAttr = &syscall.SysProcAttr{}
|
||||
if e.childCmd.SysProcAttr == nil {
|
||||
e.childCmd.SysProcAttr = &syscall.SysProcAttr{}
|
||||
}
|
||||
e.cmd.SysProcAttr.Setpgid = true
|
||||
e.childCmd.SysProcAttr.Setpgid = true
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -21,7 +21,7 @@ func (e *UniversalExecutor) setNewProcessGroup() error {
|
|||
func (e *UniversalExecutor) cleanupChildProcesses(proc *os.Process) error {
|
||||
// If new process group was created upon command execution
|
||||
// we can kill the whole process group now to cleanup any leftovers.
|
||||
if e.cmd.SysProcAttr != nil && e.cmd.SysProcAttr.Setpgid {
|
||||
if e.childCmd.SysProcAttr != nil && e.childCmd.SysProcAttr.Setpgid {
|
||||
if err := syscall.Kill(-proc.Pid, syscall.SIGKILL); err != nil && err.Error() != noSuchProcessErr {
|
||||
return err
|
||||
}
|
||||
|
@ -32,18 +32,13 @@ func (e *UniversalExecutor) cleanupChildProcesses(proc *os.Process) error {
|
|||
|
||||
// Only send the process a shutdown signal (default INT), doesn't
|
||||
// necessarily kill it.
|
||||
func (e *UniversalExecutor) shutdownProcess(proc *os.Process) error {
|
||||
// Set default kill signal, as some drivers don't support configurable
|
||||
// signals (such as rkt)
|
||||
var osSignal os.Signal
|
||||
if e.command.TaskKillSignal != nil {
|
||||
osSignal = e.command.TaskKillSignal
|
||||
} else {
|
||||
osSignal = os.Interrupt
|
||||
func (e *UniversalExecutor) shutdownProcess(sig os.Signal, proc *os.Process) error {
|
||||
if sig == nil {
|
||||
sig = os.Interrupt
|
||||
}
|
||||
|
||||
if err := proc.Signal(osSignal); err != nil && err.Error() != finishedErr {
|
||||
return fmt.Errorf("executor.shutdown error: %v", err)
|
||||
if err := proc.Signal(sig); err != nil && err.Error() != finishedErr {
|
||||
return fmt.Errorf("executor shutdown error: %v", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
|
|
|
@ -59,11 +59,11 @@ func sendCtrlBreak(pid int) error {
|
|||
|
||||
// Send the process a Ctrl-Break event, allowing it to shutdown by itself
|
||||
// before being Terminate.
|
||||
func (e *UniversalExecutor) shutdownProcess(proc *os.Process) error {
|
||||
func (e *UniversalExecutor) shutdownProcess(_ os.Signal, proc *os.Process) error {
|
||||
if err := sendCtrlBreak(proc.Pid); err != nil {
|
||||
return fmt.Errorf("executor.shutdown error: %v", err)
|
||||
return fmt.Errorf("executor shutdown error: %v", err)
|
||||
}
|
||||
e.logger.Printf("[INFO] executor: sent Ctrl-Break to process %v", proc.Pid)
|
||||
e.logger.Info("sent Ctrl-Break to process", "pid", proc.Pid)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -0,0 +1,219 @@
|
|||
package executor
|
||||
|
||||
import (
|
||||
"os"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/nomad/client/stats"
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
ps "github.com/mitchellh/go-ps"
|
||||
"github.com/shirou/gopsutil/process"
|
||||
)
|
||||
|
||||
var (
|
||||
// pidScanInterval is the interval at which the executor scans the process
|
||||
// tree for finding out the pids that the executor and it's child processes
|
||||
// have forked
|
||||
pidScanInterval = 5 * time.Second
|
||||
)
|
||||
|
||||
// pidCollector is a utility that can be embedded in an executor to collect pid
|
||||
// stats
|
||||
type pidCollector struct {
|
||||
pids map[int]*nomadPid
|
||||
pidLock sync.RWMutex
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
// nomadPid holds a pid and it's cpu percentage calculator
|
||||
type nomadPid struct {
|
||||
pid int
|
||||
cpuStatsTotal *stats.CpuStats
|
||||
cpuStatsUser *stats.CpuStats
|
||||
cpuStatsSys *stats.CpuStats
|
||||
}
|
||||
|
||||
// allPidGetter is a func which is used by the pid collector to gather
|
||||
// stats on
|
||||
type allPidGetter func() (map[int]*nomadPid, error)
|
||||
|
||||
func newPidCollector(logger hclog.Logger) *pidCollector {
|
||||
return &pidCollector{
|
||||
pids: make(map[int]*nomadPid),
|
||||
logger: logger.Named("pid_collector"),
|
||||
}
|
||||
}
|
||||
|
||||
// collectPids collects the pids of the child processes that the executor is
|
||||
// running every 5 seconds
|
||||
func (c *pidCollector) collectPids(stopCh chan interface{}, pidGetter allPidGetter) {
|
||||
// Fire the timer right away when the executor starts from there on the pids
|
||||
// are collected every scan interval
|
||||
timer := time.NewTimer(0)
|
||||
defer timer.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-timer.C:
|
||||
pids, err := pidGetter()
|
||||
if err != nil {
|
||||
c.logger.Debug("error collecting pids", "error", err)
|
||||
}
|
||||
c.pidLock.Lock()
|
||||
|
||||
// Adding pids which are not being tracked
|
||||
for pid, np := range pids {
|
||||
if _, ok := c.pids[pid]; !ok {
|
||||
c.pids[pid] = np
|
||||
}
|
||||
}
|
||||
// Removing pids which are no longer present
|
||||
for pid := range c.pids {
|
||||
if _, ok := pids[pid]; !ok {
|
||||
delete(c.pids, pid)
|
||||
}
|
||||
}
|
||||
c.pidLock.Unlock()
|
||||
timer.Reset(pidScanInterval)
|
||||
case <-stopCh:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// scanPids scans all the pids on the machine running the current executor and
|
||||
// returns the child processes of the executor.
|
||||
func scanPids(parentPid int, allPids []ps.Process) (map[int]*nomadPid, error) {
|
||||
processFamily := make(map[int]struct{})
|
||||
processFamily[parentPid] = struct{}{}
|
||||
|
||||
// A mapping of pids to their parent pids. It is used to build the process
|
||||
// tree of the executing task
|
||||
pidsRemaining := make(map[int]int, len(allPids))
|
||||
for _, pid := range allPids {
|
||||
pidsRemaining[pid.Pid()] = pid.PPid()
|
||||
}
|
||||
|
||||
for {
|
||||
// flag to indicate if we have found a match
|
||||
foundNewPid := false
|
||||
|
||||
for pid, ppid := range pidsRemaining {
|
||||
_, childPid := processFamily[ppid]
|
||||
|
||||
// checking if the pid is a child of any of the parents
|
||||
if childPid {
|
||||
processFamily[pid] = struct{}{}
|
||||
delete(pidsRemaining, pid)
|
||||
foundNewPid = true
|
||||
}
|
||||
}
|
||||
|
||||
// not scanning anymore if we couldn't find a single match
|
||||
if !foundNewPid {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
res := make(map[int]*nomadPid)
|
||||
for pid := range processFamily {
|
||||
np := nomadPid{
|
||||
pid: pid,
|
||||
cpuStatsTotal: stats.NewCpuStats(),
|
||||
cpuStatsUser: stats.NewCpuStats(),
|
||||
cpuStatsSys: stats.NewCpuStats(),
|
||||
}
|
||||
res[pid] = &np
|
||||
}
|
||||
return res, nil
|
||||
}
|
||||
|
||||
// pidStats returns the resource usage stats per pid
|
||||
func (c *pidCollector) pidStats() (map[string]*cstructs.ResourceUsage, error) {
|
||||
stats := make(map[string]*cstructs.ResourceUsage)
|
||||
c.pidLock.RLock()
|
||||
pids := make(map[int]*nomadPid, len(c.pids))
|
||||
for k, v := range c.pids {
|
||||
pids[k] = v
|
||||
}
|
||||
c.pidLock.RUnlock()
|
||||
for pid, np := range pids {
|
||||
p, err := process.NewProcess(int32(pid))
|
||||
if err != nil {
|
||||
c.logger.Trace("unable to create new process", "pid", pid, "error", err)
|
||||
continue
|
||||
}
|
||||
ms := &cstructs.MemoryStats{}
|
||||
if memInfo, err := p.MemoryInfo(); err == nil {
|
||||
ms.RSS = memInfo.RSS
|
||||
ms.Swap = memInfo.Swap
|
||||
ms.Measured = ExecutorBasicMeasuredMemStats
|
||||
}
|
||||
|
||||
cs := &cstructs.CpuStats{}
|
||||
if cpuStats, err := p.Times(); err == nil {
|
||||
cs.SystemMode = np.cpuStatsSys.Percent(cpuStats.System * float64(time.Second))
|
||||
cs.UserMode = np.cpuStatsUser.Percent(cpuStats.User * float64(time.Second))
|
||||
cs.Measured = ExecutorBasicMeasuredCpuStats
|
||||
|
||||
// calculate cpu usage percent
|
||||
cs.Percent = np.cpuStatsTotal.Percent(cpuStats.Total() * float64(time.Second))
|
||||
}
|
||||
stats[strconv.Itoa(pid)] = &cstructs.ResourceUsage{MemoryStats: ms, CpuStats: cs}
|
||||
}
|
||||
|
||||
return stats, nil
|
||||
}
|
||||
|
||||
// aggregatedResourceUsage aggregates the resource usage of all the pids and
|
||||
// returns a TaskResourceUsage data point
|
||||
func aggregatedResourceUsage(systemCpuStats *stats.CpuStats, pidStats map[string]*cstructs.ResourceUsage) *cstructs.TaskResourceUsage {
|
||||
ts := time.Now().UTC().UnixNano()
|
||||
var (
|
||||
systemModeCPU, userModeCPU, percent float64
|
||||
totalRSS, totalSwap uint64
|
||||
)
|
||||
|
||||
for _, pidStat := range pidStats {
|
||||
systemModeCPU += pidStat.CpuStats.SystemMode
|
||||
userModeCPU += pidStat.CpuStats.UserMode
|
||||
percent += pidStat.CpuStats.Percent
|
||||
|
||||
totalRSS += pidStat.MemoryStats.RSS
|
||||
totalSwap += pidStat.MemoryStats.Swap
|
||||
}
|
||||
|
||||
totalCPU := &cstructs.CpuStats{
|
||||
SystemMode: systemModeCPU,
|
||||
UserMode: userModeCPU,
|
||||
Percent: percent,
|
||||
Measured: ExecutorBasicMeasuredCpuStats,
|
||||
TotalTicks: systemCpuStats.TicksConsumed(percent),
|
||||
}
|
||||
|
||||
totalMemory := &cstructs.MemoryStats{
|
||||
RSS: totalRSS,
|
||||
Swap: totalSwap,
|
||||
Measured: ExecutorBasicMeasuredMemStats,
|
||||
}
|
||||
|
||||
resourceUsage := cstructs.ResourceUsage{
|
||||
MemoryStats: totalMemory,
|
||||
CpuStats: totalCPU,
|
||||
}
|
||||
return &cstructs.TaskResourceUsage{
|
||||
ResourceUsage: &resourceUsage,
|
||||
Timestamp: ts,
|
||||
Pids: pidStats,
|
||||
}
|
||||
}
|
||||
|
||||
func getAllPids() (map[int]*nomadPid, error) {
|
||||
allProcesses, err := ps.Processes()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return scanPids(os.Getpid(), allProcesses)
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
package executor
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
ps "github.com/mitchellh/go-ps"
|
||||
)
|
||||
|
||||
func TestScanPids(t *testing.T) {
|
||||
t.Parallel()
|
||||
p1 := NewFakeProcess(2, 5)
|
||||
p2 := NewFakeProcess(10, 2)
|
||||
p3 := NewFakeProcess(15, 6)
|
||||
p4 := NewFakeProcess(3, 10)
|
||||
p5 := NewFakeProcess(20, 18)
|
||||
|
||||
nomadPids, err := scanPids(5, []ps.Process{p1, p2, p3, p4, p5})
|
||||
if err != nil {
|
||||
t.Fatalf("error: %v", err)
|
||||
}
|
||||
if len(nomadPids) != 4 {
|
||||
t.Fatalf("expected: 4, actual: %v", len(nomadPids))
|
||||
}
|
||||
}
|
||||
|
||||
type FakeProcess struct {
|
||||
pid int
|
||||
ppid int
|
||||
}
|
||||
|
||||
func (f FakeProcess) Pid() int {
|
||||
return f.pid
|
||||
}
|
||||
|
||||
func (f FakeProcess) PPid() int {
|
||||
return f.ppid
|
||||
}
|
||||
|
||||
func (f FakeProcess) Executable() string {
|
||||
return "fake"
|
||||
}
|
||||
|
||||
func NewFakeProcess(pid int, ppid int) ps.Process {
|
||||
return FakeProcess{pid: pid, ppid: ppid}
|
||||
}
|
|
@ -2,23 +2,11 @@
|
|||
|
||||
package executor
|
||||
|
||||
import (
|
||||
dstructs "github.com/hashicorp/nomad/client/driver/structs"
|
||||
)
|
||||
|
||||
// resourceContainerContext is a platform-specific struct for managing a
|
||||
// resource container.
|
||||
type resourceContainerContext struct {
|
||||
}
|
||||
|
||||
func clientCleanup(ic *dstructs.IsolationConfig, pid int) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rc *resourceContainerContext) executorCleanup() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rc *resourceContainerContext) getIsolationConfig() *dstructs.IsolationConfig {
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -4,39 +4,22 @@ import (
|
|||
"os"
|
||||
"sync"
|
||||
|
||||
dstructs "github.com/hashicorp/nomad/client/driver/structs"
|
||||
cgroupConfig "github.com/opencontainers/runc/libcontainer/configs"
|
||||
)
|
||||
|
||||
// resourceContainerContext is a platform-specific struct for managing a
|
||||
// resource container. In the case of Linux, this is used to control Cgroups.
|
||||
type resourceContainerContext struct {
|
||||
groups *cgroupConfig.Cgroup
|
||||
cgPaths map[string]string
|
||||
cgLock sync.Mutex
|
||||
}
|
||||
|
||||
// clientCleanup removes this host's Cgroup from the Nomad Client's context
|
||||
func clientCleanup(ic *dstructs.IsolationConfig, pid int) error {
|
||||
if err := DestroyCgroup(ic.Cgroup, ic.CgroupPaths, pid); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
groups *cgroupConfig.Cgroup
|
||||
cgLock sync.Mutex
|
||||
}
|
||||
|
||||
// cleanup removes this host's Cgroup from within an Executor's context
|
||||
func (rc *resourceContainerContext) executorCleanup() error {
|
||||
rc.cgLock.Lock()
|
||||
defer rc.cgLock.Unlock()
|
||||
if err := DestroyCgroup(rc.groups, rc.cgPaths, os.Getpid()); err != nil {
|
||||
if err := DestroyCgroup(rc.groups, os.Getpid()); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rc *resourceContainerContext) getIsolationConfig() *dstructs.IsolationConfig {
|
||||
return &dstructs.IsolationConfig{
|
||||
Cgroup: rc.groups,
|
||||
CgroupPaths: rc.cgPaths,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,16 +2,15 @@ package driver
|
|||
|
||||
import (
|
||||
"encoding/gob"
|
||||
"log"
|
||||
"net/rpc"
|
||||
"os"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/nomad/client/driver/executor"
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
"github.com/hashicorp/nomad/nomad/structs"
|
||||
)
|
||||
|
||||
// Registering these types since we have to serialize and de-serialize the Task
|
||||
|
@ -26,65 +25,53 @@ func init() {
|
|||
|
||||
type ExecutorRPC struct {
|
||||
client *rpc.Client
|
||||
logger *log.Logger
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
// LaunchCmdArgs wraps a user command and the args for the purposes of RPC
|
||||
type LaunchCmdArgs struct {
|
||||
type LaunchArgs struct {
|
||||
Cmd *executor.ExecCommand
|
||||
}
|
||||
|
||||
type ExecCmdArgs struct {
|
||||
// ShutdownArgs wraps shutdown signal and grace period
|
||||
type ShutdownArgs struct {
|
||||
Signal string
|
||||
GracePeriod time.Duration
|
||||
}
|
||||
|
||||
type ExecArgs struct {
|
||||
Deadline time.Time
|
||||
Name string
|
||||
Args []string
|
||||
}
|
||||
|
||||
type ExecCmdReturn struct {
|
||||
type ExecReturn struct {
|
||||
Output []byte
|
||||
Code int
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) LaunchCmd(cmd *executor.ExecCommand) (*executor.ProcessState, error) {
|
||||
func (e *ExecutorRPC) Launch(cmd *executor.ExecCommand) (*executor.ProcessState, error) {
|
||||
var ps *executor.ProcessState
|
||||
err := e.client.Call("Plugin.LaunchCmd", LaunchCmdArgs{Cmd: cmd}, &ps)
|
||||
err := e.client.Call("Plugin.Launch", LaunchArgs{Cmd: cmd}, &ps)
|
||||
return ps, err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) LaunchSyslogServer() (*executor.SyslogServerState, error) {
|
||||
var ss *executor.SyslogServerState
|
||||
err := e.client.Call("Plugin.LaunchSyslogServer", new(interface{}), &ss)
|
||||
return ss, err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) Wait() (*executor.ProcessState, error) {
|
||||
var ps executor.ProcessState
|
||||
err := e.client.Call("Plugin.Wait", new(interface{}), &ps)
|
||||
return &ps, err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) ShutDown() error {
|
||||
return e.client.Call("Plugin.ShutDown", new(interface{}), new(interface{}))
|
||||
func (e *ExecutorRPC) Kill() error {
|
||||
return e.client.Call("Plugin.Kill", new(interface{}), new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) Exit() error {
|
||||
return e.client.Call("Plugin.Exit", new(interface{}), new(interface{}))
|
||||
func (e *ExecutorRPC) Shutdown(signal string, grace time.Duration) error {
|
||||
return e.client.Call("Plugin.Shutdown", &ShutdownArgs{signal, grace}, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) SetContext(ctx *executor.ExecutorContext) error {
|
||||
return e.client.Call("Plugin.SetContext", ctx, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) UpdateLogConfig(logConfig *structs.LogConfig) error {
|
||||
return e.client.Call("Plugin.UpdateLogConfig", logConfig, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) UpdateTask(task *structs.Task) error {
|
||||
return e.client.Call("Plugin.UpdateTask", task, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) DeregisterServices() error {
|
||||
return e.client.Call("Plugin.DeregisterServices", new(interface{}), new(interface{}))
|
||||
func (e *ExecutorRPC) UpdateResources(resources *executor.Resources) error {
|
||||
return e.client.Call("Plugin.UpdateResources", resources, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPC) Version() (*executor.ExecutorVersion, error) {
|
||||
|
@ -104,12 +91,12 @@ func (e *ExecutorRPC) Signal(s os.Signal) error {
|
|||
}
|
||||
|
||||
func (e *ExecutorRPC) Exec(deadline time.Time, name string, args []string) ([]byte, int, error) {
|
||||
req := ExecCmdArgs{
|
||||
req := ExecArgs{
|
||||
Deadline: deadline,
|
||||
Name: name,
|
||||
Args: args,
|
||||
}
|
||||
var resp *ExecCmdReturn
|
||||
var resp *ExecReturn
|
||||
err := e.client.Call("Plugin.Exec", req, &resp)
|
||||
if resp == nil {
|
||||
return nil, 0, err
|
||||
|
@ -119,25 +106,17 @@ func (e *ExecutorRPC) Exec(deadline time.Time, name string, args []string) ([]by
|
|||
|
||||
type ExecutorRPCServer struct {
|
||||
Impl executor.Executor
|
||||
logger *log.Logger
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) LaunchCmd(args LaunchCmdArgs, ps *executor.ProcessState) error {
|
||||
state, err := e.Impl.LaunchCmd(args.Cmd)
|
||||
func (e *ExecutorRPCServer) Launch(args LaunchArgs, ps *executor.ProcessState) error {
|
||||
state, err := e.Impl.Launch(args.Cmd)
|
||||
if state != nil {
|
||||
*ps = *state
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) LaunchSyslogServer(args interface{}, ss *executor.SyslogServerState) error {
|
||||
state, err := e.Impl.LaunchSyslogServer()
|
||||
if state != nil {
|
||||
*ss = *state
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) Wait(args interface{}, ps *executor.ProcessState) error {
|
||||
state, err := e.Impl.Wait()
|
||||
if state != nil {
|
||||
|
@ -146,29 +125,12 @@ func (e *ExecutorRPCServer) Wait(args interface{}, ps *executor.ProcessState) er
|
|||
return err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) ShutDown(args interface{}, resp *interface{}) error {
|
||||
return e.Impl.ShutDown()
|
||||
func (e *ExecutorRPCServer) Shutdown(args ShutdownArgs, resp *interface{}) error {
|
||||
return e.Impl.Shutdown(args.Signal, args.GracePeriod)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) Exit(args interface{}, resp *interface{}) error {
|
||||
return e.Impl.Exit()
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) SetContext(args *executor.ExecutorContext, resp *interface{}) error {
|
||||
return e.Impl.SetContext(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) UpdateLogConfig(args *structs.LogConfig, resp *interface{}) error {
|
||||
return e.Impl.UpdateLogConfig(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) UpdateTask(args *structs.Task, resp *interface{}) error {
|
||||
return e.Impl.UpdateTask(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) DeregisterServices(args interface{}, resp *interface{}) error {
|
||||
// In 0.6 this is a noop. Goes away in 0.7.
|
||||
return nil
|
||||
func (e *ExecutorRPCServer) UpdateResources(args *executor.Resources, resp *interface{}) error {
|
||||
return e.Impl.UpdateResources(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) Version(args interface{}, version *executor.ExecutorVersion) error {
|
||||
|
@ -191,9 +153,9 @@ func (e *ExecutorRPCServer) Signal(args os.Signal, resp *interface{}) error {
|
|||
return e.Impl.Signal(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServer) Exec(args ExecCmdArgs, result *ExecCmdReturn) error {
|
||||
func (e *ExecutorRPCServer) Exec(args ExecArgs, result *ExecReturn) error {
|
||||
out, code, err := e.Impl.Exec(args.Deadline, args.Name, args.Args)
|
||||
ret := &ExecCmdReturn{
|
||||
ret := &ExecReturn{
|
||||
Output: out,
|
||||
Code: code,
|
||||
}
|
||||
|
@ -202,13 +164,18 @@ func (e *ExecutorRPCServer) Exec(args ExecCmdArgs, result *ExecCmdReturn) error
|
|||
}
|
||||
|
||||
type ExecutorPlugin struct {
|
||||
logger *log.Logger
|
||||
Impl *ExecutorRPCServer
|
||||
logger hclog.Logger
|
||||
fsIsolation bool
|
||||
Impl *ExecutorRPCServer
|
||||
}
|
||||
|
||||
func (p *ExecutorPlugin) Server(*plugin.MuxBroker) (interface{}, error) {
|
||||
if p.Impl == nil {
|
||||
p.Impl = &ExecutorRPCServer{Impl: executor.NewExecutor(p.logger), logger: p.logger}
|
||||
if p.fsIsolation {
|
||||
p.Impl = &ExecutorRPCServer{Impl: executor.NewExecutorWithIsolation(p.logger), logger: p.logger}
|
||||
} else {
|
||||
p.Impl = &ExecutorRPCServer{Impl: executor.NewExecutor(p.logger), logger: p.logger}
|
||||
}
|
||||
}
|
||||
return p.Impl, nil
|
||||
}
|
||||
|
|
|
@ -0,0 +1,220 @@
|
|||
package driver
|
||||
|
||||
/*
|
||||
import (
|
||||
"encoding/gob"
|
||||
"log"
|
||||
"net/rpc"
|
||||
"os"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/go-plugin"
|
||||
executorv0 "github.com/hashicorp/nomad/client/driver/executorv0"
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
"github.com/hashicorp/nomad/nomad/structs"
|
||||
)
|
||||
|
||||
// Registering these types since we have to serialize and de-serialize the Task
|
||||
// structs over the wire between drivers and the executorv0.
|
||||
func init() {
|
||||
gob.Register([]interface{}{})
|
||||
gob.Register(map[string]interface{}{})
|
||||
gob.Register([]map[string]string{})
|
||||
gob.Register([]map[string]int{})
|
||||
gob.Register(syscall.Signal(0x1))
|
||||
}
|
||||
|
||||
type ExecutorRPCPre0_9_0 struct {
|
||||
client *rpc.Client
|
||||
logger *log.Logger
|
||||
}
|
||||
|
||||
// LaunchCmdArgs wraps a user command and the args for the purposes of RPC
|
||||
type LaunchCmdArgs struct {
|
||||
Cmd *executorv0.ExecCommand
|
||||
}
|
||||
|
||||
type ExecCmdArgs struct {
|
||||
Deadline time.Time
|
||||
Name string
|
||||
Args []string
|
||||
}
|
||||
|
||||
type ExecCmdReturn struct {
|
||||
Output []byte
|
||||
Code int
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) LaunchCmd(cmd *executorv0.ExecCommand) (*executorv0.ProcessState, error) {
|
||||
var ps *executorv0.ProcessState
|
||||
err := e.client.Call("Plugin.LaunchCmd", LaunchCmdArgs{Cmd: cmd}, &ps)
|
||||
return ps, err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) LaunchSyslogServer() (*executorv0.SyslogServerState, error) {
|
||||
var ss *executorv0.SyslogServerState
|
||||
err := e.client.Call("Plugin.LaunchSyslogServer", new(interface{}), &ss)
|
||||
return ss, err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) Wait() (*executorv0.ProcessState, error) {
|
||||
var ps executorv0.ProcessState
|
||||
err := e.client.Call("Plugin.Wait", new(interface{}), &ps)
|
||||
return &ps, err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) ShutDown() error {
|
||||
return e.client.Call("Plugin.ShutDown", new(interface{}), new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) Exit() error {
|
||||
return e.client.Call("Plugin.Exit", new(interface{}), new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) SetContext(ctx *executorv0.ExecutorContext) error {
|
||||
return e.client.Call("Plugin.SetContext", ctx, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) UpdateLogConfig(logConfig *executorv0.LogConfig) error {
|
||||
return e.client.Call("Plugin.UpdateLogConfig", logConfig, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) UpdateTask(task *structs.Task) error {
|
||||
return e.client.Call("Plugin.UpdateTask", task, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) DeregisterServices() error {
|
||||
return e.client.Call("Plugin.DeregisterServices", new(interface{}), new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) Version() (*executorv0.ExecutorVersion, error) {
|
||||
var version executorv0.ExecutorVersion
|
||||
err := e.client.Call("Plugin.Version", new(interface{}), &version)
|
||||
return &version, err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) Stats() (*cstructs.TaskResourceUsage, error) {
|
||||
var resourceUsage cstructs.TaskResourceUsage
|
||||
err := e.client.Call("Plugin.Stats", new(interface{}), &resourceUsage)
|
||||
return &resourceUsage, err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) Signal(s os.Signal) error {
|
||||
return e.client.Call("Plugin.Signal", &s, new(interface{}))
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCPre0_9_0) Exec(deadline time.Time, name string, args []string) ([]byte, int, error) {
|
||||
req := ExecCmdArgs{
|
||||
Deadline: deadline,
|
||||
Name: name,
|
||||
Args: args,
|
||||
}
|
||||
var resp *ExecCmdReturn
|
||||
err := e.client.Call("Plugin.Exec", req, &resp)
|
||||
if resp == nil {
|
||||
return nil, 0, err
|
||||
}
|
||||
return resp.Output, resp.Code, err
|
||||
}
|
||||
|
||||
type ExecutorRPCServerPre0_9_0 struct {
|
||||
Impl executorv0.Executor
|
||||
logger *log.Logger
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) LaunchCmd(args LaunchCmdArgs, ps *executorv0.ProcessState) error {
|
||||
state, err := e.Impl.LaunchCmd(args.Cmd)
|
||||
if state != nil {
|
||||
*ps = *state
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) LaunchSyslogServer(args interface{}, ss *executorv0.SyslogServerState) error {
|
||||
state, err := e.Impl.LaunchSyslogServer()
|
||||
if state != nil {
|
||||
*ss = *state
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) Wait(args interface{}, ps *executorv0.ProcessState) error {
|
||||
state, err := e.Impl.Wait()
|
||||
if state != nil {
|
||||
*ps = *state
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) ShutDown(args interface{}, resp *interface{}) error {
|
||||
return e.Impl.ShutDown()
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) Exit(args interface{}, resp *interface{}) error {
|
||||
return e.Impl.Exit()
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) SetContext(args *executorv0.ExecutorContext, resp *interface{}) error {
|
||||
return e.Impl.SetContext(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) UpdateLogConfig(args *executorv0.LogConfig, resp *interface{}) error {
|
||||
return e.Impl.UpdateLogConfig(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) UpdateTask(args *structs.Task, resp *interface{}) error {
|
||||
return e.Impl.UpdateTask(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) DeregisterServices(args interface{}, resp *interface{}) error {
|
||||
// In 0.6 this is a noop. Goes away in 0.7.
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) Version(args interface{}, version *executorv0.ExecutorVersion) error {
|
||||
ver, err := e.Impl.Version()
|
||||
if ver != nil {
|
||||
*version = *ver
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) Stats(args interface{}, resourceUsage *cstructs.TaskResourceUsage) error {
|
||||
ru, err := e.Impl.Stats()
|
||||
if ru != nil {
|
||||
*resourceUsage = *ru
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) Signal(args os.Signal, resp *interface{}) error {
|
||||
return e.Impl.Signal(args)
|
||||
}
|
||||
|
||||
func (e *ExecutorRPCServerPre0_9_0) Exec(args ExecCmdArgs, result *ExecCmdReturn) error {
|
||||
out, code, err := e.Impl.Exec(args.Deadline, args.Name, args.Args)
|
||||
ret := &ExecCmdReturn{
|
||||
Output: out,
|
||||
Code: code,
|
||||
}
|
||||
*result = *ret
|
||||
return err
|
||||
}
|
||||
|
||||
type ExecutorPluginPre0_9_0 struct {
|
||||
logger *log.Logger
|
||||
Impl *ExecutorRPCServerPre0_9_0
|
||||
}
|
||||
|
||||
func (p *ExecutorPluginPre0_9_0) Server(*plugin.MuxBroker) (interface{}, error) {
|
||||
if p.Impl == nil {
|
||||
p.Impl = &ExecutorRPCServerPre0_9_0{Impl: executorv0.NewExecutor(p.logger), logger: p.logger}
|
||||
}
|
||||
return p.Impl, nil
|
||||
}
|
||||
|
||||
func (p *ExecutorPluginPre0_9_0) Client(b *plugin.MuxBroker, c *rpc.Client) (interface{}, error) {
|
||||
return &ExecutorRPCPre0_9_0{client: c, logger: p.logger}, nil
|
||||
}
|
||||
*/
|
|
@ -55,14 +55,14 @@ type JavaDriverConfig struct {
|
|||
|
||||
// javaHandle is returned from Start/Open as a handle to the PID
|
||||
type javaHandle struct {
|
||||
pluginClient *plugin.Client
|
||||
userPid int
|
||||
executor executor.Executor
|
||||
isolationConfig *dstructs.IsolationConfig
|
||||
taskDir string
|
||||
pluginClient *plugin.Client
|
||||
userPid int
|
||||
executor executor.Executor
|
||||
taskDir string
|
||||
|
||||
killTimeout time.Duration
|
||||
maxKillTimeout time.Duration
|
||||
shutdownSignal string
|
||||
version string
|
||||
logger *log.Logger
|
||||
waitCh chan *dstructs.WaitResult
|
||||
|
@ -239,8 +239,9 @@ func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
|
||||
pluginLogFile := filepath.Join(ctx.TaskDir.Dir, "executor.out")
|
||||
executorConfig := &dstructs.ExecutorConfig{
|
||||
LogFile: pluginLogFile,
|
||||
LogLevel: d.config.LogLevel,
|
||||
LogFile: pluginLogFile,
|
||||
LogLevel: d.config.LogLevel,
|
||||
FSIsolation: true,
|
||||
}
|
||||
|
||||
execIntf, pluginClient, err := createExecutor(d.config.LogOutput, d.config, executorConfig)
|
||||
|
@ -248,25 +249,12 @@ func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
return nil, err
|
||||
}
|
||||
|
||||
// Set the context
|
||||
executorCtx := &executor.ExecutorContext{
|
||||
TaskEnv: ctx.TaskEnv,
|
||||
Driver: "java",
|
||||
Task: task,
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
LogDir: ctx.TaskDir.LogDir,
|
||||
}
|
||||
if err := execIntf.SetContext(executorCtx); err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, fmt.Errorf("failed to set executor context: %v", err)
|
||||
}
|
||||
|
||||
absPath, err := GetAbsolutePath("java")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
taskKillSignal, err := getTaskKillSignal(task.KillSignal)
|
||||
_, err = getTaskKillSignal(task.KillSignal)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -274,12 +262,20 @@ func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
execCmd := &executor.ExecCommand{
|
||||
Cmd: absPath,
|
||||
Args: args,
|
||||
FSIsolation: true,
|
||||
ResourceLimits: true,
|
||||
User: getExecutorUser(task),
|
||||
TaskKillSignal: taskKillSignal,
|
||||
Resources: &executor.Resources{
|
||||
CPU: task.Resources.CPU,
|
||||
MemoryMB: task.Resources.MemoryMB,
|
||||
IOPS: task.Resources.IOPS,
|
||||
DiskMB: task.Resources.DiskMB,
|
||||
},
|
||||
Env: ctx.TaskEnv.List(),
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
StdoutPath: ctx.StdoutFifo,
|
||||
StderrPath: ctx.StderrFifo,
|
||||
}
|
||||
ps, err := execIntf.LaunchCmd(execCmd)
|
||||
ps, err := execIntf.Launch(execCmd)
|
||||
if err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, err
|
||||
|
@ -289,17 +285,17 @@ func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
// Return a driver handle
|
||||
maxKill := d.DriverContext.config.MaxKillTimeout
|
||||
h := &javaHandle{
|
||||
pluginClient: pluginClient,
|
||||
executor: execIntf,
|
||||
userPid: ps.Pid,
|
||||
isolationConfig: ps.IsolationConfig,
|
||||
taskDir: ctx.TaskDir.Dir,
|
||||
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
|
||||
maxKillTimeout: maxKill,
|
||||
version: d.config.Version.VersionNumber(),
|
||||
logger: d.logger,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
pluginClient: pluginClient,
|
||||
executor: execIntf,
|
||||
userPid: ps.Pid,
|
||||
shutdownSignal: task.KillSignal,
|
||||
taskDir: ctx.TaskDir.Dir,
|
||||
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
|
||||
maxKillTimeout: maxKill,
|
||||
version: d.config.Version.VersionNumber(),
|
||||
logger: d.logger,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
}
|
||||
go h.run()
|
||||
return &StartResponse{Handle: h}, nil
|
||||
|
@ -308,13 +304,13 @@ func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
func (d *JavaDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }
|
||||
|
||||
type javaId struct {
|
||||
Version string
|
||||
KillTimeout time.Duration
|
||||
MaxKillTimeout time.Duration
|
||||
PluginConfig *PluginReattachConfig
|
||||
IsolationConfig *dstructs.IsolationConfig
|
||||
TaskDir string
|
||||
UserPid int
|
||||
Version string
|
||||
KillTimeout time.Duration
|
||||
MaxKillTimeout time.Duration
|
||||
PluginConfig *PluginReattachConfig
|
||||
TaskDir string
|
||||
UserPid int
|
||||
ShutdownSignal string
|
||||
}
|
||||
|
||||
func (d *JavaDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error) {
|
||||
|
@ -334,12 +330,6 @@ func (d *JavaDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
|
|||
if e := destroyPlugin(id.PluginConfig.Pid, id.UserPid); e != nil {
|
||||
merrs.Errors = append(merrs.Errors, fmt.Errorf("error destroying plugin and userpid: %v", e))
|
||||
}
|
||||
if id.IsolationConfig != nil {
|
||||
ePid := pluginConfig.Reattach.Pid
|
||||
if e := executor.ClientCleanup(id.IsolationConfig, ePid); e != nil {
|
||||
merrs.Errors = append(merrs.Errors, fmt.Errorf("destroying resource container failed: %v", e))
|
||||
}
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("error connecting to plugin: %v", merrs.ErrorOrNil())
|
||||
}
|
||||
|
@ -349,16 +339,16 @@ func (d *JavaDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
|
|||
|
||||
// Return a driver handle
|
||||
h := &javaHandle{
|
||||
pluginClient: pluginClient,
|
||||
executor: exec,
|
||||
userPid: id.UserPid,
|
||||
isolationConfig: id.IsolationConfig,
|
||||
logger: d.logger,
|
||||
version: id.Version,
|
||||
killTimeout: id.KillTimeout,
|
||||
maxKillTimeout: id.MaxKillTimeout,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
pluginClient: pluginClient,
|
||||
executor: exec,
|
||||
userPid: id.UserPid,
|
||||
shutdownSignal: id.ShutdownSignal,
|
||||
logger: d.logger,
|
||||
version: id.Version,
|
||||
killTimeout: id.KillTimeout,
|
||||
maxKillTimeout: id.MaxKillTimeout,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
}
|
||||
go h.run()
|
||||
return h, nil
|
||||
|
@ -366,13 +356,13 @@ func (d *JavaDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
|
|||
|
||||
func (h *javaHandle) ID() string {
|
||||
id := javaId{
|
||||
Version: h.version,
|
||||
KillTimeout: h.killTimeout,
|
||||
MaxKillTimeout: h.maxKillTimeout,
|
||||
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
|
||||
UserPid: h.userPid,
|
||||
IsolationConfig: h.isolationConfig,
|
||||
TaskDir: h.taskDir,
|
||||
Version: h.version,
|
||||
KillTimeout: h.killTimeout,
|
||||
MaxKillTimeout: h.maxKillTimeout,
|
||||
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
|
||||
UserPid: h.userPid,
|
||||
TaskDir: h.taskDir,
|
||||
ShutdownSignal: h.shutdownSignal,
|
||||
}
|
||||
|
||||
data, err := json.Marshal(id)
|
||||
|
@ -389,7 +379,12 @@ func (h *javaHandle) WaitCh() chan *dstructs.WaitResult {
|
|||
func (h *javaHandle) Update(task *structs.Task) error {
|
||||
// Store the updated kill timeout.
|
||||
h.killTimeout = GetKillTimeout(task.KillTimeout, h.maxKillTimeout)
|
||||
h.executor.UpdateTask(task)
|
||||
h.executor.UpdateResources(&executor.Resources{
|
||||
CPU: task.Resources.CPU,
|
||||
MemoryMB: task.Resources.MemoryMB,
|
||||
IOPS: task.Resources.IOPS,
|
||||
DiskMB: task.Resources.DiskMB,
|
||||
})
|
||||
|
||||
// Update is not possible
|
||||
return nil
|
||||
|
@ -413,11 +408,11 @@ func (d *javaHandle) Network() *cstructs.DriverNetwork {
|
|||
}
|
||||
|
||||
func (h *javaHandle) Kill() error {
|
||||
if err := h.executor.ShutDown(); err != nil {
|
||||
if err := h.executor.Shutdown(h.shutdownSignal, h.killTimeout); err != nil {
|
||||
if h.pluginClient.Exited() {
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("executor Shutdown failed: %v", err)
|
||||
return fmt.Errorf("executor Kill failed: %v", err)
|
||||
}
|
||||
|
||||
select {
|
||||
|
@ -426,8 +421,8 @@ func (h *javaHandle) Kill() error {
|
|||
if h.pluginClient.Exited() {
|
||||
break
|
||||
}
|
||||
if err := h.executor.Exit(); err != nil {
|
||||
return fmt.Errorf("executor Exit failed: %v", err)
|
||||
if err := h.executor.Shutdown(h.shutdownSignal, h.killTimeout); err != nil {
|
||||
return fmt.Errorf("executor Destroy failed: %v", err)
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -442,20 +437,13 @@ func (h *javaHandle) run() {
|
|||
ps, werr := h.executor.Wait()
|
||||
close(h.doneCh)
|
||||
if ps.ExitCode == 0 && werr != nil {
|
||||
if h.isolationConfig != nil {
|
||||
ePid := h.pluginClient.ReattachConfig().Pid
|
||||
if e := executor.ClientCleanup(h.isolationConfig, ePid); e != nil {
|
||||
h.logger.Printf("[ERR] driver.java: destroying resource container failed: %v", e)
|
||||
}
|
||||
} else {
|
||||
if e := killProcess(h.userPid); e != nil {
|
||||
h.logger.Printf("[ERR] driver.java: error killing user process: %v", e)
|
||||
}
|
||||
if e := killProcess(h.userPid); e != nil {
|
||||
h.logger.Printf("[ERR] driver.java: error killing user process: %v", e)
|
||||
}
|
||||
}
|
||||
|
||||
// Exit the executor
|
||||
h.executor.Exit()
|
||||
// Destroy the executor
|
||||
h.executor.Shutdown(h.shutdownSignal, 0)
|
||||
h.pluginClient.Kill()
|
||||
|
||||
// Send the results
|
||||
|
|
|
@ -43,7 +43,7 @@ func TestJavaDriver_Fingerprint(t *testing.T) {
|
|||
Resources: structs.DefaultResources(),
|
||||
}
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
node := &structs.Node{
|
||||
Attributes: map[string]string{
|
||||
|
@ -100,7 +100,7 @@ func TestJavaDriver_StartOpen_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the test jar into the task's directory
|
||||
|
@ -156,7 +156,7 @@ func TestJavaDriver_Start_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the test jar into the task's directory
|
||||
|
@ -222,7 +222,7 @@ func TestJavaDriver_Start_Kill_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the test jar into the task's directory
|
||||
|
@ -288,7 +288,7 @@ func TestJavaDriver_Signal(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the test jar into the task's directory
|
||||
|
@ -357,7 +357,7 @@ func TestJavaDriver_User(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -399,7 +399,7 @@ func TestJavaDriver_Start_Wait_Class(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the test jar into the task's directory
|
||||
|
@ -469,7 +469,7 @@ func TestJavaDriver_Start_Kill(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the test jar into the task's directory
|
||||
|
@ -506,7 +506,7 @@ func TestJavaDriver_Start_Kill(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewJavaDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the test jar into the task's directory
|
||||
|
|
|
@ -5,7 +5,6 @@ import (
|
|||
"bytes"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"log"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"sort"
|
||||
|
@ -13,6 +12,8 @@ import (
|
|||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -48,7 +49,7 @@ type FileRotator struct {
|
|||
bufLock sync.Mutex
|
||||
|
||||
flushTicker *time.Ticker
|
||||
logger *log.Logger
|
||||
logger hclog.Logger
|
||||
purgeCh chan struct{}
|
||||
doneCh chan struct{}
|
||||
|
||||
|
@ -58,7 +59,8 @@ type FileRotator struct {
|
|||
|
||||
// NewFileRotator returns a new file rotator
|
||||
func NewFileRotator(path string, baseFile string, maxFiles int,
|
||||
fileSize int64, logger *log.Logger) (*FileRotator, error) {
|
||||
fileSize int64, logger hclog.Logger) (*FileRotator, error) {
|
||||
logger = logger.Named("rotator")
|
||||
rotator := &FileRotator{
|
||||
MaxFiles: maxFiles,
|
||||
FileSize: fileSize,
|
||||
|
@ -71,6 +73,7 @@ func NewFileRotator(path string, baseFile string, maxFiles int,
|
|||
purgeCh: make(chan struct{}, 1),
|
||||
doneCh: make(chan struct{}, 1),
|
||||
}
|
||||
|
||||
if err := rotator.lastFile(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -93,7 +96,7 @@ func (f *FileRotator) Write(p []byte) (n int, err error) {
|
|||
f.flushBuffer()
|
||||
f.currentFile.Close()
|
||||
if err := f.nextFile(); err != nil {
|
||||
f.logger.Printf("[ERROR] driver.rotator: error creating next file: %v", err)
|
||||
f.logger.Error("error creating next file", "err", err)
|
||||
return 0, err
|
||||
}
|
||||
}
|
||||
|
@ -141,7 +144,7 @@ func (f *FileRotator) Write(p []byte) (n int, err error) {
|
|||
// Increment the total number of bytes in the file
|
||||
f.currentWr += int64(n)
|
||||
if err != nil {
|
||||
f.logger.Printf("[ERROR] driver.rotator: error writing to file: %v", err)
|
||||
f.logger.Error("error writing to file", "err", err)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
@ -210,10 +213,11 @@ func (f *FileRotator) lastFile() error {
|
|||
// createFile opens a new or existing file for writing
|
||||
func (f *FileRotator) createFile() error {
|
||||
logFileName := filepath.Join(f.path, fmt.Sprintf("%s.%d", f.baseFileName, f.logFileIdx))
|
||||
cFile, err := os.OpenFile(logFileName, os.O_RDWR|os.O_CREATE|os.O_APPEND, 0666)
|
||||
cFile, err := os.OpenFile(logFileName, os.O_RDWR|os.O_CREATE|os.O_APPEND, 0644)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
f.currentFile = cFile
|
||||
fi, err := f.currentFile.Stat()
|
||||
if err != nil {
|
||||
|
@ -257,7 +261,7 @@ func (f *FileRotator) purgeOldFiles() {
|
|||
var fIndexes []int
|
||||
files, err := ioutil.ReadDir(f.path)
|
||||
if err != nil {
|
||||
f.logger.Printf("[ERROR] driver.rotator: error getting directory listing: %v", err)
|
||||
f.logger.Error("error getting directory listing", "err", err)
|
||||
return
|
||||
}
|
||||
// Inserting all the rotated files in a slice
|
||||
|
@ -266,7 +270,7 @@ func (f *FileRotator) purgeOldFiles() {
|
|||
fileIdx := strings.TrimPrefix(fi.Name(), fmt.Sprintf("%s.", f.baseFileName))
|
||||
n, err := strconv.Atoi(fileIdx)
|
||||
if err != nil {
|
||||
f.logger.Printf("[ERROR] driver.rotator: error extracting file index: %v", err)
|
||||
f.logger.Error("error extracting file index", "err", err)
|
||||
continue
|
||||
}
|
||||
fIndexes = append(fIndexes, n)
|
||||
|
@ -287,7 +291,7 @@ func (f *FileRotator) purgeOldFiles() {
|
|||
fname := filepath.Join(f.path, fmt.Sprintf("%s.%d", f.baseFileName, fIndex))
|
||||
err := os.RemoveAll(fname)
|
||||
if err != nil {
|
||||
f.logger.Printf("[ERROR] driver.rotator: error removing file: %v", err)
|
||||
f.logger.Error("error removing file", "filename", fname, "err", err)
|
||||
}
|
||||
}
|
||||
f.oldestLogFileIdx = fIndexes[0]
|
||||
|
|
|
@ -4,10 +4,10 @@ package logging
|
|||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"strconv"
|
||||
|
||||
syslog "github.com/RackSec/srslog"
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
)
|
||||
|
||||
// Errors related to parsing priority
|
||||
|
@ -41,11 +41,11 @@ type Priority struct {
|
|||
|
||||
// DockerLogParser parses a line of log message that the docker daemon ships
|
||||
type DockerLogParser struct {
|
||||
logger *log.Logger
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
// NewDockerLogParser creates a new DockerLogParser
|
||||
func NewDockerLogParser(logger *log.Logger) *DockerLogParser {
|
||||
func NewDockerLogParser(logger hclog.Logger) *DockerLogParser {
|
||||
return &DockerLogParser{logger: logger}
|
||||
}
|
||||
|
||||
|
|
|
@ -2,9 +2,10 @@ package logging
|
|||
|
||||
import (
|
||||
"bufio"
|
||||
"log"
|
||||
"net"
|
||||
"sync"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
)
|
||||
|
||||
// SyslogServer is a server which listens to syslog messages and parses them
|
||||
|
@ -17,11 +18,12 @@ type SyslogServer struct {
|
|||
done bool
|
||||
doneLock sync.Mutex
|
||||
|
||||
logger *log.Logger
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
// NewSyslogServer creates a new syslog server
|
||||
func NewSyslogServer(l net.Listener, messages chan *SyslogMessage, logger *log.Logger) *SyslogServer {
|
||||
func NewSyslogServer(l net.Listener, messages chan *SyslogMessage, logger hclog.Logger) *SyslogServer {
|
||||
logger = logger.Named("logcollector.server")
|
||||
parser := NewDockerLogParser(logger)
|
||||
return &SyslogServer{
|
||||
listener: l,
|
||||
|
@ -48,7 +50,7 @@ func (s *SyslogServer) Start() {
|
|||
return
|
||||
}
|
||||
|
||||
s.logger.Printf("[ERR] logcollector.server: error in accepting connection: %v", err)
|
||||
s.logger.Error("error in accepting connection", "err", err)
|
||||
continue
|
||||
}
|
||||
go s.read(connection)
|
||||
|
|
|
@ -6,14 +6,13 @@ import (
|
|||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"log"
|
||||
"net"
|
||||
"os"
|
||||
"runtime"
|
||||
|
||||
syslog "github.com/RackSec/srslog"
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/nomad/client/allocdir"
|
||||
cstructs "github.com/hashicorp/nomad/client/driver/structs"
|
||||
"github.com/hashicorp/nomad/nomad/structs"
|
||||
)
|
||||
|
||||
|
@ -41,8 +40,7 @@ type LogCollectorContext struct {
|
|||
// SyslogCollectorState holds the address and isolation information of a launched
|
||||
// syslog server
|
||||
type SyslogCollectorState struct {
|
||||
IsolationConfig *cstructs.IsolationConfig
|
||||
Addr string
|
||||
Addr string
|
||||
}
|
||||
|
||||
// LogCollector is an interface which allows a driver to launch a log server
|
||||
|
@ -64,12 +62,13 @@ type SyslogCollector struct {
|
|||
syslogChan chan *SyslogMessage
|
||||
taskDir string
|
||||
|
||||
logger *log.Logger
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
// NewSyslogCollector returns an implementation of the SyslogCollector
|
||||
func NewSyslogCollector(logger *log.Logger) *SyslogCollector {
|
||||
return &SyslogCollector{logger: logger, syslogChan: make(chan *SyslogMessage, 2048)}
|
||||
func NewSyslogCollector(logger hclog.Logger) *SyslogCollector {
|
||||
return &SyslogCollector{logger: logger.Named("syslog-server"),
|
||||
syslogChan: make(chan *SyslogMessage, 2048)}
|
||||
}
|
||||
|
||||
// LaunchCollector launches a new syslog server and starts writing log lines to
|
||||
|
@ -79,7 +78,7 @@ func (s *SyslogCollector) LaunchCollector(ctx *LogCollectorContext) (*SyslogColl
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
s.logger.Printf("[DEBUG] syslog-server: launching syslog server on addr: %v", l.Addr().String())
|
||||
s.logger.Debug("launching syslog server on addr", "addr", l.Addr().String())
|
||||
s.ctx = ctx
|
||||
// configuring the task dir
|
||||
if err := s.configureTaskDir(); err != nil {
|
||||
|
|
|
@ -33,7 +33,7 @@ func TestLxcDriver_Fingerprint(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewLxcDriver(ctx.DriverCtx)
|
||||
|
||||
node := &structs.Node{
|
||||
|
@ -104,7 +104,7 @@ func TestLxcDriver_Start_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewLxcDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -191,7 +191,7 @@ func TestLxcDriver_Open_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewLxcDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -290,7 +290,7 @@ func testVolumeConfig(t *testing.T, volConfig []string) error {
|
|||
task.Config["volumes"] = volConfig
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
|
||||
driver := NewLxcDriver(ctx.DriverCtx)
|
||||
|
||||
|
@ -320,7 +320,7 @@ func TestLxcDriver_Start_NoVolumes(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
|
||||
// set lxcVolumesConfigOption to false to disallow absolute paths as the source for the bind mount
|
||||
ctx.DriverCtx.config.Options = map[string]string{lxcVolumesConfigOption: "false"}
|
||||
|
|
|
@ -14,6 +14,7 @@ import (
|
|||
|
||||
"github.com/mitchellh/mapstructure"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/nomad/client/driver/logging"
|
||||
dstructs "github.com/hashicorp/nomad/client/driver/structs"
|
||||
cstructs "github.com/hashicorp/nomad/client/structs"
|
||||
|
@ -447,7 +448,7 @@ func (h *mockDriverHandle) handleLogging() {
|
|||
// Setup a log rotator
|
||||
logFileSize := int64(h.task.LogConfig.MaxFileSizeMB * 1024 * 1024)
|
||||
lro, err := logging.NewFileRotator(h.ctx.TaskDir.LogDir, fmt.Sprintf("%v.stdout", h.taskName),
|
||||
h.task.LogConfig.MaxFiles, logFileSize, h.logger)
|
||||
h.task.LogConfig.MaxFiles, logFileSize, hclog.Default()) //TODO: plumb hclog
|
||||
if err != nil {
|
||||
h.exitErr = err
|
||||
close(h.doneCh)
|
||||
|
|
|
@ -2,12 +2,10 @@ package driver
|
|||
|
||||
import (
|
||||
"io"
|
||||
"log"
|
||||
"net"
|
||||
"strings"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/logutils"
|
||||
)
|
||||
|
||||
var HandshakeConfig = plugin.HandshakeConfig{
|
||||
|
@ -16,15 +14,15 @@ var HandshakeConfig = plugin.HandshakeConfig{
|
|||
MagicCookieValue: "e4327c2e01eabfd75a8a67adb114fb34a757d57eee7728d857a8cec6e91a7255",
|
||||
}
|
||||
|
||||
func GetPluginMap(w io.Writer, logLevel string) map[string]plugin.Plugin {
|
||||
func GetPluginMap(w io.Writer, logLevel hclog.Level, fsIsolation bool) map[string]plugin.Plugin {
|
||||
e := new(ExecutorPlugin)
|
||||
filter := &logutils.LevelFilter{
|
||||
Levels: []logutils.LogLevel{"TRACE", "DEBUG", "INFO", "WARN", "ERR"},
|
||||
MinLevel: logutils.LogLevel(strings.ToUpper(logLevel)),
|
||||
Writer: w,
|
||||
}
|
||||
|
||||
e.logger = log.New(filter, "", log.LstdFlags|log.Lmicroseconds)
|
||||
e.logger = hclog.New(&hclog.LoggerOptions{
|
||||
Output: w,
|
||||
Level: logLevel,
|
||||
})
|
||||
|
||||
e.fsIsolation = fsIsolation
|
||||
|
||||
return map[string]plugin.Plugin{
|
||||
"executor": e,
|
||||
|
|
|
@ -75,6 +75,7 @@ type qemuHandle struct {
|
|||
userPid int
|
||||
executor executor.Executor
|
||||
monitorPath string
|
||||
shutdownSignal string
|
||||
killTimeout time.Duration
|
||||
maxKillTimeout time.Duration
|
||||
logger *log.Logger
|
||||
|
@ -319,24 +320,22 @@ func (d *QemuDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
executorCtx := &executor.ExecutorContext{
|
||||
TaskEnv: ctx.TaskEnv,
|
||||
Driver: "qemu",
|
||||
Task: task,
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
LogDir: ctx.TaskDir.LogDir,
|
||||
}
|
||||
if err := exec.SetContext(executorCtx); err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, fmt.Errorf("failed to set executor context: %v", err)
|
||||
|
||||
_, err = getTaskKillSignal(task.KillSignal)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
execCmd := &executor.ExecCommand{
|
||||
Cmd: args[0],
|
||||
Args: args[1:],
|
||||
User: task.User,
|
||||
Cmd: args[0],
|
||||
Args: args[1:],
|
||||
User: task.User,
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
Env: ctx.TaskEnv.List(),
|
||||
StdoutPath: ctx.StdoutFifo,
|
||||
StderrPath: ctx.StderrFifo,
|
||||
}
|
||||
ps, err := exec.LaunchCmd(execCmd)
|
||||
ps, err := exec.Launch(execCmd)
|
||||
if err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, err
|
||||
|
@ -349,6 +348,7 @@ func (d *QemuDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse
|
|||
pluginClient: pluginClient,
|
||||
executor: exec,
|
||||
userPid: ps.Pid,
|
||||
shutdownSignal: task.KillSignal,
|
||||
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
|
||||
maxKillTimeout: maxKill,
|
||||
monitorPath: monitorPath,
|
||||
|
@ -373,6 +373,7 @@ type qemuId struct {
|
|||
MaxKillTimeout time.Duration
|
||||
UserPid int
|
||||
PluginConfig *PluginReattachConfig
|
||||
ShutdownSignal string
|
||||
}
|
||||
|
||||
func (d *QemuDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error) {
|
||||
|
@ -404,6 +405,7 @@ func (d *QemuDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, erro
|
|||
logger: d.logger,
|
||||
killTimeout: id.KillTimeout,
|
||||
maxKillTimeout: id.MaxKillTimeout,
|
||||
shutdownSignal: id.ShutdownSignal,
|
||||
version: id.Version,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
|
@ -421,6 +423,7 @@ func (h *qemuHandle) ID() string {
|
|||
MaxKillTimeout: h.maxKillTimeout,
|
||||
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
|
||||
UserPid: h.userPid,
|
||||
ShutdownSignal: h.shutdownSignal,
|
||||
}
|
||||
|
||||
data, err := json.Marshal(id)
|
||||
|
@ -437,7 +440,6 @@ func (h *qemuHandle) WaitCh() chan *dstructs.WaitResult {
|
|||
func (h *qemuHandle) Update(task *structs.Task) error {
|
||||
// Store the updated kill timeout.
|
||||
h.killTimeout = GetKillTimeout(task.KillTimeout, h.maxKillTimeout)
|
||||
h.executor.UpdateTask(task)
|
||||
|
||||
// Update is not possible
|
||||
return nil
|
||||
|
@ -470,7 +472,7 @@ func (h *qemuHandle) Kill() error {
|
|||
// the qemu process as a last resort
|
||||
if gracefulShutdownSent == false {
|
||||
h.logger.Printf("[DEBUG] driver.qemu: graceful shutdown is not enabled, sending an interrupt signal to pid: %d", h.userPid)
|
||||
if err := h.executor.ShutDown(); err != nil {
|
||||
if err := h.executor.Signal(os.Interrupt); err != nil {
|
||||
if h.pluginClient.Exited() {
|
||||
return nil
|
||||
}
|
||||
|
@ -490,8 +492,8 @@ func (h *qemuHandle) Kill() error {
|
|||
if h.pluginClient.Exited() {
|
||||
return nil
|
||||
}
|
||||
if err := h.executor.Exit(); err != nil {
|
||||
return fmt.Errorf("executor Exit failed: %v", err)
|
||||
if err := h.executor.Shutdown(h.shutdownSignal, h.killTimeout); err != nil {
|
||||
return fmt.Errorf("executor Destroy failed: %v", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -510,8 +512,8 @@ func (h *qemuHandle) run() {
|
|||
}
|
||||
close(h.doneCh)
|
||||
|
||||
// Exit the executor
|
||||
h.executor.Exit()
|
||||
// Destroy the executor
|
||||
h.executor.Shutdown(h.shutdownSignal, 0)
|
||||
h.pluginClient.Kill()
|
||||
|
||||
// Send the results
|
||||
|
|
|
@ -33,7 +33,7 @@ func TestQemuDriver_Fingerprint(t *testing.T) {
|
|||
Resources: structs.DefaultResources(),
|
||||
}
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewQemuDriver(ctx.DriverCtx)
|
||||
|
||||
node := &structs.Node{
|
||||
|
@ -100,7 +100,7 @@ func TestQemuDriver_StartOpen_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewQemuDriver(ctx.DriverCtx)
|
||||
|
||||
// Copy the test image into the task's directory
|
||||
|
@ -181,7 +181,7 @@ func TestQemuDriver_GracefulShutdown(t *testing.T) {
|
|||
|
||||
ctx := testDriverContexts(t, task)
|
||||
ctx.DriverCtx.config.MaxKillTimeout = killTimeout
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewQemuDriver(ctx.DriverCtx)
|
||||
|
||||
request := &cstructs.FingerprintRequest{Config: &config.Config{}, Node: ctx.DriverCtx.node}
|
||||
|
@ -340,7 +340,7 @@ func TestQemuDriverUser(t *testing.T) {
|
|||
|
||||
for _, task := range tasks {
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewQemuDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -391,7 +391,7 @@ func TestQemuDriverGetMonitorPathOldQemu(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
|
||||
// Simulate an older version of qemu which does not support long monitor socket paths
|
||||
ctx.DriverCtx.node.Attributes[qemuDriverVersionAttr] = "2.0.0"
|
||||
|
@ -450,7 +450,7 @@ func TestQemuDriverGetMonitorPathNewQemu(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
|
||||
// Simulate a version of qemu which supports long monitor socket paths
|
||||
ctx.DriverCtx.node.Attributes[qemuDriverVersionAttr] = "2.99.99"
|
||||
|
|
|
@ -50,18 +50,18 @@ type RawExecDriver struct {
|
|||
|
||||
// rawExecHandle is returned from Start/Open as a handle to the PID
|
||||
type rawExecHandle struct {
|
||||
version string
|
||||
pluginClient *plugin.Client
|
||||
userPid int
|
||||
executor executor.Executor
|
||||
isolationConfig *dstructs.IsolationConfig
|
||||
killTimeout time.Duration
|
||||
maxKillTimeout time.Duration
|
||||
logger *log.Logger
|
||||
waitCh chan *dstructs.WaitResult
|
||||
doneCh chan struct{}
|
||||
taskEnv *env.TaskEnv
|
||||
taskDir *allocdir.TaskDir
|
||||
version string
|
||||
pluginClient *plugin.Client
|
||||
userPid int
|
||||
executor executor.Executor
|
||||
killTimeout time.Duration
|
||||
maxKillTimeout time.Duration
|
||||
shutdownSignal string
|
||||
logger *log.Logger
|
||||
waitCh chan *dstructs.WaitResult
|
||||
doneCh chan struct{}
|
||||
taskEnv *env.TaskEnv
|
||||
taskDir *allocdir.TaskDir
|
||||
}
|
||||
|
||||
// NewRawExecDriver is used to create a new raw exec driver
|
||||
|
@ -151,31 +151,23 @@ func (d *RawExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespo
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
executorCtx := &executor.ExecutorContext{
|
||||
TaskEnv: ctx.TaskEnv,
|
||||
Driver: "raw_exec",
|
||||
Task: task,
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
LogDir: ctx.TaskDir.LogDir,
|
||||
}
|
||||
if err := exec.SetContext(executorCtx); err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, fmt.Errorf("failed to set executor context: %v", err)
|
||||
}
|
||||
|
||||
taskKillSignal, err := getTaskKillSignal(task.KillSignal)
|
||||
_, err = getTaskKillSignal(task.KillSignal)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
execCmd := &executor.ExecCommand{
|
||||
Cmd: command,
|
||||
Args: driverConfig.Args,
|
||||
Args: ctx.TaskEnv.ParseAndReplace(driverConfig.Args),
|
||||
User: task.User,
|
||||
TaskKillSignal: taskKillSignal,
|
||||
BasicProcessCgroup: d.useCgroup,
|
||||
Env: ctx.TaskEnv.List(),
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
StdoutPath: ctx.StdoutFifo,
|
||||
StderrPath: ctx.StderrFifo,
|
||||
}
|
||||
ps, err := exec.LaunchCmd(execCmd)
|
||||
ps, err := exec.Launch(execCmd)
|
||||
if err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, err
|
||||
|
@ -185,18 +177,18 @@ func (d *RawExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespo
|
|||
// Return a driver handle
|
||||
maxKill := d.DriverContext.config.MaxKillTimeout
|
||||
h := &rawExecHandle{
|
||||
pluginClient: pluginClient,
|
||||
executor: exec,
|
||||
isolationConfig: ps.IsolationConfig,
|
||||
userPid: ps.Pid,
|
||||
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
|
||||
maxKillTimeout: maxKill,
|
||||
version: d.config.Version.VersionNumber(),
|
||||
logger: d.logger,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
taskEnv: ctx.TaskEnv,
|
||||
taskDir: ctx.TaskDir,
|
||||
pluginClient: pluginClient,
|
||||
executor: exec,
|
||||
userPid: ps.Pid,
|
||||
shutdownSignal: task.KillSignal,
|
||||
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
|
||||
maxKillTimeout: maxKill,
|
||||
version: d.config.Version.VersionNumber(),
|
||||
logger: d.logger,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
taskEnv: ctx.TaskEnv,
|
||||
taskDir: ctx.TaskDir,
|
||||
}
|
||||
go h.run()
|
||||
return &StartResponse{Handle: h}, nil
|
||||
|
@ -205,12 +197,12 @@ func (d *RawExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartRespo
|
|||
func (d *RawExecDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }
|
||||
|
||||
type rawExecId struct {
|
||||
Version string
|
||||
KillTimeout time.Duration
|
||||
MaxKillTimeout time.Duration
|
||||
UserPid int
|
||||
PluginConfig *PluginReattachConfig
|
||||
IsolationConfig *dstructs.IsolationConfig
|
||||
Version string
|
||||
KillTimeout time.Duration
|
||||
MaxKillTimeout time.Duration
|
||||
UserPid int
|
||||
PluginConfig *PluginReattachConfig
|
||||
ShutdownSignal string
|
||||
}
|
||||
|
||||
func (d *RawExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error) {
|
||||
|
@ -230,12 +222,6 @@ func (d *RawExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, e
|
|||
if e := destroyPlugin(id.PluginConfig.Pid, id.UserPid); e != nil {
|
||||
merrs.Errors = append(merrs.Errors, fmt.Errorf("error destroying plugin and userpid: %v", e))
|
||||
}
|
||||
if id.IsolationConfig != nil {
|
||||
ePid := pluginConfig.Reattach.Pid
|
||||
if e := executor.ClientCleanup(id.IsolationConfig, ePid); e != nil {
|
||||
merrs.Errors = append(merrs.Errors, fmt.Errorf("destroying resource container failed: %v", e))
|
||||
}
|
||||
}
|
||||
return nil, fmt.Errorf("error connecting to plugin: %v", merrs.ErrorOrNil())
|
||||
}
|
||||
|
||||
|
@ -244,18 +230,18 @@ func (d *RawExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, e
|
|||
|
||||
// Return a driver handle
|
||||
h := &rawExecHandle{
|
||||
pluginClient: pluginClient,
|
||||
executor: exec,
|
||||
userPid: id.UserPid,
|
||||
isolationConfig: id.IsolationConfig,
|
||||
logger: d.logger,
|
||||
killTimeout: id.KillTimeout,
|
||||
maxKillTimeout: id.MaxKillTimeout,
|
||||
version: id.Version,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
taskEnv: ctx.TaskEnv,
|
||||
taskDir: ctx.TaskDir,
|
||||
pluginClient: pluginClient,
|
||||
executor: exec,
|
||||
userPid: id.UserPid,
|
||||
logger: d.logger,
|
||||
shutdownSignal: id.ShutdownSignal,
|
||||
killTimeout: id.KillTimeout,
|
||||
maxKillTimeout: id.MaxKillTimeout,
|
||||
version: id.Version,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
taskEnv: ctx.TaskEnv,
|
||||
taskDir: ctx.TaskDir,
|
||||
}
|
||||
go h.run()
|
||||
return h, nil
|
||||
|
@ -263,12 +249,12 @@ func (d *RawExecDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, e
|
|||
|
||||
func (h *rawExecHandle) ID() string {
|
||||
id := rawExecId{
|
||||
Version: h.version,
|
||||
KillTimeout: h.killTimeout,
|
||||
MaxKillTimeout: h.maxKillTimeout,
|
||||
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
|
||||
UserPid: h.userPid,
|
||||
IsolationConfig: h.isolationConfig,
|
||||
Version: h.version,
|
||||
KillTimeout: h.killTimeout,
|
||||
MaxKillTimeout: h.maxKillTimeout,
|
||||
PluginConfig: NewPluginReattachConfig(h.pluginClient.ReattachConfig()),
|
||||
UserPid: h.userPid,
|
||||
ShutdownSignal: h.shutdownSignal,
|
||||
}
|
||||
|
||||
data, err := json.Marshal(id)
|
||||
|
@ -285,14 +271,13 @@ func (h *rawExecHandle) WaitCh() chan *dstructs.WaitResult {
|
|||
func (h *rawExecHandle) Update(task *structs.Task) error {
|
||||
// Store the updated kill timeout.
|
||||
h.killTimeout = GetKillTimeout(task.KillTimeout, h.maxKillTimeout)
|
||||
h.executor.UpdateTask(task)
|
||||
|
||||
// Update is not possible
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *rawExecHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte, int, error) {
|
||||
return executor.ExecScript(ctx, h.taskDir.Dir, h.taskEnv, nil, cmd, args)
|
||||
return executor.ExecScript(ctx, h.taskDir.Dir, h.taskEnv.List(), nil, h.taskEnv.ReplaceEnv(cmd), h.taskEnv.ParseAndReplace(args))
|
||||
}
|
||||
|
||||
func (h *rawExecHandle) Signal(s os.Signal) error {
|
||||
|
@ -304,7 +289,7 @@ func (d *rawExecHandle) Network() *cstructs.DriverNetwork {
|
|||
}
|
||||
|
||||
func (h *rawExecHandle) Kill() error {
|
||||
if err := h.executor.ShutDown(); err != nil {
|
||||
if err := h.executor.Signal(os.Interrupt); err != nil {
|
||||
if h.pluginClient.Exited() {
|
||||
return nil
|
||||
}
|
||||
|
@ -318,7 +303,7 @@ func (h *rawExecHandle) Kill() error {
|
|||
if h.pluginClient.Exited() {
|
||||
return nil
|
||||
}
|
||||
if err := h.executor.Exit(); err != nil {
|
||||
if err := h.executor.Shutdown(h.shutdownSignal, h.killTimeout); err != nil {
|
||||
return fmt.Errorf("executor Exit failed: %v", err)
|
||||
}
|
||||
|
||||
|
@ -334,20 +319,13 @@ func (h *rawExecHandle) run() {
|
|||
ps, werr := h.executor.Wait()
|
||||
close(h.doneCh)
|
||||
if ps.ExitCode == 0 && werr != nil {
|
||||
if h.isolationConfig != nil {
|
||||
ePid := h.pluginClient.ReattachConfig().Pid
|
||||
if e := executor.ClientCleanup(h.isolationConfig, ePid); e != nil {
|
||||
h.logger.Printf("[ERR] driver.raw_exec: destroying resource container failed: %v", e)
|
||||
}
|
||||
} else {
|
||||
if e := killProcess(h.userPid); e != nil {
|
||||
h.logger.Printf("[ERR] driver.raw_exec: error killing user process: %v", e)
|
||||
}
|
||||
if e := killProcess(h.userPid); e != nil {
|
||||
h.logger.Printf("[ERR] driver.raw_exec: error killing user process: %v", e)
|
||||
}
|
||||
}
|
||||
|
||||
// Exit the executor
|
||||
if err := h.executor.Exit(); err != nil {
|
||||
// Destroy the executor
|
||||
if err := h.executor.Shutdown(h.shutdownSignal, 0); err != nil {
|
||||
h.logger.Printf("[ERR] driver.raw_exec: error killing executor: %v", err)
|
||||
}
|
||||
h.pluginClient.Kill()
|
||||
|
|
|
@ -30,7 +30,7 @@ func TestRawExecDriver_Fingerprint(t *testing.T) {
|
|||
Resources: structs.DefaultResources(),
|
||||
}
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
node := &structs.Node{
|
||||
Attributes: make(map[string]string),
|
||||
|
@ -83,7 +83,7 @@ func TestRawExecDriver_StartOpen_Wait(t *testing.T) {
|
|||
}
|
||||
testtask.SetTaskEnv(task)
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -130,7 +130,7 @@ func TestRawExecDriver_Start_Wait(t *testing.T) {
|
|||
}
|
||||
testtask.SetTaskEnv(task)
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -182,7 +182,7 @@ func TestRawExecDriver_Start_Wait_AllocDir(t *testing.T) {
|
|||
testtask.SetTaskEnv(task)
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -233,7 +233,7 @@ func TestRawExecDriver_Start_Kill_Wait(t *testing.T) {
|
|||
testtask.SetTaskEnv(task)
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -290,7 +290,7 @@ func TestRawExecDriver_Start_Kill_Wait_Cgroup(t *testing.T) {
|
|||
|
||||
ctx := testDriverContexts(t, task)
|
||||
ctx.DriverCtx.node.Attributes["unique.cgroup.mountpoint"] = "foo" // Enable cgroups
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -377,7 +377,7 @@ func TestRawExecDriver_HandlerExec(t *testing.T) {
|
|||
}
|
||||
testtask.SetTaskEnv(task)
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
|
|
@ -3,6 +3,7 @@
|
|||
package driver
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
|
@ -14,6 +15,7 @@ import (
|
|||
"github.com/hashicorp/nomad/helper/testtask"
|
||||
"github.com/hashicorp/nomad/nomad/structs"
|
||||
"github.com/hashicorp/nomad/testutil"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestRawExecDriver_User(t *testing.T) {
|
||||
|
@ -38,7 +40,7 @@ func TestRawExecDriver_User(t *testing.T) {
|
|||
testtask.SetTaskEnv(task)
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -73,7 +75,7 @@ func TestRawExecDriver_Signal(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRawExecDriver(ctx.DriverCtx)
|
||||
|
||||
testFile := filepath.Join(ctx.ExecCtx.TaskDir.Dir, "test.sh")
|
||||
|
@ -119,14 +121,17 @@ done
|
|||
|
||||
// Check the log file to see it exited because of the signal
|
||||
outputFile := filepath.Join(ctx.ExecCtx.TaskDir.LogDir, "signal.stdout.0")
|
||||
act, err := ioutil.ReadFile(outputFile)
|
||||
if err != nil {
|
||||
t.Fatalf("Couldn't read expected output: %v", err)
|
||||
}
|
||||
|
||||
exp := "Terminated."
|
||||
if strings.TrimSpace(string(act)) != exp {
|
||||
t.Logf("Read from %v", outputFile)
|
||||
t.Fatalf("Command outputted %v; want %v", act, exp)
|
||||
}
|
||||
testutil.WaitForResult(func() (bool, error) {
|
||||
act, err := ioutil.ReadFile(outputFile)
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("Couldn't read expected output: %v", err)
|
||||
}
|
||||
|
||||
if strings.TrimSpace(string(act)) != exp {
|
||||
t.Logf("Read from %v", outputFile)
|
||||
return false, fmt.Errorf("Command outputted %v; want %v", act, exp)
|
||||
}
|
||||
return true, nil
|
||||
}, func(err error) { require.NoError(t, err) })
|
||||
}
|
||||
|
|
|
@ -105,6 +105,7 @@ type rktHandle struct {
|
|||
logger *log.Logger
|
||||
killTimeout time.Duration
|
||||
maxKillTimeout time.Duration
|
||||
shutdownSignal string
|
||||
waitCh chan *dstructs.WaitResult
|
||||
doneCh chan struct{}
|
||||
}
|
||||
|
@ -117,6 +118,7 @@ type rktPID struct {
|
|||
ExecutorPid int
|
||||
KillTimeout time.Duration
|
||||
MaxKillTimeout time.Duration
|
||||
ShutdownSignal string
|
||||
}
|
||||
|
||||
// Retrieve pod status for the pod with the given UUID.
|
||||
|
@ -656,16 +658,10 @@ func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse,
|
|||
eb := env.NewEmptyBuilder()
|
||||
filter := strings.Split(d.config.ReadDefault("env.blacklist", config.DefaultEnvBlacklist), ",")
|
||||
rktEnv := eb.SetHostEnvvars(filter).Build()
|
||||
executorCtx := &executor.ExecutorContext{
|
||||
TaskEnv: rktEnv,
|
||||
Driver: "rkt",
|
||||
Task: task,
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
LogDir: ctx.TaskDir.LogDir,
|
||||
}
|
||||
if err := execIntf.SetContext(executorCtx); err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, fmt.Errorf("failed to set executor context: %v", err)
|
||||
|
||||
_, err = getTaskKillSignal(task.KillSignal)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Enable ResourceLimits to place the executor in a parent cgroup of
|
||||
|
@ -675,8 +671,18 @@ func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse,
|
|||
Cmd: absPath,
|
||||
Args: runArgs,
|
||||
ResourceLimits: true,
|
||||
Resources: &executor.Resources{
|
||||
CPU: task.Resources.CPU,
|
||||
MemoryMB: task.Resources.MemoryMB,
|
||||
IOPS: task.Resources.IOPS,
|
||||
DiskMB: task.Resources.DiskMB,
|
||||
},
|
||||
Env: ctx.TaskEnv.List(),
|
||||
TaskDir: ctx.TaskDir.Dir,
|
||||
StdoutPath: ctx.StdoutFifo,
|
||||
StderrPath: ctx.StderrFifo,
|
||||
}
|
||||
ps, err := execIntf.LaunchCmd(execCmd)
|
||||
ps, err := execIntf.Launch(execCmd)
|
||||
if err != nil {
|
||||
pluginClient.Kill()
|
||||
return nil, err
|
||||
|
@ -694,6 +700,7 @@ func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse,
|
|||
logger: d.logger,
|
||||
killTimeout: GetKillTimeout(task.KillTimeout, maxKill),
|
||||
maxKillTimeout: maxKill,
|
||||
shutdownSignal: task.KillSignal,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
}
|
||||
|
@ -762,6 +769,7 @@ func (d *RktDriver) Open(ctx *ExecContext, handleID string) (DriverHandle, error
|
|||
logger: d.logger,
|
||||
killTimeout: id.KillTimeout,
|
||||
maxKillTimeout: id.MaxKillTimeout,
|
||||
shutdownSignal: id.ShutdownSignal,
|
||||
doneCh: make(chan struct{}),
|
||||
waitCh: make(chan *dstructs.WaitResult, 1),
|
||||
}
|
||||
|
@ -777,6 +785,7 @@ func (h *rktHandle) ID() string {
|
|||
KillTimeout: h.killTimeout,
|
||||
MaxKillTimeout: h.maxKillTimeout,
|
||||
ExecutorPid: h.executorPid,
|
||||
ShutdownSignal: h.shutdownSignal,
|
||||
}
|
||||
data, err := json.Marshal(pid)
|
||||
if err != nil {
|
||||
|
@ -792,7 +801,6 @@ func (h *rktHandle) WaitCh() chan *dstructs.WaitResult {
|
|||
func (h *rktHandle) Update(task *structs.Task) error {
|
||||
// Store the updated kill timeout.
|
||||
h.killTimeout = GetKillTimeout(task.KillTimeout, h.maxKillTimeout)
|
||||
h.executor.UpdateTask(task)
|
||||
|
||||
// Update is not possible
|
||||
return nil
|
||||
|
@ -806,9 +814,9 @@ func (h *rktHandle) Exec(ctx context.Context, cmd string, args []string) ([]byte
|
|||
enterArgs := make([]string, 3+len(args))
|
||||
enterArgs[0] = "enter"
|
||||
enterArgs[1] = h.uuid
|
||||
enterArgs[2] = cmd
|
||||
copy(enterArgs[3:], args)
|
||||
return executor.ExecScript(ctx, h.taskDir.Dir, h.env, nil, rktCmd, enterArgs)
|
||||
enterArgs[2] = h.env.ReplaceEnv(cmd)
|
||||
copy(enterArgs[3:], h.env.ParseAndReplace(args))
|
||||
return executor.ExecScript(ctx, h.taskDir.Dir, h.env.List(), nil, rktCmd, enterArgs)
|
||||
}
|
||||
|
||||
func (h *rktHandle) Signal(s os.Signal) error {
|
||||
|
@ -823,13 +831,7 @@ func (d *rktHandle) Network() *cstructs.DriverNetwork {
|
|||
// Kill is used to terminate the task. We send an Interrupt
|
||||
// and then provide a 5 second grace period before doing a Kill.
|
||||
func (h *rktHandle) Kill() error {
|
||||
h.executor.ShutDown()
|
||||
select {
|
||||
case <-h.doneCh:
|
||||
return nil
|
||||
case <-time.After(h.killTimeout):
|
||||
return h.executor.Exit()
|
||||
}
|
||||
return h.executor.Shutdown(h.shutdownSignal, h.killTimeout)
|
||||
}
|
||||
|
||||
func (h *rktHandle) Stats() (*cstructs.TaskResourceUsage, error) {
|
||||
|
@ -845,8 +847,8 @@ func (h *rktHandle) run() {
|
|||
}
|
||||
}
|
||||
|
||||
// Exit the executor
|
||||
if err := h.executor.Exit(); err != nil {
|
||||
// Destroy the executor
|
||||
if err := h.executor.Shutdown(h.shutdownSignal, 0); err != nil {
|
||||
h.logger.Printf("[ERR] driver.rkt: error killing executor: %v", err)
|
||||
}
|
||||
h.pluginClient.Kill()
|
||||
|
|
|
@ -107,7 +107,7 @@ func TestRktDriver_Start_DNS(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -158,7 +158,7 @@ func TestRktDriver_Start_Wait(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -230,7 +230,7 @@ func TestRktDriver_Start_Wait_Skip_Trust(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -297,7 +297,7 @@ func TestRktDriver_Start_Wait_AllocDir(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -359,7 +359,7 @@ func TestRktDriver_UserGroup(t *testing.T) {
|
|||
}
|
||||
|
||||
tctx := testDriverContexts(t, task)
|
||||
defer tctx.AllocDir.Destroy()
|
||||
defer tctx.Destroy()
|
||||
d := NewRktDriver(tctx.DriverCtx)
|
||||
|
||||
_, err := d.Prestart(tctx.ExecCtx, task)
|
||||
|
@ -414,7 +414,7 @@ func TestRktTrustPrefix(t *testing.T) {
|
|||
},
|
||||
}
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -449,7 +449,7 @@ func TestRktTaskValidate(t *testing.T) {
|
|||
Resources: basicResources,
|
||||
}
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if err := d.Validate(task.Config); err != nil {
|
||||
|
@ -492,7 +492,7 @@ func TestRktDriver_PortMapping(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -556,7 +556,7 @@ func TestRktDriver_PortsMapping_Host(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -613,7 +613,7 @@ func TestRktDriver_HandlerExec(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
@ -685,7 +685,7 @@ func TestRktDriver_Stats(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := testDriverContexts(t, task)
|
||||
defer ctx.AllocDir.Destroy()
|
||||
defer ctx.Destroy()
|
||||
d := NewRktDriver(ctx.DriverCtx)
|
||||
|
||||
if _, err := d.Prestart(ctx.ExecCtx, task); err != nil {
|
||||
|
|
|
@ -64,4 +64,8 @@ type ExecutorConfig struct {
|
|||
|
||||
// LogLevel is the level of the logs to putout
|
||||
LogLevel string
|
||||
|
||||
// FSIsolation if set will use an executor implementation that support
|
||||
// filesystem isolation
|
||||
FSIsolation bool
|
||||
}
|
||||
|
|
|
@ -1,10 +0,0 @@
|
|||
package structs
|
||||
|
||||
import cgroupConfig "github.com/opencontainers/runc/libcontainer/configs"
|
||||
|
||||
// IsolationConfig has information about the isolation mechanism the executor
|
||||
// uses to put resource constraints and isolation on the user process
|
||||
type IsolationConfig struct {
|
||||
Cgroup *cgroupConfig.Cgroup
|
||||
CgroupPaths map[string]string
|
||||
}
|
|
@ -11,6 +11,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/hashicorp/consul-template/signals"
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/go-multierror"
|
||||
"github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/nomad/client/allocdir"
|
||||
|
@ -48,7 +49,7 @@ func createExecutor(w io.Writer, clientConfig *config.Config,
|
|||
Cmd: exec.Command(bin, "executor", string(c)),
|
||||
}
|
||||
config.HandshakeConfig = HandshakeConfig
|
||||
config.Plugins = GetPluginMap(w, clientConfig.LogLevel)
|
||||
config.Plugins = GetPluginMap(w, hclog.LevelFromString(clientConfig.LogLevel), executorConfig.FSIsolation)
|
||||
config.MaxPort = clientConfig.ClientMaxPort
|
||||
config.MinPort = clientConfig.ClientMinPort
|
||||
|
||||
|
@ -77,7 +78,7 @@ func createExecutorWithConfig(config *plugin.ClientConfig, w io.Writer) (executo
|
|||
|
||||
// Setting this to DEBUG since the log level at the executor server process
|
||||
// is already set, and this effects only the executor client.
|
||||
config.Plugins = GetPluginMap(w, "DEBUG")
|
||||
config.Plugins = GetPluginMap(w, hclog.Debug, false)
|
||||
|
||||
executorClient := plugin.NewClient(config)
|
||||
rpcClient, err := executorClient.Client()
|
||||
|
@ -93,11 +94,6 @@ func createExecutorWithConfig(config *plugin.ClientConfig, w io.Writer) (executo
|
|||
if !ok {
|
||||
return nil, nil, fmt.Errorf("unexpected executor rpc type: %T", raw)
|
||||
}
|
||||
// 0.6 Upgrade path: Deregister services from the executor as the Nomad
|
||||
// client agent now handles all Consul interactions. Ignore errors as
|
||||
// this shouldn't cause the alloc to fail and there's nothing useful to
|
||||
// do with them.
|
||||
executorPlugin.DeregisterServices()
|
||||
return executorPlugin, executorClient, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,31 @@
|
|||
package logmon
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/hashicorp/nomad/client/logmon/proto"
|
||||
)
|
||||
|
||||
type logmonClient struct {
|
||||
client proto.LogMonClient
|
||||
}
|
||||
|
||||
func (c *logmonClient) Start(cfg *LogConfig) error {
|
||||
req := &proto.StartRequest{
|
||||
LogDir: cfg.LogDir,
|
||||
StdoutFileName: cfg.StdoutLogFile,
|
||||
StderrFileName: cfg.StderrLogFile,
|
||||
MaxFiles: uint32(cfg.MaxFiles),
|
||||
MaxFileSizeMb: uint32(cfg.MaxFileSizeMB),
|
||||
StdoutFifo: cfg.StdoutFifo,
|
||||
StderrFifo: cfg.StderrFifo,
|
||||
}
|
||||
_, err := c.client.Start(context.Background(), req)
|
||||
return err
|
||||
}
|
||||
|
||||
func (c *logmonClient) Stop() error {
|
||||
req := &proto.StopRequest{}
|
||||
_, err := c.client.Stop(context.Background(), req)
|
||||
return err
|
||||
}
|
|
@ -0,0 +1,209 @@
|
|||
package logmon
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/nomad/client/driver/logging"
|
||||
"github.com/hashicorp/nomad/client/lib/fifo"
|
||||
)
|
||||
|
||||
const (
|
||||
// processOutputCloseTolerance is the length of time we will wait for the
|
||||
// launched process to close its stdout/stderr before we force close it. If
|
||||
// data is written after this tolerance, we will not capture it.
|
||||
processOutputCloseTolerance = 2 * time.Second
|
||||
)
|
||||
|
||||
type LogConfig struct {
|
||||
// LogDir is the host path where logs are to be written to
|
||||
LogDir string
|
||||
|
||||
// StdoutLogFile is the path relative to LogDir for stdout logging
|
||||
StdoutLogFile string
|
||||
|
||||
// StderrLogFile is the path relative to LogDir for stderr logging
|
||||
StderrLogFile string
|
||||
|
||||
// StdoutFifo is the path on the host to the stdout pipe
|
||||
StdoutFifo string
|
||||
|
||||
// StderrFifo is the path on the host to the stderr pipe
|
||||
StderrFifo string
|
||||
|
||||
// MaxFiles is the max rotated files allowed
|
||||
MaxFiles int
|
||||
|
||||
// MaxFileSizeMB is the max log file size in MB allowed before rotation occures
|
||||
MaxFileSizeMB int
|
||||
}
|
||||
|
||||
type LogMon interface {
|
||||
Start(*LogConfig) error
|
||||
Stop() error
|
||||
}
|
||||
|
||||
func NewLogMon(logger hclog.Logger) LogMon {
|
||||
return &logmonImpl{
|
||||
logger: logger,
|
||||
}
|
||||
}
|
||||
|
||||
type logmonImpl struct {
|
||||
logger hclog.Logger
|
||||
tl *TaskLogger
|
||||
}
|
||||
|
||||
func (l *logmonImpl) Start(cfg *LogConfig) error {
|
||||
tl, err := NewTaskLogger(cfg, l.logger)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
l.tl = tl
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *logmonImpl) Stop() error {
|
||||
l.tl.Close()
|
||||
return nil
|
||||
}
|
||||
|
||||
type TaskLogger struct {
|
||||
config *LogConfig
|
||||
|
||||
// rotator for stdout
|
||||
lro *logRotatorWrapper
|
||||
|
||||
// rotator for stderr
|
||||
lre *logRotatorWrapper
|
||||
}
|
||||
|
||||
func (tl *TaskLogger) Close() {
|
||||
if tl.lro != nil {
|
||||
tl.lro.Close()
|
||||
}
|
||||
if tl.lre != nil {
|
||||
tl.lre.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func NewTaskLogger(cfg *LogConfig, logger hclog.Logger) (*TaskLogger, error) {
|
||||
tl := &TaskLogger{config: cfg}
|
||||
|
||||
logFileSize := int64(cfg.MaxFileSizeMB * 1024 * 1024)
|
||||
lro, err := logging.NewFileRotator(cfg.LogDir, cfg.StdoutLogFile,
|
||||
cfg.MaxFiles, logFileSize, logger)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to create stdout logfile for %q: %v", cfg.StdoutLogFile, err)
|
||||
}
|
||||
|
||||
wrapperOut, err := newLogRotatorWrapper(cfg.StdoutFifo, logger, lro)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tl.lro = wrapperOut
|
||||
|
||||
lre, err := logging.NewFileRotator(cfg.LogDir, cfg.StderrLogFile,
|
||||
cfg.MaxFiles, logFileSize, logger)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to create stderr logfile for %q: %v", cfg.StderrLogFile, err)
|
||||
}
|
||||
|
||||
wrapperErr, err := newLogRotatorWrapper(cfg.StderrFifo, logger, lre)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tl.lre = wrapperErr
|
||||
|
||||
return tl, nil
|
||||
|
||||
}
|
||||
|
||||
// logRotatorWrapper wraps our log rotator and exposes a pipe that can feed the
|
||||
// log rotator data. The processOutWriter should be attached to the process and
|
||||
// data will be copied from the reader to the rotator.
|
||||
type logRotatorWrapper struct {
|
||||
fifoPath string
|
||||
processOutReader io.ReadCloser
|
||||
rotatorWriter *logging.FileRotator
|
||||
hasFinishedCopied chan struct{}
|
||||
logger hclog.Logger
|
||||
}
|
||||
|
||||
// newLogRotatorWrapper takes a rotator and returns a wrapper that has the
|
||||
// processOutWriter to attach to the stdout or stderr of a process.
|
||||
func newLogRotatorWrapper(path string, logger hclog.Logger, rotator *logging.FileRotator) (*logRotatorWrapper, error) {
|
||||
logger.Info("opening fifo", "path", path)
|
||||
f, err := fifo.New(path)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to create fifo for extracting logs: %v", err)
|
||||
}
|
||||
|
||||
wrap := &logRotatorWrapper{
|
||||
fifoPath: path,
|
||||
processOutReader: f,
|
||||
rotatorWriter: rotator,
|
||||
hasFinishedCopied: make(chan struct{}),
|
||||
logger: logger,
|
||||
}
|
||||
wrap.start()
|
||||
return wrap, nil
|
||||
}
|
||||
|
||||
// start starts a goroutine that copies from the pipe into the rotator. This is
|
||||
// called by the constructor and not the user of the wrapper.
|
||||
func (l *logRotatorWrapper) start() {
|
||||
go func() {
|
||||
defer close(l.hasFinishedCopied)
|
||||
_, err := io.Copy(l.rotatorWriter, l.processOutReader)
|
||||
if err != nil {
|
||||
// Close reader to propagate io error across pipe.
|
||||
// Note that this may block until the process exits on
|
||||
// Windows due to
|
||||
// https://github.com/PowerShell/PowerShell/issues/4254
|
||||
// or similar issues. Since this is already running in
|
||||
// a goroutine its safe to block until the process is
|
||||
// force-killed.
|
||||
l.processOutReader.Close()
|
||||
}
|
||||
}()
|
||||
return
|
||||
}
|
||||
|
||||
// Close closes the rotator and the process writer to ensure that the Wait
|
||||
// command exits.
|
||||
func (l *logRotatorWrapper) Close() {
|
||||
// Wait up to the close tolerance before we force close
|
||||
select {
|
||||
case <-l.hasFinishedCopied:
|
||||
case <-time.After(processOutputCloseTolerance):
|
||||
}
|
||||
|
||||
// Closing the read side of a pipe may block on Windows if the process
|
||||
// is being debugged as in:
|
||||
// https://github.com/PowerShell/PowerShell/issues/4254
|
||||
// The pipe will be closed and cleaned up when the process exits.
|
||||
closeDone := make(chan struct{})
|
||||
go func() {
|
||||
defer close(closeDone)
|
||||
err := l.processOutReader.Close()
|
||||
if err != nil && !strings.Contains(err.Error(), "file already closed") {
|
||||
l.logger.Warn("error closing read-side of process output pipe", "err", err)
|
||||
}
|
||||
|
||||
}()
|
||||
|
||||
select {
|
||||
case <-closeDone:
|
||||
case <-time.After(processOutputCloseTolerance):
|
||||
l.logger.Warn("timed out waiting for read-side of process output pipe to close")
|
||||
}
|
||||
|
||||
l.rotatorWriter.Close()
|
||||
return
|
||||
}
|
|
@ -0,0 +1,69 @@
|
|||
package logmon
|
||||
|
||||
import (
|
||||
"context"
|
||||
"os/exec"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
plugin "github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/nomad/client/logmon/proto"
|
||||
"github.com/hashicorp/nomad/helper/discover"
|
||||
"github.com/hashicorp/nomad/plugins/base"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
// LaunchLogMon an instance of logmon
|
||||
// TODO: Integrate with base plugin loader
|
||||
func LaunchLogMon(logger hclog.Logger) (LogMon, *plugin.Client, error) {
|
||||
logger = logger.Named("logmon-launcher")
|
||||
bin, err := discover.NomadExecutable()
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
client := plugin.NewClient(&plugin.ClientConfig{
|
||||
HandshakeConfig: base.Handshake,
|
||||
Plugins: map[string]plugin.Plugin{
|
||||
"logmon": NewPlugin(NewLogMon(hclog.L().Named("logmon"))),
|
||||
},
|
||||
Cmd: exec.Command(bin, "logmon"),
|
||||
AllowedProtocols: []plugin.Protocol{
|
||||
plugin.ProtocolGRPC,
|
||||
},
|
||||
})
|
||||
|
||||
rpcClient, err := client.Client()
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
raw, err := rpcClient.Dispense("logmon")
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
l := raw.(LogMon)
|
||||
return l, client, nil
|
||||
|
||||
}
|
||||
|
||||
type Plugin struct {
|
||||
plugin.NetRPCUnsupportedPlugin
|
||||
impl LogMon
|
||||
}
|
||||
|
||||
func NewPlugin(i LogMon) plugin.Plugin {
|
||||
return &Plugin{impl: i}
|
||||
}
|
||||
|
||||
func (p *Plugin) GRPCServer(broker *plugin.GRPCBroker, s *grpc.Server) error {
|
||||
proto.RegisterLogMonServer(s, &logmonServer{
|
||||
impl: p.impl,
|
||||
broker: broker,
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *Plugin) GRPCClient(ctx context.Context, broker *plugin.GRPCBroker, c *grpc.ClientConn) (interface{}, error) {
|
||||
return &logmonClient{client: proto.NewLogMonClient(c)}, nil
|
||||
}
|
|
@ -0,0 +1,338 @@
|
|||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: logmon.proto
|
||||
|
||||
package proto
|
||||
|
||||
import proto "github.com/golang/protobuf/proto"
|
||||
import fmt "fmt"
|
||||
import math "math"
|
||||
|
||||
import (
|
||||
context "golang.org/x/net/context"
|
||||
grpc "google.golang.org/grpc"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
|
||||
|
||||
type StartRequest struct {
|
||||
LogDir string `protobuf:"bytes,1,opt,name=log_dir,json=logDir,proto3" json:"log_dir,omitempty"`
|
||||
StdoutFileName string `protobuf:"bytes,2,opt,name=stdout_file_name,json=stdoutFileName,proto3" json:"stdout_file_name,omitempty"`
|
||||
StderrFileName string `protobuf:"bytes,3,opt,name=stderr_file_name,json=stderrFileName,proto3" json:"stderr_file_name,omitempty"`
|
||||
MaxFiles uint32 `protobuf:"varint,4,opt,name=max_files,json=maxFiles,proto3" json:"max_files,omitempty"`
|
||||
MaxFileSizeMb uint32 `protobuf:"varint,5,opt,name=max_file_size_mb,json=maxFileSizeMb,proto3" json:"max_file_size_mb,omitempty"`
|
||||
StdoutFifo string `protobuf:"bytes,6,opt,name=stdout_fifo,json=stdoutFifo,proto3" json:"stdout_fifo,omitempty"`
|
||||
StderrFifo string `protobuf:"bytes,7,opt,name=stderr_fifo,json=stderrFifo,proto3" json:"stderr_fifo,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *StartRequest) Reset() { *m = StartRequest{} }
|
||||
func (m *StartRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*StartRequest) ProtoMessage() {}
|
||||
func (*StartRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_logmon_0690d412b596ec9e, []int{0}
|
||||
}
|
||||
func (m *StartRequest) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_StartRequest.Unmarshal(m, b)
|
||||
}
|
||||
func (m *StartRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_StartRequest.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *StartRequest) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_StartRequest.Merge(dst, src)
|
||||
}
|
||||
func (m *StartRequest) XXX_Size() int {
|
||||
return xxx_messageInfo_StartRequest.Size(m)
|
||||
}
|
||||
func (m *StartRequest) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_StartRequest.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_StartRequest proto.InternalMessageInfo
|
||||
|
||||
func (m *StartRequest) GetLogDir() string {
|
||||
if m != nil {
|
||||
return m.LogDir
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *StartRequest) GetStdoutFileName() string {
|
||||
if m != nil {
|
||||
return m.StdoutFileName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *StartRequest) GetStderrFileName() string {
|
||||
if m != nil {
|
||||
return m.StderrFileName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *StartRequest) GetMaxFiles() uint32 {
|
||||
if m != nil {
|
||||
return m.MaxFiles
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *StartRequest) GetMaxFileSizeMb() uint32 {
|
||||
if m != nil {
|
||||
return m.MaxFileSizeMb
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *StartRequest) GetStdoutFifo() string {
|
||||
if m != nil {
|
||||
return m.StdoutFifo
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *StartRequest) GetStderrFifo() string {
|
||||
if m != nil {
|
||||
return m.StderrFifo
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type StartResponse struct {
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *StartResponse) Reset() { *m = StartResponse{} }
|
||||
func (m *StartResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StartResponse) ProtoMessage() {}
|
||||
func (*StartResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_logmon_0690d412b596ec9e, []int{1}
|
||||
}
|
||||
func (m *StartResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_StartResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *StartResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_StartResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *StartResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_StartResponse.Merge(dst, src)
|
||||
}
|
||||
func (m *StartResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_StartResponse.Size(m)
|
||||
}
|
||||
func (m *StartResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_StartResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_StartResponse proto.InternalMessageInfo
|
||||
|
||||
type StopRequest struct {
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *StopRequest) Reset() { *m = StopRequest{} }
|
||||
func (m *StopRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*StopRequest) ProtoMessage() {}
|
||||
func (*StopRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_logmon_0690d412b596ec9e, []int{2}
|
||||
}
|
||||
func (m *StopRequest) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_StopRequest.Unmarshal(m, b)
|
||||
}
|
||||
func (m *StopRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_StopRequest.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *StopRequest) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_StopRequest.Merge(dst, src)
|
||||
}
|
||||
func (m *StopRequest) XXX_Size() int {
|
||||
return xxx_messageInfo_StopRequest.Size(m)
|
||||
}
|
||||
func (m *StopRequest) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_StopRequest.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_StopRequest proto.InternalMessageInfo
|
||||
|
||||
type StopResponse struct {
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *StopResponse) Reset() { *m = StopResponse{} }
|
||||
func (m *StopResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StopResponse) ProtoMessage() {}
|
||||
func (*StopResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_logmon_0690d412b596ec9e, []int{3}
|
||||
}
|
||||
func (m *StopResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_StopResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *StopResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_StopResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *StopResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_StopResponse.Merge(dst, src)
|
||||
}
|
||||
func (m *StopResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_StopResponse.Size(m)
|
||||
}
|
||||
func (m *StopResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_StopResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_StopResponse proto.InternalMessageInfo
|
||||
|
||||
func init() {
|
||||
proto.RegisterType((*StartRequest)(nil), "hashicorp.nomad.client.logmon.proto.StartRequest")
|
||||
proto.RegisterType((*StartResponse)(nil), "hashicorp.nomad.client.logmon.proto.StartResponse")
|
||||
proto.RegisterType((*StopRequest)(nil), "hashicorp.nomad.client.logmon.proto.StopRequest")
|
||||
proto.RegisterType((*StopResponse)(nil), "hashicorp.nomad.client.logmon.proto.StopResponse")
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ context.Context
|
||||
var _ grpc.ClientConn
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the grpc package it is being compiled against.
|
||||
const _ = grpc.SupportPackageIsVersion4
|
||||
|
||||
// LogMonClient is the client API for LogMon service.
|
||||
//
|
||||
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
|
||||
type LogMonClient interface {
|
||||
Start(ctx context.Context, in *StartRequest, opts ...grpc.CallOption) (*StartResponse, error)
|
||||
Stop(ctx context.Context, in *StopRequest, opts ...grpc.CallOption) (*StopResponse, error)
|
||||
}
|
||||
|
||||
type logMonClient struct {
|
||||
cc *grpc.ClientConn
|
||||
}
|
||||
|
||||
func NewLogMonClient(cc *grpc.ClientConn) LogMonClient {
|
||||
return &logMonClient{cc}
|
||||
}
|
||||
|
||||
func (c *logMonClient) Start(ctx context.Context, in *StartRequest, opts ...grpc.CallOption) (*StartResponse, error) {
|
||||
out := new(StartResponse)
|
||||
err := c.cc.Invoke(ctx, "/hashicorp.nomad.client.logmon.proto.LogMon/Start", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *logMonClient) Stop(ctx context.Context, in *StopRequest, opts ...grpc.CallOption) (*StopResponse, error) {
|
||||
out := new(StopResponse)
|
||||
err := c.cc.Invoke(ctx, "/hashicorp.nomad.client.logmon.proto.LogMon/Stop", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
// LogMonServer is the server API for LogMon service.
|
||||
type LogMonServer interface {
|
||||
Start(context.Context, *StartRequest) (*StartResponse, error)
|
||||
Stop(context.Context, *StopRequest) (*StopResponse, error)
|
||||
}
|
||||
|
||||
func RegisterLogMonServer(s *grpc.Server, srv LogMonServer) {
|
||||
s.RegisterService(&_LogMon_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _LogMon_Start_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(StartRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(LogMonServer).Start(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/hashicorp.nomad.client.logmon.proto.LogMon/Start",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(LogMonServer).Start(ctx, req.(*StartRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _LogMon_Stop_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(StopRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(LogMonServer).Stop(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/hashicorp.nomad.client.logmon.proto.LogMon/Stop",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(LogMonServer).Stop(ctx, req.(*StopRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
var _LogMon_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "hashicorp.nomad.client.logmon.proto.LogMon",
|
||||
HandlerType: (*LogMonServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
{
|
||||
MethodName: "Start",
|
||||
Handler: _LogMon_Start_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "Stop",
|
||||
Handler: _LogMon_Stop_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{},
|
||||
Metadata: "logmon.proto",
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("logmon.proto", fileDescriptor_logmon_0690d412b596ec9e) }
|
||||
|
||||
var fileDescriptor_logmon_0690d412b596ec9e = []byte{
|
||||
// 314 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x90, 0x31, 0x6f, 0xf2, 0x30,
|
||||
0x10, 0x86, 0xbf, 0xf0, 0x41, 0x28, 0x07, 0xa1, 0xc8, 0x4b, 0x23, 0x3a, 0x14, 0xa5, 0x43, 0x99,
|
||||
0xa2, 0x42, 0xff, 0x41, 0x55, 0x75, 0x2a, 0x1d, 0x60, 0xeb, 0x12, 0x19, 0xb8, 0x04, 0x4b, 0x71,
|
||||
0x2e, 0xb5, 0x8d, 0x84, 0x58, 0xfb, 0x6b, 0xfb, 0x2f, 0xaa, 0x18, 0x13, 0x65, 0x84, 0x29, 0xca,
|
||||
0xdd, 0xf3, 0xea, 0x1e, 0xbf, 0x30, 0xc8, 0x29, 0x93, 0x54, 0xc4, 0xa5, 0x22, 0x43, 0xec, 0x71,
|
||||
0xc7, 0xf5, 0x4e, 0x6c, 0x48, 0x95, 0x71, 0x41, 0x92, 0x6f, 0xe3, 0x4d, 0x2e, 0xb0, 0x30, 0x71,
|
||||
0x13, 0x8a, 0x7e, 0x5a, 0x30, 0x58, 0x19, 0xae, 0xcc, 0x12, 0xbf, 0xf7, 0xa8, 0x0d, 0xbb, 0x83,
|
||||
0x6e, 0x4e, 0x59, 0xb2, 0x15, 0x2a, 0xf4, 0x26, 0xde, 0xb4, 0xb7, 0xf4, 0x73, 0xca, 0xde, 0x84,
|
||||
0x62, 0x53, 0x18, 0x69, 0xb3, 0xa5, 0xbd, 0x49, 0x52, 0x91, 0x63, 0x52, 0x70, 0x89, 0x61, 0xcb,
|
||||
0x12, 0xc3, 0xd3, 0xfc, 0x5d, 0xe4, 0xf8, 0xc9, 0x25, 0x3a, 0x12, 0x95, 0x6a, 0x90, 0xff, 0x6b,
|
||||
0x12, 0x95, 0xaa, 0xc9, 0x7b, 0xe8, 0x49, 0x7e, 0xb0, 0x98, 0x0e, 0xdb, 0x13, 0x6f, 0x1a, 0x2c,
|
||||
0x6f, 0x24, 0x3f, 0x54, 0x7b, 0xcd, 0x9e, 0x60, 0x74, 0x5e, 0x26, 0x5a, 0x1c, 0x31, 0x91, 0xeb,
|
||||
0xb0, 0x63, 0x99, 0xc0, 0x31, 0x2b, 0x71, 0xc4, 0xc5, 0x9a, 0x3d, 0x40, 0xbf, 0x36, 0x4b, 0x29,
|
||||
0xf4, 0xed, 0x29, 0x38, 0x4b, 0xa5, 0xe4, 0x80, 0x93, 0x50, 0x4a, 0x61, 0xb7, 0x06, 0xac, 0x4b,
|
||||
0x4a, 0xd1, 0x2d, 0x04, 0xae, 0x04, 0x5d, 0x52, 0xa1, 0x31, 0x0a, 0xa0, 0xbf, 0x32, 0x54, 0xba,
|
||||
0x52, 0xa2, 0x61, 0x55, 0x52, 0xf5, 0x7b, 0x5a, 0xcf, 0x7f, 0x3d, 0xf0, 0x3f, 0x28, 0x5b, 0x50,
|
||||
0xc1, 0x4a, 0xe8, 0xd8, 0x28, 0x9b, 0xc5, 0x17, 0xf4, 0x1d, 0x37, 0xbb, 0x1e, 0xcf, 0xaf, 0x89,
|
||||
0x38, 0xb3, 0x7f, 0x4c, 0x42, 0xbb, 0x92, 0x61, 0xcf, 0x17, 0xa6, 0xeb, 0x67, 0x8c, 0x67, 0x57,
|
||||
0x24, 0xce, 0xe7, 0x5e, 0xbb, 0x5f, 0x1d, 0x3b, 0x5f, 0xfb, 0xf6, 0xf3, 0xf2, 0x17, 0x00, 0x00,
|
||||
0xff, 0xff, 0x1b, 0x8f, 0x8b, 0x51, 0x66, 0x02, 0x00, 0x00,
|
||||
}
|
|
@ -0,0 +1,25 @@
|
|||
syntax = "proto3";
|
||||
package hashicorp.nomad.client.logmon.proto;
|
||||
option go_package = "proto";
|
||||
|
||||
service LogMon {
|
||||
rpc Start(StartRequest) returns (StartResponse) {}
|
||||
rpc Stop(StopRequest) returns (StopResponse) {}
|
||||
}
|
||||
|
||||
message StartRequest {
|
||||
string log_dir = 1;
|
||||
string stdout_file_name = 2;
|
||||
string stderr_file_name = 3;
|
||||
uint32 max_files = 4;
|
||||
uint32 max_file_size_mb = 5;
|
||||
string stdout_fifo = 6;
|
||||
string stderr_fifo = 7;
|
||||
}
|
||||
|
||||
message StartResponse {
|
||||
}
|
||||
|
||||
message StopRequest {}
|
||||
|
||||
message StopResponse {}
|
|
@ -0,0 +1,36 @@
|
|||
package logmon
|
||||
|
||||
import (
|
||||
"golang.org/x/net/context"
|
||||
|
||||
plugin "github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/nomad/client/logmon/proto"
|
||||
)
|
||||
|
||||
type logmonServer struct {
|
||||
broker *plugin.GRPCBroker
|
||||
impl LogMon
|
||||
}
|
||||
|
||||
func (s *logmonServer) Start(ctx context.Context, req *proto.StartRequest) (*proto.StartResponse, error) {
|
||||
cfg := &LogConfig{
|
||||
LogDir: req.LogDir,
|
||||
StdoutLogFile: req.StdoutFileName,
|
||||
StderrLogFile: req.StderrFileName,
|
||||
MaxFiles: int(req.MaxFiles),
|
||||
MaxFileSizeMB: int(req.MaxFileSizeMb),
|
||||
StdoutFifo: req.StdoutFifo,
|
||||
StderrFifo: req.StderrFifo,
|
||||
}
|
||||
|
||||
err := s.impl.Start(cfg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
resp := &proto.StartResponse{}
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
func (s *logmonServer) Stop(ctx context.Context, req *proto.StopRequest) (*proto.StopResponse, error) {
|
||||
return &proto.StopResponse{}, s.impl.Stop()
|
||||
}
|
|
@ -325,6 +325,11 @@ func Commands(metaPtr *Meta, agentUi cli.Ui) map[string]cli.CommandFactory {
|
|||
Meta: meta,
|
||||
}, nil
|
||||
},
|
||||
"logmon": func() (cli.Command, error) {
|
||||
return &LogMonPluginCommand{
|
||||
Meta: meta,
|
||||
}, nil
|
||||
},
|
||||
"logs": func() (cli.Command, error) {
|
||||
return &AllocLogsCommand{
|
||||
Meta: meta,
|
||||
|
|
|
@ -5,6 +5,7 @@ import (
|
|||
"os"
|
||||
"strings"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/go-plugin"
|
||||
|
||||
"github.com/hashicorp/nomad/client/driver"
|
||||
|
@ -43,7 +44,11 @@ func (e *ExecutorPluginCommand) Run(args []string) int {
|
|||
}
|
||||
plugin.Serve(&plugin.ServeConfig{
|
||||
HandshakeConfig: driver.HandshakeConfig,
|
||||
Plugins: driver.GetPluginMap(stdo, executorConfig.LogLevel),
|
||||
Plugins: driver.GetPluginMap(
|
||||
stdo,
|
||||
hclog.LevelFromString(executorConfig.LogLevel),
|
||||
executorConfig.FSIsolation,
|
||||
),
|
||||
})
|
||||
return 0
|
||||
}
|
||||
|
|
|
@ -0,0 +1,36 @@
|
|||
package command
|
||||
|
||||
import (
|
||||
"strings"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
plugin "github.com/hashicorp/go-plugin"
|
||||
"github.com/hashicorp/nomad/client/logmon"
|
||||
"github.com/hashicorp/nomad/plugins/base"
|
||||
)
|
||||
|
||||
type LogMonPluginCommand struct {
|
||||
Meta
|
||||
}
|
||||
|
||||
func (e *LogMonPluginCommand) Help() string {
|
||||
helpText := `
|
||||
This is a command used by Nomad internally to launch the logmon process"
|
||||
`
|
||||
return strings.TrimSpace(helpText)
|
||||
}
|
||||
|
||||
func (e *LogMonPluginCommand) Synopsis() string {
|
||||
return "internal - launch a logmon plugin"
|
||||
}
|
||||
|
||||
func (e *LogMonPluginCommand) Run(args []string) int {
|
||||
plugin.Serve(&plugin.ServeConfig{
|
||||
HandshakeConfig: base.Handshake,
|
||||
Plugins: map[string]plugin.Plugin{
|
||||
"logmon": logmon.NewPlugin(logmon.NewLogMon(hclog.Default().Named("logmon"))),
|
||||
},
|
||||
GRPCServer: plugin.DefaultGRPCServer,
|
||||
})
|
||||
return 0
|
||||
}
|
1
main.go
1
main.go
|
@ -29,6 +29,7 @@ var (
|
|||
"executor",
|
||||
"keygen",
|
||||
"keyring",
|
||||
"logmon",
|
||||
"node-drain",
|
||||
"node-status",
|
||||
"server-force-leave",
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"os"
|
||||
"runtime"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/opencontainers/runc/libcontainer"
|
||||
_ "github.com/opencontainers/runc/libcontainer/nsenter"
|
||||
)
|
||||
|
||||
// init is only run on linux and is used when the LibcontainerExecutor starts
|
||||
// a new process. The libcontainer shim takes over the process, setting up the
|
||||
// configured isolation and limitions before execve into the user process
|
||||
func init() {
|
||||
if len(os.Args) > 1 && os.Args[1] == "libcontainer-shim" {
|
||||
runtime.GOMAXPROCS(1)
|
||||
runtime.LockOSThread()
|
||||
factory, _ := libcontainer.New("")
|
||||
if err := factory.StartInitialization(); err != nil {
|
||||
hclog.L().Error("failed to initialize libcontainer-shim", "error", err)
|
||||
}
|
||||
panic("--this line should have never been executed, congratulations--")
|
||||
}
|
||||
}
|
|
@ -452,8 +452,7 @@ func (c *linuxContainer) newParentProcess(p *Process) (parentProcess, error) {
|
|||
}
|
||||
|
||||
func (c *linuxContainer) commandTemplate(p *Process, childPipe *os.File) (*exec.Cmd, error) {
|
||||
cmd := exec.Command(c.initPath, c.initArgs[1:]...)
|
||||
cmd.Args[0] = c.initArgs[0]
|
||||
cmd := exec.Command(c.initPath, c.initArgs...)
|
||||
cmd.Stdin = p.Stdin
|
||||
cmd.Stdout = p.Stdout
|
||||
cmd.Stderr = p.Stderr
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
"runtime/debug"
|
||||
"strconv"
|
||||
|
||||
"github.com/cyphar/filepath-securejoin"
|
||||
"github.com/opencontainers/runc/libcontainer/cgroups"
|
||||
"github.com/opencontainers/runc/libcontainer/cgroups/fs"
|
||||
"github.com/opencontainers/runc/libcontainer/cgroups/systemd"
|
||||
|
@ -42,7 +43,10 @@ func InitArgs(args ...string) func(*LinuxFactory) error {
|
|||
}
|
||||
}
|
||||
|
||||
l.InitArgs = args
|
||||
l.InitPath = args[0]
|
||||
if len(args) > 1 {
|
||||
l.InitArgs = args[1:]
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
@ -195,7 +199,10 @@ func (l *LinuxFactory) Create(id string, config *configs.Config) (Container, err
|
|||
if err := l.Validator.Validate(config); err != nil {
|
||||
return nil, newGenericError(err, ConfigInvalid)
|
||||
}
|
||||
containerRoot := filepath.Join(l.Root, id)
|
||||
containerRoot, err := securejoin.SecureJoin(l.Root, id)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if _, err := os.Stat(containerRoot); err == nil {
|
||||
return nil, newGenericError(fmt.Errorf("container with id exists: %v", id), IdInUse)
|
||||
} else if !os.IsNotExist(err) {
|
||||
|
@ -229,7 +236,14 @@ func (l *LinuxFactory) Load(id string) (Container, error) {
|
|||
if l.Root == "" {
|
||||
return nil, newGenericError(fmt.Errorf("invalid root"), ConfigInvalid)
|
||||
}
|
||||
containerRoot := filepath.Join(l.Root, id)
|
||||
//when load, we need to check id is valid or not.
|
||||
if err := l.validateID(id); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
containerRoot, err := securejoin.SecureJoin(l.Root, id)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
state, err := l.loadState(containerRoot, id)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -339,7 +353,11 @@ func (l *LinuxFactory) StartInitialization() (err error) {
|
|||
}
|
||||
|
||||
func (l *LinuxFactory) loadState(root, id string) (*State, error) {
|
||||
f, err := os.Open(filepath.Join(root, stateFilename))
|
||||
stateFilePath, err := securejoin.SecureJoin(root, stateFilename)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
f, err := os.Open(stateFilePath)
|
||||
if err != nil {
|
||||
if os.IsNotExist(err) {
|
||||
return nil, newGenericError(fmt.Errorf("container %q does not exist", id), ContainerNotExists)
|
||||
|
@ -355,7 +373,7 @@ func (l *LinuxFactory) loadState(root, id string) (*State, error) {
|
|||
}
|
||||
|
||||
func (l *LinuxFactory) validateID(id string) error {
|
||||
if !idRegex.MatchString(id) {
|
||||
if !idRegex.MatchString(id) || string(os.PathSeparator)+id != utils.CleanPath(string(os.PathSeparator)+id) {
|
||||
return newGenericError(fmt.Errorf("invalid id format: %v", id), InvalidIdFormat)
|
||||
}
|
||||
|
||||
|
|
|
@ -5,18 +5,15 @@ package libcontainer
|
|||
import (
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/opencontainers/runc/libcontainer/configs"
|
||||
"github.com/opencontainers/runc/libcontainer/utils"
|
||||
"github.com/vishvananda/netlink"
|
||||
)
|
||||
|
||||
var strategies = map[string]networkStrategy{
|
||||
"veth": &veth{},
|
||||
"loopback": &loopback{},
|
||||
}
|
||||
|
||||
|
@ -103,157 +100,3 @@ func (l *loopback) attach(n *configs.Network) (err error) {
|
|||
func (l *loopback) detach(n *configs.Network) (err error) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// veth is a network strategy that uses a bridge and creates
|
||||
// a veth pair, one that is attached to the bridge on the host and the other
|
||||
// is placed inside the container's namespace
|
||||
type veth struct {
|
||||
}
|
||||
|
||||
func (v *veth) detach(n *configs.Network) (err error) {
|
||||
return netlink.LinkSetMaster(&netlink.Device{LinkAttrs: netlink.LinkAttrs{Name: n.HostInterfaceName}}, nil)
|
||||
}
|
||||
|
||||
// attach a container network interface to an external network
|
||||
func (v *veth) attach(n *configs.Network) (err error) {
|
||||
brl, err := netlink.LinkByName(n.Bridge)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
br, ok := brl.(*netlink.Bridge)
|
||||
if !ok {
|
||||
return fmt.Errorf("Wrong device type %T", brl)
|
||||
}
|
||||
host, err := netlink.LinkByName(n.HostInterfaceName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := netlink.LinkSetMaster(host, br); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := netlink.LinkSetMTU(host, n.Mtu); err != nil {
|
||||
return err
|
||||
}
|
||||
if n.HairpinMode {
|
||||
if err := netlink.LinkSetHairpin(host, true); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if err := netlink.LinkSetUp(host); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (v *veth) create(n *network, nspid int) (err error) {
|
||||
tmpName, err := v.generateTempPeerName()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
n.TempVethPeerName = tmpName
|
||||
if n.Bridge == "" {
|
||||
return fmt.Errorf("bridge is not specified")
|
||||
}
|
||||
veth := &netlink.Veth{
|
||||
LinkAttrs: netlink.LinkAttrs{
|
||||
Name: n.HostInterfaceName,
|
||||
TxQLen: n.TxQueueLen,
|
||||
},
|
||||
PeerName: n.TempVethPeerName,
|
||||
}
|
||||
if err := netlink.LinkAdd(veth); err != nil {
|
||||
return err
|
||||
}
|
||||
defer func() {
|
||||
if err != nil {
|
||||
netlink.LinkDel(veth)
|
||||
}
|
||||
}()
|
||||
if err := v.attach(&n.Network); err != nil {
|
||||
return err
|
||||
}
|
||||
child, err := netlink.LinkByName(n.TempVethPeerName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return netlink.LinkSetNsPid(child, nspid)
|
||||
}
|
||||
|
||||
func (v *veth) generateTempPeerName() (string, error) {
|
||||
return utils.GenerateRandomName("veth", 7)
|
||||
}
|
||||
|
||||
func (v *veth) initialize(config *network) error {
|
||||
peer := config.TempVethPeerName
|
||||
if peer == "" {
|
||||
return fmt.Errorf("peer is not specified")
|
||||
}
|
||||
child, err := netlink.LinkByName(peer)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := netlink.LinkSetDown(child); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := netlink.LinkSetName(child, config.Name); err != nil {
|
||||
return err
|
||||
}
|
||||
// get the interface again after we changed the name as the index also changes.
|
||||
if child, err = netlink.LinkByName(config.Name); err != nil {
|
||||
return err
|
||||
}
|
||||
if config.MacAddress != "" {
|
||||
mac, err := net.ParseMAC(config.MacAddress)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := netlink.LinkSetHardwareAddr(child, mac); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
ip, err := netlink.ParseAddr(config.Address)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := netlink.AddrAdd(child, ip); err != nil {
|
||||
return err
|
||||
}
|
||||
if config.IPv6Address != "" {
|
||||
ip6, err := netlink.ParseAddr(config.IPv6Address)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := netlink.AddrAdd(child, ip6); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if err := netlink.LinkSetMTU(child, config.Mtu); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := netlink.LinkSetUp(child); err != nil {
|
||||
return err
|
||||
}
|
||||
if config.Gateway != "" {
|
||||
gw := net.ParseIP(config.Gateway)
|
||||
if err := netlink.RouteAdd(&netlink.Route{
|
||||
Scope: netlink.SCOPE_UNIVERSE,
|
||||
LinkIndex: child.Attrs().Index,
|
||||
Gw: gw,
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if config.IPv6Gateway != "" {
|
||||
gw := net.ParseIP(config.IPv6Gateway)
|
||||
if err := netlink.RouteAdd(&netlink.Route{
|
||||
Scope: netlink.SCOPE_UNIVERSE,
|
||||
LinkIndex: child.Attrs().Index,
|
||||
Gw: gw,
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
41
vendor/github.com/opencontainers/runc/libcontainer/nsenter/README.md
generated
vendored
Normal file
41
vendor/github.com/opencontainers/runc/libcontainer/nsenter/README.md
generated
vendored
Normal file
|
@ -0,0 +1,41 @@
|
|||
## nsenter
|
||||
|
||||
The `nsenter` package registers a special init constructor that is called before
|
||||
the Go runtime has a chance to boot. This provides us the ability to `setns` on
|
||||
existing namespaces and avoid the issues that the Go runtime has with multiple
|
||||
threads. This constructor will be called if this package is registered,
|
||||
imported, in your go application.
|
||||
|
||||
The `nsenter` package will `import "C"` and it uses [cgo](https://golang.org/cmd/cgo/)
|
||||
package. In cgo, if the import of "C" is immediately preceded by a comment, that comment,
|
||||
called the preamble, is used as a header when compiling the C parts of the package.
|
||||
So every time we import package `nsenter`, the C code function `nsexec()` would be
|
||||
called. And package `nsenter` is now only imported in `main_unix.go`, so every time
|
||||
before we call `cmd.Start` on linux, that C code would run.
|
||||
|
||||
Because `nsexec()` must be run before the Go runtime in order to use the
|
||||
Linux kernel namespace, you must `import` this library into a package if
|
||||
you plan to use `libcontainer` directly. Otherwise Go will not execute
|
||||
the `nsexec()` constructor, which means that the re-exec will not cause
|
||||
the namespaces to be joined. You can import it like this:
|
||||
|
||||
```go
|
||||
import _ "github.com/opencontainers/runc/libcontainer/nsenter"
|
||||
```
|
||||
|
||||
`nsexec()` will first get the file descriptor number for the init pipe
|
||||
from the environment variable `_LIBCONTAINER_INITPIPE` (which was opened
|
||||
by the parent and kept open across the fork-exec of the `nsexec()` init
|
||||
process). The init pipe is used to read bootstrap data (namespace paths,
|
||||
clone flags, uid and gid mappings, and the console path) from the parent
|
||||
process. `nsexec()` will then call `setns(2)` to join the namespaces
|
||||
provided in the bootstrap data (if available), `clone(2)` a child process
|
||||
with the provided clone flags, update the user and group ID mappings, do
|
||||
some further miscellaneous setup steps, and then send the PID of the
|
||||
child process to the parent of the `nsexec()` "caller". Finally,
|
||||
the parent `nsexec()` will exit and the child `nsexec()` process will
|
||||
return to allow the Go runtime take over.
|
||||
|
||||
NOTE: We do both `setns(2)` and `clone(2)` even if we don't have any
|
||||
CLONE_NEW* clone flags because we must fork a new process in order to
|
||||
enter the PID namespace.
|
32
vendor/github.com/opencontainers/runc/libcontainer/nsenter/namespace.h
generated
vendored
Normal file
32
vendor/github.com/opencontainers/runc/libcontainer/nsenter/namespace.h
generated
vendored
Normal file
|
@ -0,0 +1,32 @@
|
|||
#ifndef NSENTER_NAMESPACE_H
|
||||
#define NSENTER_NAMESPACE_H
|
||||
|
||||
#ifndef _GNU_SOURCE
|
||||
# define _GNU_SOURCE
|
||||
#endif
|
||||
#include <sched.h>
|
||||
|
||||
/* All of these are taken from include/uapi/linux/sched.h */
|
||||
#ifndef CLONE_NEWNS
|
||||
# define CLONE_NEWNS 0x00020000 /* New mount namespace group */
|
||||
#endif
|
||||
#ifndef CLONE_NEWCGROUP
|
||||
# define CLONE_NEWCGROUP 0x02000000 /* New cgroup namespace */
|
||||
#endif
|
||||
#ifndef CLONE_NEWUTS
|
||||
# define CLONE_NEWUTS 0x04000000 /* New utsname namespace */
|
||||
#endif
|
||||
#ifndef CLONE_NEWIPC
|
||||
# define CLONE_NEWIPC 0x08000000 /* New ipc namespace */
|
||||
#endif
|
||||
#ifndef CLONE_NEWUSER
|
||||
# define CLONE_NEWUSER 0x10000000 /* New user namespace */
|
||||
#endif
|
||||
#ifndef CLONE_NEWPID
|
||||
# define CLONE_NEWPID 0x20000000 /* New pid namespace */
|
||||
#endif
|
||||
#ifndef CLONE_NEWNET
|
||||
# define CLONE_NEWNET 0x40000000 /* New network namespace */
|
||||
#endif
|
||||
|
||||
#endif /* NSENTER_NAMESPACE_H */
|
12
vendor/github.com/opencontainers/runc/libcontainer/nsenter/nsenter.go
generated
vendored
Normal file
12
vendor/github.com/opencontainers/runc/libcontainer/nsenter/nsenter.go
generated
vendored
Normal file
|
@ -0,0 +1,12 @@
|
|||
// +build linux,!gccgo
|
||||
|
||||
package nsenter
|
||||
|
||||
/*
|
||||
#cgo CFLAGS: -Wall
|
||||
extern void nsexec();
|
||||
void __attribute__((constructor)) init(void) {
|
||||
nsexec();
|
||||
}
|
||||
*/
|
||||
import "C"
|
25
vendor/github.com/opencontainers/runc/libcontainer/nsenter/nsenter_gccgo.go
generated
vendored
Normal file
25
vendor/github.com/opencontainers/runc/libcontainer/nsenter/nsenter_gccgo.go
generated
vendored
Normal file
|
@ -0,0 +1,25 @@
|
|||
// +build linux,gccgo
|
||||
|
||||
package nsenter
|
||||
|
||||
/*
|
||||
#cgo CFLAGS: -Wall
|
||||
extern void nsexec();
|
||||
void __attribute__((constructor)) init(void) {
|
||||
nsexec();
|
||||
}
|
||||
*/
|
||||
import "C"
|
||||
|
||||
// AlwaysFalse is here to stay false
|
||||
// (and be exported so the compiler doesn't optimize out its reference)
|
||||
var AlwaysFalse bool
|
||||
|
||||
func init() {
|
||||
if AlwaysFalse {
|
||||
// by referencing this C init() in a noop test, it will ensure the compiler
|
||||
// links in the C function.
|
||||
// https://gcc.gnu.org/bugzilla/show_bug.cgi?id=65134
|
||||
C.init()
|
||||
}
|
||||
}
|
5
vendor/github.com/opencontainers/runc/libcontainer/nsenter/nsenter_unsupported.go
generated
vendored
Normal file
5
vendor/github.com/opencontainers/runc/libcontainer/nsenter/nsenter_unsupported.go
generated
vendored
Normal file
|
@ -0,0 +1,5 @@
|
|||
// +build !linux !cgo
|
||||
|
||||
package nsenter
|
||||
|
||||
import "C"
|
980
vendor/github.com/opencontainers/runc/libcontainer/nsenter/nsexec.c
generated
vendored
Normal file
980
vendor/github.com/opencontainers/runc/libcontainer/nsenter/nsexec.c
generated
vendored
Normal file
|
@ -0,0 +1,980 @@
|
|||
|
||||
#define _GNU_SOURCE
|
||||
#include <endian.h>
|
||||
#include <errno.h>
|
||||
#include <fcntl.h>
|
||||
#include <grp.h>
|
||||
#include <sched.h>
|
||||
#include <setjmp.h>
|
||||
#include <signal.h>
|
||||
#include <stdarg.h>
|
||||
#include <stdbool.h>
|
||||
#include <stdint.h>
|
||||
#include <stdio.h>
|
||||
#include <stdlib.h>
|
||||
#include <stdbool.h>
|
||||
#include <string.h>
|
||||
#include <unistd.h>
|
||||
|
||||
#include <sys/ioctl.h>
|
||||
#include <sys/prctl.h>
|
||||
#include <sys/socket.h>
|
||||
#include <sys/types.h>
|
||||
#include <sys/wait.h>
|
||||
|
||||
#include <linux/limits.h>
|
||||
#include <linux/netlink.h>
|
||||
#include <linux/types.h>
|
||||
|
||||
/* Get all of the CLONE_NEW* flags. */
|
||||
#include "namespace.h"
|
||||
|
||||
/* Synchronisation values. */
|
||||
enum sync_t {
|
||||
SYNC_USERMAP_PLS = 0x40, /* Request parent to map our users. */
|
||||
SYNC_USERMAP_ACK = 0x41, /* Mapping finished by the parent. */
|
||||
SYNC_RECVPID_PLS = 0x42, /* Tell parent we're sending the PID. */
|
||||
SYNC_RECVPID_ACK = 0x43, /* PID was correctly received by parent. */
|
||||
SYNC_GRANDCHILD = 0x44, /* The grandchild is ready to run. */
|
||||
SYNC_CHILD_READY = 0x45, /* The child or grandchild is ready to return. */
|
||||
|
||||
/* XXX: This doesn't help with segfaults and other such issues. */
|
||||
SYNC_ERR = 0xFF, /* Fatal error, no turning back. The error code follows. */
|
||||
};
|
||||
|
||||
/* longjmp() arguments. */
|
||||
#define JUMP_PARENT 0x00
|
||||
#define JUMP_CHILD 0xA0
|
||||
#define JUMP_INIT 0xA1
|
||||
|
||||
/* JSON buffer. */
|
||||
#define JSON_MAX 4096
|
||||
|
||||
/* Assume the stack grows down, so arguments should be above it. */
|
||||
struct clone_t {
|
||||
/*
|
||||
* Reserve some space for clone() to locate arguments
|
||||
* and retcode in this place
|
||||
*/
|
||||
char stack[4096] __attribute__ ((aligned(16)));
|
||||
char stack_ptr[0];
|
||||
|
||||
/* There's two children. This is used to execute the different code. */
|
||||
jmp_buf *env;
|
||||
int jmpval;
|
||||
};
|
||||
|
||||
struct nlconfig_t {
|
||||
char *data;
|
||||
|
||||
/* Process settings. */
|
||||
uint32_t cloneflags;
|
||||
char *oom_score_adj;
|
||||
size_t oom_score_adj_len;
|
||||
|
||||
/* User namespace settings. */
|
||||
char *uidmap;
|
||||
size_t uidmap_len;
|
||||
char *gidmap;
|
||||
size_t gidmap_len;
|
||||
char *namespaces;
|
||||
size_t namespaces_len;
|
||||
uint8_t is_setgroup;
|
||||
|
||||
/* Rootless container settings. */
|
||||
uint8_t is_rootless;
|
||||
char *uidmappath;
|
||||
size_t uidmappath_len;
|
||||
char *gidmappath;
|
||||
size_t gidmappath_len;
|
||||
};
|
||||
|
||||
/*
|
||||
* List of netlink message types sent to us as part of bootstrapping the init.
|
||||
* These constants are defined in libcontainer/message_linux.go.
|
||||
*/
|
||||
#define INIT_MSG 62000
|
||||
#define CLONE_FLAGS_ATTR 27281
|
||||
#define NS_PATHS_ATTR 27282
|
||||
#define UIDMAP_ATTR 27283
|
||||
#define GIDMAP_ATTR 27284
|
||||
#define SETGROUP_ATTR 27285
|
||||
#define OOM_SCORE_ADJ_ATTR 27286
|
||||
#define ROOTLESS_ATTR 27287
|
||||
#define UIDMAPPATH_ATTR 27288
|
||||
#define GIDMAPPATH_ATTR 27289
|
||||
|
||||
/*
|
||||
* Use the raw syscall for versions of glibc which don't include a function for
|
||||
* it, namely (glibc 2.12).
|
||||
*/
|
||||
#if __GLIBC__ == 2 && __GLIBC_MINOR__ < 14
|
||||
# define _GNU_SOURCE
|
||||
# include "syscall.h"
|
||||
# if !defined(SYS_setns) && defined(__NR_setns)
|
||||
# define SYS_setns __NR_setns
|
||||
# endif
|
||||
|
||||
#ifndef SYS_setns
|
||||
# error "setns(2) syscall not supported by glibc version"
|
||||
#endif
|
||||
|
||||
int setns(int fd, int nstype)
|
||||
{
|
||||
return syscall(SYS_setns, fd, nstype);
|
||||
}
|
||||
#endif
|
||||
|
||||
/* XXX: This is ugly. */
|
||||
static int syncfd = -1;
|
||||
|
||||
/* TODO(cyphar): Fix this so it correctly deals with syncT. */
|
||||
#define bail(fmt, ...) \
|
||||
do { \
|
||||
int ret = __COUNTER__ + 1; \
|
||||
fprintf(stderr, "nsenter: " fmt ": %m\n", ##__VA_ARGS__); \
|
||||
if (syncfd >= 0) { \
|
||||
enum sync_t s = SYNC_ERR; \
|
||||
if (write(syncfd, &s, sizeof(s)) != sizeof(s)) \
|
||||
fprintf(stderr, "nsenter: failed: write(s)"); \
|
||||
if (write(syncfd, &ret, sizeof(ret)) != sizeof(ret)) \
|
||||
fprintf(stderr, "nsenter: failed: write(ret)"); \
|
||||
} \
|
||||
exit(ret); \
|
||||
} while(0)
|
||||
|
||||
static int write_file(char *data, size_t data_len, char *pathfmt, ...)
|
||||
{
|
||||
int fd, len, ret = 0;
|
||||
char path[PATH_MAX];
|
||||
|
||||
va_list ap;
|
||||
va_start(ap, pathfmt);
|
||||
len = vsnprintf(path, PATH_MAX, pathfmt, ap);
|
||||
va_end(ap);
|
||||
if (len < 0)
|
||||
return -1;
|
||||
|
||||
fd = open(path, O_RDWR);
|
||||
if (fd < 0) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
len = write(fd, data, data_len);
|
||||
if (len != data_len) {
|
||||
ret = -1;
|
||||
goto out;
|
||||
}
|
||||
|
||||
out:
|
||||
close(fd);
|
||||
return ret;
|
||||
}
|
||||
|
||||
enum policy_t {
|
||||
SETGROUPS_DEFAULT = 0,
|
||||
SETGROUPS_ALLOW,
|
||||
SETGROUPS_DENY,
|
||||
};
|
||||
|
||||
/* This *must* be called before we touch gid_map. */
|
||||
static void update_setgroups(int pid, enum policy_t setgroup)
|
||||
{
|
||||
char *policy;
|
||||
|
||||
switch (setgroup) {
|
||||
case SETGROUPS_ALLOW:
|
||||
policy = "allow";
|
||||
break;
|
||||
case SETGROUPS_DENY:
|
||||
policy = "deny";
|
||||
break;
|
||||
case SETGROUPS_DEFAULT:
|
||||
default:
|
||||
/* Nothing to do. */
|
||||
return;
|
||||
}
|
||||
|
||||
if (write_file(policy, strlen(policy), "/proc/%d/setgroups", pid) < 0) {
|
||||
/*
|
||||
* If the kernel is too old to support /proc/pid/setgroups,
|
||||
* open(2) or write(2) will return ENOENT. This is fine.
|
||||
*/
|
||||
if (errno != ENOENT)
|
||||
bail("failed to write '%s' to /proc/%d/setgroups", policy, pid);
|
||||
}
|
||||
}
|
||||
|
||||
static int try_mapping_tool(const char *app, int pid, char *map, size_t map_len)
|
||||
{
|
||||
int child;
|
||||
|
||||
/*
|
||||
* If @app is NULL, execve will segfault. Just check it here and bail (if
|
||||
* we're in this path, the caller is already getting desparate and there
|
||||
* isn't a backup to this failing). This usually would be a configuration
|
||||
* or programming issue.
|
||||
*/
|
||||
if (!app)
|
||||
bail("mapping tool not present");
|
||||
|
||||
child = fork();
|
||||
if (child < 0)
|
||||
bail("failed to fork");
|
||||
|
||||
if (!child) {
|
||||
#define MAX_ARGV 20
|
||||
char *argv[MAX_ARGV];
|
||||
char *envp[] = { NULL };
|
||||
char pid_fmt[16];
|
||||
int argc = 0;
|
||||
char *next;
|
||||
|
||||
snprintf(pid_fmt, 16, "%d", pid);
|
||||
|
||||
argv[argc++] = (char *)app;
|
||||
argv[argc++] = pid_fmt;
|
||||
/*
|
||||
* Convert the map string into a list of argument that
|
||||
* newuidmap/newgidmap can understand.
|
||||
*/
|
||||
|
||||
while (argc < MAX_ARGV) {
|
||||
if (*map == '\0') {
|
||||
argv[argc++] = NULL;
|
||||
break;
|
||||
}
|
||||
argv[argc++] = map;
|
||||
next = strpbrk(map, "\n ");
|
||||
if (next == NULL)
|
||||
break;
|
||||
*next++ = '\0';
|
||||
map = next + strspn(next, "\n ");
|
||||
}
|
||||
|
||||
execve(app, argv, envp);
|
||||
bail("failed to execv");
|
||||
} else {
|
||||
int status;
|
||||
|
||||
while (true) {
|
||||
if (waitpid(child, &status, 0) < 0) {
|
||||
if (errno == EINTR)
|
||||
continue;
|
||||
bail("failed to waitpid");
|
||||
}
|
||||
if (WIFEXITED(status) || WIFSIGNALED(status))
|
||||
return WEXITSTATUS(status);
|
||||
}
|
||||
}
|
||||
|
||||
return -1;
|
||||
}
|
||||
|
||||
static void update_uidmap(const char *path, int pid, char *map, size_t map_len)
|
||||
{
|
||||
if (map == NULL || map_len <= 0)
|
||||
return;
|
||||
|
||||
if (write_file(map, map_len, "/proc/%d/uid_map", pid) < 0) {
|
||||
if (errno != EPERM)
|
||||
bail("failed to update /proc/%d/uid_map", pid);
|
||||
if (try_mapping_tool(path, pid, map, map_len))
|
||||
bail("failed to use newuid map on %d", pid);
|
||||
}
|
||||
}
|
||||
|
||||
static void update_gidmap(const char *path, int pid, char *map, size_t map_len)
|
||||
{
|
||||
if (map == NULL || map_len <= 0)
|
||||
return;
|
||||
|
||||
if (write_file(map, map_len, "/proc/%d/gid_map", pid) < 0) {
|
||||
if (errno != EPERM)
|
||||
bail("failed to update /proc/%d/gid_map", pid);
|
||||
if (try_mapping_tool(path, pid, map, map_len))
|
||||
bail("failed to use newgid map on %d", pid);
|
||||
}
|
||||
}
|
||||
|
||||
static void update_oom_score_adj(char *data, size_t len)
|
||||
{
|
||||
if (data == NULL || len <= 0)
|
||||
return;
|
||||
|
||||
if (write_file(data, len, "/proc/self/oom_score_adj") < 0)
|
||||
bail("failed to update /proc/self/oom_score_adj");
|
||||
}
|
||||
|
||||
/* A dummy function that just jumps to the given jumpval. */
|
||||
static int child_func(void *arg) __attribute__ ((noinline));
|
||||
static int child_func(void *arg)
|
||||
{
|
||||
struct clone_t *ca = (struct clone_t *)arg;
|
||||
longjmp(*ca->env, ca->jmpval);
|
||||
}
|
||||
|
||||
static int clone_parent(jmp_buf *env, int jmpval) __attribute__ ((noinline));
|
||||
static int clone_parent(jmp_buf *env, int jmpval)
|
||||
{
|
||||
struct clone_t ca = {
|
||||
.env = env,
|
||||
.jmpval = jmpval,
|
||||
};
|
||||
|
||||
return clone(child_func, ca.stack_ptr, CLONE_PARENT | SIGCHLD, &ca);
|
||||
}
|
||||
|
||||
/*
|
||||
* Gets the init pipe fd from the environment, which is used to read the
|
||||
* bootstrap data and tell the parent what the new pid is after we finish
|
||||
* setting up the environment.
|
||||
*/
|
||||
static int initpipe(void)
|
||||
{
|
||||
int pipenum;
|
||||
char *initpipe, *endptr;
|
||||
|
||||
initpipe = getenv("_LIBCONTAINER_INITPIPE");
|
||||
if (initpipe == NULL || *initpipe == '\0')
|
||||
return -1;
|
||||
|
||||
pipenum = strtol(initpipe, &endptr, 10);
|
||||
if (*endptr != '\0')
|
||||
bail("unable to parse _LIBCONTAINER_INITPIPE");
|
||||
|
||||
return pipenum;
|
||||
}
|
||||
|
||||
/* Returns the clone(2) flag for a namespace, given the name of a namespace. */
|
||||
static int nsflag(char *name)
|
||||
{
|
||||
if (!strcmp(name, "cgroup"))
|
||||
return CLONE_NEWCGROUP;
|
||||
else if (!strcmp(name, "ipc"))
|
||||
return CLONE_NEWIPC;
|
||||
else if (!strcmp(name, "mnt"))
|
||||
return CLONE_NEWNS;
|
||||
else if (!strcmp(name, "net"))
|
||||
return CLONE_NEWNET;
|
||||
else if (!strcmp(name, "pid"))
|
||||
return CLONE_NEWPID;
|
||||
else if (!strcmp(name, "user"))
|
||||
return CLONE_NEWUSER;
|
||||
else if (!strcmp(name, "uts"))
|
||||
return CLONE_NEWUTS;
|
||||
|
||||
/* If we don't recognise a name, fallback to 0. */
|
||||
return 0;
|
||||
}
|
||||
|
||||
static uint32_t readint32(char *buf)
|
||||
{
|
||||
return *(uint32_t *) buf;
|
||||
}
|
||||
|
||||
static uint8_t readint8(char *buf)
|
||||
{
|
||||
return *(uint8_t *) buf;
|
||||
}
|
||||
|
||||
static void nl_parse(int fd, struct nlconfig_t *config)
|
||||
{
|
||||
size_t len, size;
|
||||
struct nlmsghdr hdr;
|
||||
char *data, *current;
|
||||
|
||||
/* Retrieve the netlink header. */
|
||||
len = read(fd, &hdr, NLMSG_HDRLEN);
|
||||
if (len != NLMSG_HDRLEN)
|
||||
bail("invalid netlink header length %zu", len);
|
||||
|
||||
if (hdr.nlmsg_type == NLMSG_ERROR)
|
||||
bail("failed to read netlink message");
|
||||
|
||||
if (hdr.nlmsg_type != INIT_MSG)
|
||||
bail("unexpected msg type %d", hdr.nlmsg_type);
|
||||
|
||||
/* Retrieve data. */
|
||||
size = NLMSG_PAYLOAD(&hdr, 0);
|
||||
current = data = malloc(size);
|
||||
if (!data)
|
||||
bail("failed to allocate %zu bytes of memory for nl_payload", size);
|
||||
|
||||
len = read(fd, data, size);
|
||||
if (len != size)
|
||||
bail("failed to read netlink payload, %zu != %zu", len, size);
|
||||
|
||||
/* Parse the netlink payload. */
|
||||
config->data = data;
|
||||
while (current < data + size) {
|
||||
struct nlattr *nlattr = (struct nlattr *)current;
|
||||
size_t payload_len = nlattr->nla_len - NLA_HDRLEN;
|
||||
|
||||
/* Advance to payload. */
|
||||
current += NLA_HDRLEN;
|
||||
|
||||
/* Handle payload. */
|
||||
switch (nlattr->nla_type) {
|
||||
case CLONE_FLAGS_ATTR:
|
||||
config->cloneflags = readint32(current);
|
||||
break;
|
||||
case ROOTLESS_ATTR:
|
||||
config->is_rootless = readint8(current);
|
||||
break;
|
||||
case OOM_SCORE_ADJ_ATTR:
|
||||
config->oom_score_adj = current;
|
||||
config->oom_score_adj_len = payload_len;
|
||||
break;
|
||||
case NS_PATHS_ATTR:
|
||||
config->namespaces = current;
|
||||
config->namespaces_len = payload_len;
|
||||
break;
|
||||
case UIDMAP_ATTR:
|
||||
config->uidmap = current;
|
||||
config->uidmap_len = payload_len;
|
||||
break;
|
||||
case GIDMAP_ATTR:
|
||||
config->gidmap = current;
|
||||
config->gidmap_len = payload_len;
|
||||
break;
|
||||
case UIDMAPPATH_ATTR:
|
||||
config->uidmappath = current;
|
||||
config->uidmappath_len = payload_len;
|
||||
break;
|
||||
case GIDMAPPATH_ATTR:
|
||||
config->gidmappath = current;
|
||||
config->gidmappath_len = payload_len;
|
||||
break;
|
||||
case SETGROUP_ATTR:
|
||||
config->is_setgroup = readint8(current);
|
||||
break;
|
||||
default:
|
||||
bail("unknown netlink message type %d", nlattr->nla_type);
|
||||
}
|
||||
|
||||
current += NLA_ALIGN(payload_len);
|
||||
}
|
||||
}
|
||||
|
||||
void nl_free(struct nlconfig_t *config)
|
||||
{
|
||||
free(config->data);
|
||||
}
|
||||
|
||||
void join_namespaces(char *nslist)
|
||||
{
|
||||
int num = 0, i;
|
||||
char *saveptr = NULL;
|
||||
char *namespace = strtok_r(nslist, ",", &saveptr);
|
||||
struct namespace_t {
|
||||
int fd;
|
||||
int ns;
|
||||
char type[PATH_MAX];
|
||||
char path[PATH_MAX];
|
||||
} *namespaces = NULL;
|
||||
|
||||
if (!namespace || !strlen(namespace) || !strlen(nslist))
|
||||
bail("ns paths are empty");
|
||||
|
||||
/*
|
||||
* We have to open the file descriptors first, since after
|
||||
* we join the mnt namespace we might no longer be able to
|
||||
* access the paths.
|
||||
*/
|
||||
do {
|
||||
int fd;
|
||||
char *path;
|
||||
struct namespace_t *ns;
|
||||
|
||||
/* Resize the namespace array. */
|
||||
namespaces = realloc(namespaces, ++num * sizeof(struct namespace_t));
|
||||
if (!namespaces)
|
||||
bail("failed to reallocate namespace array");
|
||||
ns = &namespaces[num - 1];
|
||||
|
||||
/* Split 'ns:path'. */
|
||||
path = strstr(namespace, ":");
|
||||
if (!path)
|
||||
bail("failed to parse %s", namespace);
|
||||
*path++ = '\0';
|
||||
|
||||
fd = open(path, O_RDONLY);
|
||||
if (fd < 0)
|
||||
bail("failed to open %s", path);
|
||||
|
||||
ns->fd = fd;
|
||||
ns->ns = nsflag(namespace);
|
||||
strncpy(ns->path, path, PATH_MAX - 1);
|
||||
ns->path[PATH_MAX - 1] = '\0';
|
||||
} while ((namespace = strtok_r(NULL, ",", &saveptr)) != NULL);
|
||||
|
||||
/*
|
||||
* The ordering in which we join namespaces is important. We should
|
||||
* always join the user namespace *first*. This is all guaranteed
|
||||
* from the container_linux.go side of this, so we're just going to
|
||||
* follow the order given to us.
|
||||
*/
|
||||
|
||||
for (i = 0; i < num; i++) {
|
||||
struct namespace_t ns = namespaces[i];
|
||||
|
||||
if (setns(ns.fd, ns.ns) < 0)
|
||||
bail("failed to setns to %s", ns.path);
|
||||
|
||||
close(ns.fd);
|
||||
}
|
||||
|
||||
free(namespaces);
|
||||
}
|
||||
|
||||
void nsexec(void)
|
||||
{
|
||||
int pipenum;
|
||||
jmp_buf env;
|
||||
int sync_child_pipe[2], sync_grandchild_pipe[2];
|
||||
struct nlconfig_t config = { 0 };
|
||||
|
||||
/*
|
||||
* If we don't have an init pipe, just return to the go routine.
|
||||
* We'll only get an init pipe for start or exec.
|
||||
*/
|
||||
pipenum = initpipe();
|
||||
if (pipenum == -1)
|
||||
return;
|
||||
|
||||
/* Parse all of the netlink configuration. */
|
||||
nl_parse(pipenum, &config);
|
||||
|
||||
/* Set oom_score_adj. This has to be done before !dumpable because
|
||||
* /proc/self/oom_score_adj is not writeable unless you're an privileged
|
||||
* user (if !dumpable is set). All children inherit their parent's
|
||||
* oom_score_adj value on fork(2) so this will always be propagated
|
||||
* properly.
|
||||
*/
|
||||
update_oom_score_adj(config.oom_score_adj, config.oom_score_adj_len);
|
||||
|
||||
/*
|
||||
* Make the process non-dumpable, to avoid various race conditions that
|
||||
* could cause processes in namespaces we're joining to access host
|
||||
* resources (or potentially execute code).
|
||||
*
|
||||
* However, if the number of namespaces we are joining is 0, we are not
|
||||
* going to be switching to a different security context. Thus setting
|
||||
* ourselves to be non-dumpable only breaks things (like rootless
|
||||
* containers), which is the recommendation from the kernel folks.
|
||||
*/
|
||||
if (config.namespaces) {
|
||||
if (prctl(PR_SET_DUMPABLE, 0, 0, 0, 0) < 0)
|
||||
bail("failed to set process as non-dumpable");
|
||||
}
|
||||
|
||||
/* Pipe so we can tell the child when we've finished setting up. */
|
||||
if (socketpair(AF_LOCAL, SOCK_STREAM, 0, sync_child_pipe) < 0)
|
||||
bail("failed to setup sync pipe between parent and child");
|
||||
|
||||
/*
|
||||
* We need a new socketpair to sync with grandchild so we don't have
|
||||
* race condition with child.
|
||||
*/
|
||||
if (socketpair(AF_LOCAL, SOCK_STREAM, 0, sync_grandchild_pipe) < 0)
|
||||
bail("failed to setup sync pipe between parent and grandchild");
|
||||
|
||||
/* TODO: Currently we aren't dealing with child deaths properly. */
|
||||
|
||||
/*
|
||||
* Okay, so this is quite annoying.
|
||||
*
|
||||
* In order for this unsharing code to be more extensible we need to split
|
||||
* up unshare(CLONE_NEWUSER) and clone() in various ways. The ideal case
|
||||
* would be if we did clone(CLONE_NEWUSER) and the other namespaces
|
||||
* separately, but because of SELinux issues we cannot really do that. But
|
||||
* we cannot just dump the namespace flags into clone(...) because several
|
||||
* usecases (such as rootless containers) require more granularity around
|
||||
* the namespace setup. In addition, some older kernels had issues where
|
||||
* CLONE_NEWUSER wasn't handled before other namespaces (but we cannot
|
||||
* handle this while also dealing with SELinux so we choose SELinux support
|
||||
* over broken kernel support).
|
||||
*
|
||||
* However, if we unshare(2) the user namespace *before* we clone(2), then
|
||||
* all hell breaks loose.
|
||||
*
|
||||
* The parent no longer has permissions to do many things (unshare(2) drops
|
||||
* all capabilities in your old namespace), and the container cannot be set
|
||||
* up to have more than one {uid,gid} mapping. This is obviously less than
|
||||
* ideal. In order to fix this, we have to first clone(2) and then unshare.
|
||||
*
|
||||
* Unfortunately, it's not as simple as that. We have to fork to enter the
|
||||
* PID namespace (the PID namespace only applies to children). Since we'll
|
||||
* have to double-fork, this clone_parent() call won't be able to get the
|
||||
* PID of the _actual_ init process (without doing more synchronisation than
|
||||
* I can deal with at the moment). So we'll just get the parent to send it
|
||||
* for us, the only job of this process is to update
|
||||
* /proc/pid/{setgroups,uid_map,gid_map}.
|
||||
*
|
||||
* And as a result of the above, we also need to setns(2) in the first child
|
||||
* because if we join a PID namespace in the topmost parent then our child
|
||||
* will be in that namespace (and it will not be able to give us a PID value
|
||||
* that makes sense without resorting to sending things with cmsg).
|
||||
*
|
||||
* This also deals with an older issue caused by dumping cloneflags into
|
||||
* clone(2): On old kernels, CLONE_PARENT didn't work with CLONE_NEWPID, so
|
||||
* we have to unshare(2) before clone(2) in order to do this. This was fixed
|
||||
* in upstream commit 1f7f4dde5c945f41a7abc2285be43d918029ecc5, and was
|
||||
* introduced by 40a0d32d1eaffe6aac7324ca92604b6b3977eb0e. As far as we're
|
||||
* aware, the last mainline kernel which had this bug was Linux 3.12.
|
||||
* However, we cannot comment on which kernels the broken patch was
|
||||
* backported to.
|
||||
*
|
||||
* -- Aleksa "what has my life come to?" Sarai
|
||||
*/
|
||||
|
||||
switch (setjmp(env)) {
|
||||
/*
|
||||
* Stage 0: We're in the parent. Our job is just to create a new child
|
||||
* (stage 1: JUMP_CHILD) process and write its uid_map and
|
||||
* gid_map. That process will go on to create a new process, then
|
||||
* it will send us its PID which we will send to the bootstrap
|
||||
* process.
|
||||
*/
|
||||
case JUMP_PARENT:{
|
||||
int len;
|
||||
pid_t child, first_child = -1;
|
||||
char buf[JSON_MAX];
|
||||
bool ready = false;
|
||||
|
||||
/* For debugging. */
|
||||
prctl(PR_SET_NAME, (unsigned long)"runc:[0:PARENT]", 0, 0, 0);
|
||||
|
||||
/* Start the process of getting a container. */
|
||||
child = clone_parent(&env, JUMP_CHILD);
|
||||
if (child < 0)
|
||||
bail("unable to fork: child_func");
|
||||
|
||||
/*
|
||||
* State machine for synchronisation with the children.
|
||||
*
|
||||
* Father only return when both child and grandchild are
|
||||
* ready, so we can receive all possible error codes
|
||||
* generated by children.
|
||||
*/
|
||||
while (!ready) {
|
||||
enum sync_t s;
|
||||
int ret;
|
||||
|
||||
syncfd = sync_child_pipe[1];
|
||||
close(sync_child_pipe[0]);
|
||||
|
||||
if (read(syncfd, &s, sizeof(s)) != sizeof(s))
|
||||
bail("failed to sync with child: next state");
|
||||
|
||||
switch (s) {
|
||||
case SYNC_ERR:
|
||||
/* We have to mirror the error code of the child. */
|
||||
if (read(syncfd, &ret, sizeof(ret)) != sizeof(ret))
|
||||
bail("failed to sync with child: read(error code)");
|
||||
|
||||
exit(ret);
|
||||
case SYNC_USERMAP_PLS:
|
||||
/*
|
||||
* Enable setgroups(2) if we've been asked to. But we also
|
||||
* have to explicitly disable setgroups(2) if we're
|
||||
* creating a rootless container for single-entry mapping.
|
||||
* i.e. config.is_setgroup == false.
|
||||
* (this is required since Linux 3.19).
|
||||
*
|
||||
* For rootless multi-entry mapping, config.is_setgroup shall be true and
|
||||
* newuidmap/newgidmap shall be used.
|
||||
*/
|
||||
|
||||
if (config.is_rootless && !config.is_setgroup)
|
||||
update_setgroups(child, SETGROUPS_DENY);
|
||||
|
||||
/* Set up mappings. */
|
||||
update_uidmap(config.uidmappath, child, config.uidmap, config.uidmap_len);
|
||||
update_gidmap(config.gidmappath, child, config.gidmap, config.gidmap_len);
|
||||
|
||||
s = SYNC_USERMAP_ACK;
|
||||
if (write(syncfd, &s, sizeof(s)) != sizeof(s)) {
|
||||
kill(child, SIGKILL);
|
||||
bail("failed to sync with child: write(SYNC_USERMAP_ACK)");
|
||||
}
|
||||
break;
|
||||
case SYNC_RECVPID_PLS:{
|
||||
first_child = child;
|
||||
|
||||
/* Get the init_func pid. */
|
||||
if (read(syncfd, &child, sizeof(child)) != sizeof(child)) {
|
||||
kill(first_child, SIGKILL);
|
||||
bail("failed to sync with child: read(childpid)");
|
||||
}
|
||||
|
||||
/* Send ACK. */
|
||||
s = SYNC_RECVPID_ACK;
|
||||
if (write(syncfd, &s, sizeof(s)) != sizeof(s)) {
|
||||
kill(first_child, SIGKILL);
|
||||
kill(child, SIGKILL);
|
||||
bail("failed to sync with child: write(SYNC_RECVPID_ACK)");
|
||||
}
|
||||
}
|
||||
break;
|
||||
case SYNC_CHILD_READY:
|
||||
ready = true;
|
||||
break;
|
||||
default:
|
||||
bail("unexpected sync value: %u", s);
|
||||
}
|
||||
}
|
||||
|
||||
/* Now sync with grandchild. */
|
||||
|
||||
ready = false;
|
||||
while (!ready) {
|
||||
enum sync_t s;
|
||||
int ret;
|
||||
|
||||
syncfd = sync_grandchild_pipe[1];
|
||||
close(sync_grandchild_pipe[0]);
|
||||
|
||||
s = SYNC_GRANDCHILD;
|
||||
if (write(syncfd, &s, sizeof(s)) != sizeof(s)) {
|
||||
kill(child, SIGKILL);
|
||||
bail("failed to sync with child: write(SYNC_GRANDCHILD)");
|
||||
}
|
||||
|
||||
if (read(syncfd, &s, sizeof(s)) != sizeof(s))
|
||||
bail("failed to sync with child: next state");
|
||||
|
||||
switch (s) {
|
||||
case SYNC_ERR:
|
||||
/* We have to mirror the error code of the child. */
|
||||
if (read(syncfd, &ret, sizeof(ret)) != sizeof(ret))
|
||||
bail("failed to sync with child: read(error code)");
|
||||
|
||||
exit(ret);
|
||||
case SYNC_CHILD_READY:
|
||||
ready = true;
|
||||
break;
|
||||
default:
|
||||
bail("unexpected sync value: %u", s);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Send the init_func pid and the pid of the first child back to our parent.
|
||||
*
|
||||
* We need to send both back because we can't reap the first child we created (CLONE_PARENT).
|
||||
* It becomes the responsibility of our parent to reap the first child.
|
||||
*/
|
||||
len = snprintf(buf, JSON_MAX, "{\"pid\": %d, \"pid_first\": %d}\n", child, first_child);
|
||||
if (len < 0) {
|
||||
kill(child, SIGKILL);
|
||||
bail("unable to generate JSON for child pid");
|
||||
}
|
||||
if (write(pipenum, buf, len) != len) {
|
||||
kill(child, SIGKILL);
|
||||
bail("unable to send child pid to bootstrapper");
|
||||
}
|
||||
|
||||
exit(0);
|
||||
}
|
||||
|
||||
/*
|
||||
* Stage 1: We're in the first child process. Our job is to join any
|
||||
* provided namespaces in the netlink payload and unshare all
|
||||
* of the requested namespaces. If we've been asked to
|
||||
* CLONE_NEWUSER, we will ask our parent (stage 0) to set up
|
||||
* our user mappings for us. Then, we create a new child
|
||||
* (stage 2: JUMP_INIT) for PID namespace. We then send the
|
||||
* child's PID to our parent (stage 0).
|
||||
*/
|
||||
case JUMP_CHILD:{
|
||||
pid_t child;
|
||||
enum sync_t s;
|
||||
|
||||
/* We're in a child and thus need to tell the parent if we die. */
|
||||
syncfd = sync_child_pipe[0];
|
||||
close(sync_child_pipe[1]);
|
||||
|
||||
/* For debugging. */
|
||||
prctl(PR_SET_NAME, (unsigned long)"runc:[1:CHILD]", 0, 0, 0);
|
||||
|
||||
/*
|
||||
* We need to setns first. We cannot do this earlier (in stage 0)
|
||||
* because of the fact that we forked to get here (the PID of
|
||||
* [stage 2: JUMP_INIT]) would be meaningless). We could send it
|
||||
* using cmsg(3) but that's just annoying.
|
||||
*/
|
||||
if (config.namespaces)
|
||||
join_namespaces(config.namespaces);
|
||||
|
||||
/*
|
||||
* Deal with user namespaces first. They are quite special, as they
|
||||
* affect our ability to unshare other namespaces and are used as
|
||||
* context for privilege checks.
|
||||
*
|
||||
* We don't unshare all namespaces in one go. The reason for this
|
||||
* is that, while the kernel documentation may claim otherwise,
|
||||
* there are certain cases where unsharing all namespaces at once
|
||||
* will result in namespace objects being owned incorrectly.
|
||||
* Ideally we should just fix these kernel bugs, but it's better to
|
||||
* be safe than sorry, and fix them separately.
|
||||
*
|
||||
* A specific case of this is that the SELinux label of the
|
||||
* internal kern-mount that mqueue uses will be incorrect if the
|
||||
* UTS namespace is cloned before the USER namespace is mapped.
|
||||
* I've also heard of similar problems with the network namespace
|
||||
* in some scenarios. This also mirrors how LXC deals with this
|
||||
* problem.
|
||||
*/
|
||||
if (config.cloneflags & CLONE_NEWUSER) {
|
||||
if (unshare(CLONE_NEWUSER) < 0)
|
||||
bail("failed to unshare user namespace");
|
||||
config.cloneflags &= ~CLONE_NEWUSER;
|
||||
|
||||
/*
|
||||
* We don't have the privileges to do any mapping here (see the
|
||||
* clone_parent rant). So signal our parent to hook us up.
|
||||
*/
|
||||
|
||||
/* Switching is only necessary if we joined namespaces. */
|
||||
if (config.namespaces) {
|
||||
if (prctl(PR_SET_DUMPABLE, 1, 0, 0, 0) < 0)
|
||||
bail("failed to set process as dumpable");
|
||||
}
|
||||
s = SYNC_USERMAP_PLS;
|
||||
if (write(syncfd, &s, sizeof(s)) != sizeof(s))
|
||||
bail("failed to sync with parent: write(SYNC_USERMAP_PLS)");
|
||||
|
||||
/* ... wait for mapping ... */
|
||||
|
||||
if (read(syncfd, &s, sizeof(s)) != sizeof(s))
|
||||
bail("failed to sync with parent: read(SYNC_USERMAP_ACK)");
|
||||
if (s != SYNC_USERMAP_ACK)
|
||||
bail("failed to sync with parent: SYNC_USERMAP_ACK: got %u", s);
|
||||
/* Switching is only necessary if we joined namespaces. */
|
||||
if (config.namespaces) {
|
||||
if (prctl(PR_SET_DUMPABLE, 0, 0, 0, 0) < 0)
|
||||
bail("failed to set process as dumpable");
|
||||
}
|
||||
|
||||
/* Become root in the namespace proper. */
|
||||
if (setresuid(0, 0, 0) < 0)
|
||||
bail("failed to become root in user namespace");
|
||||
}
|
||||
|
||||
/*
|
||||
* Unshare all of the namespaces. Note that we don't merge this
|
||||
* with clone() because there were some old kernel versions where
|
||||
* clone(CLONE_PARENT | CLONE_NEWPID) was broken, so we'll just do
|
||||
* it the long way.
|
||||
*/
|
||||
if (unshare(config.cloneflags) < 0)
|
||||
bail("failed to unshare namespaces");
|
||||
|
||||
/*
|
||||
* TODO: What about non-namespace clone flags that we're dropping here?
|
||||
*
|
||||
* We fork again because of PID namespace, setns(2) or unshare(2) don't
|
||||
* change the PID namespace of the calling process, because doing so
|
||||
* would change the caller's idea of its own PID (as reported by getpid()),
|
||||
* which would break many applications and libraries, so we must fork
|
||||
* to actually enter the new PID namespace.
|
||||
*/
|
||||
child = clone_parent(&env, JUMP_INIT);
|
||||
if (child < 0)
|
||||
bail("unable to fork: init_func");
|
||||
|
||||
/* Send the child to our parent, which knows what it's doing. */
|
||||
s = SYNC_RECVPID_PLS;
|
||||
if (write(syncfd, &s, sizeof(s)) != sizeof(s)) {
|
||||
kill(child, SIGKILL);
|
||||
bail("failed to sync with parent: write(SYNC_RECVPID_PLS)");
|
||||
}
|
||||
if (write(syncfd, &child, sizeof(child)) != sizeof(child)) {
|
||||
kill(child, SIGKILL);
|
||||
bail("failed to sync with parent: write(childpid)");
|
||||
}
|
||||
|
||||
/* ... wait for parent to get the pid ... */
|
||||
|
||||
if (read(syncfd, &s, sizeof(s)) != sizeof(s)) {
|
||||
kill(child, SIGKILL);
|
||||
bail("failed to sync with parent: read(SYNC_RECVPID_ACK)");
|
||||
}
|
||||
if (s != SYNC_RECVPID_ACK) {
|
||||
kill(child, SIGKILL);
|
||||
bail("failed to sync with parent: SYNC_RECVPID_ACK: got %u", s);
|
||||
}
|
||||
|
||||
s = SYNC_CHILD_READY;
|
||||
if (write(syncfd, &s, sizeof(s)) != sizeof(s)) {
|
||||
kill(child, SIGKILL);
|
||||
bail("failed to sync with parent: write(SYNC_CHILD_READY)");
|
||||
}
|
||||
|
||||
/* Our work is done. [Stage 2: JUMP_INIT] is doing the rest of the work. */
|
||||
exit(0);
|
||||
}
|
||||
|
||||
/*
|
||||
* Stage 2: We're the final child process, and the only process that will
|
||||
* actually return to the Go runtime. Our job is to just do the
|
||||
* final cleanup steps and then return to the Go runtime to allow
|
||||
* init_linux.go to run.
|
||||
*/
|
||||
case JUMP_INIT:{
|
||||
/*
|
||||
* We're inside the child now, having jumped from the
|
||||
* start_child() code after forking in the parent.
|
||||
*/
|
||||
enum sync_t s;
|
||||
|
||||
/* We're in a child and thus need to tell the parent if we die. */
|
||||
syncfd = sync_grandchild_pipe[0];
|
||||
close(sync_grandchild_pipe[1]);
|
||||
close(sync_child_pipe[0]);
|
||||
close(sync_child_pipe[1]);
|
||||
|
||||
/* For debugging. */
|
||||
prctl(PR_SET_NAME, (unsigned long)"runc:[2:INIT]", 0, 0, 0);
|
||||
|
||||
if (read(syncfd, &s, sizeof(s)) != sizeof(s))
|
||||
bail("failed to sync with parent: read(SYNC_GRANDCHILD)");
|
||||
if (s != SYNC_GRANDCHILD)
|
||||
bail("failed to sync with parent: SYNC_GRANDCHILD: got %u", s);
|
||||
|
||||
if (setsid() < 0)
|
||||
bail("setsid failed");
|
||||
|
||||
if (setuid(0) < 0)
|
||||
bail("setuid failed");
|
||||
|
||||
if (setgid(0) < 0)
|
||||
bail("setgid failed");
|
||||
|
||||
if (!config.is_rootless && config.is_setgroup) {
|
||||
if (setgroups(0, NULL) < 0)
|
||||
bail("setgroups failed");
|
||||
}
|
||||
|
||||
s = SYNC_CHILD_READY;
|
||||
if (write(syncfd, &s, sizeof(s)) != sizeof(s))
|
||||
bail("failed to sync with patent: write(SYNC_CHILD_READY)");
|
||||
|
||||
/* Close sync pipes. */
|
||||
close(sync_grandchild_pipe[0]);
|
||||
|
||||
/* Free netlink data. */
|
||||
nl_free(&config);
|
||||
|
||||
/* Finish executing, let the Go runtime take over. */
|
||||
return;
|
||||
}
|
||||
default:
|
||||
bail("unexpected jump value");
|
||||
}
|
||||
|
||||
/* Should never be reached. */
|
||||
bail("should never be reached");
|
||||
}
|
|
@ -42,6 +42,12 @@ func needsSetupDev(config *configs.Config) bool {
|
|||
// finalizeRootfs after this function to finish setting up the rootfs.
|
||||
func prepareRootfs(pipe io.ReadWriter, iConfig *initConfig) (err error) {
|
||||
config := iConfig.Config
|
||||
if config.Rootfs == "/" {
|
||||
if err := unix.Chdir(config.Rootfs); err != nil {
|
||||
return newSystemErrorWithCausef(err, "changing dir to %q", config.Rootfs)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
if err := prepareRoot(config); err != nil {
|
||||
return newSystemErrorWithCause(err, "preparing rootfs")
|
||||
}
|
||||
|
@ -129,6 +135,9 @@ func prepareRootfs(pipe io.ReadWriter, iConfig *initConfig) (err error) {
|
|||
// finalizeRootfs sets anything to ro if necessary. You must call
|
||||
// prepareRootfs first.
|
||||
func finalizeRootfs(config *configs.Config) (err error) {
|
||||
if config.Rootfs == "/" {
|
||||
return nil
|
||||
}
|
||||
// remount dev as ro if specified
|
||||
for _, m := range config.Mounts {
|
||||
if libcontainerUtils.CleanPath(m.Destination) == "/dev" {
|
||||
|
@ -152,6 +161,26 @@ func finalizeRootfs(config *configs.Config) (err error) {
|
|||
return nil
|
||||
}
|
||||
|
||||
// /tmp has to be mounted as private to allow MS_MOVE to work in all situations
|
||||
func prepareTmp(topTmpDir string) (string, error) {
|
||||
tmpdir, err := ioutil.TempDir(topTmpDir, "runctop")
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
if err := unix.Mount(tmpdir, tmpdir, "bind", unix.MS_BIND, ""); err != nil {
|
||||
return "", err
|
||||
}
|
||||
if err := unix.Mount("", tmpdir, "", uintptr(unix.MS_PRIVATE), ""); err != nil {
|
||||
return "", err
|
||||
}
|
||||
return tmpdir, nil
|
||||
}
|
||||
|
||||
func cleanupTmp(tmpdir string) error {
|
||||
unix.Unmount(tmpdir, 0)
|
||||
return os.RemoveAll(tmpdir)
|
||||
}
|
||||
|
||||
func mountCmd(cmd configs.Command) error {
|
||||
command := exec.Command(cmd.Path, cmd.Args[:]...)
|
||||
command.Env = cmd.Env
|
||||
|
@ -199,7 +228,12 @@ func mountToRootfs(m *configs.Mount, rootfs, mountLabel string) error {
|
|||
}
|
||||
}
|
||||
if copyUp {
|
||||
tmpDir, err = ioutil.TempDir("/tmp", "runctmpdir")
|
||||
tmpdir, err := prepareTmp("/tmp")
|
||||
if err != nil {
|
||||
return newSystemErrorWithCause(err, "tmpcopyup: failed to setup tmpdir")
|
||||
}
|
||||
defer cleanupTmp(tmpdir)
|
||||
tmpDir, err = ioutil.TempDir(tmpdir, "runctmpdir")
|
||||
if err != nil {
|
||||
return newSystemErrorWithCause(err, "tmpcopyup: failed to create tmpdir")
|
||||
}
|
||||
|
@ -396,6 +430,7 @@ func checkMountDestination(rootfs, dest string) error {
|
|||
"/proc/stat",
|
||||
"/proc/swaps",
|
||||
"/proc/uptime",
|
||||
"/proc/loadavg",
|
||||
"/proc/net/dev",
|
||||
}
|
||||
for _, valid := range validDestinations {
|
||||
|
@ -412,7 +447,7 @@ func checkMountDestination(rootfs, dest string) error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if path == "." || !strings.HasPrefix(path, "..") {
|
||||
if path != "." && !strings.HasPrefix(path, "..") {
|
||||
return fmt.Errorf("%q cannot be mounted because it is located inside %q", dest, invalid)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -289,22 +289,23 @@
|
|||
{"path":"github.com/opencontainers/go-digest","checksumSHA1":"NTperEHVh1uBqfTy9+oKceN4tKI=","revision":"21dfd564fd89c944783d00d069f33e3e7123c448","revisionTime":"2017-01-11T18:16:59Z"},
|
||||
{"path":"github.com/opencontainers/image-spec/specs-go","checksumSHA1":"ZGlIwSRjdLYCUII7JLE++N4w7Xc=","revision":"89b51c794e9113108a2914e38e66c826a649f2b5","revisionTime":"2017-11-03T11:36:04Z"},
|
||||
{"path":"github.com/opencontainers/image-spec/specs-go/v1","checksumSHA1":"jdbXRRzeu0njLE9/nCEZG+Yg/Jk=","revision":"89b51c794e9113108a2914e38e66c826a649f2b5","revisionTime":"2017-11-03T11:36:04Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer","checksumSHA1":"cqrf+HmlptrK2lRhQ0PkYdOe5cQ=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/apparmor","checksumSHA1":"gVVY8k2G3ws+V1czsfxfuRs8log=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/cgroups","checksumSHA1":"eA9qNw7Tkpi1GLojT/Vxa99aL00=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/cgroups/fs","checksumSHA1":"YgPihDRi3OxI0qv7CxTxrqZuvfU=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/cgroups/systemd","checksumSHA1":"RVQixM4pF56oCNymhNY67I5eS0Y=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/configs","checksumSHA1":"PUv5rdj6oEGJoBij/9Elx8VO6bs=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/configs/validate","checksumSHA1":"YJq+f9izqizSzG/OuVFUOfloNEk=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/criurpc","checksumSHA1":"I1iwXoDUJeDXviilCtkvDpEF/So=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/intelrdt","checksumSHA1":"bAWJX1XUDMd4GqPLSrCkUcdiTbg=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/keys","checksumSHA1":"QXuHZwxlqhoq/oHRJFbTi6+AWLY=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/mount","checksumSHA1":"MJiogPDUU2nFr1fzQU6T+Ry1W8o=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/seccomp","checksumSHA1":"I1Qw/btE1twMqKHpYNsC98cteak=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/stacktrace","checksumSHA1":"yp/kYBgVqKtxlnpq4CmyxLFMAE4=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/system","checksumSHA1":"cjg/UcueM1/2/ExZ3N7010sa+hI=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/user","checksumSHA1":"XtLpcP6ca9SQG218re7E7UcOj3Y=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/utils","checksumSHA1":"Sb296YW4V+esieanrx4Nzt2L5lU=","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer","checksumSHA1":"6qP/ejjcc/+HelbVHmtl6cyUZmc=","origin":"github.com/hashicorp/runc/libcontainer","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/apparmor","checksumSHA1":"gVVY8k2G3ws+V1czsfxfuRs8log=","origin":"github.com/hashicorp/runc/libcontainer/apparmor","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/cgroups","checksumSHA1":"eA9qNw7Tkpi1GLojT/Vxa99aL00=","origin":"github.com/hashicorp/runc/libcontainer/cgroups","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/cgroups/fs","checksumSHA1":"YgPihDRi3OxI0qv7CxTxrqZuvfU=","origin":"github.com/hashicorp/runc/libcontainer/cgroups/fs","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/cgroups/systemd","checksumSHA1":"RVQixM4pF56oCNymhNY67I5eS0Y=","origin":"github.com/hashicorp/runc/libcontainer/cgroups/systemd","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/configs","checksumSHA1":"PUv5rdj6oEGJoBij/9Elx8VO6bs=","origin":"github.com/hashicorp/runc/libcontainer/configs","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/configs/validate","checksumSHA1":"YJq+f9izqizSzG/OuVFUOfloNEk=","origin":"github.com/hashicorp/runc/libcontainer/configs/validate","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/criurpc","checksumSHA1":"I1iwXoDUJeDXviilCtkvDpEF/So=","origin":"github.com/hashicorp/runc/libcontainer/criurpc","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/intelrdt","checksumSHA1":"bAWJX1XUDMd4GqPLSrCkUcdiTbg=","origin":"github.com/hashicorp/runc/libcontainer/intelrdt","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z","version":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/keys","checksumSHA1":"QXuHZwxlqhoq/oHRJFbTi6+AWLY=","origin":"github.com/hashicorp/runc/libcontainer/keys","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z","version":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/mount","checksumSHA1":"MJiogPDUU2nFr1fzQU6T+Ry1W8o=","origin":"github.com/hashicorp/runc/libcontainer/mount","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z","version":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/nsenter","checksumSHA1":"5p0YhO25gaLG+GUdTzqgvcRHjkE=","origin":"github.com/hashicorp/runc/libcontainer/nsenter","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/seccomp","checksumSHA1":"I1Qw/btE1twMqKHpYNsC98cteak=","origin":"github.com/hashicorp/runc/libcontainer/seccomp","revision":"8df81be073884e4e4c613d893f5877310136820f","revisionTime":"2018-09-10T14:23:11Z","version":"nomad","versionExact":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/stacktrace","checksumSHA1":"yp/kYBgVqKtxlnpq4CmyxLFMAE4=","origin":"github.com/hashicorp/runc/libcontainer/stacktrace","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z","version":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/system","checksumSHA1":"cjg/UcueM1/2/ExZ3N7010sa+hI=","origin":"github.com/hashicorp/runc/libcontainer/system","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z","version":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/user","checksumSHA1":"XtLpcP6ca9SQG218re7E7UcOj3Y=","origin":"github.com/hashicorp/runc/libcontainer/user","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z","version":"nomad"},
|
||||
{"path":"github.com/opencontainers/runc/libcontainer/utils","checksumSHA1":"Sb296YW4V+esieanrx4Nzt2L5lU=","origin":"github.com/hashicorp/runc/libcontainer/utils","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z","version":"nomad"},
|
||||
{"path":"github.com/opencontainers/runtime-spec/specs-go","checksumSHA1":"AMYc2X2O/IL6EGrq6lTl5vEhLiY=","origin":"github.com/opencontainers/runc/vendor/github.com/opencontainers/runtime-spec/specs-go","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/selinux/go-selinux","checksumSHA1":"j9efF9bPmCCag+LzqwjyB8a44B8=","origin":"github.com/opencontainers/runc/vendor/github.com/opencontainers/selinux/go-selinux","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
{"path":"github.com/opencontainers/selinux/go-selinux/label","checksumSHA1":"QbeVoKIoaJWZDH8V/588i8/Pjjs=","origin":"github.com/opencontainers/runc/vendor/github.com/opencontainers/selinux/go-selinux/label","revision":"459bfaec1fc6c17d8bfb12d0a0f69e7e7271ed2a","revisionTime":"2018-08-23T14:46:37Z"},
|
||||
|
|
Loading…
Reference in New Issue