diff --git a/pkg/cmd/roachprod/main.go b/pkg/cmd/roachprod/main.go index 71579403ff16..f4a55c6c5c7b 100644 --- a/pkg/cmd/roachprod/main.go +++ b/pkg/cmd/roachprod/main.go @@ -567,6 +567,15 @@ environment variables to the cockroach process. install.EnvOption(nodeEnv), install.NumRacksOption(numRacks), } + + // Always pick a random available port when starting virtual + // clusters. We do not expose the functionality of choosing a + // specific port, so this is fine. + // + // TODO(renato): remove this once #111052 is addressed. + startOpts.SQLPort = 0 + startOpts.AdminUIPort = 0 + return roachprod.StartServiceForVirtualCluster(context.Background(), config.Logger, targetRoachprodCluster, storageCluster, startOpts, clusterSettingsOpts...) }), diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 234d79bf799a..d9e476ada743 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1394,17 +1394,22 @@ func (c *clusterImpl) assertNoDeadNode(ctx context.Context, t test.Test) error { return err } - deadNodes := 0 + deadProcesses := 0 for info := range eventsCh { t.L().Printf("%s", info) - if _, isDeath := info.Event.(install.MonitorNodeDead); isDeath { - deadNodes++ + if _, isDeath := info.Event.(install.MonitorProcessDead); isDeath { + deadProcesses++ } } - if deadNodes > 0 { - return errors.Newf("%d dead node(s) detected", deadNodes) + var plural string + if deadProcesses > 1 { + plural = "es" + } + + if deadProcesses > 0 { + return errors.Newf("%d dead cockroach process%s detected", deadProcesses, plural) } return nil } diff --git a/pkg/cmd/roachtest/monitor.go b/pkg/cmd/roachtest/monitor.go index b66d8d12ee6e..4981feff0583 100644 --- a/pkg/cmd/roachtest/monitor.go +++ b/pkg/cmd/roachtest/monitor.go @@ -168,7 +168,7 @@ func (m *monitorImpl) startNodeMonitor() { } for info := range eventsCh { - _, isDeath := info.Event.(install.MonitorNodeDead) + _, isDeath := info.Event.(install.MonitorProcessDead) isExpectedDeath := isDeath && atomic.AddInt32(&m.expDeaths, -1) >= 0 var expectedDeathStr string if isExpectedDeath { diff --git a/pkg/roachprod/install/BUILD.bazel b/pkg/roachprod/install/BUILD.bazel index 55d7891c036a..06875d223eb6 100644 --- a/pkg/roachprod/install/BUILD.bazel +++ b/pkg/roachprod/install/BUILD.bazel @@ -51,6 +51,7 @@ go_test( name = "install_test", srcs = [ "cluster_synced_test.go", + "cockroach_test.go", "services_test.go", "staging_test.go", "start_template_test.go", diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index dba35d653d60..6de144ebede5 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -22,6 +22,7 @@ import ( "os/exec" "os/signal" "path/filepath" + "strconv" "strings" "sync" "syscall" @@ -414,7 +415,16 @@ func (c *SyncedCluster) Stop( maxWait int, virtualClusterLabel string, ) error { - display := fmt.Sprintf("%s: stopping", c.Name) + var virtualClusterDisplay string + if virtualClusterLabel != "" { + virtualClusterName, sqlInstance, err := VirtualClusterInfoFromLabel(virtualClusterLabel) + if err != nil { + return err + } + + virtualClusterDisplay = fmt.Sprintf(" virtual cluster %q, instance %d", virtualClusterName, sqlInstance) + } + display := fmt.Sprintf("%s: stopping%s", c.Name, virtualClusterDisplay) if wait { display += " and waiting" } @@ -589,18 +599,26 @@ fi return statuses, nil } -// MonitorNodeSkipped represents a node whose status was not checked. -type MonitorNodeSkipped struct{} +// MonitorProcessSkipped represents a cockroach process whose status +// was not checked. +type MonitorProcessSkipped struct { + VirtualClusterName string + SQLInstance int +} -// MonitorNodeRunning represents the cockroach process running on a +// MonitorProcessRunning represents the cockroach process running on a // node. -type MonitorNodeRunning struct { - PID string +type MonitorProcessRunning struct { + VirtualClusterName string + SQLInstance int + PID string } -// MonitorNodeDead represents the cockroach process dying on a node. -type MonitorNodeDead struct { - ExitCode string +// MonitorProcessDead represents the cockroach process dying on a node. +type MonitorProcessDead struct { + VirtualClusterName string + SQLInstance int + ExitCode string } type MonitorError struct { @@ -612,24 +630,36 @@ type NodeMonitorInfo struct { // The index of the node (in a SyncedCluster) at which the message originated. Node Node // Event describes what happened to the node; it is one of - // MonitorNodeSkipped (no store directory was found); - // MonitorNodeRunning, sent when cockroach is running on a node; - // MonitorNodeDead, when the cockroach process stops running on a - // node; or MonitorError, typically indicate networking issues - // or nodes that have (physically) shut down. + // MonitorProcessSkipped (no store directory was found); + // MonitorProcessRunning, sent when cockroach is running on a node; + // MonitorProcessDead, when the cockroach process stops running on a + // node; or MonitorError, typically indicate networking issues or + // nodes that have (physically) shut down. Event interface{} } func (nmi NodeMonitorInfo) String() string { var status string + virtualClusterDesc := func(name string, instance int) string { + if name == SystemInterfaceName { + return "system interface" + } + + return fmt.Sprintf("virtual cluster %q, instance %d", name, instance) + } + switch event := nmi.Event.(type) { - case MonitorNodeRunning: - status = fmt.Sprintf("cockroach process is running (PID: %s)", event.PID) - case MonitorNodeSkipped: - status = "node skipped" - case MonitorNodeDead: - status = fmt.Sprintf("cockroach process died (exit code %s)", event.ExitCode) + case MonitorProcessRunning: + status = fmt.Sprintf("cockroach process for %s is running (PID: %s)", + virtualClusterDesc(event.VirtualClusterName, event.SQLInstance), event.PID, + ) + case MonitorProcessSkipped: + status = fmt.Sprintf("%s was skipped", virtualClusterDesc(event.VirtualClusterName, event.SQLInstance)) + case MonitorProcessDead: + status = fmt.Sprintf("cockroach process for %s died (exit code %s)", + virtualClusterDesc(event.VirtualClusterName, event.SQLInstance), event.ExitCode, + ) case MonitorError: status = fmt.Sprintf("error: %s", event.Err.Error()) } @@ -651,9 +681,14 @@ type MonitorOpts struct { // channel is subsequently closed; otherwise the process continues indefinitely // (emitting new information as the status of the cockroach process changes). // -// If IgnoreEmptyNodes is true, nodes on which no CockroachDB data is found -// (in {store-dir}) will not be probed and single event, MonitorNodeSkipped, -// will be emitted for them. +// If IgnoreEmptyNodes is true, tenants on which no CockroachDB data is found +// (in {store-dir}) will not be probed and single event, MonitorTenantSkipped, +// will be emitted for each tenant. +// +// Note that the monitor will only send events for tenants that exist +// at the time this function is called. In other words, this function +// will not emit events for tenants started *after* a call to +// Monitor(). func (c *SyncedCluster) Monitor( l *logger.Logger, ctx context.Context, opts MonitorOpts, ) chan NodeMonitorInfo { @@ -666,6 +701,13 @@ func (c *SyncedCluster) Monitor( // that is listened to by the caller. Bails if the context is // canceled. sendEvent := func(info NodeMonitorInfo) { + // if the monitor's context is already canceled, do not attempt to + // send the error down the channel, as it is most likely *caused* + // by the cancelation itself. + if monitorCtx.Err() != nil { + return + } + select { case ch <- info: // We were able to send the info through the channel. @@ -681,39 +723,81 @@ func (c *SyncedCluster) Monitor( deadMsg = "dead" ) + wg.Add(len(nodes)) for i := range nodes { - wg.Add(1) go func(i int) { defer wg.Done() - node := nodes[i] - port, err := c.NodePort(ctx, node) - if err != nil { - err := errors.Wrap(err, "failed to get node port") + + // We first find out all cockroach processes that are currently + // running in this node. + cockroachProcessesCmd := fmt.Sprintf(`ps axeww -o command | `+ + `grep -E '%s' | `+ // processes started by roachprod + `grep -E -o 'ROACHPROD_VIRTUAL_CLUSTER=[^ ]*' | `+ // ROACHPROD_VIRTUAL_CLUSTER indicates this is a cockroach process + `cut -d= -f2`, // grab the virtual cluster label + c.roachprodEnvRegex(node), + ) + + result, err := c.runCmdOnSingleNode( + ctx, l, node, cockroachProcessesCmd, defaultCmdOpts("list-processes"), + ) + if err := errors.CombineErrors(err, result.Err); err != nil { + err := errors.Wrap(err, "failed to list cockroach processes") + sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) + return + } + + type virtualClusterInfo struct { + Name string + Instance int + } + + var vcs []virtualClusterInfo + vcLines := strings.TrimSuffix(result.CombinedOut, "\n") + if vcLines == "" { + err := errors.New("no cockroach processes running") sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) return } - // On each monitored node, we loop looking for a cockroach process. + for _, label := range strings.Split(vcLines, "\n") { + name, instance, err := VirtualClusterInfoFromLabel(label) + if err != nil { + sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) + return + } + vcs = append(vcs, virtualClusterInfo{name, instance}) + } + data := struct { OneShot bool + Node Node IgnoreEmpty bool Store string - Port int Local bool Separator string SkippedMsg string RunningMsg string DeadMsg string + Processes []virtualClusterInfo }{ OneShot: opts.OneShot, + Node: node, IgnoreEmpty: opts.IgnoreEmptyNodes, Store: c.NodeDir(node, 1 /* storeIndex */), - Port: port, Local: c.IsLocal(), Separator: separator, SkippedMsg: skippedMsg, RunningMsg: runningMsg, DeadMsg: deadMsg, + Processes: vcs, + } + + storeFor := func(name string, instance int) string { + return c.InstanceStoreDir(node, name, instance) + } + + localPIDFile := func(name string, instance int) string { + return filepath.Join(c.LogDir(node, name, instance), "cockroach.pid") } // NB.: we parse the output of every line this script @@ -721,62 +805,85 @@ func (c *SyncedCluster) Monitor( // down below in order to produce structured results to the // caller. snippet := ` -{{ if .IgnoreEmpty }} -if ! ls {{.Store}}/marker.* 1> /dev/null 2>&1; then - echo "{{.SkippedMsg}}" - exit 0 -fi -{{- end}} -# Init with -1 so that when cockroach is initially dead, we print -# a dead event for it. -lastpid=-1 -while :; do -{{ if .Local }} - pid=$(lsof -i :{{.Port}} -sTCP:LISTEN | awk '!/COMMAND/ {print $2}') - pid=${pid:-0} # default to 0 - status="unknown" -{{- else }} - # When CRDB is not running, this is zero. - pid=$(systemctl show cockroach --property MainPID --value) - status=$(systemctl show cockroach --property ExecMainStatus --value) -{{- end }} - if [[ "${lastpid}" == -1 && "${pid}" != 0 ]]; then - # On the first iteration through the loop, if the process is running, - # don't register a PID change (which would trigger an erroneous dead - # event). - lastpid=0 +{{ range .Processes }} +monitor_process_{{$.Node}}_{{.Name}}_{{.Instance}}() { + {{ if $.IgnoreEmpty }} + if ! ls {{storeFor .Name .Instance}}/marker.* 1> /dev/null 2>&1; then + echo "{{.Name}}{{$.Separator}}{{.Instance}}{{$.Separator}}{{$.SkippedMsg}}" + return 0 fi - # Output a dead event whenever the PID changes from a nonzero value to - # any other value. In particular, we emit a dead event when the node stops - # (lastpid is nonzero, pid is zero), but not when the process then starts - # again (lastpid is zero, pid is nonzero). - if [ "${pid}" != "${lastpid}" ]; then - if [ "${lastpid}" != 0 ]; then - if [ "${pid}" != 0 ]; then - # If the PID changed but neither is zero, then the status refers to - # the new incarnation. We lost the actual exit status of the old PID. - status="unknown" + {{- end}} + # Init with -1 so that when cockroach is initially dead, we print + # a dead event for it. + lastpid=-1 + while :; do + {{ if $.Local }} + pidFile=$(cat "{{pidFile .Name .Instance}}") + # Make sure the process is still running + pid=$(ps axeww -o pid -o command | grep ROACHPROD= | grep ${pidFile} | awk '{ print $1 }') + pid=${pid:-0} # default to 0 + status="unknown" + {{- else }} + # When CRDB is not running, this is zero. + pid=$(systemctl show "{{virtualClusterLabel .Name .Instance}}" --property MainPID --value) + status=$(systemctl show "{{virtualClusterLabel .Name .Instance}}" --property ExecMainStatus --value) + {{- end }} + if [[ "${lastpid}" == -1 && "${pid}" != 0 ]]; then + # On the first iteration through the loop, if the process is running, + # don't register a PID change (which would trigger an erroneous dead + # event). + lastpid=0 + fi + # Output a dead event whenever the PID changes from a nonzero value to + # any other value. In particular, we emit a dead event when the node stops + # (lastpid is nonzero, pid is zero), but not when the process then starts + # again (lastpid is zero, pid is nonzero). + if [ "${pid}" != "${lastpid}" ]; then + if [ "${lastpid}" != 0 ]; then + if [ "${pid}" != 0 ]; then + # If the PID changed but neither is zero, then the status refers to + # the new incarnation. We lost the actual exit status of the old PID. + status="unknown" + fi + echo "{{.Name}}{{$.Separator}}{{.Instance}}{{$.Separator}}{{$.DeadMsg}}{{$.Separator}}${status}" + fi + if [ "${pid}" != 0 ]; then + echo "{{.Name}}{{$.Separator}}{{.Instance}}{{$.Separator}}{{$.RunningMsg}}{{$.Separator}}${pid}" fi - echo "{{.DeadMsg}}{{.Separator}}${status}" + lastpid=${pid} fi - if [ "${pid}" != 0 ]; then - echo "{{.RunningMsg}}{{.Separator}}${pid}" + {{ if $.OneShot }} + return 0 + {{- end }} + sleep 1 + if [ "${pid}" != 0 ]; then + while kill -0 "${pid}"; do + sleep 1 + done fi - lastpid=${pid} - fi -{{ if .OneShot }} - exit 0 -{{- end }} - sleep 1 - if [ "${pid}" != 0 ]; then - while kill -0 "${pid}"; do - sleep 1 - done - fi -done + done +} +{{ end }} + +# make sure all process monitors quit when this script exits. In +# OneShot mode, this is not needed as the script should end on its own. +{{ if not .OneShot }} +trap "kill 0" EXIT +{{ end }} + +# monitor every cockroach process in parallel. +{{ range .Processes }} +monitor_process_{{$.Node}}_{{.Name}}_{{.Instance}} & +{{ end }} + +wait ` - t := template.Must(template.New("script").Parse(snippet)) + t := template.Must(template.New("script").Funcs(template.FuncMap{ + "storeFor": storeFor, + "pidFile": localPIDFile, + "virtualClusterLabel": VirtualClusterLabel, + }).Parse(snippet)) var buf bytes.Buffer if err := t.Execute(&buf, data); err != nil { err := errors.Wrap(err, "failed to execute template") @@ -792,7 +899,6 @@ done if err != nil { err := errors.Wrap(err, "failed to read stdout pipe") sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - wg.Done() return } // Request a PTY so that the script will receive a SIGPIPE when the @@ -816,42 +922,66 @@ done if err != nil { err := errors.Wrap(err, "error reading from session") sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) + return } parts := strings.Split(string(line), separator) - switch parts[0] { + ensureNumParts := func(n int) { + if len(parts) < n { + panic(fmt.Errorf("invalid output from monitor: %q", line)) + } + } + // Every event is expected to have at least 3 parts. If + // that's not the case, panic explicitly below. Otherwise, + // we'd get a slice out of bounds error and the error + // message would not include the actual problematic line, + // which would make understanding the failure more + // difficult. + ensureNumParts(3) // name, instance, event + + // Virtual cluster name and instance are the first fields of + // every event type. + name, instanceStr := parts[0], parts[1] + instance, _ := strconv.Atoi(instanceStr) + switch parts[2] { case skippedMsg: - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorNodeSkipped{}}) + sendEvent(NodeMonitorInfo{Node: node, Event: MonitorProcessSkipped{ + VirtualClusterName: name, SQLInstance: instance, + }}) case runningMsg: - pid := parts[1] - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorNodeRunning{pid}}) + ensureNumParts(4) + pid := parts[3] + sendEvent(NodeMonitorInfo{Node: node, Event: MonitorProcessRunning{ + VirtualClusterName: name, SQLInstance: instance, PID: pid, + }}) case deadMsg: - exitCode := parts[1] - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorNodeDead{exitCode}}) + ensureNumParts(4) + exitCode := parts[3] + sendEvent(NodeMonitorInfo{Node: node, Event: MonitorProcessDead{ + VirtualClusterName: name, SQLInstance: instance, ExitCode: exitCode, + }}) default: - err := fmt.Errorf("internal error: unrecognized output from monitor: %s", line) + err := fmt.Errorf("internal error: unrecognized output from monitor: %q", line) sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) } } }(p) + defer readerWg.Wait() + if err := sess.Start(); err != nil { err := errors.Wrap(err, "failed to start session") sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) return } - // Watch for context cancellation, which can happen either if - // the test fails, or if the monitor loop exits. + // Watch for context cancellation, which can happen if the test + // fails, or if the monitor loop exits. go func() { <-monitorCtx.Done() sess.Close() }() - readerWg.Wait() - // We must call `sess.Wait()` only after finishing reading from the stdout - // pipe. Otherwise it can be closed under us, causing the reader to loop - // infinitely receiving a non-`io.EOF` error. if err := sess.Wait(); err != nil { // If we got an error waiting for the session but the context // is already canceled, do not send an error through the diff --git a/pkg/roachprod/install/cockroach.go b/pkg/roachprod/install/cockroach.go index cf5154e7cdcc..2f0810b9cda4 100644 --- a/pkg/roachprod/install/cockroach.go +++ b/pkg/roachprod/install/cockroach.go @@ -19,6 +19,7 @@ import ( "os/exec" "path/filepath" "runtime" + "strconv" "strings" "text/template" @@ -598,6 +599,44 @@ func VirtualClusterLabel(virtualClusterName string, sqlInstance int) string { return fmt.Sprintf("cockroach-%s_%d", virtualClusterName, sqlInstance) } +// VirtualClusterInfoFromLabel takes as parameter a tenant label +// produced with `VirtuaLClusterLabel()` and returns the corresponding +// tenant name and instance. +func VirtualClusterInfoFromLabel(virtualClusterLabel string) (string, int, error) { + var ( + sqlInstance int + sqlInstanceStr string + labelWithoutInstance string + err error + ) + + sep := "_" + parts := strings.Split(virtualClusterLabel, sep) + + // Note that this logic assumes that virtual cluster names cannot + // have a '_' character, which is currently (Sep 2023) the case. + switch len(parts) { + case 1: + // This should be a system tenant (no instance identifier) + labelWithoutInstance = parts[0] + + case 2: + // SQL instance process: instance number is after the '_' character. + labelWithoutInstance, sqlInstanceStr = parts[0], parts[1] + sqlInstance, err = strconv.Atoi(sqlInstanceStr) + if err != nil { + return "", 0, fmt.Errorf("invalid virtual cluster label: %s", virtualClusterLabel) + } + + default: + return "", 0, fmt.Errorf("invalid virtual cluster label: %s", virtualClusterLabel) + } + + // Remove the "cockroach-" prefix added by VirtualClusterLabel. + virtualClusterName := strings.TrimPrefix(labelWithoutInstance, "cockroach-") + return virtualClusterName, sqlInstance, nil +} + func execStartTemplate(data startTemplateData) (string, error) { tpl, err := template.New("start"). Funcs(template.FuncMap{"shesc": func(i interface{}) string { diff --git a/pkg/roachprod/install/cockroach_test.go b/pkg/roachprod/install/cockroach_test.go new file mode 100644 index 000000000000..69c1d1f5b4fa --- /dev/null +++ b/pkg/roachprod/install/cockroach_test.go @@ -0,0 +1,67 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package install + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestVirtualClusterLabel(t *testing.T) { + testCases := []struct { + name string + virtualClusterName string + sqlInstance int + expectedLabel string + }{ + { + name: "empty virtual cluster name", + virtualClusterName: "", + expectedLabel: "cockroach-system", + }, + { + name: "system interface name", + virtualClusterName: "system", + expectedLabel: "cockroach-system", + }, + { + name: "simple virtual cluster name", + virtualClusterName: "a", + sqlInstance: 1, + expectedLabel: "cockroach-a_1", + }, + { + name: "virtual cluster name with hyphens", + virtualClusterName: "virtual-cluster-a-1", + sqlInstance: 1, + expectedLabel: "cockroach-virtual-cluster-a-1_1", + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + label := VirtualClusterLabel(tc.virtualClusterName, tc.sqlInstance) + require.Equal(t, tc.expectedLabel, label) + + nameFromLabel, instanceFromLabel, err := VirtualClusterInfoFromLabel(label) + require.NoError(t, err) + + expectedVirtualClusterName := tc.virtualClusterName + if tc.virtualClusterName == "" { + expectedVirtualClusterName = "system" + } + require.Equal(t, expectedVirtualClusterName, nameFromLabel) + + require.Equal(t, tc.sqlInstance, instanceFromLabel) + }) + } +} diff --git a/pkg/roachprod/install/scripts/start.sh b/pkg/roachprod/install/scripts/start.sh index 4cd9c5bacefc..5f9682be695a 100755 --- a/pkg/roachprod/install/scripts/start.sh +++ b/pkg/roachprod/install/scripts/start.sh @@ -35,7 +35,11 @@ ENV_VARS=( # End of templated code. if [[ -n "${LOCAL}" ]]; then - ARGS+=("--background") + # Write to an empty pid file. This is referenced by the roachprod + # monitor to find the PID of cockroach processes running locally. + PID_FILE="${LOG_DIR}/cockroach.pid" + rm -f "${PID_FILE}" + ARGS+=("--background" "--pid-file" "${PID_FILE}") fi if [[ -n "${LOCAL}" || "${1-}" == "run" ]]; then @@ -61,20 +65,20 @@ fi # Set up systemd unit and start it, which will recursively # invoke this script but hit the above conditional. -if systemctl is-active -q cockroach; then - echo "cockroach service already active" - echo "To get more information: systemctl status cockroach" +if systemctl is-active -q "${VIRTUAL_CLUSTER_LABEL}"; then + echo "${VIRTUAL_CLUSTER_LABEL} service already active" + echo "To get more information: systemctl status ${VIRTUAL_CLUSTER_LABEL}" exit 1 fi # If cockroach failed, the service still exists; we need to clean it up before # we can start it again. -sudo systemctl reset-failed cockroach 2>/dev/null || true +sudo systemctl reset-failed "${VIRTUAL_CLUSTER_LABEL}" 2>/dev/null || true # The first time we run, install a small script that shows some helpful # information when we ssh in. if [ ! -e "${HOME}/.profile-cockroach" ]; then - cat > "${HOME}/.profile-${VIRTUAL_CLUSTER_LABEL}" <<'EOQ' + cat > "${HOME}/.profile-${VIRTUAL_CLUSTER_LABEL}" </dev/null || true +sudo systemctl reset-failed "${VIRTUAL_CLUSTER_LABEL}" 2>/dev/null || true # The first time we run, install a small script that shows some helpful # information when we ssh in. if [ ! -e "${HOME}/.profile-cockroach" ]; then - cat > "${HOME}/.profile-${VIRTUAL_CLUSTER_LABEL}" <<'EOQ' + cat > "${HOME}/.profile-${VIRTUAL_CLUSTER_LABEL}" <