diff --git a/pkg/acceptance/cluster/testconfig.proto b/pkg/acceptance/cluster/testconfig.proto index 035e56ae83a0..96926f6a3ba1 100644 --- a/pkg/acceptance/cluster/testconfig.proto +++ b/pkg/acceptance/cluster/testconfig.proto @@ -53,6 +53,4 @@ message TestConfig { // tests such as TestPut that will run indefinitely. optional int64 duration = 3 [(gogoproto.nullable) = false, (gogoproto.casttype) = "time.Duration"]; optional InitMode init_mode = 4 [(gogoproto.nullable) = false]; - - // TODO(bram): #4559 once defined, add in a collection of chaos agents here. } diff --git a/pkg/acceptance/localcluster/cluster.go b/pkg/acceptance/localcluster/cluster.go index ecdc0b2e50c8..dee2575530d5 100644 --- a/pkg/acceptance/localcluster/cluster.go +++ b/pkg/acceptance/localcluster/cluster.go @@ -118,7 +118,7 @@ func MakePerNodeFixedPortsCfg(numNodes int) map[int]NodeConfig { // Cluster holds the state for a local cluster, providing methods for common // operations, access to the underlying nodes and per-node KV and SQL clients. type Cluster struct { - cfg ClusterConfig + Cfg ClusterConfig seq *seqGen Nodes []*Node stopper *stop.Stopper @@ -137,7 +137,7 @@ func New(cfg ClusterConfig) *Cluster { cfg.Binary = filepath.Join(repoRoot(), "cockroach") } return &Cluster{ - cfg: cfg, + Cfg: cfg, seq: new(seqGen), stopper: stop.NewStopper(), } @@ -151,32 +151,32 @@ func New(cfg ClusterConfig) *Cluster { func (c *Cluster) Start(ctx context.Context) { c.started = timeutil.Now() - chs := make([]<-chan error, c.cfg.NumNodes) - for i := 0; i < c.cfg.NumNodes; i++ { - cfg := c.cfg.PerNodeCfg[i] // zero value is ok + chs := make([]<-chan error, c.Cfg.NumNodes) + for i := 0; i < c.Cfg.NumNodes; i++ { + cfg := c.Cfg.PerNodeCfg[i] // zero value is ok if cfg.Binary == "" { - cfg.Binary = c.cfg.Binary + cfg.Binary = c.Cfg.Binary } if cfg.DataDir == "" { - cfg.DataDir = filepath.Join(c.cfg.DataDir, fmt.Sprintf("%d", i+1)) + cfg.DataDir = filepath.Join(c.Cfg.DataDir, fmt.Sprintf("%d", i+1)) } - if cfg.LogDir == "" && c.cfg.LogDir != "" { - cfg.LogDir = filepath.Join(c.cfg.LogDir, fmt.Sprintf("%d", i+1)) + if cfg.LogDir == "" && c.Cfg.LogDir != "" { + cfg.LogDir = filepath.Join(c.Cfg.LogDir, fmt.Sprintf("%d", i+1)) } if cfg.Addr == "" { cfg.Addr = "127.0.0.1" } if cfg.DB == "" { - cfg.DB = c.cfg.DB + cfg.DB = c.Cfg.DB } if cfg.NumWorkers == 0 { - cfg.NumWorkers = c.cfg.NumWorkers + cfg.NumWorkers = c.Cfg.NumWorkers } - cfg.ExtraArgs = append(append([]string(nil), c.cfg.AllNodeArgs...), cfg.ExtraArgs...) + cfg.ExtraArgs = append(append([]string(nil), c.Cfg.AllNodeArgs...), cfg.ExtraArgs...) var node *Node node, chs[i] = c.makeNode(ctx, i, cfg) c.Nodes = append(c.Nodes, node) - if i == 0 && cfg.RPCPort == 0 { + if i == 0 && cfg.RPCPort == 0 && c.Cfg.NumNodes > 1 { // The first node must know its RPCPort or we can't possibly tell // the other nodes the correct one to go to. // @@ -193,14 +193,21 @@ func (c *Cluster) Start(ctx context.Context) { } } - for i := range chs { - if err := <-chs[i]; err != nil { - log.Fatalf(ctx, "node %d: %s", i+1, err) + if c.Cfg.NumNodes > 1 { + for i := range chs { + if err := <-chs[i]; err != nil { + log.Fatalf(ctx, "node %d: %s", i+1, err) + } } } log.Infof(context.Background(), "started %.3fs", timeutil.Since(c.started).Seconds()) - c.waitForFullReplication() + + if c.Cfg.NumNodes > 1 { + c.waitForFullReplication() + } else { + log.Infof(ctx, "not waiting for initial replication") + } } // Close stops the cluster, killing all of the nodes. @@ -209,8 +216,8 @@ func (c *Cluster) Close() { n.Kill() } c.stopper.Stop(context.Background()) - if c.cfg.Ephemeral { - _ = os.RemoveAll(c.cfg.DataDir) + if c.Cfg.Ephemeral { + _ = os.RemoveAll(c.Cfg.DataDir) } } @@ -255,11 +262,6 @@ func (c *Cluster) RPCPort(nodeIdx int) string { return c.Nodes[nodeIdx].RPCPort() } -// HTTPPort returns the HTTP port of the specified node. Returns zero if unknown. -func (c *Cluster) HTTPPort(nodeIdx int) string { - return c.Nodes[nodeIdx].HTTPPort() -} - func (c *Cluster) makeNode(ctx context.Context, nodeIdx int, cfg NodeConfig) (*Node, <-chan error) { baseCtx := &base.Config{ User: security.NodeUser, @@ -437,21 +439,38 @@ type Node struct { rpcCtx *rpc.Context seq *seqGen - startSeq int32 // updated atomically on start + startSeq int32 // updated atomically on start, nonzero while running + waitErr atomic.Value // last `error`` returned from cmd.Wait() syncutil.Mutex - cmd *exec.Cmd - rpcPort, httpPort, pgURL string - db *gosql.DB - client *client.DB - statusClient serverpb.StatusClient + notRunning chan struct{} + cmd *exec.Cmd + rpcPort, pgURL string // legacy: remove once 1.0.x is no longer tested + db *gosql.DB + client *client.DB + statusClient serverpb.StatusClient } // RPCPort returns the RPC + Postgres port. func (n *Node) RPCPort() string { - n.Lock() - defer n.Unlock() - return n.rpcPort + if s := func() string { + // Legacy case. To be removed. + n.Lock() + defer n.Unlock() + if n.rpcPort != "" && n.rpcPort != "0" { + return n.rpcPort + } + return "" + }(); s != "" { + return s + } + + advAddr := addrFromFile(n.advertiseAddrFile()) + if advAddr == "" { + return "" + } + _, p, _ := net.SplitHostPort(advAddr) + return p } // RPCAddr returns the RPC + Postgres address, or an empty string if it is not known @@ -464,11 +483,9 @@ func (n *Node) RPCAddr() string { return net.JoinHostPort(n.IPAddr(), port) } -// HTTPPort returns the ui port (may be empty until known). -func (n *Node) HTTPPort() string { - n.Lock() - defer n.Unlock() - return n.httpPort +// HTTPAddr returns the HTTP address (once known). +func (n *Node) HTTPAddr() string { + return addrFromFile(n.httpAddrFile()) } // PGUrl returns the postgres connection string (may be empty until known). @@ -538,20 +555,25 @@ func (n *Node) Start(ctx context.Context, joins ...string) { } } -func (n *Node) setNotRunningLocked() { +func (n *Node) setNotRunningLocked(waitErr *exec.ExitError) { _ = os.Remove(n.listeningURLFile()) _ = os.Remove(n.advertiseAddrFile()) + _ = os.Remove(n.httpAddrFile()) + if n.notRunning != nil { + close(n.notRunning) + } + n.notRunning = make(chan struct{}) n.db = nil n.client = nil n.statusClient = nil n.cmd = nil n.rpcPort = "" - n.httpPort = "" + n.waitErr.Store(waitErr) atomic.StoreInt32(&n.startSeq, 0) } func (n *Node) startAsyncInnerLocked(ctx context.Context, joins ...string) error { - n.setNotRunningLocked() + n.setNotRunningLocked(nil) args := append([]string(nil), n.Cfg.ExtraArgs[1:]...) for _, join := range joins { @@ -584,9 +606,6 @@ func (n *Node) startAsyncInnerLocked(ctx context.Context, joins ...string) error if n.Cfg.RPCPort > 0 { n.rpcPort = fmt.Sprintf("%d", n.Cfg.RPCPort) } - if n.Cfg.HTTPPort > 0 { - n.httpPort = fmt.Sprintf("%d", n.Cfg.HTTPPort) - } if err := n.cmd.Start(); err != nil { if err := stdout.Close(); err != nil { @@ -601,8 +620,9 @@ func (n *Node) startAsyncInnerLocked(ctx context.Context, joins ...string) error log.Infof(ctx, "process %d starting: %s", n.cmd.Process.Pid, n.cmd.Args) go func(cmd *exec.Cmd) { - if err := cmd.Wait(); err != nil { - log.Warning(ctx, err) + waitErr := cmd.Wait() + if waitErr != nil { + log.Warning(ctx, waitErr) } if err := stdout.Close(); err != nil { log.Warning(ctx, err) @@ -613,16 +633,18 @@ func (n *Node) startAsyncInnerLocked(ctx context.Context, joins ...string) error log.Infof(ctx, "process %d: %s", cmd.Process.Pid, cmd.ProcessState) + execErr, _ := waitErr.(*exec.ExitError) n.Lock() - n.setNotRunningLocked() + n.setNotRunningLocked(execErr) n.Unlock() }(n.cmd) return nil } -// StartAsync starts a node asynchronously. It returns a channel that receives either -// an error, or, once the node has started up and is fully functional, `nil`. +// StartAsync starts a node asynchronously. It returns a buffered channel that +// receives either an error, or, once the node has started up and is fully +// functional, `nil`. // // StartAsync is a no-op if the node is already running. func (n *Node) StartAsync(ctx context.Context, joins ...string) <-chan error { @@ -647,44 +669,7 @@ func (n *Node) StartAsync(ctx context.Context, joins ...string) <-chan error { ch <- nil }() - // This blocking loop in the sync path is counter-intuitive but is essential - // in allowing restarts of whole clusters. Roughly the following happens: - // - // 1. The whole cluster gets killed. - // 2. A node restarts. - // 3. It will *block* here until it has written down the file which contains - // enough information to link other nodes. - // 4. When restarting other nodes, and `.joins()` is passed in, these nodes - // can connect (at least) to the first node. - // 5. the cluster can become healthy after restart. - // - // If we didn't block here, we'd start all nodes up with join addresses that - // don't make any sense, and the cluster would likely not become connected. - // - // An additional difficulty is that older versions (pre 1.1) don't write - // this file. That's why we let *every* node do this (you could try to make - // only the first one wait, but if that one is 1.0, bad luck). - // Short-circuiting the wait in the case that the listening URL file is - // written (i.e. isServing closes) makes restarts work with 1.0 servers for - // the most part. - for { - if gossipAddr := n.AdvertiseAddr(); gossipAddr != "" { - _, port, err := net.SplitHostPort(gossipAddr) - if err != nil { - ch = make(chan error, 1) - ch <- errors.Wrapf(err, "can't parse gossip address %s", gossipAddr) - return ch - } - n.rpcPort = port - return ch - } - select { - case <-isServing: - return ch - default: - time.Sleep(10 * time.Millisecond) - } - } + return ch } func portFromURL(rawURL string) (string, *url.URL, error) { @@ -714,23 +699,35 @@ func (n *Node) advertiseAddrFile() string { return filepath.Join(n.Cfg.DataDir, "cockroach.advertise-addr") } -// AdvertiseAddr returns the Node's AdvertiseAddr or empty if none is available. -func (n *Node) AdvertiseAddr() (s string) { - c, err := ioutil.ReadFile(n.advertiseAddrFile()) +func (n *Node) httpAddrFile() string { + return filepath.Join(n.Cfg.DataDir, "cockroach.http-addr") +} + +func addrFromFile(f string) string { + c, err := ioutil.ReadFile(f) if err != nil { if !os.IsNotExist(err) { panic(err) } - // The below is part of the workaround for nodes at v1.0 which don't - // write the file above, explained in more detail in StartAsync(). - if port := n.RPCPort(); port != "" { - return net.JoinHostPort(n.IPAddr(), n.RPCPort()) - } return "" } return string(c) } +// AdvertiseAddr returns the Node's AdvertiseAddr or empty if none is available. +func (n *Node) AdvertiseAddr() (s string) { + addr := addrFromFile(n.advertiseAddrFile()) + if addr != "" { + return addr + } + // The below is part of the workaround for nodes at v1.0 which don't + // write the file above, explained in more detail in StartAsync(). + if port := n.RPCPort(); port != "" { + return net.JoinHostPort(n.IPAddr(), n.RPCPort()) + } + return addr +} + func (n *Node) waitUntilLive() { ctx := context.Background() opts := retry.Options{ @@ -739,6 +736,13 @@ func (n *Node) waitUntilLive() { Multiplier: 2, } for r := retry.Start(opts); r.Next(); { + n.Lock() + exited := n.cmd == nil + n.Unlock() + if exited { + return + } + urlBytes, err := ioutil.ReadFile(n.listeningURLFile()) if err != nil { continue @@ -788,9 +792,7 @@ func (n *Node) waitUntilLive() { break } - n.Lock() - n.httpPort, uiURL, err = portFromURL(uiStr) - n.Unlock() + _, uiURL, err = portFromURL(uiStr) if err != nil { log.Info(ctx, err) // TODO(tschottdorf): see above. @@ -802,14 +804,7 @@ func (n *Node) waitUntilLive() { // Kill stops a node abruptly by sending it SIGKILL. func (n *Node) Kill() { - func() { - n.Lock() - defer n.Unlock() - if n.cmd == nil || n.cmd.Process == nil { - return - } - _ = n.cmd.Process.Kill() - }() + n.Signal(os.Kill) // Wait for the process to have been cleaned up (or a call to Start() could // turn into an unintended no-op). for ok := false; !ok; { @@ -831,3 +826,31 @@ func (n *Node) DB() *gosql.DB { defer n.Unlock() return n.db } + +// Signal sends the given signal to the process. It is a no-op if the process is +// not running. +func (n *Node) Signal(s os.Signal) { + n.Lock() + defer n.Unlock() + if n.cmd == nil || n.cmd.Process == nil { + return + } + if err := n.cmd.Process.Signal(s); err != nil { + log.Warning(context.Background(), err) + } +} + +// Wait waits for the process to terminate and returns its process' Wait(). This +// is nil if the process terminated with a zero exit code. +func (n *Node) Wait() *exec.ExitError { + n.Lock() + ch := n.notRunning + n.Unlock() + if ch == nil { + log.Warning(context.Background(), "(*Node).Wait called when node was not running") + return nil + } + <-ch + ee, _ := n.waitErr.Load().(*exec.ExitError) + return ee +} diff --git a/pkg/acceptance/localcluster/localcluster.go b/pkg/acceptance/localcluster/localcluster.go index 5d88adfe6c15..88a22234fc31 100644 --- a/pkg/acceptance/localcluster/localcluster.go +++ b/pkg/acceptance/localcluster/localcluster.go @@ -19,6 +19,7 @@ import ( "net" "os/exec" "testing" + "time" "golang.org/x/net/context" @@ -73,7 +74,7 @@ func (b *LocalCluster) AssertAndStop(ctx context.Context, t testing.TB) { // ExecCLI implements cluster.Cluster. func (b *LocalCluster) ExecCLI(ctx context.Context, i int, cmd []string) (string, string, error) { - cmd = append([]string{b.cfg.Binary}, cmd...) + cmd = append([]string{b.Cfg.Binary}, cmd...) cmd = append(cmd, "--insecure", "--port", b.Port(ctx, i)) c := exec.CommandContext(ctx, cmd[0], cmd[1:]...) var o, e bytes.Buffer @@ -95,7 +96,37 @@ func (b *LocalCluster) Kill(ctx context.Context, i int) error { // once the node is successfully connected to the cluster and serving, nil. func (b *LocalCluster) RestartAsync(ctx context.Context, i int) <-chan error { b.Nodes[i].Kill() - return b.Nodes[i].StartAsync(ctx, b.joins()...) + joins := b.joins() + ch := b.Nodes[i].StartAsync(ctx, joins...) + if len(joins) == 0 && len(b.Nodes) > 1 { + // This blocking loop in is counter-intuitive but is essential in allowing + // restarts of whole clusters. Roughly the following happens: + // + // 1. The whole cluster gets killed. + // 2. A node restarts. + // 3. It will *block* here until it has written down the file which contains + // enough information to link other nodes. + // 4. When restarting other nodes, and `.joins()` is passed in, these nodes + // can connect (at least) to the first node. + // 5. the cluster can become healthy after restart. + // + // If we didn't block here, we'd start all nodes up with join addresses that + // don't make any sense, and the cluster would likely not become connected. + // + // An additional difficulty is that older versions (pre 1.1) don't write + // this file. That's why we let *every* node do this (you could try to make + // only the first one wait, but if that one is 1.0, bad luck). + // Short-circuiting the wait in the case that the listening URL file is + // written (i.e. isServing closes) makes restarts work with 1.0 servers for + // the most part. + for { + if gossipAddr := b.Nodes[i].AdvertiseAddr(); gossipAddr != "" { + return ch + } + time.Sleep(10 * time.Millisecond) + } + } + return ch } // Restart implements cluster.Cluster. @@ -105,12 +136,16 @@ func (b *LocalCluster) Restart(ctx context.Context, i int) error { // URL implements cluster.Cluster. func (b *LocalCluster) URL(ctx context.Context, i int) string { - return "http://" + b.Addr(ctx, i, b.HTTPPort(i)) + rest := b.Nodes[i].HTTPAddr() + if rest == "" { + return "" + } + return "http://" + rest } // Addr implements cluster.Cluster. func (b *LocalCluster) Addr(ctx context.Context, i int, port string) string { - return net.JoinHostPort(b.IPAddr(i), port) + return net.JoinHostPort(b.Nodes[i].AdvertiseAddr(), port) } // Hostname implements cluster.Cluster. diff --git a/pkg/acceptance/rapid_restart_test.go b/pkg/acceptance/rapid_restart_test.go new file mode 100644 index 000000000000..d8e0f140e70d --- /dev/null +++ b/pkg/acceptance/rapid_restart_test.go @@ -0,0 +1,158 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package acceptance + +import ( + "math/rand" + "net/http" + "os" + "os/exec" + "syscall" + "testing" + "time" + + "golang.org/x/net/context" + "golang.org/x/sync/errgroup" + + "github.com/pkg/errors" + + "github.com/cockroachdb/cockroach/pkg/acceptance/cluster" + "github.com/cockroachdb/cockroach/pkg/acceptance/localcluster" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" +) + +func TestRapidRestarts(t *testing.T) { + s := log.Scope(t) + defer s.Close(t) + + ctx := context.Background() + cfg := readConfigFromFlags() + RunLocal(t, func(t *testing.T) { + deadline := timeutil.Now().Add(cfg.Duration) + // In a loop, bootstrap a new node and immediately kill it. This is more + // effective at finding problems that restarting an existing node since + // there are more moving parts the first time around. Since there could be + // future issues that only occur on a restart, each invocation of the test + // also restart-kills the existing node once. + for timeutil.Now().Before(deadline) { + testRapidRestartSingle(ctx, t, cfg) + } + }) +} + +func unexpectedExitCode(exitErr *exec.ExitError) error { + if exitErr == nil { + // Server shut down cleanly. Note that returning `err` here would create + // an error interface wrapping a nil *ExitError, which is *not* nil + // itself. + return nil + } + // NB: the docs suggest[1] that there are platform-independent types, but an + // inspection shows that at the time of writing, everything seems to return a + // syscall.WaitStatus. + // + // [1]: https://golang.org/pkg/os/#ProcessState.Sys + s, ok := exitErr.Sys().(syscall.WaitStatus) + if !ok { + return errors.Wrapf(exitErr, "unsupported architecture: %T", exitErr.Sys()) + } + status := s.ExitStatus() + switch status { + case -1: + // Received SIGINT before setting up our own signal handlers. + case 1: + // Exit code from a SIGINT received by our signal handlers. + default: + return errors.Wrapf(exitErr, "unexpected exit status %d", status) + } + return nil +} + +func testRapidRestartSingle(ctx context.Context, t *testing.T, cfg cluster.TestConfig) { + // Make this a single-node cluster which unlocks optimizations in + // LocalCluster that skip all the waiting so that we get to kill the process + // early in its boot sequence. + cfg.Nodes = cfg.Nodes[:1] + + c := StartCluster(ctx, t, cfg) + defer c.AssertAndStop(ctx, t) + + lc := c.(*localcluster.LocalCluster) + + interrupt := func() { + t.Helper() + time.Sleep(time.Duration(rand.Int63n(int64(time.Second)))) + lc.Nodes[0].Signal(os.Interrupt) + } + + check := func() { + t.Helper() + if err := unexpectedExitCode(lc.Nodes[0].Wait()); err != nil { + lc.Cfg.Ephemeral = false // keep log dir + t.Fatalf("node did not terminate cleanly: %v", err) + } + } + + const count = 2 + // NB: the use of Group makes no sense with count=2, but this way you can + // bump it and the whole thing still works. + var g errgroup.Group + + getVars := func(ch <-chan error) func() error { + return func() error { + for { + base := c.URL(ctx, 0) + if base != "" { + // Torture the prometheus endpoint to prevent regression of #19559. + const varsEndpoint = "/_status/vars" + resp, err := cluster.HTTPClient.Get(base + varsEndpoint) + if err == nil { + if resp.StatusCode != http.StatusNotFound && resp.StatusCode != http.StatusOK { + return errors.Errorf("unexpected status code from %s: %d", varsEndpoint, resp.StatusCode) + } + } + } + select { + case err := <-ch: + return err + default: + time.Sleep(time.Millisecond) + } + } + } + } + closedCh := make(chan error) + close(closedCh) + + for i := 0; i < count; i++ { + g.Go(getVars(closedCh)) + + if i > 0 { + ch := lc.RestartAsync(ctx, 0) + g.Go(getVars(ch)) + } + + log.Info(ctx, "interrupting node") + interrupt() + + log.Info(ctx, "waiting for exit code") + check() + } + + if err := g.Wait(); err != nil { + t.Fatal(err) + } +} diff --git a/pkg/acceptance/status_server_test.go b/pkg/acceptance/status_server_test.go index 8bf88d4bb5bd..b8a6086cc45b 100644 --- a/pkg/acceptance/status_server_test.go +++ b/pkg/acceptance/status_server_test.go @@ -98,6 +98,8 @@ func checkNode( get(ctx, t, c.URL(ctx, i), fmt.Sprintf("/_status/logs/%s", urlID)) get(ctx, t, c.URL(ctx, i), fmt.Sprintf("/_status/stacks/%s", urlID)) } + + get(ctx, t, c.URL(ctx, i), "/_status/vars") } // TestStatusServer starts up an N node cluster and tests the status server on diff --git a/pkg/acceptance/util_cluster.go b/pkg/acceptance/util_cluster.go index 85c39d1be11f..7d990a349d62 100644 --- a/pkg/acceptance/util_cluster.go +++ b/pkg/acceptance/util_cluster.go @@ -187,7 +187,9 @@ func StartCluster(ctx context.Context, t *testing.T, cfg cluster.TestConfig) (c } } - if cfg.InitMode != cluster.INIT_NONE { + // Don't bother waiting for a single-node cluster. This is useful for + // TestRapidRestarts which wants to kill the process early. + if len(cfg.Nodes) > 1 && cfg.InitMode != cluster.INIT_NONE { wantedReplicas := 3 if numNodes := c.NumNodes(); numNodes < wantedReplicas { wantedReplicas = numNodes @@ -196,7 +198,6 @@ func StartCluster(ctx context.Context, t *testing.T, cfg cluster.TestConfig) (c // Looks silly, but we actually start zero-node clusters in the // reference tests. if wantedReplicas > 0 { - log.Infof(ctx, "waiting for first range to have %d replicas", wantedReplicas) testutils.SucceedsSoon(t, func() error { diff --git a/pkg/acceptance/version_upgrade_test.go b/pkg/acceptance/version_upgrade_test.go index 6d3fbd6e685b..d13b0417808f 100644 --- a/pkg/acceptance/version_upgrade_test.go +++ b/pkg/acceptance/version_upgrade_test.go @@ -66,6 +66,7 @@ func testVersionUpgrade(ctx context.Context, t *testing.T, cfg cluster.TestConfi // Verify that the nodes are *really* at the versions configured. This // tests the CI harness. + log.Info(ctx, "verify that configured versions are actual versions") for i := 0; i < c.NumNodes(); i++ { db, err := gosql.Open("postgres", c.PGUrl(ctx, i)) if err != nil { @@ -110,11 +111,11 @@ func testVersionUpgrade(ctx context.Context, t *testing.T, cfg cluster.TestConfi } lc := c.(*localcluster.LocalCluster) - // Upgrade the first node's binary to match the other nodes (i.e. the testing binary). + log.Info(ctx, "upgrading the first node's binary to match the other nodes (i.e. the testing binary)") lc.Nodes[0].Cfg.ExtraArgs[0] = lc.Nodes[1].Cfg.ExtraArgs[0] var chs []<-chan error - // Restart the nodes asynchronously. + log.Info(ctx, "restarting the nodes asynchronously") for i := 0; i < c.NumNodes(); i++ { chs = append(chs, lc.RestartAsync(ctx, i)) }