diff --git a/pkg/cmd/roachprod/flags.go b/pkg/cmd/roachprod/flags.go index 54e42efa7adc..3e56c25d9fcb 100644 --- a/pkg/cmd/roachprod/flags.go +++ b/pkg/cmd/roachprod/flags.go @@ -43,18 +43,18 @@ var ( "COCKROACH_ENABLE_RPC_COMPRESSION=false", "COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED=true", } - tag string - external = false - certsDir string - adminurlOpen = false - adminurlPath = "" - adminurlIPs = false - useTreeDist = true - quiet = false - sig = 9 - waitFlag = false - createVMOpts = vm.DefaultCreateOpts() - startOpts = install.StartOpts{ + tag string + external = false + pgurlCertsDir string + adminurlOpen = false + adminurlPath = "" + adminurlIPs = false + useTreeDist = true + quiet = false + sig = 9 + waitFlag = false + createVMOpts = vm.DefaultCreateOpts() + startOpts = install.StartOpts{ Encrypt: false, Sequential: true, SkipInit: false, @@ -152,7 +152,7 @@ func initFlags() { pgurlCmd.Flags().BoolVar(&external, "external", false, "return pgurls for external connections") - pgurlCmd.Flags().StringVar(&certsDir, + pgurlCmd.Flags().StringVar(&pgurlCertsDir, "certs-dir", "./certs", "cert dir to use for secure connections") pprofCmd.Flags().DurationVar(&pprofOptions.duration, diff --git a/pkg/cmd/roachprod/main.go b/pkg/cmd/roachprod/main.go index a7e365f5ebcc..f5a33dfde7a7 100644 --- a/pkg/cmd/roachprod/main.go +++ b/pkg/cmd/roachprod/main.go @@ -85,7 +85,7 @@ func wrap(f func(cmd *cobra.Command, args []string) error) func(cmd *cobra.Comma func clusterOpts() install.ClusterSettings { return install.ClusterSettings{ Tag: tag, - CertsDir: certsDir, + PGUrlCertsDir: pgurlCertsDir, Secure: secure, Quiet: quiet || !term.IsTerminal(int(os.Stdout.Fd())), UseTreeDist: useTreeDist, diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index 27297af58924..86d2d45dd88a 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -45,21 +45,10 @@ import ( "golang.org/x/sync/errgroup" ) -// ClusterImpl TODO(peter): document -type ClusterImpl interface { - Start(c *SyncedCluster, opts StartOpts) error - CertsDir(c *SyncedCluster, index int) string - NodeDir(c *SyncedCluster, index, storeIndex int) string - LogDir(c *SyncedCluster, index int) string - NodeURL(c *SyncedCluster, host string, port int) string - NodePort(c *SyncedCluster, index int) int - NodeUIPort(c *SyncedCluster, index int) int -} - // ClusterSettings contains various knobs that affect operations on a cluster. type ClusterSettings struct { Secure bool - CertsDir string + PGUrlCertsDir string Env []string Tag string UseTreeDist bool @@ -73,11 +62,11 @@ type ClusterSettings struct { // DefaultClusterSettings returns the default settings. func DefaultClusterSettings() ClusterSettings { return ClusterSettings{ - Tag: "", - CertsDir: "./certs", - Secure: false, - Quiet: false, - UseTreeDist: true, + Tag: "", + PGUrlCertsDir: "./certs", + Secure: false, + Quiet: false, + UseTreeDist: true, Env: []string{ "COCKROACH_ENABLE_RPC_COMPRESSION=false", "COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED=true", @@ -102,8 +91,6 @@ type SyncedCluster struct { ClusterSettings - Impl ClusterImpl - Localities []string // AuthorizedKeys is used by SetupSSH to add additional authorized keys. @@ -120,7 +107,6 @@ func NewSyncedCluster( c := &SyncedCluster{ Cluster: *metadata, ClusterSettings: settings, - Impl: Cockroach{}, } c.Localities = make([]string, len(c.VMs)) for i := range c.VMs { @@ -166,14 +152,14 @@ func (c *SyncedCluster) localVMDir(nodeIdx int) string { return local.VMDir(c.Name, nodeIdx) } -// ServerNodes is the fully expanded, ordered list of nodes that any given +// TargetNodes is the fully expanded, ordered list of nodes that any given // roachprod command is intending to target. // // $ roachprod create local -n 4 // $ roachprod start local # [1, 2, 3, 4] // $ roachprod start local:2-4 # [2, 3, 4] // $ roachprod start local:2,1,4 # [1, 2, 4] -func (c *SyncedCluster) ServerNodes() []int { +func (c *SyncedCluster) TargetNodes() []int { return append([]int{}, c.Nodes...) } @@ -262,11 +248,6 @@ type StartOpts struct { ExtraArgs []string } -// Start TODO(peter): document -func (c *SyncedCluster) Start(startOpts StartOpts) error { - return c.Impl.Start(c, startOpts) -} - func (c *SyncedCluster) newSession(i int) (session, error) { if c.IsLocal() { return newLocalSession(), nil @@ -306,7 +287,7 @@ func (c *SyncedCluster) Stop(sig int, wait bool) error { done echo "${pid}: dead" >> %[1]s/roachprod.log done`, - c.Impl.LogDir(c, c.Nodes[i]), // [1] + c.LogDir(c.Nodes[i]), // [1] ) } @@ -323,7 +304,7 @@ if [ -n "${pids}" ]; then kill -%[3]d ${pids} %[4]s fi`, - c.Impl.LogDir(c, c.Nodes[i]), // [1] + c.LogDir(c.Nodes[i]), // [1] c.roachprodEnvRegex(c.Nodes[i]), // [2] sig, // [3] waitCmd, // [4] @@ -442,7 +423,7 @@ type NodeMonitorInfo struct { // be emitted for them. func (c *SyncedCluster) Monitor(ignoreEmptyNodes bool, oneShot bool) chan NodeMonitorInfo { ch := make(chan NodeMonitorInfo) - nodes := c.ServerNodes() + nodes := c.TargetNodes() var wg sync.WaitGroup for i := range nodes { @@ -474,8 +455,8 @@ func (c *SyncedCluster) Monitor(ignoreEmptyNodes bool, oneShot bool) chan NodeMo }{ OneShot: oneShot, IgnoreEmpty: ignoreEmptyNodes, - Store: Cockroach{}.NodeDir(c, nodes[i], 1 /* storeIndex */), - Port: Cockroach{}.NodePort(c, nodes[i]), + Store: c.NodeDir(nodes[i], 1 /* storeIndex */), + Port: c.NodePort(nodes[i]), Local: c.IsLocal(), } @@ -1377,7 +1358,7 @@ func (c *SyncedCluster) Logs( if c.IsLocal() { // This here is a bit of a hack to guess that the parent of the log dir is // the "home" for the local node and that the srcBase is relative to that. - localHome := filepath.Dir(c.Impl.LogDir(c, idx)) + localHome := filepath.Dir(c.LogDir(idx)) remote = filepath.Join(localHome, src) + "/" } else { logDir := src @@ -1706,7 +1687,7 @@ func (c *SyncedCluster) pgurls(nodes []int) (map[int]string, error) { } m := make(map[int]string, len(hosts)) for node, host := range hosts { - m[node] = c.Impl.NodeURL(c, host, c.Impl.NodePort(c, node)) + m[node] = c.NodeURL(host, c.NodePort(node)) } return m, nil } @@ -1942,19 +1923,16 @@ func (c *SyncedCluster) ParallelE( return nil, nil } -// Init initializes the cluster. It does it through node 1 (as per ServerNodes) +// Init initializes the cluster. It does it through node 1 (as per TargetNodes) // to maintain parity with auto-init behavior of `roachprod start` (when // --skip-init) is not specified. The implementation should be kept in -// sync with Cockroach.Start. +// sync with Start(). func (c *SyncedCluster) Init() error { - r := c.Impl.(Cockroach) - h := &crdbInstallHelper{c: c, r: r} - - // See (Cockroach).Start. We reserve a few special operations for the first - // node, so we strive to maintain the same here for interoperability. + // See Start(). We reserve a few special operations for the first node, so we + // strive to maintain the same here for interoperability. const firstNodeIdx = 0 - vers, err := getCockroachVersion(c, c.ServerNodes()[firstNodeIdx]) + vers, err := getCockroachVersion(c, c.TargetNodes()[firstNodeIdx]) if err != nil { return errors.WithDetail(err, "install.Init() failed: unable to retrieve cockroach version.") } @@ -1963,8 +1941,8 @@ func (c *SyncedCluster) Init() error { return errors.New("install.Init() failed: `roachprod init` only supported for v20.1 and beyond") } - fmt.Printf("%s: initializing cluster\n", h.c.Name) - initOut, err := h.initializeCluster(firstNodeIdx) + fmt.Printf("%s: initializing cluster\n", c.Name) + initOut, err := c.initializeCluster(firstNodeIdx) if err != nil { return errors.WithDetail(err, "install.Init() failed: unable to initialize cluster.") } @@ -1972,8 +1950,8 @@ func (c *SyncedCluster) Init() error { fmt.Println(initOut) } - fmt.Printf("%s: setting cluster settings\n", h.c.Name) - clusterSettingsOut, err := h.setClusterSettings(firstNodeIdx) + fmt.Printf("%s: setting cluster settings\n", c.Name) + clusterSettingsOut, err := c.setClusterSettings(firstNodeIdx) if err != nil { return errors.WithDetail(err, "install.Init() failed: unable to set cluster settings.") } diff --git a/pkg/roachprod/install/cockroach.go b/pkg/roachprod/install/cockroach.go index 8d653c571c06..c8f3ee9b2104 100644 --- a/pkg/roachprod/install/cockroach.go +++ b/pkg/roachprod/install/cockroach.go @@ -33,9 +33,6 @@ import ( //go:embed scripts/start.sh var startScript string -// Cockroach TODO(peter): document -type Cockroach struct{} - func cockroachNodeBinary(c *SyncedCluster, node int) string { if filepath.IsAbs(config.Binary) { return config.Binary @@ -123,20 +120,20 @@ func argExists(args []string, target string) int { return -1 } -// Start implements the ClusterImpl.NodeDir interface, and powers `roachprod -// start`. Starting the first node is special-cased quite a bit, it's used to -// distribute certs, set cluster settings, and initialize the cluster. Also, -// if we're only starting a single node in the cluster and it happens to be the -// "first" node (node 1, as understood by SyncedCluster.ServerNodes), we use +// Start the cockroach process on the cluster. +// +// Starting the first node is special-cased quite a bit, it's used to distribute +// certs, set cluster settings, and initialize the cluster. Also, if we're only +// starting a single node in the cluster and it happens to be the "first" node +// (node 1, as understood by SyncedCluster.TargetNodes), we use // `start-single-node` (this was written to provide a short hand to start a // single node cluster with a replication factor of one). -func (r Cockroach) Start(c *SyncedCluster, startOpts StartOpts) error { - h := &crdbInstallHelper{c: c, r: r} - if err := h.distributeCerts(); err != nil { +func (c *SyncedCluster) Start(startOpts StartOpts) error { + if err := c.distributeCerts(); err != nil { return err } - nodes := c.ServerNodes() + nodes := c.TargetNodes() var parallelism = 0 if startOpts.Sequential { parallelism = 1 @@ -151,7 +148,7 @@ func (r Cockroach) Start(c *SyncedCluster, startOpts StartOpts) error { // NB: if cockroach started successfully, we ignore the output as it is // some harmless start messaging. - if _, err := h.startNode(nodeIdx, startOpts, vers); err != nil { + if _, err := c.startNode(nodeIdx, startOpts, vers); err != nil { return nil, err } @@ -175,10 +172,10 @@ func (r Cockroach) Start(c *SyncedCluster, startOpts StartOpts) error { return nil, nil } - shouldInit := !h.useStartSingleNode(vers) && vers.AtLeast(version.MustParse("v20.1.0")) + shouldInit := !c.useStartSingleNode(vers) && vers.AtLeast(version.MustParse("v20.1.0")) if shouldInit { - fmt.Printf("%s: initializing cluster\n", h.c.Name) - initOut, err := h.initializeCluster(nodeIdx) + fmt.Printf("%s: initializing cluster\n", c.Name) + initOut, err := c.initializeCluster(nodeIdx) if err != nil { return nil, errors.WithDetail(err, "unable to initialize cluster") } @@ -198,8 +195,8 @@ func (r Cockroach) Start(c *SyncedCluster, startOpts StartOpts) error { // addressing #51897. // // TODO(irfansharif): Remove this once #51897 is resolved. - markBootstrap := fmt.Sprintf("touch %s/%s", h.c.Impl.NodeDir(h.c, nodes[nodeIdx], 1 /* storeIndex */), "cluster-bootstrapped") - cmdOut, err := h.run(nodeIdx, markBootstrap) + markBootstrap := fmt.Sprintf("touch %s/%s", c.NodeDir(nodes[nodeIdx], 1 /* storeIndex */), "cluster-bootstrapped") + cmdOut, err := c.run(nodeIdx, markBootstrap) if err != nil { return nil, errors.WithDetail(err, "unable to run cmd") } @@ -211,8 +208,8 @@ func (r Cockroach) Start(c *SyncedCluster, startOpts StartOpts) error { // We're sure to set cluster settings after having initialized the // cluster. - fmt.Printf("%s: setting cluster settings\n", h.c.Name) - clusterSettingsOut, err := h.setClusterSettings(nodeIdx) + fmt.Printf("%s: setting cluster settings\n", c.Name) + clusterSettingsOut, err := c.setClusterSettings(nodeIdx) if err != nil { return nil, errors.Wrap(err, "unable to set cluster settings") } @@ -223,44 +220,44 @@ func (r Cockroach) Start(c *SyncedCluster, startOpts StartOpts) error { }) } -// NodeDir implements the ClusterImpl.NodeDir interface. -func (Cockroach) NodeDir(c *SyncedCluster, index, storeIndex int) string { +// NodeDir returns the data directory for the given node and store. +func (c *SyncedCluster) NodeDir(nodeIndex, storeIndex int) string { if c.IsLocal() { if storeIndex != 1 { - panic("Cockroach.NodeDir only supports one store for local deployments") + panic("NodeDir only supports one store for local deployments") } - return filepath.Join(c.localVMDir(index), "data") + return filepath.Join(c.localVMDir(nodeIndex), "data") } return fmt.Sprintf("/mnt/data%d/cockroach", storeIndex) } -// LogDir implements the ClusterImpl.NodeDir interface. -func (Cockroach) LogDir(c *SyncedCluster, index int) string { +// LogDir returns the logs directory for the given node. +func (c *SyncedCluster) LogDir(nodeIndex int) string { if c.IsLocal() { - return filepath.Join(c.localVMDir(index), "logs") + return filepath.Join(c.localVMDir(nodeIndex), "logs") } return "logs" } -// CertsDir implements the ClusterImpl.NodeDir interface. -func (Cockroach) CertsDir(c *SyncedCluster, index int) string { +// CertsDir returns the certificate directory for the given node. +func (c *SyncedCluster) CertsDir(nodeIndex int) string { if c.IsLocal() { - return filepath.Join(c.localVMDir(index), "certs") + return filepath.Join(c.localVMDir(nodeIndex), "certs") } return "certs" } -// NodeURL implements the ClusterImpl.NodeDir interface. -func (Cockroach) NodeURL(c *SyncedCluster, host string, port int) string { +// NodeURL constructs a postgres URL. +func (c *SyncedCluster) NodeURL(host string, port int) string { var u url.URL u.User = url.User("root") u.Scheme = "postgres" u.Host = fmt.Sprintf("%s:%d", host, port) v := url.Values{} if c.Secure { - v.Add("sslcert", c.CertsDir+"/client.root.crt") - v.Add("sslkey", c.CertsDir+"/client.root.key") - v.Add("sslrootcert", c.CertsDir+"/ca.crt") + v.Add("sslcert", c.PGUrlCertsDir+"/client.root.crt") + v.Add("sslkey", c.PGUrlCertsDir+"/client.root.key") + v.Add("sslrootcert", c.PGUrlCertsDir+"/ca.crt") v.Add("sslmode", "verify-full") } else { v.Add("sslmode", "disable") @@ -269,25 +266,31 @@ func (Cockroach) NodeURL(c *SyncedCluster, host string, port int) string { return "'" + u.String() + "'" } -// NodePort implements the ClusterImpl.NodeDir interface. -func (Cockroach) NodePort(c *SyncedCluster, index int) int { - return c.VMs[index-1].SQLPort +// NodePort returns the SQL port for the given node. +func (c *SyncedCluster) NodePort(nodeIndex int) int { + return c.VMs[nodeIndex-1].SQLPort } -// NodeUIPort implements the ClusterImpl.NodeDir interface. -func (r Cockroach) NodeUIPort(c *SyncedCluster, index int) int { - return c.VMs[index-1].AdminUIPort +// NodeUIPort returns the AdminUI port for the given node. +func (c *SyncedCluster) NodeUIPort(nodeIndex int) int { + return c.VMs[nodeIndex-1].AdminUIPort } -// SQL implements the ClusterImpl.NodeDir interface. -func (r Cockroach) SQL(c *SyncedCluster, args []string) error { +// SQL runs `cockroach sql`, which starts a SQL shell or runs a SQL command. +// +// In interactive mode, there must be exactly one node target (as per +// TargetNodes). +// +// In non-interactive mode, a command specified via the `-e` flag is run against +// all nodes. +func (c *SyncedCluster) SQL(args []string) error { if len(args) == 0 || len(c.Nodes) == 1 { // If no arguments, we're going to get an interactive SQL shell. Require // exactly one target and ask SSH to provide a pseudoterminal. if len(args) == 0 && len(c.Nodes) != 1 { return fmt.Errorf("invalid number of nodes for interactive sql: %d", len(c.Nodes)) } - url := r.NodeURL(c, "localhost", r.NodePort(c, c.Nodes[0])) + url := c.NodeURL("localhost", c.NodePort(c.Nodes[0])) binary := cockroachNodeBinary(c, c.Nodes[0]) allArgs := []string{binary, "sql", "--url", url} allArgs = append(allArgs, ssh.Escape(args)) @@ -315,7 +318,7 @@ func (r Cockroach) SQL(c *SyncedCluster, args []string) error { cmd = fmt.Sprintf(`cd %s ; `, c.localVMDir(c.Nodes[nodeIdx])) } cmd += cockroachNodeBinary(c, c.Nodes[nodeIdx]) + " sql --url " + - r.NodeURL(c, "localhost", r.NodePort(c, c.Nodes[nodeIdx])) + " " + + c.NodeURL("localhost", c.NodePort(c.Nodes[nodeIdx])) + " " + ssh.Escape(args) out, err := sess.CombinedOutput(cmd) @@ -343,22 +346,17 @@ func (r Cockroach) SQL(c *SyncedCluster, args []string) error { return nil } -type crdbInstallHelper struct { - c *SyncedCluster - r Cockroach -} - -func (h *crdbInstallHelper) startNode( +func (c *SyncedCluster) startNode( nodeIdx int, startOpts StartOpts, vers *version.Version, ) (string, error) { - startCmd, err := h.generateStartCmd(nodeIdx, startOpts, vers) + startCmd, err := c.generateStartCmd(nodeIdx, startOpts, vers) if err != nil { return "", err } - nodes := h.c.ServerNodes() + nodes := c.TargetNodes() if err := func() error { - sess, err := h.c.newSession(nodes[nodeIdx]) + sess, err := c.newSession(nodes[nodeIdx]) if err != nil { return err } @@ -366,8 +364,8 @@ func (h *crdbInstallHelper) startNode( sess.SetStdin(strings.NewReader(startCmd)) var cmd string - if h.c.IsLocal() { - cmd = fmt.Sprintf(`cd %s ; `, h.c.localVMDir(nodes[nodeIdx])) + if c.IsLocal() { + cmd = fmt.Sprintf(`cd %s ; `, c.localVMDir(nodes[nodeIdx])) } cmd += `cat > cockroach.sh && chmod +x cockroach.sh` if out, err := sess.CombinedOutput(cmd); err != nil { @@ -379,15 +377,15 @@ func (h *crdbInstallHelper) startNode( return "", err } - sess, err := h.c.newSession(nodes[nodeIdx]) + sess, err := c.newSession(nodes[nodeIdx]) if err != nil { return "", err } defer sess.Close() var cmd string - if h.c.IsLocal() { - cmd = fmt.Sprintf(`cd %s ; `, h.c.localVMDir(nodes[nodeIdx])) + if c.IsLocal() { + cmd = fmt.Sprintf(`cd %s ; `, c.localVMDir(nodes[nodeIdx])) } cmd += "./cockroach.sh" out, err := sess.CombinedOutput(cmd) @@ -397,11 +395,11 @@ func (h *crdbInstallHelper) startNode( return strings.TrimSpace(string(out)), nil } -func (h *crdbInstallHelper) generateStartCmd( +func (c *SyncedCluster) generateStartCmd( nodeIdx int, startOpts StartOpts, vers *version.Version, ) (string, error) { - args, advertiseFirstIP, err := h.generateStartArgs(nodeIdx, startOpts, vers) + args, advertiseFirstIP, err := c.generateStartArgs(nodeIdx, startOpts, vers) if err != nil { return "", err } @@ -409,25 +407,25 @@ func (h *crdbInstallHelper) generateStartCmd( // For a one-node cluster, use `start-single-node` to disable replication. // For everything else we'll fall back to using `cockroach start`. var startCmd string - if h.useStartSingleNode(vers) { + if c.useStartSingleNode(vers) { startCmd = "start-single-node" } else { startCmd = "start" } - nodes := h.c.ServerNodes() + nodes := c.TargetNodes() return execStartTemplate(startTemplateData{ - LogDir: h.c.Impl.LogDir(h.c, nodes[nodeIdx]), - KeyCmd: h.generateKeyCmd(nodeIdx, startOpts), + LogDir: c.LogDir(nodes[nodeIdx]), + KeyCmd: c.generateKeyCmd(nodeIdx, startOpts), EnvVars: append(append([]string{ - fmt.Sprintf("ROACHPROD=%s", h.c.roachprodEnvValue(nodes[nodeIdx])), + fmt.Sprintf("ROACHPROD=%s", c.roachprodEnvValue(nodes[nodeIdx])), "GOTRACEBACK=crash", "COCKROACH_SKIP_ENABLING_DIAGNOSTIC_REPORTING=1", - }, h.c.Env...), h.getEnvVars()...), - Binary: cockroachNodeBinary(h.c, nodes[nodeIdx]), + }, c.Env...), getEnvVars()...), + Binary: cockroachNodeBinary(c, nodes[nodeIdx]), StartCmd: startCmd, Args: args, MemoryMax: config.MemoryMax, - Local: h.c.IsLocal(), + Local: c.IsLocal(), AdvertiseFirstIP: advertiseFirstIP, }) } @@ -461,14 +459,14 @@ func execStartTemplate(data startTemplateData) (string, error) { return buf.String(), nil } -func (h *crdbInstallHelper) generateStartArgs( +func (c *SyncedCluster) generateStartArgs( nodeIdx int, startOpts StartOpts, vers *version.Version, ) (_ []string, _advertiseFirstIP bool, _ error) { var args []string - nodes := h.c.ServerNodes() + nodes := c.TargetNodes() - if h.c.Secure { - args = append(args, `--certs-dir`, h.c.Impl.CertsDir(h.c, nodes[nodeIdx])) + if c.Secure { + args = append(args, `--certs-dir`, c.CertsDir(nodes[nodeIdx])) } else { args = append(args, "--insecure") } @@ -476,7 +474,7 @@ func (h *crdbInstallHelper) generateStartArgs( var storeDirs []string if idx := argExists(startOpts.ExtraArgs, "--store"); idx == -1 { for i := 1; i <= startOpts.StoreCount; i++ { - storeDir := h.c.Impl.NodeDir(h.c, nodes[nodeIdx], i) + storeDir := c.NodeDir(nodes[nodeIdx], i) storeDirs = append(storeDirs, storeDir) // Place a store{i} attribute on each store to allow for zone configs // that use specific stores. @@ -498,7 +496,7 @@ func (h *crdbInstallHelper) generateStartArgs( } } - logDir := h.c.Impl.LogDir(h.c, nodes[nodeIdx]) + logDir := c.LogDir(nodes[nodeIdx]) if vers.AtLeast(version.MustParse("v21.1.0-alpha.0")) { // Specify exit-on-error=false to work around #62763. args = append(args, "--log", `file-defaults: {dir: '`+logDir+`', exit-on-error: false}`) @@ -508,7 +506,7 @@ func (h *crdbInstallHelper) generateStartArgs( if vers.AtLeast(version.MustParse("v1.1.0")) { cache := 25 - if h.c.IsLocal() { + if c.IsLocal() { cache /= len(nodes) if cache == 0 { cache = 1 @@ -517,7 +515,7 @@ func (h *crdbInstallHelper) generateStartArgs( args = append(args, fmt.Sprintf("--cache=%d%%", cache)) args = append(args, fmt.Sprintf("--max-sql-memory=%d%%", cache)) } - if h.c.IsLocal() { + if c.IsLocal() { // This avoids annoying firewall prompts on Mac OS X. if vers.AtLeast(version.MustParse("v2.1.0")) { args = append(args, "--listen-addr=127.0.0.1") @@ -527,16 +525,16 @@ func (h *crdbInstallHelper) generateStartArgs( } args = append(args, - fmt.Sprintf("--port=%d", h.r.NodePort(h.c, nodes[nodeIdx])), - fmt.Sprintf("--http-port=%d", h.r.NodeUIPort(h.c, nodes[nodeIdx])), + fmt.Sprintf("--port=%d", c.NodePort(nodes[nodeIdx])), + fmt.Sprintf("--http-port=%d", c.NodeUIPort(nodes[nodeIdx])), ) - if locality := h.c.locality(nodes[nodeIdx]); locality != "" { + if locality := c.locality(nodes[nodeIdx]); locality != "" { if idx := argExists(startOpts.ExtraArgs, "--locality"); idx == -1 { args = append(args, "--locality="+locality) } } - if !h.useStartSingleNode(vers) { + if !c.useStartSingleNode(vers) { // --join flags are unsupported/unnecessary in `cockroach // start-single-node`. That aside, setting up --join flags is a bit // precise. We have every node point to node 1. For clusters running @@ -545,14 +543,14 @@ func (h *crdbInstallHelper) generateStartArgs( // node 1 also points to itself, and an explicit `cockroach init` is // needed. if nodes[nodeIdx] != 1 || vers.AtLeast(version.MustParse("v20.1.0")) { - args = append(args, fmt.Sprintf("--join=%s:%d", h.c.host(1), h.r.NodePort(h.c, 1))) + args = append(args, fmt.Sprintf("--join=%s:%d", c.host(1), c.NodePort(1))) } } var advertiseFirstIP bool - if h.shouldAdvertisePublicIP() { - args = append(args, fmt.Sprintf("--advertise-host=%s", h.c.host(nodeIdx+1))) - } else if !h.c.IsLocal() { + if c.shouldAdvertisePublicIP() { + args = append(args, fmt.Sprintf("--advertise-host=%s", c.host(nodeIdx+1))) + } else if !c.IsLocal() { // Explicitly advertise by IP address so that we don't need to // deal with cross-region name resolution. The `hostname -I` // prints all IP addresses for the host and then we'll select @@ -566,7 +564,7 @@ func (h *crdbInstallHelper) generateStartArgs( node: nodes[nodeIdx], } for _, arg := range startOpts.ExtraArgs { - expandedArg, err := e.expand(h.c, arg) + expandedArg, err := e.expand(c, arg) if err != nil { return nil, false, err } @@ -576,11 +574,11 @@ func (h *crdbInstallHelper) generateStartArgs( return args, advertiseFirstIP, nil } -func (h *crdbInstallHelper) initializeCluster(nodeIdx int) (string, error) { - nodes := h.c.ServerNodes() - initCmd := h.generateInitCmd(nodeIdx) +func (c *SyncedCluster) initializeCluster(nodeIdx int) (string, error) { + nodes := c.TargetNodes() + initCmd := c.generateInitCmd(nodeIdx) - sess, err := h.c.newSession(nodes[nodeIdx]) + sess, err := c.newSession(nodes[nodeIdx]) if err != nil { return "", err } @@ -593,11 +591,11 @@ func (h *crdbInstallHelper) initializeCluster(nodeIdx int) (string, error) { return strings.TrimSpace(string(out)), nil } -func (h *crdbInstallHelper) setClusterSettings(nodeIdx int) (string, error) { - nodes := h.c.ServerNodes() - clusterSettingCmd := h.generateClusterSettingCmd(nodeIdx) +func (c *SyncedCluster) setClusterSettings(nodeIdx int) (string, error) { + nodes := c.TargetNodes() + clusterSettingCmd := c.generateClusterSettingCmd(nodeIdx) - sess, err := h.c.newSession(nodes[nodeIdx]) + sess, err := c.newSession(nodes[nodeIdx]) if err != nil { return "", err } @@ -610,22 +608,22 @@ func (h *crdbInstallHelper) setClusterSettings(nodeIdx int) (string, error) { return strings.TrimSpace(string(out)), nil } -func (h *crdbInstallHelper) generateClusterSettingCmd(nodeIdx int) string { - nodes := h.c.ServerNodes() +func (c *SyncedCluster) generateClusterSettingCmd(nodeIdx int) string { + nodes := c.TargetNodes() license := envutil.EnvOrDefaultString("COCKROACH_DEV_LICENSE", "") if license == "" { fmt.Printf("%s: COCKROACH_DEV_LICENSE unset: enterprise features will be unavailable\n", - h.c.Name) + c.Name) } var clusterSettingCmd string - if h.c.IsLocal() { - clusterSettingCmd = fmt.Sprintf(`cd %s ; `, h.c.localVMDir(1)) + if c.IsLocal() { + clusterSettingCmd = fmt.Sprintf(`cd %s ; `, c.localVMDir(1)) } - binary := cockroachNodeBinary(h.c, nodes[nodeIdx]) - path := fmt.Sprintf("%s/%s", h.c.Impl.NodeDir(h.c, nodes[nodeIdx], 1 /* storeIndex */), "settings-initialized") - url := h.r.NodeURL(h.c, "localhost", h.r.NodePort(h.c, 1)) + binary := cockroachNodeBinary(c, nodes[nodeIdx]) + path := fmt.Sprintf("%s/%s", c.NodeDir(nodes[nodeIdx], 1 /* storeIndex */), "settings-initialized") + url := c.NodeURL("localhost", c.NodePort(1)) // We ignore failures to set remote_debugging.mode, which was // removed in v21.2. @@ -640,17 +638,17 @@ func (h *crdbInstallHelper) generateClusterSettingCmd(nodeIdx int) string { return clusterSettingCmd } -func (h *crdbInstallHelper) generateInitCmd(nodeIdx int) string { - nodes := h.c.ServerNodes() +func (c *SyncedCluster) generateInitCmd(nodeIdx int) string { + nodes := c.TargetNodes() var initCmd string - if h.c.IsLocal() { - initCmd = fmt.Sprintf(`cd %s ; `, h.c.localVMDir(1)) + if c.IsLocal() { + initCmd = fmt.Sprintf(`cd %s ; `, c.localVMDir(1)) } - path := fmt.Sprintf("%s/%s", h.c.Impl.NodeDir(h.c, nodes[nodeIdx], 1 /* storeIndex */), "cluster-bootstrapped") - url := h.r.NodeURL(h.c, "localhost", h.r.NodePort(h.c, nodes[nodeIdx])) - binary := cockroachNodeBinary(h.c, nodes[nodeIdx]) + path := fmt.Sprintf("%s/%s", c.NodeDir(nodes[nodeIdx], 1 /* storeIndex */), "cluster-bootstrapped") + url := c.NodeURL("localhost", c.NodePort(nodes[nodeIdx])) + binary := cockroachNodeBinary(c, nodes[nodeIdx]) initCmd += fmt.Sprintf(` if ! test -e %[1]s ; then COCKROACH_CONNECT_TIMEOUT=0 %[2]s init --url %[3]s && touch %[1]s @@ -658,16 +656,16 @@ func (h *crdbInstallHelper) generateInitCmd(nodeIdx int) string { return initCmd } -func (h *crdbInstallHelper) generateKeyCmd(nodeIdx int, startOpts StartOpts) string { +func (c *SyncedCluster) generateKeyCmd(nodeIdx int, startOpts StartOpts) string { if !startOpts.Encrypt { return "" } - nodes := h.c.ServerNodes() + nodes := c.TargetNodes() var storeDirs []string if idx := argExists(startOpts.ExtraArgs, "--store"); idx == -1 { for i := 1; i <= startOpts.StoreCount; i++ { - storeDir := h.c.Impl.NodeDir(h.c, nodes[nodeIdx], i) + storeDir := c.NodeDir(nodes[nodeIdx], i) storeDirs = append(storeDirs, storeDir) } } else { @@ -687,37 +685,36 @@ func (h *crdbInstallHelper) generateKeyCmd(nodeIdx int, startOpts StartOpts) str return keyCmd.String() } -func (h *crdbInstallHelper) useStartSingleNode(vers *version.Version) bool { - return len(h.c.VMs) == 1 && vers.AtLeast(version.MustParse("v19.2.0")) +func (c *SyncedCluster) useStartSingleNode(vers *version.Version) bool { + return len(c.VMs) == 1 && vers.AtLeast(version.MustParse("v19.2.0")) } -// distributeCerts, like the name suggests, distributes certs if it's a secure -// cluster and we're starting n1. -func (h *crdbInstallHelper) distributeCerts() error { - for _, node := range h.c.ServerNodes() { - if node == 1 && h.c.Secure { - if err := h.c.DistributeCerts(); err != nil { - return err - } - break +// distributeCerts distributes certs if it's a secure cluster and we're +// starting n1. +func (c *SyncedCluster) distributeCerts() error { + for _, node := range c.TargetNodes() { + if node == 1 && c.Secure { + return c.DistributeCerts() } } return nil } -func (h *crdbInstallHelper) shouldAdvertisePublicIP() bool { +func (c *SyncedCluster) shouldAdvertisePublicIP() bool { // If we're creating nodes that span VPC (e.g. AWS multi-region or // multi-cloud), we'll tell the nodes to advertise their public IPs // so that attaching nodes to the cluster Just Works. - for i := range h.c.VMs { - if i > 0 && h.c.VMs[i].VPC != h.c.VMs[0].VPC { + for i := range c.VMs { + if i > 0 && c.VMs[i].VPC != c.VMs[0].VPC { return true } } return false } -func (h *crdbInstallHelper) getEnvVars() []string { +// getEnvVars returns all COCKROACH_* environment variables, in the form +// "key=value". +func getEnvVars() []string { var sl []string for _, v := range os.Environ() { if strings.HasPrefix(v, "COCKROACH_") { @@ -727,10 +724,10 @@ func (h *crdbInstallHelper) getEnvVars() []string { return sl } -func (h *crdbInstallHelper) run(nodeIdx int, cmd string) (string, error) { - nodes := h.c.ServerNodes() +func (c *SyncedCluster) run(nodeIdx int, cmd string) (string, error) { + nodes := c.TargetNodes() - sess, err := h.c.newSession(nodes[nodeIdx]) + sess, err := c.newSession(nodes[nodeIdx]) if err != nil { return "", err } diff --git a/pkg/roachprod/install/expander.go b/pkg/roachprod/install/expander.go index f2ce1c4ca5ca..3962b6e7a77a 100644 --- a/pkg/roachprod/install/expander.go +++ b/pkg/roachprod/install/expander.go @@ -153,7 +153,7 @@ func (e *expander) maybeExpandPgPort(c *SyncedCluster, s string) (string, bool, if e.pgPorts == nil { e.pgPorts = make(map[int]string, len(c.VMs)) for _, i := range allNodes(len(c.VMs)) { - e.pgPorts[i] = fmt.Sprint(c.Impl.NodePort(c, i)) + e.pgPorts[i] = fmt.Sprint(c.NodePort(i)) } } @@ -171,7 +171,7 @@ func (e *expander) maybeExpandUIPort(c *SyncedCluster, s string) (string, bool, if e.uiPorts == nil { e.uiPorts = make(map[int]string, len(c.VMs)) for _, i := range allNodes(len(c.VMs)) { - e.uiPorts[i] = fmt.Sprint(c.Impl.NodeUIPort(c, i)) + e.uiPorts[i] = fmt.Sprint(c.NodeUIPort(i)) } } @@ -184,7 +184,7 @@ func (e *expander) maybeExpandStoreDir(c *SyncedCluster, s string) (string, bool if !storeDirRe.MatchString(s) { return s, false, nil } - return c.Impl.NodeDir(c, e.node, 1 /* storeIndex */), true, nil + return c.NodeDir(e.node, 1 /* storeIndex */), true, nil } // maybeExpandLogDir is an expanderFunc for "{log-dir}" @@ -192,7 +192,7 @@ func (e *expander) maybeExpandLogDir(c *SyncedCluster, s string) (string, bool, if !logDirRe.MatchString(s) { return s, false, nil } - return c.Impl.LogDir(c, e.node), true, nil + return c.LogDir(e.node), true, nil } // maybeExpandCertsDir is an expanderFunc for "{certs-dir}" @@ -200,5 +200,5 @@ func (e *expander) maybeExpandCertsDir(c *SyncedCluster, s string) (string, bool if !certsDirRe.MatchString(s) { return s, false, nil } - return c.Impl.CertsDir(c, e.node), true, nil + return c.CertsDir(e.node), true, nil } diff --git a/pkg/roachprod/roachprod.go b/pkg/roachprod/roachprod.go index f90d8775db4d..ef6dc04850df 100644 --- a/pkg/roachprod/roachprod.go +++ b/pkg/roachprod/roachprod.go @@ -366,11 +366,7 @@ func SQL(name string, clusterOpts install.ClusterSettings, cmdArray []string) er if err != nil { return err } - cockroach, ok := c.Impl.(install.Cockroach) - if !ok { - return errors.New("sql is only valid on cockroach clusters") - } - return cockroach.SQL(c, cmdArray) + return c.SQL(cmdArray) } // IP gets the ip addresses of the nodes in a cluster. @@ -380,7 +376,7 @@ func IP(name string, clusterOpts install.ClusterSettings, external bool) ([]stri return nil, err } - nodes := c.ServerNodes() + nodes := c.TargetNodes() ips := make([]string, len(nodes)) if external { @@ -700,7 +696,7 @@ func PgURL(name string, clusterOpts install.ClusterSettings, external bool) erro if err != nil { return err } - nodes := c.ServerNodes() + nodes := c.TargetNodes() ips := make([]string, len(nodes)) if external { @@ -722,7 +718,7 @@ func PgURL(name string, clusterOpts install.ClusterSettings, external bool) erro if ip == "" { return errors.Errorf("empty ip: %v", ips) } - urls = append(urls, c.Impl.NodeURL(c, ip, c.Impl.NodePort(c, nodes[i]))) + urls = append(urls, c.NodeURL(ip, c.NodePort(nodes[i]))) } fmt.Println(strings.Join(urls, " ")) if len(urls) != len(nodes) { @@ -743,7 +739,7 @@ func AdminURL( return err } - for i, node := range c.ServerNodes() { + for i, node := range c.TargetNodes() { host := vm.Name(c.Name, node) + "." + gce.Subdomain // verify DNS is working / fallback to IPs if not. @@ -757,7 +753,7 @@ func AdminURL( if adminurlIPs { host = c.VMs[node-1].PublicIP } - port := c.Impl.NodeUIPort(c, node) + port := c.NodeUIPort(node) scheme := "http" if c.Secure { scheme = "https" @@ -812,9 +808,9 @@ func Pprof( httpClient := httputil.NewClientWithTimeout(timeout) startTime := timeutil.Now().Unix() - failed, err := c.ParallelE(description, len(c.ServerNodes()), 0, func(i int) ([]byte, error) { + failed, err := c.ParallelE(description, len(c.TargetNodes()), 0, func(i int) ([]byte, error) { host := c.VMs[i].PublicIP - port := c.Impl.NodeUIPort(c, i) + port := c.NodeUIPort(i) scheme := "http" if c.Secure { scheme = "https"