15954: Logging and env var fixes.
[arvados.git] / lib / boot / cmd.go
index 4ac6d4a2141fcd079f937ddbf6ee265aa98cbef4..2c3103202427237e94863d5393d1d1bee698a360 100644 (file)
@@ -17,6 +17,7 @@ import (
        "os"
        "os/exec"
        "os/signal"
+       "os/user"
        "path/filepath"
        "strings"
        "sync"
@@ -25,48 +26,55 @@ import (
 
        "git.arvados.org/arvados.git/lib/cmd"
        "git.arvados.org/arvados.git/lib/config"
-       "git.arvados.org/arvados.git/lib/controller"
-       "git.arvados.org/arvados.git/lib/dispatchcloud"
+       "git.arvados.org/arvados.git/lib/service"
        "git.arvados.org/arvados.git/sdk/go/arvados"
        "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       "git.arvados.org/arvados.git/sdk/go/health"
        "github.com/sirupsen/logrus"
 )
 
-var Command cmd.Handler = &bootCommand{}
+var Command cmd.Handler = bootCommand{}
 
-type bootCommand struct {
-       sourcePath  string // e.g., /home/username/src/arvados
-       libPath     string // e.g., /var/lib/arvados
-       clusterType string // e.g., production
+type bootTask interface {
+       // Execute the task. Run should return nil when the task is
+       // done enough to satisfy a dependency relationship (e.g., the
+       // service is running and ready). If the task starts a
+       // goroutine that fails after Run returns (e.g., the service
+       // shuts down), it should call cancel.
+       Run(ctx context.Context, fail func(error), boot *Booter) error
+       String() string
+}
 
-       stdout io.Writer
-       stderr io.Writer
+type bootCommand struct{}
 
-       setupRubyOnce sync.Once
-       setupRubyErr  error
-       goMutex       sync.Mutex
-}
+func (bootCommand) RunCommand(prog string, args []string, stdin io.Reader, stdout, stderr io.Writer) int {
+       boot := &Booter{
+               Stderr: stderr,
+               logger: ctxlog.New(stderr, "json", "info"),
+       }
 
-func (boot *bootCommand) RunCommand(prog string, args []string, stdin io.Reader, stdout, stderr io.Writer) int {
-       boot.stdout = stdout
-       boot.stderr = stderr
-       log := ctxlog.New(stderr, "json", "info")
+       ctx := ctxlog.Context(context.Background(), boot.logger)
+       ctx, cancel := context.WithCancel(ctx)
+       defer cancel()
 
        var err error
        defer func() {
                if err != nil {
-                       log.WithError(err).Info("exiting")
+                       boot.logger.WithError(err).Info("exiting")
                }
        }()
 
        flags := flag.NewFlagSet(prog, flag.ContinueOnError)
        flags.SetOutput(stderr)
-       loader := config.NewLoader(stdin, log)
+       loader := config.NewLoader(stdin, boot.logger)
        loader.SetupFlags(flags)
        versionFlag := flags.Bool("version", false, "Write version information to stdout and exit 0")
-       flags.StringVar(&boot.sourcePath, "source", ".", "arvados source tree `directory`")
-       flags.StringVar(&boot.libPath, "lib", "/var/lib/arvados", "`directory` to install dependencies and library files")
-       flags.StringVar(&boot.clusterType, "type", "production", "cluster `type`: development, test, or production")
+       flags.StringVar(&boot.SourcePath, "source", ".", "arvados source tree `directory`")
+       flags.StringVar(&boot.LibPath, "lib", "/var/lib/arvados", "`directory` to install dependencies and library files")
+       flags.StringVar(&boot.ClusterType, "type", "production", "cluster `type`: development, test, or production")
+       flags.StringVar(&boot.ListenHost, "listen-host", "localhost", "host name or interface address for service listeners")
+       flags.StringVar(&boot.ControllerAddr, "controller-address", ":0", "desired controller address, `host:port` or `:port`")
+       flags.BoolVar(&boot.OwnTemporaryDatabase, "own-temporary-database", false, "bring up a postgres server and create a temporary database")
        err = flags.Parse(args)
        if err == flag.ErrHelp {
                err = nil
@@ -75,319 +83,579 @@ func (boot *bootCommand) RunCommand(prog string, args []string, stdin io.Reader,
                return 2
        } else if *versionFlag {
                return cmd.Version.RunCommand(prog, args, stdin, stdout, stderr)
-       } else if boot.clusterType != "development" && boot.clusterType != "test" && boot.clusterType != "production" {
+       } else if boot.ClusterType != "development" && boot.ClusterType != "test" && boot.ClusterType != "production" {
                err = fmt.Errorf("cluster type must be 'development', 'test', or 'production'")
                return 2
        }
 
-       cwd, err := os.Getwd()
+       loader.SkipAPICalls = true
+       cfg, err := loader.Load()
        if err != nil {
                return 1
        }
-       if !strings.HasPrefix(boot.sourcePath, "/") {
-               boot.sourcePath = filepath.Join(cwd, boot.sourcePath)
-       }
-       boot.sourcePath, err = filepath.EvalSymlinks(boot.sourcePath)
-       if err != nil {
+
+       boot.Start(ctx, cfg)
+       defer boot.Stop()
+       if url, ok := boot.WaitReady(); ok {
+               fmt.Fprintln(stdout, url)
+               // Wait for signal/crash + orderly shutdown
+               <-boot.done
+               return 0
+       } else {
                return 1
        }
+}
 
-       loader.SkipAPICalls = true
-       cfg, err := loader.Load()
+type Booter struct {
+       SourcePath           string // e.g., /home/username/src/arvados
+       LibPath              string // e.g., /var/lib/arvados
+       ClusterType          string // e.g., production
+       ListenHost           string // e.g., localhost
+       ControllerAddr       string // e.g., 127.0.0.1:8000
+       OwnTemporaryDatabase bool
+       Stderr               io.Writer
+
+       logger  logrus.FieldLogger
+       cluster *arvados.Cluster
+
+       ctx           context.Context
+       cancel        context.CancelFunc
+       done          chan struct{}
+       healthChecker *health.Aggregator
+       tasksReady    map[string]chan bool
+       waitShutdown  sync.WaitGroup
+
+       tempdir    string
+       configfile string
+       environ    []string // for child processes
+
+       setupRubyOnce sync.Once
+       setupRubyErr  error
+       goMutex       sync.Mutex
+}
+
+func (boot *Booter) Start(ctx context.Context, cfg *arvados.Config) {
+       boot.ctx, boot.cancel = context.WithCancel(ctx)
+       boot.done = make(chan struct{})
+
+       go func() {
+               sigch := make(chan os.Signal)
+               signal.Notify(sigch, syscall.SIGINT, syscall.SIGTERM)
+               defer signal.Stop(sigch)
+               go func() {
+                       for sig := range sigch {
+                               boot.logger.WithField("signal", sig).Info("caught signal")
+                               boot.cancel()
+                       }
+               }()
+
+               err := boot.run(cfg)
+               if err != nil {
+                       fmt.Fprintln(boot.Stderr, err)
+               }
+               close(boot.done)
+       }()
+}
+
+func (boot *Booter) run(cfg *arvados.Config) error {
+       cwd, err := os.Getwd()
        if err != nil {
-               return 1
+               return err
+       }
+       if !strings.HasPrefix(boot.SourcePath, "/") {
+               boot.SourcePath = filepath.Join(cwd, boot.SourcePath)
+       }
+       boot.SourcePath, err = filepath.EvalSymlinks(boot.SourcePath)
+       if err != nil {
+               return err
        }
 
-       tempdir, err := ioutil.TempDir("", "arvados-server-boot-")
+       boot.tempdir, err = ioutil.TempDir("", "arvados-server-boot-")
        if err != nil {
-               return 1
+               return err
        }
-       defer os.RemoveAll(tempdir)
+       defer os.RemoveAll(boot.tempdir)
 
        // Fill in any missing config keys, and write the resulting
        // config in the temp dir for child services to use.
-       err = boot.autofillConfig(cfg, log)
+       err = boot.autofillConfig(cfg, boot.logger)
        if err != nil {
-               return 1
+               return err
        }
-       conffile, err := os.OpenFile(filepath.Join(tempdir, "config.yml"), os.O_CREATE|os.O_WRONLY, 0777)
+       conffile, err := os.OpenFile(filepath.Join(boot.tempdir, "config.yml"), os.O_CREATE|os.O_WRONLY, 0777)
        if err != nil {
-               return 1
+               return err
        }
        defer conffile.Close()
        err = json.NewEncoder(conffile).Encode(cfg)
        if err != nil {
-               return 1
+               return err
        }
        err = conffile.Close()
        if err != nil {
-               return 1
+               return err
        }
-       os.Setenv("ARVADOS_CONFIG", conffile.Name())
+       boot.configfile = conffile.Name()
 
-       os.Setenv("RAILS_ENV", boot.clusterType)
+       boot.environ = os.Environ()
+       boot.cleanEnv()
+       boot.setEnv("ARVADOS_CONFIG", boot.configfile)
+       boot.setEnv("RAILS_ENV", boot.ClusterType)
+       boot.setEnv("TMPDIR", boot.tempdir)
+       boot.prependEnv("PATH", filepath.Join(boot.LibPath, "bin")+":")
 
+       boot.cluster, err = cfg.GetCluster("")
+       if err != nil {
+               return err
+       }
        // Now that we have the config, replace the bootstrap logger
        // with a new one according to the logging config.
-       cluster, err := cfg.GetCluster("")
-       if err != nil {
-               return 1
+       loglevel := boot.cluster.SystemLogs.LogLevel
+       if s := os.Getenv("ARVADOS_DEBUG"); s != "" && s != "0" {
+               loglevel = "debug"
        }
-       log = ctxlog.New(stderr, cluster.SystemLogs.Format, cluster.SystemLogs.LogLevel)
-       logger := log.WithFields(logrus.Fields{
+       boot.logger = ctxlog.New(boot.Stderr, boot.cluster.SystemLogs.Format, loglevel).WithFields(logrus.Fields{
                "PID": os.Getpid(),
        })
-       ctx := ctxlog.Context(context.Background(), logger)
-       ctx, cancel := context.WithCancel(ctx)
-       defer cancel()
-
-       ch := make(chan os.Signal)
-       signal.Notify(ch, syscall.SIGINT)
-       go func() {
-               for sig := range ch {
-                       logger.WithField("signal", sig).Info("caught signal")
-                       cancel()
-               }
-       }()
 
-       for _, dir := range []string{boot.libPath, filepath.Join(boot.libPath, "bin")} {
+       for _, dir := range []string{boot.LibPath, filepath.Join(boot.LibPath, "bin")} {
                if _, err = os.Stat(filepath.Join(dir, ".")); os.IsNotExist(err) {
                        err = os.Mkdir(dir, 0755)
                        if err != nil {
-                               return 1
+                               return err
                        }
                } else if err != nil {
-                       return 1
+                       return err
                }
        }
-       os.Setenv("PATH", filepath.Join(boot.libPath, "bin")+":"+os.Getenv("PATH"))
-
-       err = boot.installGoProgram(ctx, "cmd/arvados-server")
+       err = boot.installGoProgram(boot.ctx, "cmd/arvados-server")
        if err != nil {
-               return 1
+               return err
+       }
+       err = boot.setupRubyEnv()
+       if err != nil {
+               return err
        }
 
-       var wg sync.WaitGroup
-       for _, cmpt := range []component{
-               {name: "controller", cmdHandler: controller.Command},
-               {name: "dispatchcloud", cmdHandler: dispatchcloud.Command, notIfTest: true},
-               {name: "keepproxy", goProg: "services/keepproxy"},
-               {name: "railsAPI", svc: cluster.Services.RailsAPI, railsApp: "services/api"},
-       } {
-               cmpt := cmpt
-               wg.Add(1)
+       tasks := []bootTask{
+               createCertificates{},
+               runPostgreSQL{},
+               runNginx{},
+               runServiceCommand{name: "controller", svc: boot.cluster.Services.Controller, depends: []bootTask{runPostgreSQL{}}},
+               runGoProgram{src: "services/arv-git-httpd"},
+               runGoProgram{src: "services/health"},
+               runGoProgram{src: "services/keepproxy", depends: []bootTask{runPassenger{src: "services/api"}}},
+               runGoProgram{src: "services/keepstore", svc: boot.cluster.Services.Keepstore},
+               runGoProgram{src: "services/keep-web"},
+               runGoProgram{src: "services/ws", depends: []bootTask{runPostgreSQL{}}},
+               installPassenger{src: "services/api"},
+               runPassenger{src: "services/api", svc: boot.cluster.Services.RailsAPI, depends: []bootTask{createCertificates{}, runPostgreSQL{}, installPassenger{src: "services/api"}}},
+               installPassenger{src: "apps/workbench", depends: []bootTask{installPassenger{src: "services/api"}}}, // dependency ensures workbench doesn't delay api startup
+               runPassenger{src: "apps/workbench", svc: boot.cluster.Services.Workbench1, depends: []bootTask{installPassenger{src: "apps/workbench"}}},
+               seedDatabase{},
+       }
+       if boot.ClusterType != "test" {
+               tasks = append(tasks,
+                       runServiceCommand{name: "dispatch-cloud", svc: boot.cluster.Services.Controller},
+                       runGoProgram{src: "services/keep-balance"},
+               )
+       }
+       boot.tasksReady = map[string]chan bool{}
+       for _, task := range tasks {
+               boot.tasksReady[task.String()] = make(chan bool)
+       }
+       for _, task := range tasks {
+               task := task
+               fail := func(err error) {
+                       if boot.ctx.Err() != nil {
+                               return
+                       }
+                       boot.cancel()
+                       boot.logger.WithField("task", task.String()).WithError(err).Error("task failed")
+               }
                go func() {
-                       defer wg.Done()
-                       defer cancel()
-                       logger.WithField("component", cmpt.name).Info("starting")
-                       err := cmpt.Run(ctx, boot, stdout, stderr)
+                       boot.logger.WithField("task", task.String()).Info("starting")
+                       err := task.Run(boot.ctx, fail, boot)
                        if err != nil {
-                               logger.WithError(err).WithField("component", cmpt.name).Info("exited")
+                               fail(err)
+                               return
                        }
+                       close(boot.tasksReady[task.String()])
                }()
        }
-       <-ctx.Done()
-       wg.Wait()
-       return 0
+       err = boot.wait(boot.ctx, tasks...)
+       if err != nil {
+               return err
+       }
+       boot.logger.Info("all startup tasks are complete; starting health checks")
+       boot.healthChecker = &health.Aggregator{Cluster: boot.cluster}
+       <-boot.ctx.Done()
+       boot.logger.Info("shutting down")
+       boot.waitShutdown.Wait()
+       return boot.ctx.Err()
+}
+
+func (boot *Booter) wait(ctx context.Context, tasks ...bootTask) error {
+       for _, task := range tasks {
+               ch, ok := boot.tasksReady[task.String()]
+               if !ok {
+                       return fmt.Errorf("no such task: %s", task)
+               }
+               boot.logger.WithField("task", task.String()).Info("waiting")
+               select {
+               case <-ch:
+                       boot.logger.WithField("task", task.String()).Info("ready")
+               case <-ctx.Done():
+                       boot.logger.WithField("task", task.String()).Info("task was never ready")
+                       return ctx.Err()
+               }
+       }
+       return nil
+}
+
+func (boot *Booter) Stop() {
+       boot.cancel()
+       <-boot.done
+}
+
+func (boot *Booter) WaitReady() (*arvados.URL, bool) {
+       ticker := time.NewTicker(time.Second)
+       defer ticker.Stop()
+       for waiting := true; waiting; {
+               select {
+               case <-ticker.C:
+               case <-boot.ctx.Done():
+                       return nil, false
+               }
+               if boot.healthChecker == nil {
+                       // not set up yet
+                       continue
+               }
+               resp := boot.healthChecker.ClusterHealth()
+               // The overall health check (resp.Health=="OK") might
+               // never pass due to missing components (like
+               // arvados-dispatch-cloud in a test cluster), so
+               // instead we wait for all configured components to
+               // pass.
+               waiting = false
+               for target, check := range resp.Checks {
+                       if check.Health != "OK" {
+                               waiting = true
+                               boot.logger.WithField("target", target).Debug("waiting")
+                       }
+               }
+       }
+       u := boot.cluster.Services.Controller.ExternalURL
+       return &u, true
+}
+
+func (boot *Booter) prependEnv(key, prepend string) {
+       for i, s := range boot.environ {
+               if strings.HasPrefix(s, key+"=") {
+                       boot.environ[i] = key + "=" + prepend + s[len(key)+1:]
+                       return
+               }
+       }
+       boot.environ = append(boot.environ, key+"="+prepend)
+}
+
+var cleanEnvPrefixes = []string{
+       "GEM_HOME=",
+       "GEM_PATH=",
+       "ARVADOS_",
+}
+
+func (boot *Booter) cleanEnv() {
+       var cleaned []string
+       for _, s := range boot.environ {
+               drop := false
+               for _, p := range cleanEnvPrefixes {
+                       if strings.HasPrefix(s, p) {
+                               drop = true
+                               break
+                       }
+               }
+               if !drop {
+                       cleaned = append(cleaned, s)
+               }
+       }
+       boot.environ = cleaned
+}
+
+func (boot *Booter) setEnv(key, val string) {
+       for i, s := range boot.environ {
+               if strings.HasPrefix(s, key+"=") {
+                       boot.environ[i] = key + "=" + val
+                       return
+               }
+       }
+       boot.environ = append(boot.environ, key+"="+val)
 }
 
-func (boot *bootCommand) installGoProgram(ctx context.Context, srcpath string) error {
+// Remove all but the first occurrence of each env var.
+func dedupEnv(in []string) []string {
+       saw := map[string]bool{}
+       var out []string
+       for _, kv := range in {
+               if split := strings.Index(kv, "="); split < 1 {
+                       panic("invalid environment var: " + kv)
+               } else if saw[kv[:split]] {
+                       continue
+               } else {
+                       saw[kv[:split]] = true
+                       out = append(out, kv)
+               }
+       }
+       return out
+}
+
+func (boot *Booter) installGoProgram(ctx context.Context, srcpath string) error {
        boot.goMutex.Lock()
        defer boot.goMutex.Unlock()
-       env := append([]string{"GOPATH=" + boot.libPath}, os.Environ()...)
-       return boot.RunProgram(ctx, filepath.Join(boot.sourcePath, srcpath), nil, env, "go", "install")
+       return boot.RunProgram(ctx, filepath.Join(boot.SourcePath, srcpath), nil, []string{"GOPATH=" + boot.LibPath}, "go", "install")
 }
 
-func (boot *bootCommand) setupRubyEnv() error {
-       boot.setupRubyOnce.Do(func() {
-               buf, err := exec.Command("gem", "env", "gempath").Output() // /var/lib/arvados/.gem/ruby/2.5.0/bin:...
-               if err != nil || len(buf) == 0 {
-                       boot.setupRubyErr = fmt.Errorf("gem env gempath: %v", err)
+func (boot *Booter) setupRubyEnv() error {
+       cmd := exec.Command("gem", "env", "gempath")
+       cmd.Env = boot.environ
+       buf, err := cmd.Output() // /var/lib/arvados/.gem/ruby/2.5.0/bin:...
+       if err != nil || len(buf) == 0 {
+               return fmt.Errorf("gem env gempath: %v", err)
+       }
+       gempath := string(bytes.Split(buf, []byte{':'})[0])
+       boot.prependEnv("PATH", gempath+"/bin:")
+       boot.setEnv("GEM_HOME", gempath)
+       boot.setEnv("GEM_PATH", gempath)
+       // Passenger install doesn't work unless $HOME is ~user
+       u, err := user.Current()
+       if err != nil {
+               return err
+       }
+       boot.setEnv("HOME", u.HomeDir)
+       return nil
+}
+
+func (boot *Booter) lookPath(prog string) string {
+       for _, val := range boot.environ {
+               if strings.HasPrefix(val, "PATH=") {
+                       for _, dir := range filepath.SplitList(val[5:]) {
+                               path := filepath.Join(dir, prog)
+                               if fi, err := os.Stat(path); err == nil && fi.Mode()&0111 != 0 {
+                                       return path
+                               }
+                       }
                }
-               gempath := string(bytes.Split(buf, []byte{':'})[0])
-               os.Setenv("PATH", gempath+"/bin:"+os.Getenv("PATH"))
-               os.Setenv("GEM_HOME", gempath)
-               os.Setenv("GEM_PATH", gempath)
-       })
-       return boot.setupRubyErr
+       }
+       return prog
 }
 
-func (boot *bootCommand) RunProgram(ctx context.Context, dir string, output io.Writer, env []string, prog string, args ...string) error {
+// Run prog with args, using dir as working directory. If ctx is
+// cancelled while the child is running, RunProgram terminates the
+// child, waits for it to exit, then returns.
+//
+// Child's environment will have our env vars, plus any given in env.
+//
+// Child's stdout will be written to output if non-nil, otherwise the
+// boot command's stderr.
+func (boot *Booter) RunProgram(ctx context.Context, dir string, output io.Writer, env []string, prog string, args ...string) error {
        cmdline := fmt.Sprintf("%s", append([]string{prog}, args...))
-       fmt.Fprintf(boot.stderr, "%s executing in %s\n", cmdline, dir)
-       cmd := exec.Command(prog, args...)
+       boot.logger.WithField("command", cmdline).WithField("dir", dir).Info("executing")
+
+       logprefix := prog
+       if prog == "bundle" && len(args) > 2 && args[0] == "exec" {
+               logprefix = args[1]
+       } else if prog == "arvados-server" && len(args) > 1 {
+               logprefix = args[0]
+       }
+       if !strings.HasPrefix(dir, "/") {
+               logprefix = dir + ": " + logprefix
+       }
+
+       cmd := exec.Command(boot.lookPath(prog), args...)
+       stdout, err := cmd.StdoutPipe()
+       if err != nil {
+               return err
+       }
+       stderr, err := cmd.StderrPipe()
+       if err != nil {
+               return err
+       }
+       logwriter := &service.LogPrefixer{Writer: boot.Stderr, Prefix: []byte("[" + logprefix + "] ")}
+       go io.Copy(logwriter, stderr)
        if output == nil {
-               cmd.Stdout = boot.stderr
+               go io.Copy(logwriter, stdout)
        } else {
-               cmd.Stdout = output
+               go io.Copy(output, stdout)
        }
-       cmd.Stderr = boot.stderr
+
        if strings.HasPrefix(dir, "/") {
                cmd.Dir = dir
        } else {
-               cmd.Dir = filepath.Join(boot.sourcePath, dir)
-       }
-       if env != nil {
-               cmd.Env = env
+               cmd.Dir = filepath.Join(boot.SourcePath, dir)
        }
+       env = append([]string(nil), env...)
+       env = append(env, boot.environ...)
+       cmd.Env = dedupEnv(env)
+
+       exited := false
+       defer func() { exited = true }()
        go func() {
                <-ctx.Done()
                log := ctxlog.FromContext(ctx).WithFields(logrus.Fields{"dir": dir, "cmdline": cmdline})
-               for cmd.ProcessState != nil {
+               for !exited {
                        if cmd.Process == nil {
-                               log.Infof("waiting for child process to start")
-                               time.Sleep(time.Second)
+                               log.Debug("waiting for child process to start")
+                               time.Sleep(time.Second / 2)
                        } else {
-                               cmd.Process.Signal(syscall.SIGINT)
-                               log.WithField("PID", cmd.Process.Pid).Infof("waiting for child process to exit after SIGINT")
+                               log.WithField("PID", cmd.Process.Pid).Debug("sending SIGTERM")
+                               cmd.Process.Signal(syscall.SIGTERM)
                                time.Sleep(5 * time.Second)
+                               if !exited {
+                                       stdout.Close()
+                                       stderr.Close()
+                                       log.WithField("PID", cmd.Process.Pid).Warn("still waiting for child process to exit 5s after SIGTERM")
+                               }
                        }
                }
        }()
-       err := cmd.Run()
+
+       err = cmd.Start()
        if err != nil {
+               return err
+       }
+       err = cmd.Wait()
+       if err != nil && ctx.Err() == nil {
+               // Only report errors that happen before the context ends.
                return fmt.Errorf("%s: error: %v", cmdline, err)
        }
        return nil
 }
 
-type component struct {
-       name       string
-       svc        arvados.Service
-       cmdHandler cmd.Handler
-       railsApp   string // source dir in arvados tree, e.g., "services/api"
-       goProg     string // source dir in arvados tree, e.g., "services/keepstore"
-       notIfTest  bool   // don't run this component on a test cluster
-}
-
-func (cmpt *component) Run(ctx context.Context, boot *bootCommand, stdout, stderr io.Writer) error {
-       if cmpt.notIfTest && boot.clusterType == "test" {
-               fmt.Fprintf(stderr, "skipping component %q\n", cmpt.name)
-               <-ctx.Done()
-               return nil
+func (boot *Booter) autofillConfig(cfg *arvados.Config, log logrus.FieldLogger) error {
+       cluster, err := cfg.GetCluster("")
+       if err != nil {
+               return err
        }
-       fmt.Fprintf(stderr, "starting component %q\n", cmpt.name)
-       if cmpt.cmdHandler != nil {
-               errs := make(chan error, 1)
-               go func() {
-                       defer close(errs)
-                       exitcode := cmpt.cmdHandler.RunCommand(cmpt.name, nil, bytes.NewBuffer(nil), stdout, stderr)
-                       if exitcode != 0 {
-                               errs <- fmt.Errorf("exit code %d", exitcode)
+       usedPort := map[string]bool{}
+       nextPort := func() string {
+               for {
+                       port, err := availablePort(":0")
+                       if err != nil {
+                               panic(err)
                        }
-               }()
-               select {
-               case err := <-errs:
-                       return err
-               case <-ctx.Done():
-                       // cmpt.cmdHandler.RunCommand() doesn't have
-                       // access to our context, so it won't shut
-                       // down by itself. We just abandon it.
-                       return nil
-               }
-       }
-       if cmpt.goProg != "" {
-               return boot.RunProgram(ctx, cmpt.goProg, nil, nil, "go", "run", ".")
-       }
-       if cmpt.railsApp != "" {
-               port := "-"
-               for u := range cmpt.svc.InternalURLs {
-                       if _, p, err := net.SplitHostPort(u.Host); err != nil {
-                               return err
-                       } else if p != "" {
-                               port = p
-                       } else if u.Scheme == "https" {
-                               port = "443"
-                       } else {
-                               port = "80"
+                       if usedPort[port] {
+                               continue
                        }
-                       break
-               }
-               if port == "-" {
-                       return fmt.Errorf("bug: no InternalURLs for component %q: %v", cmpt.name, cmpt.svc.InternalURLs)
+                       usedPort[port] = true
+                       return port
                }
-
-               err := boot.setupRubyEnv()
+       }
+       if cluster.Services.Controller.ExternalURL.Host == "" {
+               h, p, err := net.SplitHostPort(boot.ControllerAddr)
                if err != nil {
                        return err
                }
-               var buf bytes.Buffer
-               err = boot.RunProgram(ctx, cmpt.railsApp, &buf, nil, "gem", "list", "--details", "bundler")
-               if err != nil {
-                       return err
+               if h == "" {
+                       h = boot.ListenHost
                }
-               for _, version := range []string{"1.11.0", "1.17.3", "2.0.2"} {
-                       if !strings.Contains(buf.String(), "("+version+")") {
-                               err = boot.RunProgram(ctx, cmpt.railsApp, nil, nil, "gem", "install", "--user", "bundler:1.11", "bundler:1.17.3", "bundler:2.0.2")
-                               if err != nil {
-                                       return err
-                               }
-                               break
+               if p == "0" {
+                       p, err = availablePort(":0")
+                       if err != nil {
+                               return err
                        }
+                       usedPort[p] = true
                }
-               err = boot.RunProgram(ctx, cmpt.railsApp, nil, nil, "bundle", "install", "--jobs", "4", "--path", filepath.Join(os.Getenv("HOME"), ".gem"))
-               if err != nil {
-                       return err
-               }
-               err = boot.RunProgram(ctx, cmpt.railsApp, nil, nil, "bundle", "exec", "passenger-config", "build-native-support")
-               if err != nil {
-                       return err
-               }
-               err = boot.RunProgram(ctx, cmpt.railsApp, nil, nil, "bundle", "exec", "passenger-config", "install-standalone-runtime")
-               if err != nil {
-                       return err
-               }
-               err = boot.RunProgram(ctx, cmpt.railsApp, nil, nil, "bundle", "exec", "passenger-config", "validate-install")
-               if err != nil {
-                       return err
-               }
-               err = boot.RunProgram(ctx, cmpt.railsApp, nil, nil, "bundle", "exec", "passenger", "start", "-p", port)
-               if err != nil {
-                       return err
-               }
-       }
-       return fmt.Errorf("bug: component %q has nothing to run", cmpt.name)
-}
-
-func (boot *bootCommand) autofillConfig(cfg *arvados.Config, log logrus.FieldLogger) error {
-       cluster, err := cfg.GetCluster("")
-       if err != nil {
-               return err
+               cluster.Services.Controller.ExternalURL = arvados.URL{Scheme: "https", Host: net.JoinHostPort(h, p)}
        }
-       port := 9000
        for _, svc := range []*arvados.Service{
                &cluster.Services.Controller,
                &cluster.Services.DispatchCloud,
+               &cluster.Services.GitHTTP,
+               &cluster.Services.Health,
+               &cluster.Services.Keepproxy,
+               &cluster.Services.Keepstore,
                &cluster.Services.RailsAPI,
+               &cluster.Services.WebDAV,
+               &cluster.Services.WebDAVDownload,
+               &cluster.Services.Websocket,
+               &cluster.Services.Workbench1,
        } {
+               if svc == &cluster.Services.DispatchCloud && boot.ClusterType == "test" {
+                       continue
+               }
+               if svc.ExternalURL.Host == "" && (svc == &cluster.Services.Controller ||
+                       svc == &cluster.Services.GitHTTP ||
+                       svc == &cluster.Services.Keepproxy ||
+                       svc == &cluster.Services.WebDAV ||
+                       svc == &cluster.Services.WebDAVDownload ||
+                       svc == &cluster.Services.Websocket ||
+                       svc == &cluster.Services.Workbench1) {
+                       svc.ExternalURL = arvados.URL{Scheme: "https", Host: fmt.Sprintf("%s:%s", boot.ListenHost, nextPort())}
+               }
                if len(svc.InternalURLs) == 0 {
-                       port++
                        svc.InternalURLs = map[arvados.URL]arvados.ServiceInstance{
-                               arvados.URL{Scheme: "http", Host: fmt.Sprintf("localhost:%d", port)}: arvados.ServiceInstance{},
+                               arvados.URL{Scheme: "http", Host: fmt.Sprintf("%s:%s", boot.ListenHost, nextPort())}: arvados.ServiceInstance{},
                        }
                }
        }
-       if cluster.Services.Controller.ExternalURL.Host == "" {
-               for k := range cluster.Services.Controller.InternalURLs {
-                       cluster.Services.Controller.ExternalURL = k
-               }
-       }
        if cluster.SystemRootToken == "" {
                cluster.SystemRootToken = randomHexString(64)
        }
+       if cluster.ManagementToken == "" {
+               cluster.ManagementToken = randomHexString(64)
+       }
        if cluster.API.RailsSessionSecretToken == "" {
                cluster.API.RailsSessionSecretToken = randomHexString(64)
        }
        if cluster.Collections.BlobSigningKey == "" {
                cluster.Collections.BlobSigningKey = randomHexString(64)
        }
-       if boot.clusterType != "production" && cluster.Containers.DispatchPrivateKey == "" {
-               buf, err := ioutil.ReadFile(filepath.Join(boot.sourcePath, "lib", "dispatchcloud", "test", "sshkey_dispatch"))
+       if boot.ClusterType != "production" && cluster.Containers.DispatchPrivateKey == "" {
+               buf, err := ioutil.ReadFile(filepath.Join(boot.SourcePath, "lib", "dispatchcloud", "test", "sshkey_dispatch"))
                if err != nil {
                        return err
                }
                cluster.Containers.DispatchPrivateKey = string(buf)
        }
+       if boot.ClusterType != "production" {
+               cluster.TLS.Insecure = true
+       }
+       if boot.ClusterType == "test" {
+               // Add a second keepstore process.
+               cluster.Services.Keepstore.InternalURLs[arvados.URL{Scheme: "http", Host: fmt.Sprintf("%s:%s", boot.ListenHost, nextPort())}] = arvados.ServiceInstance{}
+
+               // Create a directory-backed volume for each keepstore
+               // process.
+               cluster.Volumes = map[string]arvados.Volume{}
+               for url := range cluster.Services.Keepstore.InternalURLs {
+                       volnum := len(cluster.Volumes)
+                       datadir := fmt.Sprintf("%s/keep%d.data", boot.tempdir, volnum)
+                       if _, err = os.Stat(datadir + "/."); err == nil {
+                       } else if !os.IsNotExist(err) {
+                               return err
+                       } else if err = os.Mkdir(datadir, 0777); err != nil {
+                               return err
+                       }
+                       cluster.Volumes[fmt.Sprintf(cluster.ClusterID+"-nyw5e-%015d", volnum)] = arvados.Volume{
+                               Driver:           "Directory",
+                               DriverParameters: json.RawMessage(fmt.Sprintf(`{"Root":%q}`, datadir)),
+                               AccessViaHosts: map[arvados.URL]arvados.VolumeAccess{
+                                       url: {},
+                               },
+                       }
+               }
+       }
+       if boot.OwnTemporaryDatabase {
+               cluster.PostgreSQL.Connection = arvados.PostgreSQLConnection{
+                       "client_encoding": "utf8",
+                       "host":            "localhost",
+                       "port":            nextPort(),
+                       "dbname":          "arvados_test",
+                       "user":            "arvados",
+                       "password":        "insecure_arvados_test",
+               }
+       }
+
        cfg.Clusters[cluster.ClusterID] = *cluster
        return nil
 }
@@ -400,3 +668,59 @@ func randomHexString(chars int) string {
        }
        return fmt.Sprintf("%x", b)
 }
+
+func internalPort(svc arvados.Service) (string, error) {
+       for u := range svc.InternalURLs {
+               if _, p, err := net.SplitHostPort(u.Host); err != nil {
+                       return "", err
+               } else if p != "" {
+                       return p, nil
+               } else if u.Scheme == "https" {
+                       return "443", nil
+               } else {
+                       return "80", nil
+               }
+       }
+       return "", fmt.Errorf("service has no InternalURLs")
+}
+
+func externalPort(svc arvados.Service) (string, error) {
+       if _, p, err := net.SplitHostPort(svc.ExternalURL.Host); err != nil {
+               return "", err
+       } else if p != "" {
+               return p, nil
+       } else if svc.ExternalURL.Scheme == "https" {
+               return "443", nil
+       } else {
+               return "80", nil
+       }
+}
+
+func availablePort(addr string) (string, error) {
+       ln, err := net.Listen("tcp", addr)
+       if err != nil {
+               return "", err
+       }
+       defer ln.Close()
+       _, port, err := net.SplitHostPort(ln.Addr().String())
+       if err != nil {
+               return "", err
+       }
+       return port, nil
+}
+
+// Try to connect to addr until it works, then close ch. Give up if
+// ctx cancels.
+func waitForConnect(ctx context.Context, addr string) error {
+       dialer := net.Dialer{Timeout: time.Second}
+       for ctx.Err() == nil {
+               conn, err := dialer.DialContext(ctx, "tcp", addr)
+               if err != nil {
+                       time.Sleep(time.Second / 10)
+                       continue
+               }
+               conn.Close()
+               return nil
+       }
+       return ctx.Err()
+}