From 9aa5bc1005b33bcfbfc65a517bef328b5114b7a6 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 9 Jul 2020 16:28:20 -0400 Subject: [PATCH] cli: remove auto-init with `cockroach start` without `--join` Fixes #44116. Informs #24118. Reviving #44112. Release note (backward-incompatible change): A CockroachDB node started with cockroach start without the --join flag no longer automatically initializes the cluster. The cockrach init command is now mandatory. The auto-initialization behavior had been deprecated in version 19.2. --- pkg/acceptance/cluster/dockercluster.go | 8 +- pkg/acceptance/cluster/testconfig.pb.go | 80 +++++++++---------- pkg/acceptance/cluster/testconfig.proto | 4 +- pkg/base/config.go | 9 +++ pkg/base/test_server_args.go | 5 ++ pkg/cli/context.go | 1 + pkg/cli/demo_cluster.go | 8 +- pkg/cli/demo_test.go | 2 + pkg/cli/init.go | 17 +--- .../test_disable_replication.tcl | 5 +- pkg/cli/interactive_tests/test_dump_sig.tcl | 2 +- pkg/cli/interactive_tests/test_flags.tcl | 9 --- pkg/cli/start.go | 23 ++---- pkg/cmd/roachprod/install/cockroach.go | 63 ++++++++++++--- pkg/cmd/roachtest/cluster_init.go | 38 +-------- pkg/server/server.go | 25 +++--- pkg/server/testserver.go | 1 + pkg/testutils/testcluster/testcluster.go | 5 +- 18 files changed, 150 insertions(+), 155 deletions(-) diff --git a/pkg/acceptance/cluster/dockercluster.go b/pkg/acceptance/cluster/dockercluster.go index 654491c6dadd..cadb8397a783 100644 --- a/pkg/acceptance/cluster/dockercluster.go +++ b/pkg/acceptance/cluster/dockercluster.go @@ -480,10 +480,12 @@ func (l *DockerCluster) startNode(ctx context.Context, node *testNode) { } cmd = append(cmd, fmt.Sprintf("--store=%s", storeSpec)) } - // Append --join flag (for all nodes except first in bootstrap-node-zero mode) - if node.index > 0 || l.config.InitMode != INIT_BOOTSTRAP_NODE_ZERO { - cmd = append(cmd, "--join="+net.JoinHostPort(l.Nodes[0].nodeStr, base.DefaultPort)) + // Append --join flag for all nodes. + firstNodeAddr := "" + if node.index > 0 { + firstNodeAddr = l.Nodes[0].nodeStr } + cmd = append(cmd, "--join="+net.JoinHostPort(firstNodeAddr, base.DefaultPort)) dockerLogDir := "/logs/" + node.nodeStr localLogDir := filepath.Join(l.volumesDir, "logs", node.nodeStr) diff --git a/pkg/acceptance/cluster/testconfig.pb.go b/pkg/acceptance/cluster/testconfig.pb.go index a2eecc82c2e0..9825aff03499 100644 --- a/pkg/acceptance/cluster/testconfig.pb.go +++ b/pkg/acceptance/cluster/testconfig.pb.go @@ -30,9 +30,6 @@ const ( // INIT_COMMAND starts every node with a join flag and issues the // init command. INIT_COMMAND InitMode = 0 - // INIT_BOOTSTRAP_NODE_ZERO uses the legacy protocol of omitting the - // join flag from node zero. - INIT_BOOTSTRAP_NODE_ZERO InitMode = 1 // INIT_NONE starts every node with a join flag and leaves the // cluster uninitialized. INIT_NONE InitMode = 2 @@ -40,13 +37,11 @@ const ( var InitMode_name = map[int32]string{ 0: "INIT_COMMAND", - 1: "INIT_BOOTSTRAP_NODE_ZERO", 2: "INIT_NONE", } var InitMode_value = map[string]int32{ - "INIT_COMMAND": 0, - "INIT_BOOTSTRAP_NODE_ZERO": 1, - "INIT_NONE": 2, + "INIT_COMMAND": 0, + "INIT_NONE": 2, } func (x InitMode) Enum() *InitMode { @@ -66,7 +61,7 @@ func (x *InitMode) UnmarshalJSON(data []byte) error { return nil } func (InitMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_testconfig_af28e3620e9a1d6b, []int{0} + return fileDescriptor_testconfig_a82c0aa336d029cb, []int{0} } // StoreConfig holds the configuration of a collection of similar stores. @@ -78,7 +73,7 @@ func (m *StoreConfig) Reset() { *m = StoreConfig{} } func (m *StoreConfig) String() string { return proto.CompactTextString(m) } func (*StoreConfig) ProtoMessage() {} func (*StoreConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_testconfig_af28e3620e9a1d6b, []int{0} + return fileDescriptor_testconfig_a82c0aa336d029cb, []int{0} } func (m *StoreConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -113,7 +108,7 @@ func (m *NodeConfig) Reset() { *m = NodeConfig{} } func (m *NodeConfig) String() string { return proto.CompactTextString(m) } func (*NodeConfig) ProtoMessage() {} func (*NodeConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_testconfig_af28e3620e9a1d6b, []int{1} + return fileDescriptor_testconfig_a82c0aa336d029cb, []int{1} } func (m *NodeConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -154,7 +149,7 @@ func (m *TestConfig) Reset() { *m = TestConfig{} } func (m *TestConfig) String() string { return proto.CompactTextString(m) } func (*TestConfig) ProtoMessage() {} func (*TestConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_testconfig_af28e3620e9a1d6b, []int{2} + return fileDescriptor_testconfig_a82c0aa336d029cb, []int{2} } func (m *TestConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -810,36 +805,35 @@ var ( ) func init() { - proto.RegisterFile("acceptance/cluster/testconfig.proto", fileDescriptor_testconfig_af28e3620e9a1d6b) -} - -var fileDescriptor_testconfig_af28e3620e9a1d6b = []byte{ - // 422 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x90, 0xcd, 0x6e, 0xd3, 0x40, - 0x14, 0x85, 0x3d, 0xf9, 0x21, 0xc9, 0x2d, 0x45, 0xd1, 0x08, 0x24, 0xab, 0x2a, 0x53, 0x2b, 0x95, - 0x90, 0xcb, 0xc2, 0x11, 0x79, 0x83, 0xa6, 0x89, 0x90, 0x17, 0xb1, 0x91, 0x6b, 0x09, 0xa9, 0x1b, - 0x6b, 0x34, 0x1e, 0xcc, 0x08, 0x3c, 0x53, 0xd9, 0x13, 0xe8, 0x23, 0xc0, 0x8e, 0x77, 0xe0, 0x65, - 0xb2, 0xec, 0xb2, 0xab, 0x0a, 0x9c, 0xb7, 0x60, 0x85, 0xec, 0x8e, 0x1b, 0xd8, 0x64, 0x67, 0x9f, - 0x7b, 0xce, 0xf9, 0xee, 0x1d, 0x38, 0xa5, 0x8c, 0xf1, 0x6b, 0x4d, 0x25, 0xe3, 0x53, 0xf6, 0x79, - 0x5d, 0x6a, 0x5e, 0x4c, 0x35, 0x2f, 0x35, 0x53, 0xf2, 0x83, 0xc8, 0xbc, 0xeb, 0x42, 0x69, 0x85, - 0x8f, 0x99, 0x62, 0x9f, 0x0a, 0x45, 0xd9, 0x47, 0x6f, 0x67, 0xf7, 0x8c, 0xfd, 0xe8, 0x79, 0xa6, - 0x32, 0xd5, 0x18, 0xa7, 0xf5, 0xd7, 0x43, 0x66, 0x32, 0x83, 0x83, 0x4b, 0xad, 0x0a, 0x7e, 0xd1, - 0x14, 0xe1, 0x53, 0x80, 0x9c, 0xde, 0x24, 0x05, 0x95, 0x19, 0x2f, 0xed, 0x8e, 0x83, 0xdc, 0xfe, - 0xbc, 0xb7, 0xb9, 0x3f, 0xb1, 0xa2, 0x51, 0x4e, 0x6f, 0xa2, 0x46, 0x9e, 0xac, 0x01, 0x02, 0x95, - 0xb6, 0x11, 0x02, 0x83, 0x2f, 0xbc, 0x28, 0x85, 0x92, 0x36, 0x72, 0x90, 0x3b, 0x32, 0xfe, 0x56, - 0xc4, 0x6f, 0xe1, 0x49, 0x59, 0x13, 0xea, 0xba, 0xae, 0x7b, 0x30, 0x3b, 0xf3, 0xf6, 0xad, 0xe9, - 0xfd, 0xb3, 0x8d, 0x69, 0x32, 0xf1, 0xc9, 0xf7, 0x0e, 0x40, 0xcc, 0x4b, 0x6d, 0xb8, 0x36, 0xf4, - 0x24, 0xcd, 0xf9, 0x7f, 0xd0, 0x46, 0xc1, 0x0b, 0xe8, 0x4b, 0x95, 0x3e, 0x02, 0xdd, 0xfd, 0xc0, - 0xdd, 0x29, 0xa6, 0xe4, 0x21, 0x8c, 0xdf, 0xc0, 0x30, 0x5d, 0x17, 0x54, 0xd7, 0x87, 0x75, 0x1d, - 0xe4, 0x76, 0xe7, 0x2f, 0xea, 0xf1, 0x9f, 0xfb, 0x93, 0x43, 0x2d, 0x72, 0xee, 0x2d, 0xcc, 0x30, - 0x7a, 0xb4, 0x61, 0x1f, 0x46, 0x42, 0x0a, 0x9d, 0xe4, 0x2a, 0xe5, 0x76, 0xcf, 0x41, 0xee, 0xb3, - 0xd9, 0xab, 0xfd, 0x70, 0x5f, 0x0a, 0xbd, 0x52, 0x29, 0x37, 0xe8, 0xa1, 0x30, 0xff, 0xf8, 0x25, - 0x0c, 0xa4, 0x4a, 0xbe, 0x52, 0xa1, 0xed, 0xbe, 0x83, 0xdc, 0x61, 0xfb, 0x16, 0x52, 0xbd, 0xa7, - 0x42, 0xbf, 0x0e, 0x61, 0xd8, 0x46, 0xf1, 0x18, 0x9e, 0xfa, 0x81, 0x1f, 0x27, 0x17, 0xe1, 0x6a, - 0x75, 0x1e, 0x2c, 0xc6, 0x16, 0x3e, 0x06, 0xbb, 0x51, 0xe6, 0x61, 0x18, 0x5f, 0xc6, 0xd1, 0xf9, - 0xbb, 0x24, 0x08, 0x17, 0xcb, 0xe4, 0x6a, 0x19, 0x85, 0x63, 0x84, 0x0f, 0x61, 0xd4, 0x4c, 0x83, - 0x30, 0x58, 0x8e, 0x3b, 0x47, 0xbd, 0x6f, 0x3f, 0x89, 0x35, 0x3f, 0xdb, 0xfc, 0x26, 0xd6, 0xa6, - 0x22, 0xe8, 0xb6, 0x22, 0xe8, 0xae, 0x22, 0xe8, 0x57, 0x45, 0xd0, 0x8f, 0x2d, 0xb1, 0x6e, 0xb7, - 0xc4, 0xba, 0xdb, 0x12, 0xeb, 0x6a, 0x60, 0x76, 0xfe, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x5a, 0x33, - 0x2b, 0x9d, 0x8c, 0x02, 0x00, 0x00, + proto.RegisterFile("acceptance/cluster/testconfig.proto", fileDescriptor_testconfig_a82c0aa336d029cb) +} + +var fileDescriptor_testconfig_a82c0aa336d029cb = []byte{ + // 405 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x90, 0xc1, 0x6e, 0xd3, 0x40, + 0x14, 0x45, 0x3d, 0x89, 0x43, 0xec, 0x57, 0x8a, 0xac, 0x11, 0x48, 0x56, 0x05, 0x53, 0xcb, 0x95, + 0x90, 0xcb, 0xc2, 0x11, 0xd9, 0xb0, 0x26, 0x0d, 0x42, 0x5e, 0xc4, 0x95, 0x42, 0x25, 0x24, 0x36, + 0xd6, 0x68, 0x3c, 0x98, 0x11, 0x78, 0xa6, 0xb2, 0x27, 0xd0, 0x4f, 0x80, 0x1d, 0xff, 0xc0, 0xcf, + 0x64, 0xd9, 0x65, 0x57, 0x15, 0x38, 0x7f, 0xc1, 0x0a, 0xd9, 0x1d, 0x27, 0x74, 0x93, 0x9d, 0x7d, + 0xdf, 0xbd, 0xf7, 0xbc, 0x37, 0x70, 0x42, 0x19, 0xe3, 0x97, 0x9a, 0x4a, 0xc6, 0x27, 0xec, 0xcb, + 0xaa, 0xd6, 0xbc, 0x9a, 0x68, 0x5e, 0x6b, 0xa6, 0xe4, 0x47, 0x51, 0xc4, 0x97, 0x95, 0xd2, 0x0a, + 0x3f, 0x65, 0x8a, 0x7d, 0xae, 0x14, 0x65, 0x9f, 0xe2, 0x9d, 0x3d, 0x36, 0xf6, 0xa3, 0xc7, 0x85, + 0x2a, 0x54, 0x67, 0x9c, 0xb4, 0x5f, 0x77, 0x99, 0x70, 0x0a, 0x07, 0xef, 0xb4, 0xaa, 0xf8, 0x59, + 0x57, 0x84, 0x4f, 0x00, 0x4a, 0x7a, 0x95, 0x55, 0x54, 0x16, 0xbc, 0xf6, 0x07, 0x01, 0x8a, 0x46, + 0x33, 0x7b, 0x7d, 0x7b, 0x6c, 0x2d, 0xdd, 0x92, 0x5e, 0x2d, 0x3b, 0x39, 0x5c, 0x01, 0xa4, 0x2a, + 0xef, 0x23, 0x04, 0xc6, 0x5f, 0x79, 0x55, 0x0b, 0x25, 0x7d, 0x14, 0xa0, 0xc8, 0x35, 0xfe, 0x5e, + 0xc4, 0x6f, 0xe1, 0x41, 0xdd, 0x12, 0xda, 0xba, 0x61, 0x74, 0x30, 0x3d, 0x8d, 0xf7, 0xad, 0x19, + 0xff, 0xb7, 0x8d, 0x69, 0x32, 0xf1, 0xf0, 0xc7, 0x00, 0xe0, 0x82, 0xd7, 0xda, 0x70, 0x7d, 0xb0, + 0x25, 0x2d, 0xf9, 0x3d, 0x68, 0xa7, 0xe0, 0x39, 0x8c, 0xa4, 0xca, 0xb7, 0xc0, 0x68, 0x3f, 0x70, + 0x77, 0x8a, 0x29, 0xb9, 0x0b, 0xe3, 0x97, 0xe0, 0xe4, 0xab, 0x8a, 0xea, 0xf6, 0xb0, 0x61, 0x80, + 0xa2, 0xe1, 0xec, 0x49, 0x3b, 0xfe, 0x7b, 0x7b, 0x7c, 0xa8, 0x45, 0xc9, 0xe3, 0xb9, 0x19, 0x2e, + 0xb7, 0x36, 0x9c, 0x80, 0x2b, 0xa4, 0xd0, 0x59, 0xa9, 0x72, 0xee, 0xdb, 0x01, 0x8a, 0x1e, 0x4d, + 0x9f, 0xef, 0x87, 0x27, 0x52, 0xe8, 0x85, 0xca, 0xb9, 0x41, 0x3b, 0xc2, 0xfc, 0xe3, 0x67, 0x30, + 0x96, 0x2a, 0xfb, 0x46, 0x85, 0xf6, 0x47, 0x01, 0x8a, 0x9c, 0xfe, 0x2d, 0xa4, 0x7a, 0x4f, 0x85, + 0x7e, 0xf1, 0x0a, 0x9c, 0x3e, 0x8a, 0x3d, 0x78, 0x98, 0xa4, 0xc9, 0x45, 0x76, 0x76, 0xbe, 0x58, + 0xbc, 0x4e, 0xe7, 0x9e, 0x85, 0x0f, 0xc1, 0xed, 0x94, 0xf4, 0x3c, 0x7d, 0xe3, 0x0d, 0x8e, 0xec, + 0xef, 0xbf, 0x88, 0x15, 0xda, 0x0e, 0xf2, 0xd0, 0xec, 0x74, 0xfd, 0x87, 0x58, 0xeb, 0x86, 0xa0, + 0xeb, 0x86, 0xa0, 0x9b, 0x86, 0xa0, 0xdf, 0x0d, 0x41, 0x3f, 0x37, 0xc4, 0xba, 0xde, 0x10, 0xeb, + 0x66, 0x43, 0xac, 0x0f, 0x63, 0xb3, 0xdb, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x09, 0x22, 0x2a, + 0xa7, 0x74, 0x02, 0x00, 0x00, } diff --git a/pkg/acceptance/cluster/testconfig.proto b/pkg/acceptance/cluster/testconfig.proto index d727cec1458f..cc09659e660e 100644 --- a/pkg/acceptance/cluster/testconfig.proto +++ b/pkg/acceptance/cluster/testconfig.proto @@ -22,9 +22,7 @@ enum InitMode { // init command. INIT_COMMAND = 0; - // INIT_BOOTSTRAP_NODE_ZERO uses the legacy protocol of omitting the - // join flag from node zero. - INIT_BOOTSTRAP_NODE_ZERO = 1; + reserved 1; // INIT_NONE starts every node with a join flag and leaves the // cluster uninitialized. diff --git a/pkg/base/config.go b/pkg/base/config.go index 23823921a0a2..69a7a1842fc4 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -208,6 +208,15 @@ type Config struct { // Enables the use of an PTP hardware clock user space API for HLC current time. // This contains the path to the device to be used (i.e. /dev/ptp0) ClockDevicePath string + + // AutoInitializeCluster, if set, causes the server to bootstrap the + // cluster. Note that if two nodes are started with this flag set + // and also configured to join each other, each node will bootstrap + // its own unique cluster and the join will fail. + // + // The flag exists mostly for the benefit of tests, and for + // `cockroach start-single-node`. + AutoInitializeCluster bool } // HistogramWindowInterval is used to determine the approximate length of time diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index 1d0fb7211c0f..4404dade8654 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -89,6 +89,11 @@ type TestServerArgs struct { SQLMemoryPoolSize int64 CacheSize int64 + // By default, test servers have AutoInitializeCluster=true set in + // their config. If WaitForBootstrap is set, that behavior is disabled + // and the test becomes responsible for initializing the cluster. + WaitForBootstrap bool + // If set, this will be appended to the Postgres URL by functions that // automatically open a connection to the server. That's equivalent to running // SET DATABASE=foo, which works even if the database doesn't (yet) exist. diff --git a/pkg/cli/context.go b/pkg/cli/context.go index a6560f3e464a..d9eaee7834fd 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -91,6 +91,7 @@ func setServerContextDefaults() { serverCfg.KVConfig.GoroutineDumpDirName = "" serverCfg.KVConfig.HeapProfileDirName = "" + serverCfg.AutoInitializeCluster = false serverCfg.KVConfig.ReadyFn = nil serverCfg.KVConfig.DelayedBootstrapFn = nil serverCfg.KVConfig.JoinList = nil diff --git a/pkg/cli/demo_cluster.go b/pkg/cli/demo_cluster.go index 9877493e1cdb..84968e249ce1 100644 --- a/pkg/cli/demo_cluster.go +++ b/pkg/cli/demo_cluster.go @@ -120,11 +120,15 @@ func setupTransientCluster( for i := 0; i < demoCtx.nodes; i++ { // All the nodes connect to the address of the first server created. var joinAddr string - if c.s != nil { + if i != 0 { joinAddr = c.s.ServingRPCAddr() } nodeID := roachpb.NodeID(i + 1) args := testServerArgsForTransientCluster(c.sockForServer(nodeID), nodeID, joinAddr, c.demoDir) + if i == 0 { + // The first node also auto-inits the cluster. + args.WaitForBootstrap = false + } // servRPCReadyCh is used if latency simulation is requested to notify that a test server has // successfully computed its RPC address. @@ -143,7 +147,6 @@ func setupTransientCluster( } serv := serverFactory.New(args).(*server.TestServer) - if i == 0 { c.s = serv } @@ -301,6 +304,7 @@ func testServerArgsForTransientCluster( StoreSpecs: []base.StoreSpec{storeSpec}, SQLMemoryPoolSize: demoCtx.sqlPoolMemorySize, CacheSize: demoCtx.cacheSize, + WaitForBootstrap: true, } if demoCtx.localities != nil { diff --git a/pkg/cli/demo_test.go b/pkg/cli/demo_test.go index a9d65c79ab22..a9053d055850 100644 --- a/pkg/cli/demo_test.go +++ b/pkg/cli/demo_test.go @@ -44,6 +44,7 @@ func TestTestServerArgsForTransientCluster(t *testing.T) { DisableTLSForHTTP: true, SQLMemoryPoolSize: 2 << 10, CacheSize: 1 << 10, + WaitForBootstrap: true, }, }, { @@ -57,6 +58,7 @@ func TestTestServerArgsForTransientCluster(t *testing.T) { DisableTLSForHTTP: true, SQLMemoryPoolSize: 4 << 10, CacheSize: 4 << 10, + WaitForBootstrap: true, }, }, } diff --git a/pkg/cli/init.go b/pkg/cli/init.go index 92bbc3a7c4e8..c3aa0e79d2ad 100644 --- a/pkg/cli/init.go +++ b/pkg/cli/init.go @@ -14,10 +14,8 @@ import ( "context" "fmt" "os" - "strings" "time" - "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -34,11 +32,7 @@ Perform one-time-only initialization of a CockroachDB cluster. After starting one or more nodes with --join flags, run the init command on one node (passing the same --host and certificate flags -you would use for the sql command). The target of the init command -must appear in the --join flags of other nodes. - -A node started without the --join flag initializes itself as a -single-node cluster, so the init command is not used in that case. +you would use for the sql command). `, Args: cobra.NoArgs, RunE: maybeShoutError(MaybeDecorateGRPCError(runInit)), @@ -59,15 +53,6 @@ func runInit(cmd *cobra.Command, args []string) error { c := serverpb.NewInitClient(conn) if _, err = c.Bootstrap(ctx, &serverpb.BootstrapRequest{}); err != nil { - if strings.Contains(err.Error(), server.ErrClusterInitialized.Error()) { - // We really want to use errors.Is() here but this would require - // error serialization support in gRPC. - // This is not yet performed in CockroachDB even though - // the error library now has infrastructure to do so, see: - // https://github.com/cockroachdb/errors/pull/14 - return errors.WithHint(err, - "Please ensure all your start commands are using --join.") - } return err } diff --git a/pkg/cli/interactive_tests/test_disable_replication.tcl b/pkg/cli/interactive_tests/test_disable_replication.tcl index f2c73f3f4e8e..8a43f18b59c7 100644 --- a/pkg/cli/interactive_tests/test_disable_replication.tcl +++ b/pkg/cli/interactive_tests/test_disable_replication.tcl @@ -36,8 +36,9 @@ 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'" +system "$argv start --insecure --pid-file=server_pid --background -s=path=logs/db >>logs/expect-cmd.log 2>&1" +system "$argv init --insecure" +system "$argv sql -e 'select 1'" # restart with start-single-node stop_server $argv start_server $argv diff --git a/pkg/cli/interactive_tests/test_dump_sig.tcl b/pkg/cli/interactive_tests/test_dump_sig.tcl index 192ee865f446..2a8fb651cda3 100644 --- a/pkg/cli/interactive_tests/test_dump_sig.tcl +++ b/pkg/cli/interactive_tests/test_dump_sig.tcl @@ -8,7 +8,7 @@ send "PS1='\\h:''/# '\r" eexpect ":/# " start_test "Check that the server emits a goroutine dump upon receiving signal" -send "$argv start --insecure --pid-file=server_pid --log-dir=logs --logtostderr\r" +send "$argv start-single-node --insecure --pid-file=server_pid --log-dir=logs --logtostderr\r" eexpect "CockroachDB node starting" system "kill -QUIT `cat server_pid`" diff --git a/pkg/cli/interactive_tests/test_flags.tcl b/pkg/cli/interactive_tests/test_flags.tcl index 899c0b2d3e0d..e895cb954990 100644 --- a/pkg/cli/interactive_tests/test_flags.tcl +++ b/pkg/cli/interactive_tests/test_flags.tcl @@ -72,15 +72,6 @@ eexpect {Failed running "cockroach"} eexpect ":/# " end_test -start_test "Check that start without --join reports a deprecation warning" -send "$argv start --insecure\r" -eexpect "running 'cockroach start' without --join is deprecated." -eexpect "node starting" -interrupt -eexpect ":/# " -end_test - - start_server $argv start_test "Check that a client can connect using the URL env var" diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 49cd13f65f5d..9a21783227c9 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -76,10 +76,6 @@ Specify the --join flag to point to another node or nodes that are part of the same cluster. The other nodes do not need to be started yet, and if the address of the other nodes to be added are not yet known it is legal for the first node to join itself. - -If --join is not specified, the cluster will also be initialized. -THIS BEHAVIOR IS DEPRECATED; consider using 'cockroach init' or -'cockroach start-single-node' instead. `, Example: ` cockroach start --insecure --store=attrs=ssd,path=/mnt/ssd1 --join=host:port,[host:port]`, Args: cobra.NoArgs, @@ -446,9 +442,10 @@ func runStartSingleNode(cmd *cobra.Command, args []string) error { if joinFlag.Changed { return errCannotUseJoin } - // Now actually set the flag as changed so that the start code - // doesn't warn that it was not set. - joinFlag.Changed = true + + // Make the node auto-init the cluster if not done already. + serverCfg.AutoInitializeCluster = true + return runStart(cmd, args, true /*disableReplication*/) } @@ -461,9 +458,8 @@ func runStartJoin(cmd *cobra.Command, args []string) error { // of other active nodes used to join this node to the cockroach // cluster, if this is its first time connecting. // -// If the argument disableReplication is true and we are starting -// a fresh cluster, the replication factor will be disabled in -// all zone configs. +// If the argument disableReplication is set the replication factor +// will be set to 1 all zone configs. func runStart(cmd *cobra.Command, args []string, disableReplication bool) error { tBegin := timeutil.Now() @@ -537,13 +533,6 @@ func runStart(cmd *cobra.Command, args []string, disableReplication bool) error // but when actually starting a server, we enable them. grpcutil.SetSeverity(log.Severity_WARNING) - // Check the --join flag. - if !flagSetForCmd(cmd).Lookup(cliflags.Join.Name).Changed { - log.Shout(ctx, log.Severity_WARNING, - "running 'cockroach start' without --join is deprecated.\n"+ - "Consider using 'cockroach start-single-node' or 'cockroach init' instead.") - } - // Now perform additional configuration tweaks specific to the start // command. diff --git a/pkg/cmd/roachprod/install/cockroach.go b/pkg/cmd/roachprod/install/cockroach.go index 7a865835c489..e81c9a312ba5 100644 --- a/pkg/cmd/roachprod/install/cockroach.go +++ b/pkg/cmd/roachprod/install/cockroach.go @@ -112,17 +112,17 @@ func argExists(args []string, target string) int { // Start implements the ClusterImpl.NodeDir interface. func (r Cockroach) Start(c *SyncedCluster, extraArgs []string) { - // Check to see if node 1 was started indicating the cluster was + // Check to see if node 1 was started, indicating the cluster is to be // bootstrapped. - var bootstrapped bool + var bootstrap bool for _, i := range c.ServerNodes() { if i == 1 { - bootstrapped = true + bootstrap = true break } } - if c.Secure && bootstrapped { + if c.Secure && bootstrap { c.DistributeCerts() } @@ -217,7 +217,8 @@ func (r Cockroach) Start(c *SyncedCluster, extraArgs []string) { args = append(args, "--locality="+locality) } } - if nodes[i] != 1 { + // `cockroach start` without `--join` is no longer supported as 20.2. + if nodes[i] != 1 || vers.AtLeast(version.MustParse("v20.1.0")) { args = append(args, fmt.Sprintf("--join=%s:%d", host1, r.NodePort(c, 1))) } if advertisePublicIP { @@ -271,11 +272,13 @@ func (r Cockroach) Start(c *SyncedCluster, extraArgs []string) { // unhelpful empty error (since everything has been redirected away). This is // unfortunately equally awkward to address. cmd := "ulimit -c unlimited; mkdir -p " + logDir + "; " + // TODO(peter): The ps and lslocks stuff is intended to debug why killing // of a cockroach process sometimes doesn't release file locks immediately. cmd += `echo ">>> roachprod start: $(date)" >> ` + logDir + "/roachprod.log; " + `ps axeww -o pid -o command >> ` + logDir + "/roachprod.log; " + `[ -x /usr/bin/lslocks ] && /usr/bin/lslocks >> ` + logDir + "/roachprod.log; " + cmd += keyCmd + fmt.Sprintf(" export ROACHPROD=%d%s && ", nodes[i], c.Tag) + "GOTRACEBACK=crash " + @@ -297,14 +300,54 @@ func (r Cockroach) Start(c *SyncedCluster, extraArgs []string) { return nil, nil }) - if bootstrapped { + if bootstrap { + var initOut string + display = fmt.Sprintf("%s: bootstrapping cluster", c.Name) + c.Parallel(display, 1, 0, func(i int) ([]byte, error) { + vers, err := getCockroachVersion(c, nodes[i]) + if err != nil { + return nil, err + } + if !vers.AtLeast(version.MustParse("v20.1.0")) { + return nil, nil + } + sess, err := c.newSession(1) + if err != nil { + return nil, err + } + defer sess.Close() + + var cmd string + if c.IsLocal() { + cmd = `cd ${HOME}/local/1 ; ` + } + path := fmt.Sprintf("%s/%s", c.Impl.NodeDir(c, nodes[i]), "cluster-bootstrapped") + binary := cockroachNodeBinary(c, 1) + url := r.NodeURL(c, "localhost", r.NodePort(c, 1)) + cmd += fmt.Sprintf(` + if ! test -e %s ; then + COCKROACH_CONNECT_TIMEOUT=0 %s init --url %s && touch %s + fi`, path, binary, url, path) + + out, err := sess.CombinedOutput(cmd) + if err != nil { + return nil, errors.Wrapf(err, "~ %s\n%s", cmd, out) + } + initOut = strings.TrimSpace(string(out)) + return nil, nil + }) + + if initOut != "" { + fmt.Println(initOut) + } + license := envutil.EnvOrDefaultString("COCKROACH_DEV_LICENSE", "") if license == "" { fmt.Printf("%s: COCKROACH_DEV_LICENSE unset: enterprise features will be unavailable\n", c.Name) } - var msg string + var clusterSettingsOut string display = fmt.Sprintf("%s: initializing cluster settings", c.Name) c.Parallel(display, 1, 0, func(i int) ([]byte, error) { sess, err := c.newSession(1) @@ -333,12 +376,12 @@ fi if err != nil { return nil, errors.Wrapf(err, "~ %s\n%s", cmd, out) } - msg = strings.TrimSpace(string(out)) + clusterSettingsOut = strings.TrimSpace(string(out)) return nil, nil }) - if msg != "" { - fmt.Println(msg) + if clusterSettingsOut != "" { + fmt.Println(clusterSettingsOut) } } } diff --git a/pkg/cmd/roachtest/cluster_init.go b/pkg/cmd/roachtest/cluster_init.go index 66b709f8bc76..dad8194ee5d2 100644 --- a/pkg/cmd/roachtest/cluster_init.go +++ b/pkg/cmd/roachtest/cluster_init.go @@ -38,42 +38,8 @@ func runClusterInit(ctx context.Context, t *test, c *cluster) { t.Fatal("no address for first node") } - // Legacy-style init where we start node 1 without a join flag and then point - // the other nodes at it. - func() { - var g errgroup.Group - g.Go(func() error { - return c.RunE(ctx, c.Node(1), - `mkdir -p {log-dir} && `+ - `./cockroach start --insecure --background --store={store-dir} `+ - `--log-dir={log-dir} --cache=10% --max-sql-memory=10% `+ - `--listen-addr=:{pgport:1} --http-port=$[{pgport:1}+1] `+ - `> {log-dir}/cockroach.stdout 2> {log-dir}/cockroach.stderr`) - }) - for i := 2; i <= c.spec.NodeCount; i++ { - i := i - g.Go(func() error { - return c.RunE(ctx, c.Node(i), - fmt.Sprintf( - `mkdir -p {log-dir} && `+ - `./cockroach start --insecure --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=`+addrs[0]+ - `> {log-dir}/cockroach.stdout 2> {log-dir}/cockroach.stderr`, i)) - }) - } - if err := g.Wait(); err != nil { - t.Fatal(err) - } - - db := c.Conn(ctx, 1) - defer db.Close() - waitForFullReplication(t, db) - }() - - // New-style init where we start all nodes with the same join flags and then - // issue an "init" command to one of the nodes. + // We start all nodes with the same join flags and then issue an "init" + // command to one of the nodes. for _, initNode := range []int{1, 2} { c.Wipe(ctx) diff --git a/pkg/server/server.go b/pkg/server/server.go index ae4c51ad09e2..4e6c2809275d 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -1283,18 +1283,10 @@ func (s *Server) Start(ctx context.Context) error { defer time.AfterFunc(30*time.Second, s.cfg.DelayedBootstrapFn).Stop() } - // Set up calling s.cfg.ReadyFn at the right time. Essentially, this call - // determines when `./cockroach [...] --background` returns. For any initialized - // nodes (i.e. already part of a cluster) this is when this method returns - // (assuming there's no error). For nodes that need to join a cluster, we - // return once the initServer is ready to accept requests. - var onSuccessfulReturnFn, onInitServerReady func() - selfBootstrap := initServer.NeedsInit() && len(s.cfg.GossipBootstrapResolvers) == 0 + // We self bootstrap for when we're configured to do so, which should only + // happen during tests and for `cockroach start-single-node`. + selfBootstrap := s.cfg.AutoInitializeCluster && initServer.NeedsInit() if selfBootstrap { - // If a new node is started without join flags, self-bootstrap. - // - // Note: this is behavior slated for removal, see: - // https://github.com/cockroachdb/cockroach/pull/44112 _, err := initServer.Bootstrap(ctx, &serverpb.BootstrapRequest{}) switch { case err == nil: @@ -1304,6 +1296,15 @@ func (s *Server) Start(ctx context.Context) error { // Process is shutting down. } } + + log.Info(ctx, "awaiting init command or join with an already initialized node.") + + // Set up calling s.cfg.ReadyFn at the right time. Essentially, this call + // determines when `./cockroach [...] --background` returns. For any + // initialized nodes (i.e. already part of a cluster) this is when this + // method returns (assuming there's no error). For nodes that need to join a + // cluster, we return once the initServer is ready to accept requests. + var onSuccessfulReturnFn, onInitServerReady func() { readyFn := func(bool) {} if s.cfg.ReadyFn != nil { @@ -1319,7 +1320,7 @@ func (s *Server) Start(ctx context.Context) error { } // This opens the main listener. When the listener is open, we can call - // initServerReadyFn since any request initated to the initServer at that + // initServerReadyFn since any request initiated to the initServer at that // point will reach it once ServeAndWait starts handling the queue of incoming // connections. startRPCServer(workersCtx) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index c30819718f2b..130525035653 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -146,6 +146,7 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { } cfg.ClusterName = params.ClusterName cfg.Insecure = params.Insecure + cfg.AutoInitializeCluster = !params.WaitForBootstrap cfg.SocketFile = params.SocketFile cfg.RetryOptions = params.RetryOptions cfg.Locality = params.Locality diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index 98e14cae707b..c232750fc73c 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -199,7 +199,7 @@ func StartTestCluster(t testing.TB, nodes int, args base.TestClusterArgs) *TestC serverArgs.Knobs.Server.(*server.TestingKnobs).RPCListener = firstListener serverArgs.Addr = firstListener.Addr().String() } else { - //serverArgs.JoinAddr = tc.Servers[0].ServingRPCAddr() + serverArgs.WaitForBootstrap = true serverArgs.JoinAddr = firstListener.Addr().String() } @@ -322,6 +322,9 @@ func (tc *TestCluster) AddServer(t testing.TB, serverArgs base.TestServerArgs) { func (tc *TestCluster) doAddServer(t testing.TB, serverArgs base.TestServerArgs) error { serverArgs.PartOfCluster = true + if serverArgs.JoinAddr != "" { + serverArgs.WaitForBootstrap = true + } // Check args even though they might have been checked in StartTestCluster; // this method might be called for servers being added after the cluster was // started, in which case the check has not been performed.