Merge branch '18947-githttpd'
[arvados.git] / lib / boot / supervisor.go
index 323f672348b84f7b45688ee25b05ad81f511517d..94cd5d000023ce4fb3c5689073871f091cb22066 100644 (file)
@@ -37,25 +37,47 @@ import (
 )
 
 type Supervisor struct {
-       SourcePath           string // e.g., /home/username/src/arvados
-       SourceVersion        string // e.g., acbd1324...
-       ClusterType          string // e.g., production
-       ListenHost           string // e.g., localhost
-       ControllerAddr       string // e.g., 127.0.0.1:8000
-       Workbench2Source     string // e.g., /home/username/src/arvados-workbench2
+       // Config file location like "/etc/arvados/config.yml", or "-"
+       // to read from Stdin (see below).
+       ConfigPath string
+       // Literal config file (useful for test suites). If non-empty,
+       // this is used instead of ConfigPath.
+       ConfigYAML string
+       // Path to arvados source tree. Only used for dev/test
+       // clusters.
+       SourcePath string
+       // Version number to build into binaries. Only used for
+       // dev/test clusters.
+       SourceVersion string
+       // "production", "development", or "test".
+       ClusterType string
+       // Listening address for external services, and internal
+       // services whose InternalURLs are not explicitly configured.
+       // If blank, listen on the configured controller ExternalURL
+       // host; if that is also blank, listen on all addresses
+       // (0.0.0.0).
+       ListenHost string
+       // Default host:port for controller ExternalURL if not
+       // explicitly configured in config file. If blank, use a
+       // random port on ListenHost.
+       ControllerAddr string
+       // Path to arvados-workbench2 source tree checkout.
+       Workbench2Source     string
        NoWorkbench1         bool
        NoWorkbench2         bool
        OwnTemporaryDatabase bool
+       Stdin                io.Reader
        Stderr               io.Writer
 
-       logger  logrus.FieldLogger
-       cluster *arvados.Cluster
+       logger   logrus.FieldLogger
+       cluster  *arvados.Cluster       // nil if this is a multi-cluster supervisor
+       children map[string]*Supervisor // nil if this is a single-cluster supervisor
 
        ctx           context.Context
        cancel        context.CancelFunc
-       done          chan struct{} // closed when child procs/services have shut down
-       err           error         // error that caused shutdown (valid when done is closed)
-       healthChecker *health.Aggregator
+       done          chan struct{}      // closed when child procs/services have shut down
+       err           error              // error that caused shutdown (valid when done is closed)
+       healthChecker *health.Aggregator // nil if this is a multi-cluster supervisor, or still booting
        tasksReady    map[string]chan bool
        waitShutdown  sync.WaitGroup
 
@@ -66,73 +88,161 @@ type Supervisor struct {
        environ    []string // for child processes
 }
 
-func (super *Supervisor) Cluster() *arvados.Cluster { return super.cluster }
+func (super *Supervisor) Clusters() map[string]*arvados.Cluster {
+       m := map[string]*arvados.Cluster{}
+       if super.cluster != nil {
+               m[super.cluster.ClusterID] = super.cluster
+       }
+       for id, super2 := range super.children {
+               m[id] = super2.Cluster("")
+       }
+       return m
+}
 
-func (super *Supervisor) Start(ctx context.Context, cfg *arvados.Config, cfgPath string) {
+func (super *Supervisor) Cluster(id string) *arvados.Cluster {
+       if super.children != nil {
+               return super.children[id].Cluster(id)
+       } else {
+               return super.cluster
+       }
+}
+
+func (super *Supervisor) Start(ctx context.Context) {
+       super.logger = ctxlog.FromContext(ctx)
        super.ctx, super.cancel = context.WithCancel(ctx)
        super.done = make(chan struct{})
 
+       sigch := make(chan os.Signal)
+       signal.Notify(sigch, syscall.SIGINT, syscall.SIGTERM)
+       defer signal.Stop(sigch)
        go func() {
-               defer close(super.done)
+               for sig := range sigch {
+                       super.logger.WithField("signal", sig).Info("caught signal")
+                       if super.err == nil {
+                               super.err = fmt.Errorf("caught signal %s", sig)
+                       }
+                       super.cancel()
+               }
+       }()
 
-               sigch := make(chan os.Signal)
-               signal.Notify(sigch, syscall.SIGINT, syscall.SIGTERM)
-               defer signal.Stop(sigch)
-               go func() {
-                       for sig := range sigch {
-                               super.logger.WithField("signal", sig).Info("caught signal")
-                               if super.err == nil {
-                                       super.err = fmt.Errorf("caught signal %s", sig)
-                               }
-                               super.cancel()
+       hupch := make(chan os.Signal)
+       signal.Notify(hupch, syscall.SIGHUP)
+       defer signal.Stop(hupch)
+       go func() {
+               for sig := range hupch {
+                       super.logger.WithField("signal", sig).Info("caught signal")
+                       if super.err == nil {
+                               super.err = errNeedConfigReload
                        }
-               }()
+                       super.cancel()
+               }
+       }()
 
-               hupch := make(chan os.Signal)
-               signal.Notify(hupch, syscall.SIGHUP)
-               defer signal.Stop(hupch)
+       loaderStdin := super.Stdin
+       if super.ConfigYAML != "" {
+               loaderStdin = bytes.NewBufferString(super.ConfigYAML)
+       }
+       loader := config.NewLoader(loaderStdin, super.logger)
+       loader.SkipLegacy = true
+       loader.SkipAPICalls = true
+       loader.Path = super.ConfigPath
+       if super.ConfigYAML != "" {
+               loader.Path = "-"
+       }
+       cfg, err := loader.Load()
+       if err != nil {
+               super.err = err
+               close(super.done)
+               super.cancel()
+               return
+       }
+
+       if super.ConfigPath != "" && super.ConfigPath != "-" && cfg.AutoReloadConfig {
+               go watchConfig(super.ctx, super.logger, super.ConfigPath, copyConfig(cfg), func() {
+                       if super.err == nil {
+                               super.err = errNeedConfigReload
+                       }
+                       super.cancel()
+               })
+       }
+
+       if len(cfg.Clusters) > 1 {
+               super.startFederation(cfg)
                go func() {
-                       for sig := range hupch {
-                               super.logger.WithField("signal", sig).Info("caught signal")
+                       defer super.cancel()
+                       defer close(super.done)
+                       for _, super2 := range super.children {
+                               err := super2.Wait()
                                if super.err == nil {
-                                       super.err = errNeedConfigReload
+                                       super.err = err
                                }
-                               super.cancel()
                        }
                }()
-
-               if cfgPath != "" && cfgPath != "-" && cfg.AutoReloadConfig {
-                       go watchConfig(super.ctx, super.logger, cfgPath, copyConfig(cfg), func() {
+       } else {
+               go func() {
+                       defer super.cancel()
+                       defer close(super.done)
+                       super.cluster, super.err = cfg.GetCluster("")
+                       if super.err != nil {
+                               return
+                       }
+                       err := super.runCluster()
+                       if err != nil {
+                               super.logger.WithError(err).Info("supervisor shut down")
                                if super.err == nil {
-                                       super.err = errNeedConfigReload
+                                       super.err = err
                                }
-                               super.cancel()
-                       })
-               }
-
-               err := super.run(cfg)
-               if err != nil {
-                       super.logger.WithError(err).Warn("supervisor shut down")
-                       if super.err == nil {
-                               super.err = err
                        }
-               }
-       }()
+               }()
+       }
 }
 
+// Wait returns when all child processes and goroutines have exited.
 func (super *Supervisor) Wait() error {
        <-super.done
        return super.err
 }
 
-func (super *Supervisor) run(cfg *arvados.Config) error {
-       defer super.cancel()
+// startFederation starts a child Supervisor for each cluster in the
+// given config. Each is a copy of the original/parent with the
+// original config reduced to a single cluster.
+func (super *Supervisor) startFederation(cfg *arvados.Config) {
+       super.children = map[string]*Supervisor{}
+       for id, cc := range cfg.Clusters {
+               super2 := *super
+               yaml, err := json.Marshal(arvados.Config{Clusters: map[string]arvados.Cluster{id: cc}})
+               if err != nil {
+                       panic(fmt.Sprintf("json.Marshal partial config: %s", err))
+               }
+               super2.ConfigYAML = string(yaml)
+               super2.ConfigPath = "-"
+               super2.children = nil
 
+               if super2.ClusterType == "test" {
+                       super2.Stderr = &service.LogPrefixer{
+                               Writer: super.Stderr,
+                               Prefix: []byte("[" + id + "] "),
+                       }
+               }
+               super2.Start(super.ctx)
+               super.children[id] = &super2
+       }
+}
+
+func (super *Supervisor) runCluster() error {
        cwd, err := os.Getwd()
        if err != nil {
                return err
        }
-       if !strings.HasPrefix(super.SourcePath, "/") {
+       if super.ClusterType == "test" && super.SourcePath == "" {
+               // When invoked by test suite, default to current
+               // source tree
+               buf, err := exec.Command("git", "rev-parse", "--show-toplevel").CombinedOutput()
+               if err != nil {
+                       return fmt.Errorf("git rev-parse: %w", err)
+               }
+               super.SourcePath = strings.TrimSuffix(string(buf), "\n")
+       } else if !strings.HasPrefix(super.SourcePath, "/") {
                super.SourcePath = filepath.Join(cwd, super.SourcePath)
        }
        super.SourcePath, err = filepath.EvalSymlinks(super.SourcePath)
@@ -140,6 +250,18 @@ func (super *Supervisor) run(cfg *arvados.Config) error {
                return err
        }
 
+       if super.ListenHost == "" {
+               if urlhost := super.cluster.Services.Controller.ExternalURL.Host; urlhost != "" {
+                       if h, _, _ := net.SplitHostPort(urlhost); h != "" {
+                               super.ListenHost = h
+                       } else {
+                               super.ListenHost = urlhost
+                       }
+               } else {
+                       super.ListenHost = "0.0.0.0"
+               }
+       }
+
        // Choose bin and temp dirs: /var/lib/arvados/... in
        // production, transient tempdir otherwise.
        if super.ClusterType == "production" {
@@ -164,7 +286,7 @@ func (super *Supervisor) run(cfg *arvados.Config) error {
 
        // Fill in any missing config keys, and write the resulting
        // config in the temp dir for child services to use.
-       err = super.autofillConfig(cfg)
+       err = super.autofillConfig()
        if err != nil {
                return err
        }
@@ -173,7 +295,9 @@ func (super *Supervisor) run(cfg *arvados.Config) error {
                return err
        }
        defer conffile.Close()
-       err = json.NewEncoder(conffile).Encode(cfg)
+       err = json.NewEncoder(conffile).Encode(arvados.Config{
+               Clusters: map[string]arvados.Cluster{
+                       super.cluster.ClusterID: *super.cluster}})
        if err != nil {
                return err
        }
@@ -193,10 +317,6 @@ func (super *Supervisor) run(cfg *arvados.Config) error {
                super.prependEnv("PATH", super.tempdir+"/bin:")
        }
 
-       super.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.
        loglevel := super.cluster.SystemLogs.LogLevel
@@ -244,9 +364,9 @@ func (super *Supervisor) run(cfg *arvados.Config) error {
                runPostgreSQL{},
                runNginx{},
                runServiceCommand{name: "controller", svc: super.cluster.Services.Controller, depends: []supervisedTask{seedDatabase{}}},
-               runGoProgram{src: "services/arv-git-httpd", svc: super.cluster.Services.GitHTTP},
+               runServiceCommand{name: "git-httpd", svc: super.cluster.Services.GitHTTP},
                runGoProgram{src: "services/health", svc: super.cluster.Services.Health},
-               runGoProgram{src: "services/keepproxy", svc: super.cluster.Services.Keepproxy, depends: []supervisedTask{runPassenger{src: "services/api"}}},
+               runServiceCommand{name: "keepproxy", svc: super.cluster.Services.Keepproxy, depends: []supervisedTask{runPassenger{src: "services/api"}}},
                runServiceCommand{name: "keepstore", svc: super.cluster.Services.Keepstore},
                runGoProgram{src: "services/keep-web", svc: super.cluster.Services.WebDAV},
                runServiceCommand{name: "ws", svc: super.cluster.Services.Websocket, depends: []supervisedTask{seedDatabase{}}},
@@ -307,36 +427,60 @@ func (super *Supervisor) run(cfg *arvados.Config) error {
 }
 
 func (super *Supervisor) wait(ctx context.Context, tasks ...supervisedTask) error {
+       ticker := time.NewTicker(15 * time.Second)
+       defer ticker.Stop()
        for _, task := range tasks {
                ch, ok := super.tasksReady[task.String()]
                if !ok {
                        return fmt.Errorf("no such task: %s", task)
                }
                super.logger.WithField("task", task.String()).Info("waiting")
-               select {
-               case <-ch:
-                       super.logger.WithField("task", task.String()).Info("ready")
-               case <-ctx.Done():
-                       super.logger.WithField("task", task.String()).Info("task was never ready")
-                       return ctx.Err()
+               for {
+                       select {
+                       case <-ch:
+                               super.logger.WithField("task", task.String()).Info("ready")
+                       case <-ctx.Done():
+                               super.logger.WithField("task", task.String()).Info("task was never ready")
+                               return ctx.Err()
+                       case <-ticker.C:
+                               super.logger.WithField("task", task.String()).Info("still waiting...")
+                               continue
+                       }
+                       break
                }
        }
        return nil
 }
 
+// Stop shuts down all child processes and goroutines, and returns
+// when all of them have exited.
 func (super *Supervisor) Stop() {
        super.cancel()
        <-super.done
 }
 
-func (super *Supervisor) WaitReady() (*arvados.URL, bool) {
+// WaitReady waits for the cluster(s) to be ready to handle requests,
+// then returns true. If startup fails, it returns false.
+func (super *Supervisor) WaitReady() bool {
+       if super.children != nil {
+               for id, super2 := range super.children {
+                       super.logger.Infof("waiting for %s to be ready", id)
+                       if !super2.WaitReady() {
+                               super.logger.Infof("%s startup failed", id)
+                               return false
+                       }
+                       super.logger.Infof("%s is ready", id)
+               }
+               super.logger.Info("all clusters are ready")
+               return true
+       }
        ticker := time.NewTicker(time.Second)
        defer ticker.Stop()
        for waiting := "all"; waiting != ""; {
                select {
                case <-ticker.C:
                case <-super.ctx.Done():
-                       return nil, false
+                       return false
                }
                if super.healthChecker == nil {
                        // not set up yet
@@ -358,8 +502,7 @@ func (super *Supervisor) WaitReady() (*arvados.URL, bool) {
                        super.logger.WithField("targets", waiting[1:]).Info("waiting")
                }
        }
-       u := super.cluster.Services.Controller.ExternalURL
-       return &u, true
+       return true
 }
 
 func (super *Supervisor) prependEnv(key, prepend string) {
@@ -631,11 +774,7 @@ func (super *Supervisor) RunProgram(ctx context.Context, dir string, opts runOpt
        return nil
 }
 
-func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
-       cluster, err := cfg.GetCluster("")
-       if err != nil {
-               return err
-       }
+func (super *Supervisor) autofillConfig() error {
        usedPort := map[string]bool{}
        nextPort := func(host string) (string, error) {
                for {
@@ -653,39 +792,39 @@ func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
                        return port, nil
                }
        }
-       if cluster.Services.Controller.ExternalURL.Host == "" {
+       if super.cluster.Services.Controller.ExternalURL.Host == "" {
                h, p, err := net.SplitHostPort(super.ControllerAddr)
-               if err != nil {
-                       return fmt.Errorf("SplitHostPort(ControllerAddr): %w", err)
+               if err != nil && super.ControllerAddr != "" {
+                       return fmt.Errorf("SplitHostPort(ControllerAddr %q): %w", super.ControllerAddr, err)
                }
                if h == "" {
                        h = super.ListenHost
                }
-               if p == "0" {
+               if p == "0" || p == "" {
                        p, err = nextPort(h)
                        if err != nil {
                                return err
                        }
                }
-               cluster.Services.Controller.ExternalURL = arvados.URL{Scheme: "https", Host: net.JoinHostPort(h, p), Path: "/"}
+               super.cluster.Services.Controller.ExternalURL = arvados.URL{Scheme: "https", Host: net.JoinHostPort(h, p), Path: "/"}
        }
-       u := url.URL(cluster.Services.Controller.ExternalURL)
+       u := url.URL(super.cluster.Services.Controller.ExternalURL)
        defaultExtHost := u.Hostname()
        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,
-               &cluster.Services.Workbench2,
+               &super.cluster.Services.Controller,
+               &super.cluster.Services.DispatchCloud,
+               &super.cluster.Services.GitHTTP,
+               &super.cluster.Services.Health,
+               &super.cluster.Services.Keepproxy,
+               &super.cluster.Services.Keepstore,
+               &super.cluster.Services.RailsAPI,
+               &super.cluster.Services.WebDAV,
+               &super.cluster.Services.WebDAVDownload,
+               &super.cluster.Services.Websocket,
+               &super.cluster.Services.Workbench1,
+               &super.cluster.Services.Workbench2,
        } {
-               if svc == &cluster.Services.DispatchCloud && super.ClusterType == "test" {
+               if svc == &super.cluster.Services.DispatchCloud && super.ClusterType == "test" {
                        continue
                }
                if svc.ExternalURL.Host == "" {
@@ -694,21 +833,21 @@ func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
                                return err
                        }
                        host := net.JoinHostPort(defaultExtHost, port)
-                       if svc == &cluster.Services.Controller ||
-                               svc == &cluster.Services.GitHTTP ||
-                               svc == &cluster.Services.Health ||
-                               svc == &cluster.Services.Keepproxy ||
-                               svc == &cluster.Services.WebDAV ||
-                               svc == &cluster.Services.WebDAVDownload ||
-                               svc == &cluster.Services.Workbench1 ||
-                               svc == &cluster.Services.Workbench2 {
+                       if svc == &super.cluster.Services.Controller ||
+                               svc == &super.cluster.Services.GitHTTP ||
+                               svc == &super.cluster.Services.Health ||
+                               svc == &super.cluster.Services.Keepproxy ||
+                               svc == &super.cluster.Services.WebDAV ||
+                               svc == &super.cluster.Services.WebDAVDownload ||
+                               svc == &super.cluster.Services.Workbench1 ||
+                               svc == &super.cluster.Services.Workbench2 {
                                svc.ExternalURL = arvados.URL{Scheme: "https", Host: host, Path: "/"}
-                       } else if svc == &cluster.Services.Websocket {
+                       } else if svc == &super.cluster.Services.Websocket {
                                svc.ExternalURL = arvados.URL{Scheme: "wss", Host: host, Path: "/websocket"}
                        }
                }
-               if super.NoWorkbench1 && svc == &cluster.Services.Workbench1 ||
-                       super.NoWorkbench2 && svc == &cluster.Services.Workbench2 {
+               if super.NoWorkbench1 && svc == &super.cluster.Services.Workbench1 ||
+                       super.NoWorkbench2 && svc == &super.cluster.Services.Workbench2 {
                        // When workbench1 is disabled, it gets an
                        // ExternalURL (so we have a valid listening
                        // port to write in our Nginx config) but no
@@ -728,26 +867,26 @@ func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
                }
        }
        if super.ClusterType != "production" {
-               if cluster.SystemRootToken == "" {
-                       cluster.SystemRootToken = randomHexString(64)
+               if super.cluster.SystemRootToken == "" {
+                       super.cluster.SystemRootToken = randomHexString(64)
                }
-               if cluster.ManagementToken == "" {
-                       cluster.ManagementToken = randomHexString(64)
+               if super.cluster.ManagementToken == "" {
+                       super.cluster.ManagementToken = randomHexString(64)
                }
-               if cluster.Collections.BlobSigningKey == "" {
-                       cluster.Collections.BlobSigningKey = randomHexString(64)
+               if super.cluster.Collections.BlobSigningKey == "" {
+                       super.cluster.Collections.BlobSigningKey = randomHexString(64)
                }
-               if cluster.Users.AnonymousUserToken == "" {
-                       cluster.Users.AnonymousUserToken = randomHexString(64)
+               if super.cluster.Users.AnonymousUserToken == "" {
+                       super.cluster.Users.AnonymousUserToken = randomHexString(64)
                }
-               if cluster.Containers.DispatchPrivateKey == "" {
+               if super.cluster.Containers.DispatchPrivateKey == "" {
                        buf, err := ioutil.ReadFile(filepath.Join(super.SourcePath, "lib", "dispatchcloud", "test", "sshkey_dispatch"))
                        if err != nil {
                                return err
                        }
-                       cluster.Containers.DispatchPrivateKey = string(buf)
+                       super.cluster.Containers.DispatchPrivateKey = string(buf)
                }
-               cluster.TLS.Insecure = true
+               super.cluster.TLS.Insecure = true
        }
        if super.ClusterType == "test" {
                // Add a second keepstore process.
@@ -756,13 +895,13 @@ func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
                        return err
                }
                host := net.JoinHostPort(super.ListenHost, port)
-               cluster.Services.Keepstore.InternalURLs[arvados.URL{Scheme: "http", Host: host, Path: "/"}] = arvados.ServiceInstance{}
+               super.cluster.Services.Keepstore.InternalURLs[arvados.URL{Scheme: "http", Host: host, Path: "/"}] = 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)
+               super.cluster.Volumes = map[string]arvados.Volume{}
+               for url := range super.cluster.Services.Keepstore.InternalURLs {
+                       volnum := len(super.cluster.Volumes)
                        datadir := fmt.Sprintf("%s/keep%d.data", super.tempdir, volnum)
                        if _, err = os.Stat(datadir + "/."); err == nil {
                        } else if !os.IsNotExist(err) {
@@ -770,7 +909,7 @@ func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
                        } else if err = os.Mkdir(datadir, 0755); err != nil {
                                return err
                        }
-                       cluster.Volumes[fmt.Sprintf(cluster.ClusterID+"-nyw5e-%015d", volnum)] = arvados.Volume{
+                       super.cluster.Volumes[fmt.Sprintf(super.cluster.ClusterID+"-nyw5e-%015d", volnum)] = arvados.Volume{
                                Driver:           "Directory",
                                DriverParameters: json.RawMessage(fmt.Sprintf(`{"Root":%q}`, datadir)),
                                AccessViaHosts: map[arvados.URL]arvados.VolumeAccess{
@@ -783,7 +922,7 @@ func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
                                },
                        }
                }
-               cluster.StorageClasses = map[string]arvados.StorageClassConfig{
+               super.cluster.StorageClasses = map[string]arvados.StorageClassConfig{
                        "default": {Default: true},
                        "foo":     {},
                        "bar":     {},
@@ -794,7 +933,7 @@ func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
                if err != nil {
                        return err
                }
-               cluster.PostgreSQL.Connection = arvados.PostgreSQLConnection{
+               super.cluster.PostgreSQL.Connection = arvados.PostgreSQLConnection{
                        "client_encoding": "utf8",
                        "host":            "localhost",
                        "port":            port,
@@ -803,13 +942,15 @@ func (super *Supervisor) autofillConfig(cfg *arvados.Config) error {
                        "password":        "insecure_arvados_test",
                }
        }
-
-       cfg.Clusters[cluster.ClusterID] = *cluster
        return nil
 }
 
 func addrIsLocal(addr string) (bool, error) {
-       return true, nil
+       if h, _, err := net.SplitHostPort(addr); err != nil {
+               return false, err
+       } else {
+               addr = net.JoinHostPort(h, "0")
+       }
        listener, err := net.Listen("tcp", addr)
        if err == nil {
                listener.Close()
@@ -876,6 +1017,7 @@ func availablePort(host string) (string, error) {
 // Try to connect to addr until it works, then close ch. Give up if
 // ctx cancels.
 func waitForConnect(ctx context.Context, addr string) error {
+       ctxlog.FromContext(ctx).WithField("addr", addr).Info("waitForConnect")
        dialer := net.Dialer{Timeout: time.Second}
        for ctx.Err() == nil {
                conn, err := dialer.DialContext(ctx, "tcp", addr)