diff --git a/pkg/cli/demo.go b/pkg/cli/demo.go index b6c2ea7e483b..0783b7f34452 100644 --- a/pkg/cli/demo.go +++ b/pkg/cli/demo.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cli/cliflags" - "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" @@ -28,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log/logflags" "github.com/cockroachdb/cockroach/pkg/workload" "github.com/cockroachdb/cockroach/pkg/workload/workloadsql" - "github.com/gogo/protobuf/proto" "github.com/pkg/errors" "github.com/spf13/cobra" "github.com/spf13/pflag" @@ -123,27 +121,11 @@ func setupTransientServers( } cleanup = func() { stopper.Stop(ctx) } - // Set up the default zone configuration. We are using an in-memory store - // so we really want to disable replication. - cfg := config.DefaultZoneConfig() - sysCfg := config.DefaultSystemZoneConfig() - - if demoCtx.nodes < 3 { - cfg.NumReplicas = proto.Int32(1) - sysCfg.NumReplicas = proto.Int32(1) - } - // Create the first transient server. The others will join this one. args := base.TestServerArgs{ PartOfCluster: true, Insecure: true, - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DefaultZoneConfigOverride: &cfg, - DefaultSystemZoneConfigOverride: &sysCfg, - }, - }, - Stopper: stopper, + Stopper: stopper, } serverFactory := server.TestServerFactory @@ -166,6 +148,14 @@ func setupTransientServers( } } + if demoCtx.nodes < 3 { + // Set up the default zone configuration. We are using an in-memory store + // so we really want to disable replication. + if err := cliDisableReplication(ctx, s.Server); err != nil { + return ``, ``, cleanup, err + } + } + // Prepare the URL for use by the SQL shell. options := url.Values{} options.Add("sslmode", "disable") diff --git a/pkg/cli/disable_replication.go b/pkg/cli/disable_replication.go new file mode 100644 index 000000000000..3fa70539f798 --- /dev/null +++ b/pkg/cli/disable_replication.go @@ -0,0 +1,48 @@ +// Copyright 2019 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 cli + +import ( + "context" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +// cliDisableReplication changes the replication factor on +// all defined zones to become 1. This is used by start-single-node +// and demo to define single-node clusters, so as to avoid +// churn in the log files. +// +// The change is effected using the internal SQL interface of the +// given server object. +func cliDisableReplication(ctx context.Context, s *server.Server) error { + return s.RunLocalSQL(ctx, + func(ctx context.Context, ie *sql.InternalExecutor) error { + rows, err := ie.Query(ctx, "get-zones", nil, + "SELECT target FROM crdb_internal.zones") + if err != nil { + return err + } + + for _, row := range rows { + zone := string(*row[0].(*tree.DString)) + if _, err := ie.Exec(ctx, "set-zone", nil, + fmt.Sprintf("ALTER %s CONFIGURE ZONE USING num_replicas = 1", zone)); err != nil { + return err + } + } + + return nil + }) +} diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index 90191b7d64cf..66cdae9fcadc 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log/logflags" "github.com/cockroachdb/cockroach/pkg/util/netutil" "github.com/cockroachdb/errors" - "github.com/gogo/protobuf/proto" "github.com/spf13/cobra" "github.com/spf13/pflag" ) @@ -250,13 +249,6 @@ func init() { }) } - // start-single-node starts with default replication of 1. - AddPersistentPreRunE(startSingleNodeCmd, func(cmd *cobra.Command, _ []string) error { - serverCfg.DefaultSystemZoneConfig.NumReplicas = proto.Int32(1) - serverCfg.DefaultZoneConfig.NumReplicas = proto.Int32(1) - return nil - }) - // Map any flags registered in the standard "flag" package into the // top-level cockroach command. pf := cockroachCmd.PersistentFlags() diff --git a/pkg/cli/interactive_tests/test_disable_replication.tcl b/pkg/cli/interactive_tests/test_disable_replication.tcl new file mode 100644 index 000000000000..f2c73f3f4e8e --- /dev/null +++ b/pkg/cli/interactive_tests/test_disable_replication.tcl @@ -0,0 +1,51 @@ +#! /usr/bin/env expect -f + +source [file join [file dirname $argv0] common.tcl] + +spawn /bin/bash +send "PS1=':''/# '\r" +eexpect ":/# " + +start_test "Check that demo disables replication properly" +send "$argv demo -e 'show zone configuration for range default'\r" +eexpect "num_replicas = 1" +eexpect ":/# " +end_test + +start_test "Check that start-single-node disables replication properly" +system "rm -rf logs/db" +start_server $argv +send "$argv sql -e 'show zone configuration for range default'\r" +eexpect "num_replicas = 1" +eexpect ":/# " +end_test + +start_test "Check that it remains possible to reset the replication factor" +send "$argv sql -e 'alter range default configure zone using num_replicas = 3'\r" +eexpect "CONFIGURE ZONE" +eexpect ":/# " +stop_server $argv +start_server $argv +send "$argv sql -e 'show zone configuration for range default'\r" +eexpect "num_replicas = 3" +eexpect ":/# " +end_test + +stop_server $argv + +start_test "Check that start-single-node on a regular cluster does not reset the replication factor" +# make a fresh server but using the regular 'start' +system "rm -rf logs/db" +system "$argv start --insecure --pid-file=server_pid --background -s=path=logs/db >>logs/expect-cmd.log 2>&1; + $argv sql -e 'select 1'" +# restart with start-single-node +stop_server $argv +start_server $argv +# check that the replication factor was unchanged +send "$argv sql -e 'show zone configuration for range default'\r" +eexpect "num_replicas = 3" +eexpect ":/# " +end_test + +send "exit 0\r" +eexpect eof diff --git a/pkg/cli/interactive_tests/test_flags.tcl b/pkg/cli/interactive_tests/test_flags.tcl index 6b5b2ea04bc1..188c1602138a 100644 --- a/pkg/cli/interactive_tests/test_flags.tcl +++ b/pkg/cli/interactive_tests/test_flags.tcl @@ -80,14 +80,5 @@ interrupt eexpect ":/# " end_test -start_test "Check that start-single-node disables replication properly" -system "rm -rf logs/db" -start_server $argv -send "$argv sql -e 'show zone configuration for range default'\r" -eexpect "num_replicas = 1" -eexpect ":/# " -stop_server $argv -end_test - send "exit 0\r" eexpect eof diff --git a/pkg/cli/interactive_tests/test_multiple_nodes.tcl b/pkg/cli/interactive_tests/test_multiple_nodes.tcl new file mode 100644 index 000000000000..bd5ecb411c9d --- /dev/null +++ b/pkg/cli/interactive_tests/test_multiple_nodes.tcl @@ -0,0 +1,27 @@ +#! /usr/bin/env expect -f + +source [file join [file dirname $argv0] common.tcl] + +start_server $argv + +start_test "Check that it is possible to add nodes to a server started with start-single-node" + +system "$argv start --insecure --port=26258 --http-port=8083 --pid-file=server_pid2 --background -s=path=logs/db2 --join=:26257 >>logs/expect-cmd.log 2>&1; + $argv sql -e 'select 1' --port=26258" + +system "$argv start --insecure --port=26259 --http-port=8084 --pid-file=server_pid3 --background -s=path=logs/db3 --join=:26257 >>logs/expect-cmd.log 2>&1; + $argv sql -e 'select 1' --port=26259" + +# Check the number of nodes +spawn $argv node ls +eexpect id +eexpect "3 rows" +eexpect eof + +# Remove the additional nodes. +system "$argv quit --port=26258" +system "$argv quit --port=26259" + +end_test + +stop_server $argv diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 667a84bf8101..0d072fffd597 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -416,18 +416,22 @@ func runStartSingleNode(cmd *cobra.Command, args []string) error { // Now actually set the flag as changed so that the start code // doesn't warn that it was not set. joinFlag.Changed = true - return runStart(cmd, args) + return runStart(cmd, args, true /*disableReplication*/) } func runStartJoin(cmd *cobra.Command, args []string) error { - return runStart(cmd, args) + return runStart(cmd, args, false /*disableReplication*/) } // runStart starts the cockroach node using --store as the list of // storage devices ("stores") on this machine and --join as the list // of other active nodes used to join this node to the cockroach // cluster, if this is its first time connecting. -func runStart(cmd *cobra.Command, args []string) error { +// +// If the argument disableReplication is true and we are starting +// a fresh cluster, the replication factor will be disabled in +// all zone configs. +func runStart(cmd *cobra.Command, args []string, disableReplication bool) error { tBegin := timeutil.Now() // First things first: if the user wants background processing, @@ -710,6 +714,21 @@ If problems persist, please see ` + base.DocsURL("cluster-setup-troubleshooting. s.PeriodicallyCheckForUpdates(ctx) } + initialBoot := s.InitialBoot() + + if disableReplication && initialBoot { + // For start-single-node, set the default replication factor to + // 1 so as to avoid warning message and unnecessary rebalance + // churn. + if err := cliDisableReplication(ctx, s); err != nil { + log.Errorf(ctx, "could not disable replication: %v", err) + return err + } + log.Shout(ctx, log.Severity_INFO, + "Replication was disabled for this cluster.\n"+ + "When/if adding nodes in the future, update zone configurations to increase the replication factor.") + } + // Now inform the user that the server is running and tell the // user about its run-time derived parameters. var buf bytes.Buffer @@ -725,9 +744,9 @@ If problems persist, please see ` + base.DocsURL("cluster-setup-troubleshooting. pgURL, err := serverCfg.PGURL(url.User(security.RootUser)) if err != nil { log.Errorf(ctx, "failed computing the URL: %v", err) - } else { - fmt.Fprintf(tw, "sql:\t%s\n", pgURL) + return err } + fmt.Fprintf(tw, "sql:\t%s\n", pgURL) fmt.Fprintf(tw, "RPC client flags:\t%s\n", clientFlagsRPC()) if len(serverCfg.SocketFile) != 0 { @@ -754,7 +773,6 @@ If problems persist, please see ` + base.DocsURL("cluster-setup-troubleshooting. for i, spec := range serverCfg.Stores.Specs { fmt.Fprintf(tw, "store[%d]:\t%s\n", i, spec) } - initialBoot := s.InitialBoot() nodeID := s.NodeID() if initialBoot { if nodeID == server.FirstNodeID { diff --git a/pkg/server/server.go b/pkg/server/server.go index f4528b9c88ca..e810f44f1ae1 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -2227,3 +2227,19 @@ func listen( } return ln, nil } + +// RunLocalSQL calls fn on a SQL internal executor on this server. +// This is meant for use for SQL initialization during bootstrapping. +// +// The internal SQL interface should be used instead of a regular SQL +// network connection for SQL initializations when setting up a new +// server, because it is possible for the server to listen on a +// network interface that is not reachable from loopback. It is also +// possible for the TLS certificates to be invalid when used locally +// (e.g. if the hostname in the cert is an advertised address that's +// only reachable externally). +func (s *Server) RunLocalSQL( + ctx context.Context, fn func(ctx context.Context, sqlExec *sql.InternalExecutor) error, +) error { + return fn(ctx, s.internalExecutor) +}