14406: Merge branch 'master'
[arvados.git] / services / crunch-run / crunchrun.go
index 2f9ccf52460a667215cdfb9156b7df56605712a5..1deb74031667d7ade04968344d3b262b3ccf1dd1 100644 (file)
@@ -32,6 +32,7 @@ import (
        "git.curoverse.com/arvados.git/sdk/go/arvadosclient"
        "git.curoverse.com/arvados.git/sdk/go/keepclient"
        "git.curoverse.com/arvados.git/sdk/go/manifest"
+       "github.com/shirou/gopsutil/process"
        "golang.org/x/net/context"
 
        dockertypes "github.com/docker/docker/api/types"
@@ -60,6 +61,7 @@ type IKeepClient interface {
        PutB(buf []byte) (string, int, error)
        ReadAt(locator string, p []byte, off int) (int, error)
        ManifestFileReader(m manifest.Manifest, filename string) (arvados.File, error)
+       LocalLocator(locator string) (string, error)
        ClearBlockCache()
 }
 
@@ -78,45 +80,54 @@ type ThinDockerClient interface {
        ContainerStart(ctx context.Context, container string, options dockertypes.ContainerStartOptions) error
        ContainerRemove(ctx context.Context, container string, options dockertypes.ContainerRemoveOptions) error
        ContainerWait(ctx context.Context, container string, condition dockercontainer.WaitCondition) (<-chan dockercontainer.ContainerWaitOKBody, <-chan error)
+       ContainerInspect(ctx context.Context, id string) (dockertypes.ContainerJSON, error)
        ImageInspectWithRaw(ctx context.Context, image string) (dockertypes.ImageInspect, []byte, error)
        ImageLoad(ctx context.Context, input io.Reader, quiet bool) (dockertypes.ImageLoadResponse, error)
        ImageRemove(ctx context.Context, image string, options dockertypes.ImageRemoveOptions) ([]dockertypes.ImageDeleteResponseItem, error)
 }
 
+type PsProcess interface {
+       CmdlineSlice() ([]string, error)
+}
+
 // ContainerRunner is the main stateful struct used for a single execution of a
 // container.
 type ContainerRunner struct {
-       Docker    ThinDockerClient
-       client    *arvados.Client
-       ArvClient IArvadosClient
-       Kc        IKeepClient
-       arvados.Container
+       Docker          ThinDockerClient
+       client          *arvados.Client
+       ArvClient       IArvadosClient
+       Kc              IKeepClient
+       Container       arvados.Container
        ContainerConfig dockercontainer.Config
-       dockercontainer.HostConfig
-       token       string
-       ContainerID string
-       ExitCode    *int
-       NewLogWriter
-       loggingDone   chan bool
-       CrunchLog     *ThrottledLogger
-       Stdout        io.WriteCloser
-       Stderr        io.WriteCloser
-       LogCollection arvados.CollectionFileSystem
-       LogsPDH       *string
-       RunArvMount
-       MkTempDir
-       ArvMount      *exec.Cmd
-       ArvMountPoint string
-       HostOutputDir string
-       Binds         []string
-       Volumes       map[string]struct{}
-       OutputPDH     *string
-       SigChan       chan os.Signal
-       ArvMountExit  chan error
-       SecretMounts  map[string]arvados.Mount
-       MkArvClient   func(token string) (IArvadosClient, error)
-       finalState    string
-       parentTemp    string
+       HostConfig      dockercontainer.HostConfig
+       token           string
+       ContainerID     string
+       ExitCode        *int
+       NewLogWriter    NewLogWriter
+       loggingDone     chan bool
+       CrunchLog       *ThrottledLogger
+       Stdout          io.WriteCloser
+       Stderr          io.WriteCloser
+       logUUID         string
+       logMtx          sync.Mutex
+       LogCollection   arvados.CollectionFileSystem
+       LogsPDH         *string
+       RunArvMount     RunArvMount
+       MkTempDir       MkTempDir
+       ArvMount        *exec.Cmd
+       ArvMountPoint   string
+       HostOutputDir   string
+       Binds           []string
+       Volumes         map[string]struct{}
+       OutputPDH       *string
+       SigChan         chan os.Signal
+       ArvMountExit    chan error
+       SecretMounts    map[string]arvados.Mount
+       MkArvClient     func(token string) (IArvadosClient, IKeepClient, error)
+       finalState      string
+       parentTemp      string
+
+       ListProcesses func() ([]PsProcess, error)
 
        statLogger       io.WriteCloser
        statReporter     *crunchstat.Reporter
@@ -140,10 +151,14 @@ type ContainerRunner struct {
 
        cStateLock sync.Mutex
        cCancelled bool // StopContainer() invoked
+       cRemoved   bool // docker confirmed the container no longer exists
 
-       enableNetwork string // one of "default" or "always"
-       networkMode   string // passed through to HostConfig.NetworkMode
-       arvMountLog   *ThrottledLogger
+       enableNetwork   string // one of "default" or "always"
+       networkMode     string // passed through to HostConfig.NetworkMode
+       arvMountLog     *ThrottledLogger
+       checkContainerd time.Duration
+
+       containerWatchdogInterval time.Duration
 }
 
 // setupSignals sets up signal handling to gracefully terminate the underlying
@@ -177,31 +192,39 @@ func (runner *ContainerRunner) stop(sig os.Signal) {
        if err != nil {
                runner.CrunchLog.Printf("error removing container: %s", err)
        }
+       if err == nil || strings.Contains(err.Error(), "No such container: "+runner.ContainerID) {
+               runner.cRemoved = true
+       }
 }
 
 var errorBlacklist = []string{
        "(?ms).*[Cc]annot connect to the Docker daemon.*",
        "(?ms).*oci runtime error.*starting container process.*container init.*mounting.*to rootfs.*no such file or directory.*",
+       "(?ms).*grpc: the connection is unavailable.*",
 }
 var brokenNodeHook *string = flag.String("broken-node-hook", "", "Script to run if node is detected to be broken (for example, Docker daemon is not running)")
 
+func (runner *ContainerRunner) runBrokenNodeHook() {
+       if *brokenNodeHook == "" {
+               runner.CrunchLog.Printf("No broken node hook provided, cannot mark node as broken.")
+       } else {
+               runner.CrunchLog.Printf("Running broken node hook %q", *brokenNodeHook)
+               // run killme script
+               c := exec.Command(*brokenNodeHook)
+               c.Stdout = runner.CrunchLog
+               c.Stderr = runner.CrunchLog
+               err := c.Run()
+               if err != nil {
+                       runner.CrunchLog.Printf("Error running broken node hook: %v", err)
+               }
+       }
+}
+
 func (runner *ContainerRunner) checkBrokenNode(goterr error) bool {
        for _, d := range errorBlacklist {
                if m, e := regexp.MatchString(d, goterr.Error()); m && e == nil {
                        runner.CrunchLog.Printf("Error suggests node is unable to run containers: %v", goterr)
-                       if *brokenNodeHook == "" {
-                               runner.CrunchLog.Printf("No broken node hook provided, cannot mark node as broken.")
-                       } else {
-                               runner.CrunchLog.Printf("Running broken node hook %q", *brokenNodeHook)
-                               // run killme script
-                               c := exec.Command(*brokenNodeHook)
-                               c.Stdout = runner.CrunchLog
-                               c.Stderr = runner.CrunchLog
-                               err := c.Run()
-                               if err != nil {
-                                       runner.CrunchLog.Printf("Error running broken node hook: %v", err)
-                               }
-                       }
+                       runner.runBrokenNodeHook()
                        return true
                }
        }
@@ -215,8 +238,17 @@ func (runner *ContainerRunner) LoadImage() (err error) {
 
        runner.CrunchLog.Printf("Fetching Docker image from collection '%s'", runner.Container.ContainerImage)
 
+       tok, err := runner.ContainerToken()
+       if err != nil {
+               return fmt.Errorf("While getting container token (LoadImage): %v", err)
+       }
+       arvClient, kc, err := runner.MkArvClient(tok)
+       if err != nil {
+               return fmt.Errorf("While creating arv client (LoadImage): %v", err)
+       }
+
        var collection arvados.Collection
-       err = runner.ArvClient.Get("collections", runner.Container.ContainerImage, nil, &collection)
+       err = arvClient.Get("collections", runner.Container.ContainerImage, nil, &collection)
        if err != nil {
                return fmt.Errorf("While getting container image collection: %v", err)
        }
@@ -237,7 +269,7 @@ func (runner *ContainerRunner) LoadImage() (err error) {
                runner.CrunchLog.Print("Loading Docker image from keep")
 
                var readCloser io.ReadCloser
-               readCloser, err = runner.Kc.ManifestFileReader(manifest, img)
+               readCloser, err = kc.ManifestFileReader(manifest, img)
                if err != nil {
                        return fmt.Errorf("While creating ManifestFileReader for container image: %v", err)
                }
@@ -259,7 +291,7 @@ func (runner *ContainerRunner) LoadImage() (err error) {
 
        runner.ContainerConfig.Image = imageID
 
-       runner.Kc.ClearBlockCache()
+       kc.ClearBlockCache()
 
        return nil
 }
@@ -729,6 +761,7 @@ func (runner *ContainerRunner) startCrunchstat() error {
                CgroupParent: runner.expectCgroupParent,
                CgroupRoot:   runner.cgroupRoot,
                PollPeriod:   runner.statInterval,
+               TempDir:      runner.parentTemp,
        }
        runner.statReporter.Start()
        return nil
@@ -1001,6 +1034,10 @@ func (runner *ContainerRunner) CreateContainer() error {
        runner.ContainerConfig.Volumes = runner.Volumes
 
        maxRAM := int64(runner.Container.RuntimeConstraints.RAM)
+       if maxRAM < 4*1024*1024 {
+               // Docker daemon won't let you set a limit less than 4 MiB
+               maxRAM = 4 * 1024 * 1024
+       }
        runner.HostConfig = dockercontainer.HostConfig{
                Binds: runner.Binds,
                LogConfig: dockercontainer.LogConfig{
@@ -1071,13 +1108,86 @@ func (runner *ContainerRunner) StartContainer() error {
        return nil
 }
 
+// checkContainerd checks if "containerd" is present in the process list.
+func (runner *ContainerRunner) CheckContainerd() error {
+       if runner.checkContainerd == 0 {
+               return nil
+       }
+       p, _ := runner.ListProcesses()
+       for _, i := range p {
+               e, _ := i.CmdlineSlice()
+               if len(e) > 0 {
+                       if strings.Index(e[0], "containerd") > -1 {
+                               return nil
+                       }
+               }
+       }
+
+       // Not found
+       runner.runBrokenNodeHook()
+       runner.stop(nil)
+       return fmt.Errorf("'containerd' not found in process list.")
+}
+
 // WaitFinish waits for the container to terminate, capture the exit code, and
 // close the stdout/stderr logging.
 func (runner *ContainerRunner) WaitFinish() error {
+       var runTimeExceeded <-chan time.Time
        runner.CrunchLog.Print("Waiting for container to finish")
 
        waitOk, waitErr := runner.Docker.ContainerWait(context.TODO(), runner.ContainerID, dockercontainer.WaitConditionNotRunning)
        arvMountExit := runner.ArvMountExit
+       if timeout := runner.Container.SchedulingParameters.MaxRunTime; timeout > 0 {
+               runTimeExceeded = time.After(time.Duration(timeout) * time.Second)
+       }
+
+       containerGone := make(chan struct{})
+       go func() {
+               defer close(containerGone)
+               if runner.containerWatchdogInterval < 1 {
+                       runner.containerWatchdogInterval = time.Minute
+               }
+               for range time.NewTicker(runner.containerWatchdogInterval).C {
+                       ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(runner.containerWatchdogInterval))
+                       ctr, err := runner.Docker.ContainerInspect(ctx, runner.ContainerID)
+                       cancel()
+                       runner.cStateLock.Lock()
+                       done := runner.cRemoved || runner.ExitCode != nil
+                       runner.cStateLock.Unlock()
+                       if done {
+                               return
+                       } else if err != nil {
+                               runner.CrunchLog.Printf("Error inspecting container: %s", err)
+                               runner.checkBrokenNode(err)
+                               return
+                       } else if ctr.State == nil || !(ctr.State.Running || ctr.State.Status == "created") {
+                               runner.CrunchLog.Printf("Container is not running: State=%v", ctr.State)
+                               return
+                       }
+               }
+       }()
+
+       containerdGone := make(chan error)
+       defer close(containerdGone)
+       if runner.checkContainerd > 0 {
+               go func() {
+                       ticker := time.NewTicker(time.Duration(runner.checkContainerd))
+                       defer ticker.Stop()
+                       for {
+                               select {
+                               case <-ticker.C:
+                                       if ck := runner.CheckContainerd(); ck != nil {
+                                               containerdGone <- ck
+                                               return
+                                       }
+                               case <-containerdGone:
+                                       // Channel closed, quit goroutine
+                                       return
+                               }
+                       }
+               }()
+       }
+
        for {
                select {
                case waitBody := <-waitOk:
@@ -1098,10 +1208,69 @@ func (runner *ContainerRunner) WaitFinish() error {
                        // arvMountExit will always be ready now that
                        // it's closed, but that doesn't interest us.
                        arvMountExit = nil
+
+               case <-runTimeExceeded:
+                       runner.CrunchLog.Printf("maximum run time exceeded. Stopping container.")
+                       runner.stop(nil)
+                       runTimeExceeded = nil
+
+               case <-containerGone:
+                       return errors.New("docker client never returned status")
+
+               case err := <-containerdGone:
+                       return err
                }
        }
 }
 
+func (runner *ContainerRunner) updateLogs() {
+       ticker := time.NewTicker(crunchLogUpdatePeriod / 360)
+       defer ticker.Stop()
+
+       sigusr1 := make(chan os.Signal, 1)
+       signal.Notify(sigusr1, syscall.SIGUSR1)
+       defer signal.Stop(sigusr1)
+
+       saveAtTime := time.Now().Add(crunchLogUpdatePeriod)
+       saveAtSize := crunchLogUpdateSize
+       var savedSize int64
+       for {
+               select {
+               case <-ticker.C:
+               case <-sigusr1:
+                       saveAtTime = time.Now()
+               }
+               runner.logMtx.Lock()
+               done := runner.LogsPDH != nil
+               runner.logMtx.Unlock()
+               if done {
+                       return
+               }
+               size := runner.LogCollection.Size()
+               if size == savedSize || (time.Now().Before(saveAtTime) && size < saveAtSize) {
+                       continue
+               }
+               saveAtTime = time.Now().Add(crunchLogUpdatePeriod)
+               saveAtSize = runner.LogCollection.Size() + crunchLogUpdateSize
+               saved, err := runner.saveLogCollection(false)
+               if err != nil {
+                       runner.CrunchLog.Printf("error updating log collection: %s", err)
+                       continue
+               }
+
+               var updated arvados.Container
+               err = runner.ArvClient.Update("containers", runner.Container.UUID, arvadosclient.Dict{
+                       "container": arvadosclient.Dict{"log": saved.PortableDataHash},
+               }, &updated)
+               if err != nil {
+                       runner.CrunchLog.Printf("error updating container log to %s: %s", saved.PortableDataHash, err)
+                       continue
+               }
+
+               savedSize = size
+       }
+}
+
 // CaptureOutput saves data from the container's output directory if
 // needed, and updates the container output accordingly.
 func (runner *ContainerRunner) CaptureOutput() error {
@@ -1134,6 +1303,17 @@ func (runner *ContainerRunner) CaptureOutput() error {
        if err != nil {
                return err
        }
+       if n := len(regexp.MustCompile(` [0-9a-f]+\+\S*\+R`).FindAllStringIndex(txt, -1)); n > 0 {
+               runner.CrunchLog.Printf("Copying %d data blocks from remote input collections...", n)
+               fs, err := (&arvados.Collection{ManifestText: txt}).FileSystem(runner.client, runner.Kc)
+               if err != nil {
+                       return err
+               }
+               txt, err = fs.MarshalManifest(".")
+               if err != nil {
+                       return err
+               }
+       }
        var resp arvados.Collection
        err = runner.ArvClient.Create("collections", arvadosclient.Dict{
                "ensure_unique_name": true,
@@ -1239,26 +1419,51 @@ func (runner *ContainerRunner) CommitLogs() error {
                // -- it exists only to send logs to other channels.
                return nil
        }
+       saved, err := runner.saveLogCollection(true)
+       if err != nil {
+               return fmt.Errorf("error saving log collection: %s", err)
+       }
+       runner.logMtx.Lock()
+       defer runner.logMtx.Unlock()
+       runner.LogsPDH = &saved.PortableDataHash
+       return nil
+}
 
+func (runner *ContainerRunner) saveLogCollection(final bool) (response arvados.Collection, err error) {
+       runner.logMtx.Lock()
+       defer runner.logMtx.Unlock()
+       if runner.LogsPDH != nil {
+               // Already finalized.
+               return
+       }
        mt, err := runner.LogCollection.MarshalManifest(".")
        if err != nil {
-               return fmt.Errorf("While creating log manifest: %v", err)
-       }
-
-       var response arvados.Collection
-       err = runner.ArvClient.Create("collections",
-               arvadosclient.Dict{
-                       "ensure_unique_name": true,
-                       "collection": arvadosclient.Dict{
-                               "is_trashed":    true,
-                               "name":          "logs for " + runner.Container.UUID,
-                               "manifest_text": mt}},
-               &response)
+               err = fmt.Errorf("error creating log manifest: %v", err)
+               return
+       }
+       updates := arvadosclient.Dict{
+               "name":          "logs for " + runner.Container.UUID,
+               "manifest_text": mt,
+       }
+       if final {
+               updates["is_trashed"] = true
+       } else {
+               exp := time.Now().Add(crunchLogUpdatePeriod * 24)
+               updates["trash_at"] = exp
+               updates["delete_at"] = exp
+       }
+       reqBody := arvadosclient.Dict{"collection": updates}
+       if runner.logUUID == "" {
+               reqBody["ensure_unique_name"] = true
+               err = runner.ArvClient.Create("collections", reqBody, &response)
+       } else {
+               err = runner.ArvClient.Update("collections", runner.logUUID, reqBody, &response)
+       }
        if err != nil {
-               return fmt.Errorf("While creating log collection: %v", err)
+               return
        }
-       runner.LogsPDH = &response.PortableDataHash
-       return nil
+       runner.logUUID = response.UUID
+       return
 }
 
 // UpdateContainerRunning updates the container state to "Running"
@@ -1284,7 +1489,7 @@ func (runner *ContainerRunner) ContainerToken() (string, error) {
        if err != nil {
                return "", err
        }
-       runner.token = auth.APIToken
+       runner.token = fmt.Sprintf("v2/%s/%s/%s", auth.UUID, auth.APIToken, runner.Container.UUID)
        return runner.token, nil
 }
 
@@ -1355,11 +1560,11 @@ func (runner *ContainerRunner) Run() (err error) {
                // hasn't already been assigned when Run() returns,
                // this cleanup func will cause Run() to return the
                // first non-nil error that is passed to checkErr().
-               checkErr := func(e error) {
+               checkErr := func(errorIn string, e error) {
                        if e == nil {
                                return
                        }
-                       runner.CrunchLog.Print(e)
+                       runner.CrunchLog.Printf("error in %s: %v", errorIn, e)
                        if err == nil {
                                err = e
                        }
@@ -1370,7 +1575,7 @@ func (runner *ContainerRunner) Run() (err error) {
                }
 
                // Log the error encountered in Run(), if any
-               checkErr(err)
+               checkErr("Run", err)
 
                if runner.finalState == "Queued" {
                        runner.UpdateContainerFinal()
@@ -1383,10 +1588,10 @@ func (runner *ContainerRunner) Run() (err error) {
                        // capture partial output and write logs
                }
 
-               checkErr(runner.CaptureOutput())
-               checkErr(runner.stopHoststat())
-               checkErr(runner.CommitLogs())
-               checkErr(runner.UpdateContainerFinal())
+               checkErr("CaptureOutput", runner.CaptureOutput())
+               checkErr("stopHoststat", runner.stopHoststat())
+               checkErr("CommitLogs", runner.CommitLogs())
+               checkErr("UpdateContainerFinal", runner.UpdateContainerFinal())
        }()
 
        err = runner.fetchContainerRecord()
@@ -1399,6 +1604,12 @@ func (runner *ContainerRunner) Run() (err error) {
                return
        }
 
+       // Sanity check that containerd is running.
+       err = runner.CheckContainerd()
+       if err != nil {
+               return
+       }
+
        // check for and/or load image
        err = runner.LoadImage()
        if err != nil {
@@ -1489,7 +1700,7 @@ func (runner *ContainerRunner) fetchContainerRecord() error {
                return fmt.Errorf("error getting container token: %v", err)
        }
 
-       containerClient, err := runner.MkArvClient(containerToken)
+       containerClient, _, err := runner.MkArvClient(containerToken)
        if err != nil {
                return fmt.Errorf("error creating container API client: %v", err)
        }
@@ -1518,13 +1729,28 @@ func NewContainerRunner(client *arvados.Client, api IArvadosClient, kc IKeepClie
        cr.NewLogWriter = cr.NewArvLogWriter
        cr.RunArvMount = cr.ArvMountCmd
        cr.MkTempDir = ioutil.TempDir
-       cr.MkArvClient = func(token string) (IArvadosClient, error) {
-               cl, err := arvadosclient.MakeArvadosClient()
+       cr.ListProcesses = func() ([]PsProcess, error) {
+               pr, err := process.Processes()
                if err != nil {
                        return nil, err
                }
+               ps := make([]PsProcess, len(pr))
+               for i, j := range pr {
+                       ps[i] = j
+               }
+               return ps, nil
+       }
+       cr.MkArvClient = func(token string) (IArvadosClient, IKeepClient, error) {
+               cl, err := arvadosclient.MakeArvadosClient()
+               if err != nil {
+                       return nil, nil, err
+               }
                cl.ApiToken = token
-               return cl, nil
+               kc, err := keepclient.MakeKeepClient(cl)
+               if err != nil {
+                       return nil, nil, err
+               }
+               return cl, kc, nil
        }
        var err error
        cr.LogCollection, err = (&arvados.Collection{}).FileSystem(cr.client, cr.Kc)
@@ -1540,6 +1766,7 @@ func NewContainerRunner(client *arvados.Client, api IArvadosClient, kc IKeepClie
        cr.CrunchLog.Immediate = log.New(os.Stderr, containerUUID+" ", 0)
 
        loadLogThrottleParams(api)
+       go cr.updateLogs()
 
        return cr, nil
 }
@@ -1560,6 +1787,7 @@ func main() {
        `)
        memprofile := flag.String("memprofile", "", "write memory profile to `file` after running container")
        getVersion := flag.Bool("version", false, "Print version information and exit.")
+       checkContainerd := flag.Duration("check-containerd", 60*time.Second, "Periodic check if (docker-)containerd is running (use 0s to disable).")
        flag.Parse()
 
        // Print version information if requested
@@ -1615,6 +1843,7 @@ func main() {
        cr.expectCgroupParent = *cgroupParent
        cr.enableNetwork = *enableNetwork
        cr.networkMode = *networkMode
+       cr.checkContainerd = *checkContainerd
        if *cgroupParentSubsystem != "" {
                p := findCgroup(*cgroupParentSubsystem)
                cr.setCgroupParent = p