Skip to content

Commit

Permalink
roachprod: default to secure clusters
Browse files Browse the repository at this point in the history
Before: roachprod defaulted to making insecure clusters and the useneeded
to use the --secure flag to create a secure cluster.

Why change? We tell our users that the best practice is to run in secure mode
but we don't dog food that configuration enough. Setting roachprod to secure
by default will require us to debug problems in secure mode, which means we'll
make it easer to run secure mode and we'll get better knowledge of how to run
in secure mode.

Now: clusters will be spun up in secure mode by default. To get an insecure
cluster, use the --insecure flag.

Fixes cockroachdb#38539.

Release note: None
  • Loading branch information
jlinder authored and rail committed Apr 15, 2021
1 parent 29b6209 commit 3195270
Show file tree
Hide file tree
Showing 31 changed files with 142 additions and 81 deletions.
3 changes: 3 additions & 0 deletions pkg/cmd/roachprod/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,9 @@ roachprod start ${CLUSTER}

# Check the admin UI.
roachprod admin --open ${CLUSTER}:1
# For secure clusters, create a user and grant admin privileges
roachprod sql ${CLUSTER}:1 -- -e "CREATE USER craig WITH PASSWORD 'cockroach';"
roachprod sql ${CLUSTER}:1 -- -e "GRANT ADMIN TO craig;"

# Run a workload.
roachprod run ${CLUSTER}:4 -- ./workload init kv
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachprod/install/cluster_synced.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ type SyncedCluster struct {
VPCs []string
// all other fields are populated in newCluster.
Nodes []int
Secure bool
Insecure bool
Env string
Args []string
Tag string
Expand Down
14 changes: 7 additions & 7 deletions pkg/cmd/roachprod/install/cockroach.go
Original file line number Diff line number Diff line change
Expand Up @@ -260,11 +260,11 @@ func (Cockroach) CertsDir(c *SyncedCluster, index int) string {
// NodeURL implements the ClusterImpl.NodeDir interface.
func (Cockroach) NodeURL(c *SyncedCluster, host string, port int) string {
url := fmt.Sprintf("'postgres://root@%s:%d", host, port)
if c.Secure {
if c.Insecure {
url += "?sslmode=disable"
} else {
url += "?sslcert=certs%2Fclient.root.crt&sslkey=certs%2Fclient.root.key&" +
"sslrootcert=certs%2Fca.crt&sslmode=verify-full"
} else {
url += "?sslmode=disable"
}
url += "'"
return url
Expand Down Expand Up @@ -428,10 +428,10 @@ func (h *crdbInstallHelper) generateStartArgs(
nodes := h.c.ServerNodes()

args = append(args, "--background")
if h.c.Secure {
args = append(args, "--certs-dir="+h.c.Impl.CertsDir(h.c, nodes[nodeIdx]))
} else {
if h.c.Insecure {
args = append(args, "--insecure")
} else {
args = append(args, "--certs-dir="+h.c.Impl.CertsDir(h.c, nodes[nodeIdx]))
}

var storeDirs []string
Expand Down Expand Up @@ -646,7 +646,7 @@ func (h *crdbInstallHelper) useStartSingleNode(vers *version.Version) bool {
// cluster and we're starting n1.
func (h *crdbInstallHelper) distributeCerts() {
for _, node := range h.c.ServerNodes() {
if node == 1 && h.c.Secure {
if node == 1 && !h.c.Insecure {
h.c.DistributeCerts()
break
}
Expand Down
25 changes: 13 additions & 12 deletions pkg/cmd/roachprod/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ var (
listJSON bool
listMine bool
clusterType = "cockroach"
secure = false
insecure = false
nodeEnv = []string{
"COCKROACH_ENABLE_RPC_COMPRESSION=false",
"COCKROACH_UI_RELEASE_NOTES_SIGNUP_DISMISSED=true",
Expand Down Expand Up @@ -186,7 +186,7 @@ Available clusters:
}
}
c.Nodes = nodes
c.Secure = secure
c.Insecure = insecure
c.Env = strings.Join(nodeEnv, " ")
c.Args = nodeArgs
if tag != "" {
Expand Down Expand Up @@ -984,10 +984,11 @@ var startCmd = &cobra.Command{
Short: "start nodes on a cluster",
Long: `Start nodes on a cluster.
The --secure flag can be used to start nodes in secure mode (i.e. using
certs). When specified, there is a one time initialization for the cluster to
create and distribute the certs. Note that running some modes in secure mode
and others in insecure mode is not a supported Cockroach configuration.
The --insecure flag can be used to start nodes in insecure mode (i.e. without
using certs). When it is specified, the one time initialization for the
cluster to create and distribute the certs is skipped. Note that running some
nodes in insecure mode and others in secure mode is not a supported Cockroach
configuration.
As a debugging aid, the --sequential flag starts the nodes sequentially so node
IDs match hostnames. Otherwise nodes are started are parallel.
Expand Down Expand Up @@ -1601,7 +1602,7 @@ Examples:
host := c.VMs[i]
port := install.GetAdminUIPort(c.Impl.NodePort(c, i))
scheme := "http"
if c.Secure {
if !c.Insecure {
scheme = "https"
}
outputFile := fmt.Sprintf("pprof-%s-%d-%s-%04d.out", profType, startTime, c.Name, i+1)
Expand Down Expand Up @@ -1716,9 +1717,9 @@ var adminurlCmd = &cobra.Command{
host = c.VMs[node-1]
}
port := install.GetAdminUIPort(c.Impl.NodePort(c, node))
scheme := "http"
if c.Secure {
scheme = "https"
scheme := "https"
if c.Insecure {
scheme = "http"
}
if !strings.HasPrefix(adminurlPath, "/") {
adminurlPath = "/" + adminurlPath
Expand Down Expand Up @@ -1921,7 +1922,7 @@ func main() {
&external, "external", false, "return external IP addresses")

runCmd.Flags().BoolVar(
&secure, "secure", false, "use a secure cluster")
&insecure, "insecure", false, "use an insecure cluster")

startCmd.Flags().IntVarP(&numRacks,
"racks", "r", 0, "the number of racks to partition the nodes into")
Expand Down Expand Up @@ -2008,7 +2009,7 @@ func main() {
fallthrough
case pgurlCmd, adminurlCmd:
cmd.Flags().BoolVar(
&secure, "secure", false, "use a secure cluster")
&insecure, "insecure", false, "use an insecure cluster")
}

if cmd.Long == "" {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -366,7 +366,7 @@ FROM crdb_internal.kv_store_status

decom := func(id int) {
c.Run(ctx, c.Node(1),
fmt.Sprintf("./cockroach node decommission --insecure --wait=none %d", id))
fmt.Sprintf("./cockroach node decommission %s --wait=none %d", c.secureFlags(), id))
}

// Decommission a node. The ranges should down-replicate to 7 replicas.
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/autoupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,8 @@ func registerAutoUpgrade(r *testRegistry) {
t.WorkerStatus("decommission")
port := fmt.Sprintf("{pgport:%d}", node)
if err := c.RunE(ctx, c.Node(node),
fmt.Sprintf("./cockroach node decommission %d --insecure --port=%s", node, port)); err != nil {
fmt.Sprintf("./cockroach node decommission %d %s --port=%s", node, c.secureFlags(),
port)); err != nil {
return err
}
t.WorkerStatus("stop")
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,8 +152,8 @@ func registerBackup(r *testRegistry) {
m := newMonitor(ctx, c)
m.Go(func(ctx context.Context) error {
t.Status(`running backup`)
c.Run(ctx, c.Node(1), `./cockroach sql --insecure -e "
BACKUP bank.bank TO 'gs://cockroachdb-backup-testing/`+dest+`'"`)
c.Run(ctx, c.Node(1),
fmt.Sprintf("./cockroach sql %s -e BACKUP bank.bank TO 'gs://cockroachdb-backup-testing/%s'", c.secureFlags(), dest))
return nil
})
m.Wait()
Expand Down
13 changes: 11 additions & 2 deletions pkg/cmd/roachtest/build_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package main

import (
"context"
"crypto/tls"
"net/http"
"os/exec"

Expand All @@ -24,8 +25,16 @@ func runBuildInfo(ctx context.Context, t *test, c *cluster) {
c.Start(ctx, t)

var details serverpb.DetailsResponse
url := `http://` + c.ExternalAdminUIAddr(ctx, c.Node(1))[0] + `/_status/details/local`
err := httputil.GetJSON(http.Client{}, url, &details)
urlSchema := "https://"
if insecure {
urlSchema = "http://"
}
url := urlSchema + c.ExternalAdminUIAddr(ctx, c.Node(1))[0] + `/_status/details/local`
// TODO(rail): figure out the way to accept expected certs only
tr := &http.Transport{
TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
}
err := httputil.GetJSON(http.Client{Transport: tr}, url, &details)
if err != nil {
t.Fatal(err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/cancel.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ func registerCancel(r *testRegistry) {

const cancelQuery = `CANCEL QUERIES
SELECT query_id FROM [SHOW CLUSTER QUERIES] WHERE query not like '%SHOW CLUSTER QUERIES%'`
c.Run(ctx, c.Node(1), `./cockroach sql --insecure -e "`+cancelQuery+`"`)
c.Run(ctx, c.Node(1), `./cockroach sql `+c.secureFlags()+` -e "`+cancelQuery+`"`)
cancelStartTime := timeutil.Now()

select {
Expand Down
4 changes: 3 additions & 1 deletion pkg/cmd/roachtest/clearrange.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,9 @@ func runClearRange(ctx context.Context, t *test, c *cluster, aggressiveChecks bo
// Use a 120s connect timeout to work around the fact that the server will
// declare itself ready before it's actually 100% ready. See:
// https://github.com/cockroachdb/cockroach/issues/34897#issuecomment-465089057
c.Run(ctx, c.Node(1), `COCKROACH_CONNECT_TIMEOUT=120 ./cockroach sql --insecure -e "DROP DATABASE IF EXISTS tinybank"`)
c.Run(ctx, c.Node(1),
`COCKROACH_CONNECT_TIMEOUT=120 ./cockroach sql `+c.secureFlags()+
` -e "DROP DATABASE IF EXISTS tinybank"`)
c.Run(ctx, c.Node(1), "./cockroach", "workload", "fixtures", "import", "bank", "--db=tinybank",
"--payload-bytes=100", "--ranges=10", "--rows=800", "--seed=1")

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/cli.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func runCLINodeStatus(ctx context.Context, t *test, c *cluster) {

nodeStatus := func() (raw string, _ []string) {
out, err := c.RunWithBuffer(ctx, t.l, c.Node(1),
"./cockroach node status --insecure -p {pgport:1}")
"./cockroach node status -p {pgport:1} "+c.secureFlags())
if err != nil {
t.Fatalf("%v\n%s", err, out)
}
Expand Down
10 changes: 9 additions & 1 deletion pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ var (
libraryFilePaths []string
cloud = gce
encrypt encryptValue = "false"
insecure bool
instanceType string
localSSD bool
workload string
Expand Down Expand Up @@ -1151,6 +1152,13 @@ func (c *cluster) String() string {
return fmt.Sprintf("%s [tag:%s] (%d nodes)", c.name, c.tag, c.spec.NodeCount)
}

func (c *cluster) secureFlags() string {
if insecure {
return "--insecure"
}
return "--certs-dir=certs"
}

type destroyState struct {
// owned is set if this instance is responsible for `roachprod destroy`ing the
// cluster. It is set when a new cluster is created, but not when we attach to
Expand Down Expand Up @@ -1423,7 +1431,7 @@ func (c *cluster) StopCockroachGracefullyOnNode(ctx context.Context, node int) e
// Note that the following command line needs to run against both v2.1
// and the current branch. Do not change it in a manner that is
// incompatible with 2.1.
if err := c.RunE(ctx, c.Node(node), "./cockroach quit --insecure --port="+port); err != nil {
if err := c.RunE(ctx, c.Node(node), fmt.Sprintf("./cockroach quit --port=%s %s", port, c.secureFlags())); err != nil {
return err
}
// TODO (rohany): This comment below might be out of date.
Expand Down
16 changes: 9 additions & 7 deletions pkg/cmd/roachtest/cluster_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) {
return c.RunE(ctx, c.Node(i),
fmt.Sprintf(
`mkdir -p {log-dir} && `+
`./cockroach start --insecure --background --store={store-dir} `+
`./cockroach start `+c.secureFlags()+` --background --store={store-dir} `+
`--log-dir={log-dir} --cache=10%% --max-sql-memory=10%% `+
`--listen-addr=:{pgport:%[1]d} --http-port=$[{pgport:%[1]d}+1] `+
`--join=`+strings.Join(addrs, ",")+
Expand All @@ -61,7 +61,11 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) {

urlMap := make(map[int]string)
for i, addr := range c.ExternalAdminUIAddr(ctx, c.All()) {
urlMap[i+1] = `http://` + addr
urlSchema := "https://"
if insecure {
urlSchema = "http://"
}
urlMap[i+1] = urlSchema + addr
}

// Wait for the servers to bind their ports.
Expand Down Expand Up @@ -128,7 +132,7 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) {
// The actual contents of the cookie don't matter; the presence of
// a valid encoded cookie is enough to trigger the authentication
// code paths.
}, false /* forHTTPSOnly - cluster is insecure */)
}, !insecure /* forHTTPSOnly - cluster is insecure */)
if err != nil {
t.Fatal(err)
}
Expand All @@ -149,7 +153,7 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) {
}

c.Run(ctx, c.Node(initNode),
fmt.Sprintf(`./cockroach init --insecure --port={pgport:%d}`, initNode))
fmt.Sprintf(`./cockroach init %s --port={pgport:%d}`, c.secureFlags(), initNode))
if err := g.Wait(); err != nil {
t.Fatal(err)
}
Expand All @@ -158,10 +162,8 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) {
waitForFullReplication(t, dbs[0])

execCLI := func(runNode int, extraArgs ...string) (string, error) {
args := []string{"./cockroach"}
args := []string{"./cockroach", c.secureFlags(), fmt.Sprintf("--port={pgport:%d}", runNode)}
args = append(args, extraArgs...)
args = append(args, "--insecure")
args = append(args, fmt.Sprintf("--port={pgport:%d}", runNode))
buf, err := c.RunWithBuffer(ctx, c.l, c.Node(runNode), args...)
t.l.Printf("%s\n", buf)
return string(buf), err
Expand Down
9 changes: 4 additions & 5 deletions pkg/cmd/roachtest/decommission.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,13 +212,14 @@ func runDecommission(ctx context.Context, t *test, c *cluster, nodes int, durati
stop := func(node int) error {
port := fmt.Sprintf("{pgport:%d}", node)
defer time.Sleep(time.Second) // work around quit returning too early
return c.RunE(ctx, c.Node(node), "./cockroach quit --insecure --host=:"+port)
return c.RunE(ctx, c.Node(node), fmt.Sprintf("./cockroach quit --host=:%s %s", port, c.secureFlags()))
}

decom := func(id int) error {
port := fmt.Sprintf("{pgport:%d}", pinnedNode) // always use the pinned node
t.Status(fmt.Sprintf("decommissioning node %d", id))
return c.RunE(ctx, c.Node(pinnedNode), fmt.Sprintf("./cockroach node decommission --insecure --wait=all --host=:%s %d", port, id))
return c.RunE(ctx, c.Node(pinnedNode), fmt.Sprintf("./cockroach node decommission %s --wait=all --host=:%s %d",
c.secureFlags(), port, id))
}

tBegin, whileDown := timeutil.Now(), true
Expand Down Expand Up @@ -1107,10 +1108,8 @@ func (h *decommTestHelper) getRandNodeOtherThan(ids ...int) int {
func execCLI(
ctx context.Context, t *test, c *cluster, runNode int, extraArgs ...string,
) (string, error) {
args := []string{"./cockroach"}
args := []string{"./cockroach", c.secureFlags(), fmt.Sprintf("--port={pgport:%d}", runNode)}
args = append(args, extraArgs...)
args = append(args, "--insecure")
args = append(args, fmt.Sprintf("--port={pgport:%d}", runNode))
buf, err := c.RunWithBuffer(ctx, t.l, c.Node(runNode), args...)
t.l.Printf("%s\n", buf)
return string(buf), err
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/disk_stall.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,8 @@ func runDiskStalledDetection(
t.WorkerStatus("running server")
out, err := c.RunWithBuffer(ctx, l, n,
fmt.Sprintf("timeout --signal 9 %ds env COCKROACH_ENGINE_MAX_SYNC_DURATION_DEFAULT=%s COCKROACH_LOG_MAX_SYNC_DURATION=%s "+
"./cockroach start-single-node --insecure --store {store-dir}/%s --log '{sinks: {stderr: {filter: INFO}}, file-defaults: {dir: \"{store-dir}/%s\"}}'",
"./cockroach start-single-node "+c.secureFlags()+" --store {store-dir}/%s --log '{sinks: {stderr: {filter: INFO}}, "+
"file-defaults: {dir: \"{store-dir}/%s\"}}'",
int(dur.Seconds()), maxDataSync, maxLogSync, dataDir, logDir,
),
)
Expand Down
7 changes: 4 additions & 3 deletions pkg/cmd/roachtest/election.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,15 +38,15 @@ func registerElectionAfterRestart(r *testRegistry) {
time.Sleep(3 * time.Second)

t.Status("creating table and splits")
c.Run(ctx, c.Node(1), `./cockroach sql --insecure -e "
c.Run(ctx, c.Node(1), `./cockroach sql `+c.secureFlags()+` -e "
CREATE DATABASE IF NOT EXISTS test;
CREATE TABLE test.kv (k INT PRIMARY KEY, v INT);
-- Prevent the merge queue from immediately discarding our splits.
SET CLUSTER SETTING kv.range_merge.queue_enabled = false;
ALTER TABLE test.kv SPLIT AT SELECT generate_series(0, 10000, 100)"`)

start := timeutil.Now()
c.Run(ctx, c.Node(1), `./cockroach sql --insecure -e "
c.Run(ctx, c.Node(1), `./cockroach sql `+c.secureFlags()+` -e "
SELECT * FROM test.kv"`)
duration := timeutil.Since(start)
t.l.Printf("pre-restart, query took %s\n", duration)
Expand All @@ -73,7 +73,8 @@ func registerElectionAfterRestart(r *testRegistry) {
// takes ages (perhaps due to some cli-internal query taking a
// very long time), we fail with the duration check below and
// not an opaque error from the cli.
buf, err := c.RunWithBuffer(ctx, t.l, c.Node(1), `COCKROACH_CONNECT_TIMEOUT=240 ./cockroach sql --insecure -e "
buf, err := c.RunWithBuffer(ctx, t.l, c.Node(1),
`COCKROACH_CONNECT_TIMEOUT=240 ./cockroach sql `+c.secureFlags()+` -e "
SET TRACING = on;
SELECT * FROM test.kv;
SET TRACING = off;
Expand Down
Loading

0 comments on commit 3195270

Please sign in to comment.