Skip to content

Commit

Permalink
DEVPROD-10457 contain and kill subprocess.exec/shell.exec processes (e…
Browse files Browse the repository at this point in the history
  • Loading branch information
ybrill authored Oct 14, 2024
1 parent d1ee299 commit c9fcf30
Show file tree
Hide file tree
Showing 17 changed files with 85 additions and 18 deletions.
2 changes: 1 addition & 1 deletion agent/agent.go
Original file line number Diff line number Diff line change
Expand Up @@ -1222,7 +1222,7 @@ func (a *Agent) killProcs(ctx context.Context, tc *taskContext, ignoreTaskGroupC

if tc.task.ID != "" && tc.taskConfig != nil {
logger.Infof("Cleaning up processes for task: '%s'.", tc.task.ID)
if err := agentutil.KillSpawnedProcs(ctx, tc.task.ID, tc.taskConfig.WorkDir, logger); err != nil {
if err := agentutil.KillSpawnedProcs(ctx, tc.task.ID, tc.taskConfig.WorkDir, tc.taskConfig.Distro.ExecUser, logger); err != nil {
// If the host is in a state where ps is timing out we need human intervention.
if psErr := errors.Cause(err); psErr == agentutil.ErrPSTimeout {
disableErr := a.comm.DisableHost(ctx, a.opts.HostID, apimodels.DisableInfo{Reason: psErr.Error()})
Expand Down
10 changes: 7 additions & 3 deletions agent/command/exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -220,7 +220,7 @@ func addTempDirs(env map[string]string, dir string) {
}
}

func (c *subprocessExec) getProc(ctx context.Context, execPath, taskID string, logger client.LoggerProducer) *jasper.Command {
func (c *subprocessExec) getProc(ctx context.Context, execPath string, conf *internal.TaskConfig, logger client.LoggerProducer) *jasper.Command {
cmd := c.JasperManager().CreateCommand(ctx).Add(append([]string{execPath}, c.Args...)).
Background(c.Background).Environment(c.Env).Directory(c.WorkingDir).
SuppressStandardError(c.IgnoreStandardError).SuppressStandardOutput(c.IgnoreStandardOutput).RedirectErrorToOutput(c.RedirectStandardErrorToOutput).
Expand Down Expand Up @@ -250,7 +250,7 @@ func (c *subprocessExec) getProc(ctx context.Context, execPath, taskID string, l

pid := proc.Info(ctx).PID

agentutil.TrackProcess(taskID, pid, logger.System())
agentutil.TrackProcess(conf.Task.Id, pid, logger.System())

if c.Background {
logger.Execution().Debugf("Running process in the background with pid %d.", pid)
Expand All @@ -277,6 +277,10 @@ func (c *subprocessExec) getProc(ctx context.Context, execPath, taskID string, l
}
}

if execUser := conf.Distro.ExecUser; execUser != "" {
cmd.SudoAs(execUser)
}

return cmd
}

Expand Down Expand Up @@ -389,7 +393,7 @@ func (c *subprocessExec) Execute(ctx context.Context, comm client.Communicator,
})
}

err = errors.WithStack(c.runCommand(ctx, c.getProc(ctx, execPath, conf.Task.Id, logger), logger))
err = errors.WithStack(c.runCommand(ctx, c.getProc(ctx, execPath, conf, logger), logger))

if ctxErr := ctx.Err(); ctxErr != nil {
logger.System().Debugf("Canceled command '%s', dumping running processes.", c.Name())
Expand Down
11 changes: 6 additions & 5 deletions agent/command/exec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"github.com/evergreen-ci/evergreen/agent/internal"
"github.com/evergreen-ci/evergreen/agent/internal/client"
agentutil "github.com/evergreen-ci/evergreen/agent/util"
"github.com/evergreen-ci/evergreen/apimodels"
"github.com/evergreen-ci/evergreen/model"
"github.com/evergreen-ci/evergreen/model/task"
"github.com/evergreen-ci/evergreen/testutil"
Expand Down Expand Up @@ -52,7 +53,7 @@ func (s *execCmdSuite) SetupTest() {
var err error

s.comm = client.NewMock("http://localhost.com")
s.conf = &internal.TaskConfig{Expansions: util.Expansions{}, Task: task.Task{}, Project: model.Project{}}
s.conf = &internal.TaskConfig{Expansions: util.Expansions{}, Distro: &apimodels.DistroView{}, Task: task.Task{}, Project: model.Project{}}
s.logger, err = s.comm.GetLoggerProducer(s.ctx, &s.conf.Task, nil)
s.Require().NoError(err)
}
Expand Down Expand Up @@ -179,7 +180,7 @@ func (s *execCmdSuite) TestRunCommand() {
}
cmd.SetJasperManager(s.jasper)
s.NoError(cmd.ParseParams(map[string]interface{}{}))
exec := cmd.getProc(s.ctx, cmd.Binary, "foo", s.logger)
exec := cmd.getProc(s.ctx, cmd.Binary, &internal.TaskConfig{Task: task.Task{Id: "foo"}, Distro: &apimodels.DistroView{}}, s.logger)
s.NoError(cmd.runCommand(s.ctx, exec, s.logger))
}

Expand All @@ -189,7 +190,7 @@ func (s *execCmdSuite) TestRunCommandPropagatesError() {
}
cmd.SetJasperManager(s.jasper)
s.NoError(cmd.ParseParams(map[string]interface{}{}))
exec := cmd.getProc(s.ctx, cmd.Binary, "foo", s.logger)
exec := cmd.getProc(s.ctx, cmd.Binary, &internal.TaskConfig{Task: task.Task{Id: "foo"}, Distro: &apimodels.DistroView{}}, s.logger)
err := cmd.runCommand(s.ctx, exec, s.logger)
s.Require().NotNil(err)
s.Contains(err.Error(), "process encountered problem: exit code 1")
Expand All @@ -203,7 +204,7 @@ func (s *execCmdSuite) TestRunCommandContinueOnErrorNoError() {
}
cmd.SetJasperManager(s.jasper)
s.NoError(cmd.ParseParams(map[string]interface{}{}))
exec := cmd.getProc(s.ctx, cmd.Binary, "foo", s.logger)
exec := cmd.getProc(s.ctx, cmd.Binary, &internal.TaskConfig{Task: task.Task{Id: "foo"}, Distro: &apimodels.DistroView{}}, s.logger)
s.NoError(cmd.runCommand(s.ctx, exec, s.logger))
}

Expand All @@ -215,7 +216,7 @@ func (s *execCmdSuite) TestRunCommandBackgroundAlwaysNil() {
}
cmd.SetJasperManager(s.jasper)
s.NoError(cmd.ParseParams(map[string]interface{}{}))
exec := cmd.getProc(s.ctx, cmd.Binary, "foo", s.logger)
exec := cmd.getProc(s.ctx, cmd.Binary, &internal.TaskConfig{Task: task.Task{Id: "foo"}, Distro: &apimodels.DistroView{}}, s.logger)
s.NoError(cmd.runCommand(s.ctx, exec, s.logger))
}

Expand Down
4 changes: 4 additions & 0 deletions agent/command/shell.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,6 +226,10 @@ func (c *shellExec) Execute(ctx context.Context, _ client.Communicator, logger c
}
}

if execUser := conf.Distro.ExecUser; execUser != "" {
cmd.SudoAs(execUser)
}

err = cmd.Run(ctx)
if !c.Background && err != nil {
if exitCode, _ := cmd.Wait(ctx); exitCode != 0 {
Expand Down
2 changes: 2 additions & 0 deletions agent/command/shell_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"github.com/evergreen-ci/evergreen/agent/internal"
"github.com/evergreen-ci/evergreen/agent/internal/client"
agentutil "github.com/evergreen-ci/evergreen/agent/util"
"github.com/evergreen-ci/evergreen/apimodels"
"github.com/evergreen-ci/evergreen/model"
"github.com/evergreen-ci/evergreen/model/task"
"github.com/evergreen-ci/evergreen/testutil"
Expand Down Expand Up @@ -55,6 +56,7 @@ func (s *shellExecuteCommandSuite) SetupTest() {
Secret: "task_secret",
},
Project: model.Project{},
Distro: &apimodels.DistroView{},
}
s.logger, err = s.comm.GetLoggerProducer(s.ctx, &s.conf.Task, nil)
s.Require().NoError(err)
Expand Down
3 changes: 1 addition & 2 deletions agent/internal/testutil/task_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,7 @@ func MakeTaskConfigFromModelData(ctx context.Context, settings *evergreen.Settin
if err != nil {
return nil, errors.Wrap(err, "populating expansions")
}
var dv *apimodels.DistroView
config, err := internal.NewTaskConfig(data.Host.Distro.WorkDir, dv, data.Project, data.Task, data.ProjectRef, nil, &apimodels.ExpansionsAndVars{Expansions: exp})
config, err := internal.NewTaskConfig(data.Host.Distro.WorkDir, &apimodels.DistroView{}, data.Project, data.Task, data.ProjectRef, nil, &apimodels.ExpansionsAndVars{Expansions: exp})
if err != nil {
return nil, errors.Wrap(err, "making task config from test model data")
}
Expand Down
2 changes: 1 addition & 1 deletion agent/task_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,7 @@ func (a *Agent) makeTaskConfig(ctx context.Context, tc *taskContext) (*internal.
}

grip.Info("Fetching distro configuration.")
var confDistro *apimodels.DistroView
confDistro := &apimodels.DistroView{}
if a.opts.Mode == globals.HostMode {
var err error
confDistro, err = a.comm.GetDistroView(ctx, tc.task)
Expand Down
25 changes: 24 additions & 1 deletion agent/util/subtree_unix.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (

"github.com/evergreen-ci/utility"
"github.com/mongodb/grip"
"github.com/mongodb/jasper"
"github.com/pkg/errors"
)

Expand All @@ -20,6 +21,8 @@ const (
cleanupCheckTimeoutMin = 100 * time.Millisecond
cleanupCheckTimeoutMax = time.Second
contextTimeout = 10 * time.Minute

pkillNoMatchingProcessesExitCode = 1
)

// TrackProcess is a noop by default if we don't need to do any special
Expand All @@ -28,7 +31,13 @@ func TrackProcess(key string, pid int, logger grip.Journaler) {}

// KillSpawnedProcs kills processes that descend from the agent and waits
// for them to terminate.
func KillSpawnedProcs(ctx context.Context, key, workingDir string, logger grip.Journaler) error {
func KillSpawnedProcs(ctx context.Context, key, workingDir, execUser string, logger grip.Journaler) error {
// When execUser is set all subprocess.exec and shell.exec processes are run under that user. This facilitates
// easy cleanup of processes started by a task. It is enabled on a distro-by-distro basis.
if execUser != "" {
return killUserProcesses(ctx, execUser)
}

pidsToKill, err := getPIDsToKill(ctx, key, workingDir, logger)
if err != nil {
return errors.Wrap(err, "getting list of PIDs to kill")
Expand Down Expand Up @@ -56,6 +65,20 @@ func KillSpawnedProcs(ctx context.Context, key, workingDir string, logger grip.J

}

func killUserProcesses(ctx context.Context, execUser string) error {
if execUser == "" {
return errors.New("execUser cannot be empty")
}
cmd := jasper.NewCommand().Add([]string{"pkill", "-SIGKILL", "-U", execUser}).Sudo(true)
if err := cmd.Run(ctx); err != nil {
exitCode, _ := cmd.Wait(ctx)
if exitCode != pkillNoMatchingProcessesExitCode {
return errors.Wrapf(err, "killing processes for user '%s'", execUser)
}
}
return nil
}

func getPIDsToKill(ctx context.Context, key, workingDir string, logger grip.Journaler) ([]int, error) {
var pidsToKill []int

Expand Down
6 changes: 3 additions & 3 deletions agent/util/subtree_unix_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,20 +77,20 @@ func TestKillSpawnedProcs(t *testing.T) {
expiredContext, cancel := context.WithTimeout(ctx, -time.Second)
defer cancel()

err := KillSpawnedProcs(expiredContext, "", "", grip.GetDefaultJournaler())
err := KillSpawnedProcs(expiredContext, "", "", "", grip.GetDefaultJournaler())
assert.Error(t, err)
assert.Equal(t, ErrPSTimeout, errors.Cause(err))
},
"ErrorsWithContextCancelled": func(ctx context.Context, t *testing.T) {
cancelledContext, cancel := context.WithCancel(ctx)
cancel()

err := KillSpawnedProcs(cancelledContext, "", "", grip.GetDefaultJournaler())
err := KillSpawnedProcs(cancelledContext, "", "", "", grip.GetDefaultJournaler())
assert.Error(t, err)
assert.NotEqual(t, ErrPSTimeout, errors.Cause(err))
},
"SucceedsWithNoContextError": func(ctx context.Context, t *testing.T) {
err := KillSpawnedProcs(ctx, "", "", grip.GetDefaultJournaler())
err := KillSpawnedProcs(ctx, "", "", "", grip.GetDefaultJournaler())
assert.NoError(t, err)
},
} {
Expand Down
2 changes: 1 addition & 1 deletion agent/util/subtree_windows.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ func TrackProcess(taskId string, pid int, logger grip.Journaler) {
// cleanup() has a windows-specific implementation which finds the job object associated with the
// given task key, and if it exists, terminates it. This will guarantee that any shell processes
// started throughout the task run are destroyed, as long as they were captured in trackProcess.
func KillSpawnedProcs(ctx context.Context, key, workingDir string, logger grip.Journaler) error {
func KillSpawnedProcs(ctx context.Context, key, workingDir, _ string, logger grip.Journaler) error {
job, err := processMapping.getJob(key)
if err != nil {
return nil
Expand Down
1 change: 1 addition & 0 deletions apimodels/agent_models.go
Original file line number Diff line number Diff line change
Expand Up @@ -376,6 +376,7 @@ type GeneratePollResponse struct {
type DistroView struct {
DisableShallowClone bool `json:"disable_shallow_clone"`
Mountpoints []string `json:"mountpoints"`
ExecUser string `json:"exec_user"`
}

// ExpansionsAndVars represents expansions, project variables, and parameters
Expand Down
2 changes: 1 addition & 1 deletion config.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ var (

// Agent version to control agent rollover. The format is the calendar date
// (YYYY-MM-DD).
AgentVersion = "2024-10-03"
AgentVersion = "2024-10-14"
)

const (
Expand Down
22 changes: 22 additions & 0 deletions docs/decisions/2024-10-14_task_process_termination.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
# 2024-10-14 cleanup task processes by user

* status: accepted
* date: 2024-10-14
* authors: Jonathan Brill

## Context and Problem Statement

Among the inter-task cleanup steps the agent does is terminating processes started by the previous task. On UNIX systems, picking out the specific processes started by a task has always been a challenge. The current solution is to mark processes by setting an environment variable and rely on the task to propagate its environment to any child processes. This doesn't always work so we added an additional check for processes with a working directory set to the task's working directory ([EVG-12948](https://jira.mongodb.org/browse/EVG-12948)). This is not foolproof either. Additionally, when SIP is enabled on macOS a process is not allowed to inspect another processes's environment variables ([DEVPROD-6539](https://jira.mongodb.org/browse/DEVPROD-6539)).

## Considered Options
1. For macOS specifically, the process sandbox ([sandbox-exec](https://reverse.put.as/wp-content/uploads/2011/09/Apple-Sandbox-Guide-v1.0.pdf)) was investigated. It was discovered the sandbox does not support containing processes as it is more geared to limiting the permissions of a process. Additionally, the sandbox is officially deprecated.

2. Manually tracking process forks to reconstruct the process tree despite daemonized processes getting reparented to init. This is overly complex and its implementation would be coupled with the underlying system. For example, auditd could be configured to log when a process forks but it's disabled on macOS and will soon be removed.

3. An easy solution to [DEVPROD-6539](https://jira.mongodb.org/browse/DEVPROD-6539) would have been if we could have somehow made the agent able to read the environment variables of other processes, but it appears Apple has really locked it down. A short list of exceptions appears [here](https://github.com/apple-oss-distributions/xnu/blob/main/bsd/kern/kern_sysctl.c#L1380-L1386) but there's no way to get the agent to qualify as any of them since the entitlement doesn't work.

## Decision Outcome
Start processes as a user that's dedicated to task processes. A utility such as pkill can signal all processes belonging to that user. Turning it on for a distro will require that distro to have an agent user with passwordless sudo and another user to use for tasks.

## More Information
Giving processes a new user isolated from the agent's user will be good for other reasons as well. For one, the agent user's permissions vary and can include passwordless sudo, so a new user will give us the opportunity to apply a more consistently secure profile.
3 changes: 3 additions & 0 deletions model/distro/distro.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,9 @@ type Distro struct {
Mountpoints []string `bson:"mountpoints,omitempty" json:"mountpoints,omitempty" mapstructure:"mountpoints,omitempty"`
// ImageID is not equivalent to AMI. It is the identifier of the base image for the distro.
ImageID string `bson:"image_id,omitempty" json:"image_id,omitempty" mapstructure:"image_id,omitempty"`

// ExecUser is the user to run shell.exec and subprocess.exec processes as. If unset, processes are run as the regular distro User.
ExecUser string `bson:"exec_user,omitempty" json:"exec_user,omitempty" mapstructure:"exec_user,omitempty"`
}

// DistroData is the same as a distro, with the only difference being that all
Expand Down
3 changes: 3 additions & 0 deletions rest/model/distro.go
Original file line number Diff line number Diff line change
Expand Up @@ -365,6 +365,7 @@ type APIDistro struct {
ValidProjects []*string `json:"valid_projects"`
Mountpoints []string `json:"mountpoints"`
ImageID *string `json:"image_id"`
ExecUser *string `json:"exec_user"`
}

// BuildFromService converts from service level distro.Distro to an APIDistro
Expand All @@ -390,6 +391,7 @@ func (apiDistro *APIDistro) BuildFromService(d distro.Distro) {
apiDistro.ValidProjects = utility.ToStringPtrSlice(d.ValidProjects)
apiDistro.Mountpoints = d.Mountpoints
apiDistro.ImageID = utility.ToStringPtr(d.ImageID)
apiDistro.ExecUser = utility.ToStringPtr(d.ExecUser)

if d.Expansions != nil {
apiDistro.Expansions = []APIExpansion{}
Expand Down Expand Up @@ -455,6 +457,7 @@ func (apiDistro *APIDistro) ToService() *distro.Distro {
d.Disabled = apiDistro.Disabled
d.ContainerPool = utility.FromStringPtr(apiDistro.ContainerPool)
d.ImageID = utility.FromStringPtr(apiDistro.ImageID)
d.ExecUser = utility.FromStringPtr(apiDistro.ExecUser)

d.FinderSettings = apiDistro.FinderSettings.ToService()
d.PlannerSettings = apiDistro.PlannerSettings.ToService()
Expand Down
4 changes: 4 additions & 0 deletions rest/model/distro_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ func TestDistroBuildFromService(t *testing.T) {
ConfigPath: "config_path",
},
Mountpoints: []string{"/", "/data"},
ExecUser: "exec_user",
}
apiDistro := &APIDistro{}
apiDistro.BuildFromService(d)
Expand All @@ -48,6 +49,7 @@ func TestDistroBuildFromService(t *testing.T) {
assert.Equal(t, d.IceCreamSettings.SchedulerHost, utility.FromStringPtr(apiDistro.IcecreamSettings.SchedulerHost))
assert.Equal(t, d.IceCreamSettings.ConfigPath, utility.FromStringPtr(apiDistro.IcecreamSettings.ConfigPath))
assert.Equal(t, d.Mountpoints, apiDistro.Mountpoints)
assert.Equal(t, d.ExecUser, utility.FromStringPtr(apiDistro.ExecUser))
}

func TestDistroBuildFromServiceDefaults(t *testing.T) {
Expand Down Expand Up @@ -97,6 +99,7 @@ func TestDistroToService(t *testing.T) {
ConfigPath: utility.ToStringPtr("config_path"),
},
Mountpoints: []string{"/", "/data"},
ExecUser: utility.ToStringPtr("exec_user"),
}

d := apiDistro.ToService()
Expand Down Expand Up @@ -124,6 +127,7 @@ func TestDistroToService(t *testing.T) {
assert.Equal(t, utility.FromStringPtr(apiDistro.IcecreamSettings.SchedulerHost), d.IceCreamSettings.SchedulerHost)
assert.Equal(t, utility.FromStringPtr(apiDistro.IcecreamSettings.ConfigPath), d.IceCreamSettings.ConfigPath)
assert.Equal(t, apiDistro.Mountpoints, d.Mountpoints)
assert.Equal(t, utility.FromStringPtr(apiDistro.ExecUser), d.ExecUser)
}

func TestDistroToServiceDefaults(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions rest/route/agent.go
Original file line number Diff line number Diff line change
Expand Up @@ -613,6 +613,7 @@ func (h *getDistroViewHandler) Run(ctx context.Context) gimlet.Responder {
dv := apimodels.DistroView{
DisableShallowClone: host.Distro.DisableShallowClone,
Mountpoints: host.Distro.Mountpoints,
ExecUser: host.Distro.ExecUser,
}
return gimlet.NewJSONResponse(dv)
}
Expand Down

0 comments on commit c9fcf30

Please sign in to comment.