19094: Note docker/singularity/arv-mount versions in container log.
[arvados.git] / lib / crunchrun / crunchrun.go
index e15303a3155afe81d72e8ce61e881ce76d5282d7..0253ac3fa8f7e6409c5eab5d87741dcac9dd4a05 100644 (file)
@@ -6,6 +6,7 @@ package crunchrun
 
 import (
        "bytes"
+       "context"
        "encoding/json"
        "errors"
        "flag"
@@ -13,9 +14,12 @@ import (
        "io"
        "io/ioutil"
        "log"
+       "net"
+       "net/http"
        "os"
        "os/exec"
        "os/signal"
+       "os/user"
        "path"
        "path/filepath"
        "regexp"
@@ -28,18 +32,28 @@ import (
        "time"
 
        "git.arvados.org/arvados.git/lib/cmd"
+       "git.arvados.org/arvados.git/lib/config"
        "git.arvados.org/arvados.git/lib/crunchstat"
        "git.arvados.org/arvados.git/sdk/go/arvados"
        "git.arvados.org/arvados.git/sdk/go/arvadosclient"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
        "git.arvados.org/arvados.git/sdk/go/keepclient"
        "git.arvados.org/arvados.git/sdk/go/manifest"
-       "golang.org/x/net/context"
+       "golang.org/x/sys/unix"
 )
 
 type command struct{}
 
 var Command = command{}
 
+// ConfigData contains environment variables and (when needed) cluster
+// configuration, passed from dispatchcloud to crunch-run on stdin.
+type ConfigData struct {
+       Env         map[string]string
+       KeepBuffers int
+       Cluster     *arvados.Cluster
+}
+
 // IArvadosClient is the minimal Arvados API methods used by crunch-run.
 type IArvadosClient interface {
        Create(resourceType string, parameters arvadosclient.Dict, output interface{}) error
@@ -66,7 +80,7 @@ type IKeepClient interface {
 // NewLogWriter is a factory function to create a new log writer.
 type NewLogWriter func(name string) (io.WriteCloser, error)
 
-type RunArvMount func(args []string, tok string) (*exec.Cmd, error)
+type RunArvMount func(cmdline []string, tok string) (*exec.Cmd, error)
 
 type MkTempDir func(string, string) (string, error)
 
@@ -127,6 +141,8 @@ type ContainerRunner struct {
        finalState    string
        parentTemp    string
 
+       keepstoreLogger  io.WriteCloser
+       keepstoreLogbuf  *bufThenWrite
        statLogger       io.WriteCloser
        statReporter     *crunchstat.Reporter
        hoststatLogger   io.WriteCloser
@@ -153,6 +169,7 @@ type ContainerRunner struct {
        enableMemoryLimit bool
        enableNetwork     string // one of "default" or "always"
        networkMode       string // "none", "host", or "" -- passed through to executor
+       brokenNodeHook    string // script to run if node appears to be broken
        arvMountLog       *ThrottledLogger
 
        containerWatchdogInterval time.Duration
@@ -196,10 +213,9 @@ var errorBlacklist = []string{
        "(?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 == "" {
+       if runner.brokenNodeHook == "" {
                path := filepath.Join(lockdir, brokenfile)
                runner.CrunchLog.Printf("Writing %s to mark node as broken", path)
                f, err := os.OpenFile(path, os.O_CREATE|os.O_RDWR, 0700)
@@ -209,9 +225,9 @@ func (runner *ContainerRunner) runBrokenNodeHook() {
                }
                f.Close()
        } else {
-               runner.CrunchLog.Printf("Running broken node hook %q", *brokenNodeHook)
+               runner.CrunchLog.Printf("Running broken node hook %q", runner.brokenNodeHook)
                // run killme script
-               c := exec.Command(*brokenNodeHook)
+               c := exec.Command(runner.brokenNodeHook)
                c.Stdout = runner.CrunchLog
                c.Stderr = runner.CrunchLog
                err := c.Run()
@@ -273,8 +289,8 @@ func (runner *ContainerRunner) LoadImage() (string, error) {
        return imageID, nil
 }
 
-func (runner *ContainerRunner) ArvMountCmd(arvMountCmd []string, token string) (c *exec.Cmd, err error) {
-       c = exec.Command("arv-mount", arvMountCmd...)
+func (runner *ContainerRunner) ArvMountCmd(cmdline []string, token string) (c *exec.Cmd, err error) {
+       c = exec.Command(cmdline[0], cmdline[1:]...)
 
        // Copy our environment, but override ARVADOS_API_TOKEN with
        // the container auth token.
@@ -291,8 +307,16 @@ func (runner *ContainerRunner) ArvMountCmd(arvMountCmd []string, token string) (
                return nil, err
        }
        runner.arvMountLog = NewThrottledLogger(w)
+       scanner := logScanner{
+               Patterns: []string{
+                       "Keep write error",
+                       "Block not found error",
+                       "Unhandled exception during FUSE operation",
+               },
+               ReportFunc: runner.reportArvMountWarning,
+       }
        c.Stdout = runner.arvMountLog
-       c.Stderr = io.MultiWriter(runner.arvMountLog, os.Stderr)
+       c.Stderr = io.MultiWriter(runner.arvMountLog, os.Stderr, &scanner)
 
        runner.CrunchLog.Printf("Running %v", c.Args)
 
@@ -392,12 +416,16 @@ func (runner *ContainerRunner) SetupMounts() (map[string]bindmount, error) {
        pdhOnly := true
        tmpcount := 0
        arvMountCmd := []string{
+               "arv-mount",
                "--foreground",
-               "--allow-other",
                "--read-write",
                "--storage-classes", strings.Join(runner.Container.OutputStorageClasses, ","),
                fmt.Sprintf("--crunchstat-interval=%v", runner.statInterval.Seconds())}
 
+       if _, isdocker := runner.executor.(*dockerExecutor); isdocker {
+               arvMountCmd = append(arvMountCmd, "--allow-other")
+       }
+
        if runner.Container.RuntimeConstraints.KeepCacheRAM > 0 {
                arvMountCmd = append(arvMountCmd, "--file-cache", fmt.Sprintf("%d", runner.Container.RuntimeConstraints.KeepCacheRAM))
        }
@@ -428,8 +456,8 @@ func (runner *ContainerRunner) SetupMounts() (map[string]bindmount, error) {
        sort.Strings(binds)
 
        for _, bind := range binds {
-               mnt, ok := runner.Container.Mounts[bind]
-               if !ok {
+               mnt, notSecret := runner.Container.Mounts[bind]
+               if !notSecret {
                        mnt = runner.SecretMounts[bind]
                }
                if bind == "stdout" || bind == "stderr" {
@@ -498,8 +526,7 @@ func (runner *ContainerRunner) SetupMounts() (map[string]bindmount, error) {
                                }
                        } else {
                                src = fmt.Sprintf("%s/tmp%d", runner.ArvMountPoint, tmpcount)
-                               arvMountCmd = append(arvMountCmd, "--mount-tmp")
-                               arvMountCmd = append(arvMountCmd, fmt.Sprintf("tmp%d", tmpcount))
+                               arvMountCmd = append(arvMountCmd, "--mount-tmp", fmt.Sprintf("tmp%d", tmpcount))
                                tmpcount++
                        }
                        if mnt.Writable {
@@ -559,9 +586,32 @@ func (runner *ContainerRunner) SetupMounts() (map[string]bindmount, error) {
                        if err != nil {
                                return nil, fmt.Errorf("writing temp file: %v", err)
                        }
-                       if strings.HasPrefix(bind, runner.Container.OutputPath+"/") {
+                       if strings.HasPrefix(bind, runner.Container.OutputPath+"/") && (notSecret || runner.Container.Mounts[runner.Container.OutputPath].Kind != "collection") {
+                               // In most cases, if the container
+                               // specifies a literal file inside the
+                               // output path, we copy it into the
+                               // output directory (either a mounted
+                               // collection or a staging area on the
+                               // host fs). If it's a secret, it will
+                               // be skipped when copying output from
+                               // staging to Keep later.
                                copyFiles = append(copyFiles, copyFile{tmpfn, runner.HostOutputDir + bind[len(runner.Container.OutputPath):]})
                        } else {
+                               // If a secret is outside OutputPath,
+                               // we bind mount the secret file
+                               // directly just like other mounts. We
+                               // also use this strategy when a
+                               // secret is inside OutputPath but
+                               // OutputPath is a live collection, to
+                               // avoid writing the secret to
+                               // Keep. Attempting to remove a
+                               // bind-mounted secret file from
+                               // inside the container will return a
+                               // "Device or resource busy" error
+                               // that might not be handled well by
+                               // the container, which is why we
+                               // don't use this strategy when
+                               // OutputPath is a staging directory.
                                bindmounts[bind] = bindmount{HostPath: tmpfn, ReadOnly: true}
                        }
 
@@ -593,10 +643,15 @@ func (runner *ContainerRunner) SetupMounts() (map[string]bindmount, error) {
        }
 
        if pdhOnly {
-               arvMountCmd = append(arvMountCmd, "--mount-by-pdh", "by_id")
+               // If we are only mounting collections by pdh, make
+               // sure we don't subscribe to websocket events to
+               // avoid putting undesired load on the API server
+               arvMountCmd = append(arvMountCmd, "--mount-by-pdh", "by_id", "--disable-event-listening")
        } else {
                arvMountCmd = append(arvMountCmd, "--mount-by-id", "by_id")
        }
+       // the by_uuid mount point is used by singularity when writing
+       // out docker images converted to SIF
        arvMountCmd = append(arvMountCmd, "--mount-by-id", "by_uuid")
        arvMountCmd = append(arvMountCmd, runner.ArvMountPoint)
 
@@ -957,20 +1012,26 @@ func (runner *ContainerRunner) CreateContainer(imageID string, bindmounts map[st
        runner.executorStdin = stdin
        runner.executorStdout = stdout
        runner.executorStderr = stderr
+
+       if runner.Container.RuntimeConstraints.CUDA.DeviceCount > 0 {
+               nvidiaModprobe(runner.CrunchLog)
+       }
+
        return runner.executor.Create(containerSpec{
-               Image:         imageID,
-               VCPUs:         runner.Container.RuntimeConstraints.VCPUs,
-               RAM:           ram,
-               WorkingDir:    workdir,
-               Env:           env,
-               BindMounts:    bindmounts,
-               Command:       runner.Container.Command,
-               EnableNetwork: enableNetwork,
-               NetworkMode:   runner.networkMode,
-               CgroupParent:  runner.setCgroupParent,
-               Stdin:         stdin,
-               Stdout:        stdout,
-               Stderr:        stderr,
+               Image:           imageID,
+               VCPUs:           runner.Container.RuntimeConstraints.VCPUs,
+               RAM:             ram,
+               WorkingDir:      workdir,
+               Env:             env,
+               BindMounts:      bindmounts,
+               Command:         runner.Container.Command,
+               EnableNetwork:   enableNetwork,
+               CUDADeviceCount: runner.Container.RuntimeConstraints.CUDA.DeviceCount,
+               NetworkMode:     runner.networkMode,
+               CgroupParent:    runner.setCgroupParent,
+               Stdin:           stdin,
+               Stdout:          stdout,
+               Stderr:          stderr,
        })
 }
 
@@ -1021,6 +1082,20 @@ func (runner *ContainerRunner) WaitFinish() error {
        }
        runner.ExitCode = &exitcode
 
+       extra := ""
+       if exitcode&0x80 != 0 {
+               // Convert raw exit status (0x80 + signal number) to a
+               // string to log after the code, like " (signal 101)"
+               // or " (signal 9, killed)"
+               sig := syscall.WaitStatus(exitcode).Signal()
+               if name := unix.SignalName(sig); name != "" {
+                       extra = fmt.Sprintf(" (signal %d, %s)", sig, name)
+               } else {
+                       extra = fmt.Sprintf(" (signal %d)", sig)
+               }
+       }
+       runner.CrunchLog.Printf("Container exited with status code %d%s", exitcode, extra)
+
        var returnErr error
        if err = runner.executorStdin.Close(); err != nil {
                err = fmt.Errorf("error closing container stdin: %s", err)
@@ -1100,6 +1175,21 @@ func (runner *ContainerRunner) updateLogs() {
        }
 }
 
+func (runner *ContainerRunner) reportArvMountWarning(pattern, text string) {
+       var updated arvados.Container
+       err := runner.DispatcherArvClient.Update("containers", runner.Container.UUID, arvadosclient.Dict{
+               "container": arvadosclient.Dict{
+                       "runtime_status": arvadosclient.Dict{
+                               "warning":       "arv-mount: " + pattern,
+                               "warningDetail": text,
+                       },
+               },
+       }, &updated)
+       if err != nil {
+               runner.CrunchLog.Printf("error updating container runtime_status: %s", err)
+       }
+}
+
 // CaptureOutput saves data from the container's output directory if
 // needed, and updates the container output accordingly.
 func (runner *ContainerRunner) CaptureOutput(bindmounts map[string]bindmount) error {
@@ -1170,6 +1260,7 @@ func (runner *ContainerRunner) CleanupDirs() {
 
                if umnterr != nil {
                        runner.CrunchLog.Printf("Error unmounting: %v", umnterr)
+                       runner.ArvMount.Process.Kill()
                } else {
                        // If arv-mount --unmount gets stuck for any reason, we
                        // don't want to wait for it forever.  Do Wait() in a goroutine
@@ -1242,6 +1333,16 @@ func (runner *ContainerRunner) CommitLogs() error {
                runner.CrunchLog.Immediate = log.New(os.Stderr, runner.Container.UUID+" ", 0)
        }()
 
+       if runner.keepstoreLogger != nil {
+               // Flush any buffered logs from our local keepstore
+               // process.  Discard anything logged after this point
+               // -- it won't end up in the log collection, so
+               // there's no point writing it to the collectionfs.
+               runner.keepstoreLogbuf.SetWriter(io.Discard)
+               runner.keepstoreLogger.Close()
+               runner.keepstoreLogger = nil
+       }
+
        if runner.LogsPDH != nil {
                // If we have already assigned something to LogsPDH,
                // we must be closing the re-opened log, which won't
@@ -1250,6 +1351,7 @@ 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)
@@ -1376,7 +1478,11 @@ func (runner *ContainerRunner) NewArvLogWriter(name string) (io.WriteCloser, err
 // Run the full container lifecycle.
 func (runner *ContainerRunner) Run() (err error) {
        runner.CrunchLog.Printf("crunch-run %s started", cmd.Version.String())
-       runner.CrunchLog.Printf("Executing container '%s'", runner.Container.UUID)
+       runner.CrunchLog.Printf("%s", currentUserAndGroups())
+       v, _ := exec.Command("arv-mount", "--version").CombinedOutput()
+       runner.CrunchLog.Printf("Using FUSE mount: %s", v)
+       runner.CrunchLog.Printf("Using container runtime: %s", runner.executor.Runtime())
+       runner.CrunchLog.Printf("Executing container: %s", runner.Container.UUID)
 
        hostname, hosterr := os.Hostname()
        if hosterr != nil {
@@ -1612,6 +1718,7 @@ func NewContainerRunner(dispatcherClient *arvados.Client,
 }
 
 func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, stderr io.Writer) int {
+       log := log.New(stderr, "", 0)
        flags := flag.NewFlagSet(prog, flag.ContinueOnError)
        statInterval := flags.Duration("crunchstat-interval", 10*time.Second, "sampling period for periodic resource usage reporting")
        cgroupRoot := flags.String("cgroup-root", "/sys/fs/cgroup", "path to sysfs cgroup tree")
@@ -1619,7 +1726,8 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
        cgroupParentSubsystem := flags.String("cgroup-parent-subsystem", "", "use current cgroup for given subsystem as parent cgroup for container")
        caCertsPath := flags.String("ca-certs", "", "Path to TLS root certificates")
        detach := flags.Bool("detach", false, "Detach from parent process and run in the background")
-       stdinEnv := flags.Bool("stdin-env", false, "Load environment variables from JSON message on stdin")
+       stdinConfig := flags.Bool("stdin-config", false, "Load config and environment variables from JSON message on stdin")
+       configFile := flags.String("config", arvados.DefaultConfigFile, "filename of cluster config file to try loading if -stdin-config=false (default is $ARVADOS_CONFIG)")
        sleep := flags.Duration("sleep", 0, "Delay before starting (testing use only)")
        kill := flags.Int("kill", -1, "Send signal to an existing crunch-run process for given UUID")
        list := flags.Bool("list", false, "List UUIDs of existing crunch-run processes")
@@ -1628,6 +1736,7 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
        networkMode := flags.String("container-network-mode", "default", `Docker network mode for container (use any argument valid for docker --net)`)
        memprofile := flags.String("memprofile", "", "write memory profile to `file` after running container")
        runtimeEngine := flags.String("runtime-engine", "docker", "container runtime: docker or singularity")
+       brokenNodeHook := flags.String("broken-node-hook", "", "script to run if node is detected to be broken (for example, Docker daemon is not running)")
        flags.Duration("check-containerd", 0, "Ignored. Exists for compatibility with older versions.")
 
        ignoreDetachFlag := false
@@ -1642,40 +1751,55 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
                ignoreDetachFlag = true
        }
 
-       if err := flags.Parse(args); err == flag.ErrHelp {
-               return 0
-       } else if err != nil {
-               log.Print(err)
-               return 1
-       }
-
-       if *stdinEnv && !ignoreDetachFlag {
-               // Load env vars on stdin if asked (but not in a
-               // detached child process, in which case stdin is
-               // /dev/null).
-               err := loadEnv(os.Stdin)
-               if err != nil {
-                       log.Print(err)
-                       return 1
-               }
+       if ok, code := cmd.ParseFlags(flags, prog, args, "container-uuid", stderr); !ok {
+               return code
+       } else if !*list && flags.NArg() != 1 {
+               fmt.Fprintf(stderr, "missing required argument: container-uuid (try -help)\n")
+               return 2
        }
 
        containerUUID := flags.Arg(0)
 
        switch {
        case *detach && !ignoreDetachFlag:
-               return Detach(containerUUID, prog, args, os.Stdout, os.Stderr)
+               return Detach(containerUUID, prog, args, os.Stdin, os.Stdout, os.Stderr)
        case *kill >= 0:
                return KillProcess(containerUUID, syscall.Signal(*kill), os.Stdout, os.Stderr)
        case *list:
                return ListProcesses(os.Stdout, os.Stderr)
        }
 
-       if containerUUID == "" {
+       if len(containerUUID) != 27 {
                log.Printf("usage: %s [options] UUID", prog)
                return 1
        }
 
+       var keepstoreLogbuf bufThenWrite
+       var conf ConfigData
+       if *stdinConfig {
+               err := json.NewDecoder(stdin).Decode(&conf)
+               if err != nil {
+                       log.Printf("decode stdin: %s", err)
+                       return 1
+               }
+               for k, v := range conf.Env {
+                       err = os.Setenv(k, v)
+                       if err != nil {
+                               log.Printf("setenv(%q): %s", k, err)
+                               return 1
+                       }
+               }
+               if conf.Cluster != nil {
+                       // ClusterID is missing from the JSON
+                       // representation, but we need it to generate
+                       // a valid config file for keepstore, so we
+                       // fill it using the container UUID prefix.
+                       conf.Cluster.ClusterID = containerUUID[:5]
+               }
+       } else {
+               conf = hpcConfData(containerUUID, *configFile, io.MultiWriter(&keepstoreLogbuf, stderr))
+       }
+
        log.Printf("crunch-run %s started", cmd.Version.String())
        time.Sleep(*sleep)
 
@@ -1683,6 +1807,15 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
                arvadosclient.CertFiles = []string{*caCertsPath}
        }
 
+       keepstore, err := startLocalKeepstore(conf, io.MultiWriter(&keepstoreLogbuf, stderr))
+       if err != nil {
+               log.Print(err)
+               return 1
+       }
+       if keepstore != nil {
+               defer keepstore.Process.Kill()
+       }
+
        api, err := arvadosclient.MakeArvadosClient()
        if err != nil {
                log.Printf("%s: %v", containerUUID, err)
@@ -1690,9 +1823,9 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
        }
        api.Retries = 8
 
-       kc, kcerr := keepclient.MakeKeepClient(api)
-       if kcerr != nil {
-               log.Printf("%s: %v", containerUUID, kcerr)
+       kc, err := keepclient.MakeKeepClient(api)
+       if err != nil {
+               log.Printf("%s: %v", containerUUID, err)
                return 1
        }
        kc.BlockCache = &keepclient.BlockCache{MaxBlocks: 2}
@@ -1704,6 +1837,43 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
                return 1
        }
 
+       if keepstore == nil {
+               // Log explanation (if any) for why we're not running
+               // a local keepstore.
+               var buf bytes.Buffer
+               keepstoreLogbuf.SetWriter(&buf)
+               if buf.Len() > 0 {
+                       cr.CrunchLog.Printf("%s", strings.TrimSpace(buf.String()))
+               }
+       } else if logWhat := conf.Cluster.Containers.LocalKeepLogsToContainerLog; logWhat == "none" {
+               cr.CrunchLog.Printf("using local keepstore process (pid %d) at %s", keepstore.Process.Pid, os.Getenv("ARVADOS_KEEP_SERVICES"))
+               keepstoreLogbuf.SetWriter(io.Discard)
+       } else {
+               cr.CrunchLog.Printf("using local keepstore process (pid %d) at %s, writing logs to keepstore.txt in log collection", keepstore.Process.Pid, os.Getenv("ARVADOS_KEEP_SERVICES"))
+               logwriter, err := cr.NewLogWriter("keepstore")
+               if err != nil {
+                       log.Print(err)
+                       return 1
+               }
+               cr.keepstoreLogger = NewThrottledLogger(logwriter)
+
+               var writer io.WriteCloser = cr.keepstoreLogger
+               if logWhat == "errors" {
+                       writer = &filterKeepstoreErrorsOnly{WriteCloser: writer}
+               } else if logWhat != "all" {
+                       // should have been caught earlier by
+                       // dispatcher's config loader
+                       log.Printf("invalid value for Containers.LocalKeepLogsToContainerLog: %q", logWhat)
+                       return 1
+               }
+               err = keepstoreLogbuf.SetWriter(writer)
+               if err != nil {
+                       log.Print(err)
+                       return 1
+               }
+               cr.keepstoreLogbuf = &keepstoreLogbuf
+       }
+
        switch *runtimeEngine {
        case "docker":
                cr.executor, err = newDockerExecutor(containerUUID, cr.CrunchLog.Printf, cr.containerWatchdogInterval)
@@ -1722,6 +1892,8 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
        }
        defer cr.executor.Close()
 
+       cr.brokenNodeHook = *brokenNodeHook
+
        gwAuthSecret := os.Getenv("GatewayAuthSecret")
        os.Unsetenv("GatewayAuthSecret")
        if gwAuthSecret == "" {
@@ -1762,7 +1934,11 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
        cr.enableNetwork = *enableNetwork
        cr.networkMode = *networkMode
        if *cgroupParentSubsystem != "" {
-               p := findCgroup(*cgroupParentSubsystem)
+               p, err := findCgroup(*cgroupParentSubsystem)
+               if err != nil {
+                       log.Printf("fatal: cgroup parent subsystem: %s", err)
+                       return 1
+               }
                cr.setCgroupParent = p
                cr.expectCgroupParent = p
        }
@@ -1791,21 +1967,179 @@ func (command) RunCommand(prog string, args []string, stdin io.Reader, stdout, s
        return 0
 }
 
-func loadEnv(rdr io.Reader) error {
-       buf, err := ioutil.ReadAll(rdr)
+// Try to load ConfigData in hpc (slurm/lsf) environment. This means
+// loading the cluster config from the specified file and (if that
+// works) getting the runtime_constraints container field from
+// controller to determine # VCPUs so we can calculate KeepBuffers.
+func hpcConfData(uuid string, configFile string, stderr io.Writer) ConfigData {
+       var conf ConfigData
+       conf.Cluster = loadClusterConfigFile(configFile, stderr)
+       if conf.Cluster == nil {
+               // skip loading the container record -- we won't be
+               // able to start local keepstore anyway.
+               return conf
+       }
+       arv, err := arvadosclient.MakeArvadosClient()
+       if err != nil {
+               fmt.Fprintf(stderr, "error setting up arvadosclient: %s\n", err)
+               return conf
+       }
+       arv.Retries = 8
+       var ctr arvados.Container
+       err = arv.Call("GET", "containers", uuid, "", arvadosclient.Dict{"select": []string{"runtime_constraints"}}, &ctr)
+       if err != nil {
+               fmt.Fprintf(stderr, "error getting container record: %s\n", err)
+               return conf
+       }
+       if ctr.RuntimeConstraints.VCPUs > 0 {
+               conf.KeepBuffers = ctr.RuntimeConstraints.VCPUs * conf.Cluster.Containers.LocalKeepBlobBuffersPerVCPU
+       }
+       return conf
+}
+
+// Load cluster config file from given path. If an error occurs, log
+// the error to stderr and return nil.
+func loadClusterConfigFile(path string, stderr io.Writer) *arvados.Cluster {
+       ldr := config.NewLoader(&bytes.Buffer{}, ctxlog.New(stderr, "plain", "info"))
+       ldr.Path = path
+       cfg, err := ldr.Load()
+       if err != nil {
+               fmt.Fprintf(stderr, "could not load config file %s: %s\n", path, err)
+               return nil
+       }
+       cluster, err := cfg.GetCluster("")
+       if err != nil {
+               fmt.Fprintf(stderr, "could not use config file %s: %s\n", path, err)
+               return nil
+       }
+       fmt.Fprintf(stderr, "loaded config file %s\n", path)
+       return cluster
+}
+
+func startLocalKeepstore(configData ConfigData, logbuf io.Writer) (*exec.Cmd, error) {
+       if configData.KeepBuffers < 1 {
+               fmt.Fprintf(logbuf, "not starting a local keepstore process because KeepBuffers=%v in config\n", configData.KeepBuffers)
+               return nil, nil
+       }
+       if configData.Cluster == nil {
+               fmt.Fprint(logbuf, "not starting a local keepstore process because cluster config file was not loaded\n")
+               return nil, nil
+       }
+       for uuid, vol := range configData.Cluster.Volumes {
+               if len(vol.AccessViaHosts) > 0 {
+                       fmt.Fprintf(logbuf, "not starting a local keepstore process because a volume (%s) uses AccessViaHosts\n", uuid)
+                       return nil, nil
+               }
+               if !vol.ReadOnly && vol.Replication < configData.Cluster.Collections.DefaultReplication {
+                       fmt.Fprintf(logbuf, "not starting a local keepstore process because a writable volume (%s) has replication less than Collections.DefaultReplication (%d < %d)\n", uuid, vol.Replication, configData.Cluster.Collections.DefaultReplication)
+                       return nil, nil
+               }
+       }
+
+       // Rather than have an alternate way to tell keepstore how
+       // many buffers to use when starting it this way, we just
+       // modify the cluster configuration that we feed it on stdin.
+       configData.Cluster.API.MaxKeepBlobBuffers = configData.KeepBuffers
+
+       ln, err := net.Listen("tcp", "localhost:0")
        if err != nil {
-               return fmt.Errorf("read stdin: %s", err)
+               return nil, err
        }
-       var env map[string]string
-       err = json.Unmarshal(buf, &env)
+       _, port, err := net.SplitHostPort(ln.Addr().String())
        if err != nil {
-               return fmt.Errorf("decode stdin: %s", err)
+               ln.Close()
+               return nil, err
        }
-       for k, v := range env {
-               err = os.Setenv(k, v)
+       ln.Close()
+       url := "http://localhost:" + port
+
+       fmt.Fprintf(logbuf, "starting keepstore on %s\n", url)
+
+       var confJSON bytes.Buffer
+       err = json.NewEncoder(&confJSON).Encode(arvados.Config{
+               Clusters: map[string]arvados.Cluster{
+                       configData.Cluster.ClusterID: *configData.Cluster,
+               },
+       })
+       if err != nil {
+               return nil, err
+       }
+       cmd := exec.Command("/proc/self/exe", "keepstore", "-config=-")
+       if target, err := os.Readlink(cmd.Path); err == nil && strings.HasSuffix(target, ".test") {
+               // If we're a 'go test' process, running
+               // /proc/self/exe would start the test suite in a
+               // child process, which is not what we want.
+               cmd.Path, _ = exec.LookPath("go")
+               cmd.Args = append([]string{"go", "run", "../../cmd/arvados-server"}, cmd.Args[1:]...)
+               cmd.Env = os.Environ()
+       }
+       cmd.Stdin = &confJSON
+       cmd.Stdout = logbuf
+       cmd.Stderr = logbuf
+       cmd.Env = append(cmd.Env,
+               "GOGC=10",
+               "ARVADOS_SERVICE_INTERNAL_URL="+url)
+       err = cmd.Start()
+       if err != nil {
+               return nil, fmt.Errorf("error starting keepstore process: %w", err)
+       }
+       cmdExited := false
+       go func() {
+               cmd.Wait()
+               cmdExited = true
+       }()
+       ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(time.Second*10))
+       defer cancel()
+       poll := time.NewTicker(time.Second / 10)
+       defer poll.Stop()
+       client := http.Client{}
+       for range poll.C {
+               testReq, err := http.NewRequestWithContext(ctx, "GET", url+"/_health/ping", nil)
+               testReq.Header.Set("Authorization", "Bearer "+configData.Cluster.ManagementToken)
                if err != nil {
-                       return fmt.Errorf("setenv(%q): %s", k, err)
+                       return nil, err
+               }
+               resp, err := client.Do(testReq)
+               if err == nil {
+                       resp.Body.Close()
+                       if resp.StatusCode == http.StatusOK {
+                               break
+                       }
+               }
+               if cmdExited {
+                       return nil, fmt.Errorf("keepstore child process exited")
+               }
+               if ctx.Err() != nil {
+                       return nil, fmt.Errorf("timed out waiting for new keepstore process to report healthy")
                }
        }
-       return nil
+       os.Setenv("ARVADOS_KEEP_SERVICES", url)
+       return cmd, nil
+}
+
+// return current uid, gid, groups in a format suitable for logging:
+// "crunch-run process has uid=1234(arvados) gid=1234(arvados)
+// groups=1234(arvados),114(fuse)"
+func currentUserAndGroups() string {
+       u, err := user.Current()
+       if err != nil {
+               return fmt.Sprintf("error getting current user ID: %s", err)
+       }
+       s := fmt.Sprintf("crunch-run process has uid=%s(%s) gid=%s", u.Uid, u.Username, u.Gid)
+       if g, err := user.LookupGroupId(u.Gid); err == nil {
+               s += fmt.Sprintf("(%s)", g.Name)
+       }
+       s += " groups="
+       if gids, err := u.GroupIds(); err == nil {
+               for i, gid := range gids {
+                       if i > 0 {
+                               s += ","
+                       }
+                       s += gid
+                       if g, err := user.LookupGroupId(gid); err == nil {
+                               s += fmt.Sprintf("(%s)", g.Name)
+                       }
+               }
+       }
+       return s
 }