From 0da2825120bf9143ce55c2192374b56f80b4a05e Mon Sep 17 00:00:00 2001 From: Rogger Vasquez Date: Fri, 12 Aug 2022 10:21:59 -0500 Subject: [PATCH 1/2] rpk: use redpanda.yaml instead of bootstrap file redpanda don't do backcompat redpanda.yaml config stripping (legacy mode) if .bootstrap.yaml is present. We are removing the bootstrap file writing because it could break clients that still use --set to set cluster properties on first start. --- src/go/rpk/pkg/cli/cmd/redpanda/start.go | 66 ++------- src/go/rpk/pkg/cli/cmd/redpanda/start_test.go | 135 ++++++++---------- 2 files changed, 73 insertions(+), 128 deletions(-) diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/start.go b/src/go/rpk/pkg/cli/cmd/redpanda/start.go index 803569317d10..cb6c11234a66 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/start.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/start.go @@ -15,7 +15,6 @@ package redpanda import ( "errors" "fmt" - "io" "os" "path/filepath" "regexp" @@ -168,12 +167,8 @@ func NewStartCommand(fs afero.Fs, launcher rp.Launcher) *cobra.Command { switch mode { case "container": fmt.Fprintln(os.Stderr, "WARNING: This is a setup for development purposes only; in this mode your clusters may run unrealistically fast and data can be corrupted any time your computer shuts down uncleanly.") - cfg.Redpanda.DeveloperMode = true setContainerModeFlags(cmd) - err := setClusterProperties(fs, cfg.FileLocation()) - if err != nil { - fmt.Fprintf(os.Stderr, "unable to set cluster properties: %v\n", err) - } + setContainerModeCfgFields(cfg) case "help": fmt.Println(helpMode) return nil @@ -1094,58 +1089,17 @@ func setContainerModeFlags(cmd *cobra.Command) { } } -// setClusterProperties generates a .bootstrap.yaml file in the same path as -// the redpanda.yaml, this will be picked up by redpanda on first start and -// set the passed properties. -func setClusterProperties(fs afero.Fs, cfgFileLocation string) (rerr error) { - const props = `auto_create_topics_enabled: true -group_topic_partitions: 3 -storage_min_free_bytes: 10485760 -topic_partitions_per_shard: 1000 -` - cfgDir := filepath.Dir(cfgFileLocation) - - tmp, err := afero.TempFile(fs, cfgDir, "bootstrap-*.yaml") - if err != nil { - return err - } - - defer func() { - if rerr != nil { - suggestion := "you can run 'rpk cluster config set ' to set the following properties: " + props - if removeErr := fs.Remove(tmp.Name()); removeErr != nil { - rerr = fmt.Errorf("%s, unable to remove temp file: %v; %s", rerr, removeErr, suggestion) - } else { - rerr = fmt.Errorf("%s, temp file removed from disk; %s", rerr, suggestion) - } - } - }() - - _, err = io.WriteString(tmp, props) - tmp.Close() - if err != nil { - return fmt.Errorf("error writing to temporary file: %v", err) - } - - // If we already have a redpanda.yaml we want to have the same file - // ownership for .boostrap.yaml - if exists, _ := afero.Exists(fs, cfgFileLocation); exists { - stat, err := fs.Stat(cfgFileLocation) - if err != nil { - return fmt.Errorf("unable to stat existing file: %v", err) - } - err = config.PreserveUnixOwnership(fs, stat, tmp.Name()) - if err != nil { - return err - } - } +func setContainerModeCfgFields(cfg *config.Config) { + cfg.Redpanda.DeveloperMode = true - err = fs.Rename(tmp.Name(), filepath.Join(cfgDir, ".bootstrap.yaml")) - if err != nil { - return err + // cluster properties: + if cfg.Redpanda.Other == nil { + cfg.Redpanda.Other = make(map[string]interface{}) } - - return nil + cfg.Redpanda.Other["auto_create_topics_enabled"] = true + cfg.Redpanda.Other["group_topic_partitions"] = 3 + cfg.Redpanda.Other["storage_min_free_bytes"] = 10485760 + cfg.Redpanda.Other["topic_partitions_per_shard"] = 1000 } const helpMode = `Mode uses well-known configuration properties for development or tests diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go b/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go index 10b4f854add7..51f8cf4c5a70 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go @@ -241,6 +241,14 @@ func TestStartCommand(t *testing.T) { path, ) c := config.Default() + // We are adding now this cluster properties as default with + // redpanda.developer_mode: true. + c.Redpanda.Other = map[string]interface{}{ + "auto_create_topics_enabled": true, + "group_topic_partitions": 3, + "storage_min_free_bytes": 10485760, + "topic_partitions_per_shard": 1000, + } conf, err := new(config.Params).Load(fs) require.NoError(st, err) @@ -1384,16 +1392,26 @@ func TestStartCommand(t *testing.T) { rpArgs *redpanda.RedpandaArgs, st *testing.T, ) { + // Flags: require.Equal(st, "true", rpArgs.SeastarFlags["overprovisioned"]) require.Equal(st, "0M", rpArgs.SeastarFlags["reserve-memory"]) require.Equal(st, "true", rpArgs.SeastarFlags["unsafe-bypass-fsync"]) + + // Config: conf, err := new(config.Params).Load(fs) require.NoError(st, err) require.Equal(st, 0, conf.Redpanda.ID) require.Equal(st, true, conf.Redpanda.DeveloperMode) + expectedClusterFields := map[string]interface{}{ + "auto_create_topics_enabled": true, + "group_topic_partitions": 3, + "storage_min_free_bytes": 10485760, + "topic_partitions_per_shard": 1000, + } + require.Equal(st, expectedClusterFields, conf.Redpanda.Other) }, }, { - name: "override values set by --mode container", + name: "override flags set by --mode container", args: []string{ "--install-dir", "/var/lib/redpanda", "--mode", "container", "--reserve-memory", "2M", @@ -1413,61 +1431,6 @@ func TestStartCommand(t *testing.T) { require.Equal(st, 0, conf.Redpanda.ID) require.Equal(st, true, conf.Redpanda.DeveloperMode) }, - }, { - name: ".bootstrap.yaml created with --mode container", - args: []string{ - "--install-dir", "/var/lib/redpanda", - "--mode", "container", - }, - postCheck: func( - fs afero.Fs, - _ *redpanda.RedpandaArgs, - st *testing.T, - ) { - bFile := "/etc/redpanda/.bootstrap.yaml" - exists, err := afero.Exists(fs, bFile) - require.NoError(st, err) - require.True(st, exists) - file, err := afero.ReadFile(fs, bFile) - require.NoError(st, err) - require.Equal( - st, - `auto_create_topics_enabled: true -group_topic_partitions: 3 -storage_min_free_bytes: 10485760 -topic_partitions_per_shard: 1000 -`, - string(file), - ) - }, - }, { - name: ".bootstrap.yaml created with --mode container in arbitrary path", - args: []string{ - "--install-dir", "/var/lib/redpanda", - "--mode", "container", "--config", - "/arbitrary/path/redpanda.yaml", - }, - postCheck: func( - fs afero.Fs, - _ *redpanda.RedpandaArgs, - st *testing.T, - ) { - bFile := "/arbitrary/path/.bootstrap.yaml" - exists, err := afero.Exists(fs, bFile) - require.NoError(st, err) - require.True(st, exists) - file, err := afero.ReadFile(fs, bFile) - require.NoError(st, err) - require.Equal( - st, - `auto_create_topics_enabled: true -group_topic_partitions: 3 -storage_min_free_bytes: 10485760 -topic_partitions_per_shard: 1000 -`, - string(file), - ) - }, }, { name: "redpanda.developer_mode: true behaves like --mode container", args: []string{"--install-dir", "/var/lib/redpanda"}, @@ -1488,25 +1451,53 @@ topic_partitions_per_shard: 1000 require.NoError(st, err) // Config: + expectedClusterFields := map[string]interface{}{ + "auto_create_topics_enabled": true, + "group_topic_partitions": 3, + "storage_min_free_bytes": 10485760, + "topic_partitions_per_shard": 1000, + } require.Equal(st, 0, conf.Redpanda.ID) require.Equal(st, true, conf.Redpanda.DeveloperMode) - - // Bootstrap Yaml - bFile := "/etc/redpanda/.bootstrap.yaml" - exists, err := afero.Exists(fs, bFile) - require.NoError(st, err) - require.True(st, exists) - file, err := afero.ReadFile(fs, bFile) - require.NoError(st, err) - require.Equal( - st, - `auto_create_topics_enabled: true -group_topic_partitions: 3 -storage_min_free_bytes: 10485760 -topic_partitions_per_shard: 1000 -`, - string(file), + require.Equal(st, expectedClusterFields, conf.Redpanda.Other) + }, + }, { + name: "--set overrides cluster configs set by --mode container", + args: []string{ + "--install-dir", "/var/lib/redpanda", + "--mode", "container", + }, + before: func(fs afero.Fs) error { + // --set flags are parsed "outside" of Cobra, directly from + // os.Args. + os.Args = append( + os.Args, + // A single int value + "--set", "redpanda.auto_create_topics_enabled=false", + // A single bool value + "--set", "redpanda.group_topic_partitions=1", ) + return nil + }, + after: func() { + for i, a := range os.Args { + if a == setFlag { + os.Args = os.Args[:i] + return + } + } + }, + postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) { + conf, _ := new(config.Params).Load(fs) + expectedClusterFields := map[string]interface{}{ + // set by --set flag + "auto_create_topics_enabled": false, + "group_topic_partitions": 1, + // rest of --mode container cfg fields + "storage_min_free_bytes": 10485760, + "topic_partitions_per_shard": 1000, + } + require.Exactly(st, expectedClusterFields, conf.Redpanda.Other) }, }, { name: "Fails if unknown mode is passed", From 2375a7bbf876fc2d8fe759c67f9aa46b25cdbf12 Mon Sep 17 00:00:00 2001 From: Rogger Vasquez Date: Fri, 12 Aug 2022 11:42:27 -0500 Subject: [PATCH 2/2] rpk: rename mode container to dev-container --- .../rpk/pkg/cli/cmd/container/common/common.go | 2 +- src/go/rpk/pkg/cli/cmd/redpanda/start.go | 10 +++++----- src/go/rpk/pkg/cli/cmd/redpanda/start_test.go | 18 +++++++++--------- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/go/rpk/pkg/cli/cmd/container/common/common.go b/src/go/rpk/pkg/cli/cmd/container/common/common.go index 740042c704e6..7e0bb8526000 100644 --- a/src/go/rpk/pkg/cli/cmd/container/common/common.go +++ b/src/go/rpk/pkg/cli/cmd/container/common/common.go @@ -282,7 +282,7 @@ func CreateNode( AdvertiseAddresses(ip, config.DefaultProxyPort, proxyPort), "--advertise-rpc-addr", net.JoinHostPort(ip, strconv.Itoa(config.Default().Redpanda.RPCServer.Port)), - "--mode container", + "--mode dev-container", } containerConfig := container.Config{ Image: image, diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/start.go b/src/go/rpk/pkg/cli/cmd/redpanda/start.go index cb6c11234a66..fbd077f61aed 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/start.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/start.go @@ -162,10 +162,10 @@ func NewStartCommand(fs afero.Fs, launcher rp.Launcher) *cobra.Command { // configuration itself. cfg = cfg.FileOrDefaults() if cfg.Redpanda.DeveloperMode && len(mode) == 0 { - mode = "container" + mode = "dev-container" } switch mode { - case "container": + case "dev-container": fmt.Fprintln(os.Stderr, "WARNING: This is a setup for development purposes only; in this mode your clusters may run unrealistically fast and data can be corrupted any time your computer shuts down uncleanly.") setContainerModeFlags(cmd) setContainerModeCfgFields(cfg) @@ -1071,7 +1071,7 @@ func mergeMaps(a, b map[string]string) map[string]string { return a } -// setContainerModeFlags sets flags bundled into --mode container flag. +// setContainerModeFlags sets flags bundled into --mode dev-container flag. func setContainerModeFlags(cmd *cobra.Command) { devMap := map[string]string{ overprovisionedFlag: "true", @@ -1080,7 +1080,7 @@ func setContainerModeFlags(cmd *cobra.Command) { unsafeBypassFsyncFlag: "true", } // We don't override the values set during command execution, e.g: - // rpk redpanda start --mode container --smp 2 + // rpk redpanda start --mode dev-container --smp 2 // will apply all dev flags, but smp will be 2. for k, v := range devMap { if !cmd.Flags().Changed(k) { @@ -1105,7 +1105,7 @@ func setContainerModeCfgFields(cfg *config.Config) { const helpMode = `Mode uses well-known configuration properties for development or tests environments: ---mode container +--mode dev-container Bundled flags: * --overprovisioned * --reserve-memory 0M diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go b/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go index 51f8cf4c5a70..24fe47806964 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go @@ -1382,10 +1382,10 @@ func TestStartCommand(t *testing.T) { require.Equal(st, expected, rpArgs.ExtraArgs) }, }, { - name: "--mode container flag set required bundle of flags", + name: "--mode dev-container flag set required bundle of flags", args: []string{ "--install-dir", "/var/lib/redpanda", - "--mode", "container", + "--mode", "dev-container", }, postCheck: func( fs afero.Fs, @@ -1411,10 +1411,10 @@ func TestStartCommand(t *testing.T) { require.Equal(st, expectedClusterFields, conf.Redpanda.Other) }, }, { - name: "override flags set by --mode container", + name: "override flags set by --mode dev-container", args: []string{ "--install-dir", "/var/lib/redpanda", - "--mode", "container", "--reserve-memory", "2M", + "--mode", "dev-container", "--reserve-memory", "2M", }, postCheck: func( fs afero.Fs, @@ -1423,7 +1423,7 @@ func TestStartCommand(t *testing.T) { ) { // override value: require.Equal(st, "2M", rpArgs.SeastarFlags["reserve-memory"]) - // rest of --mode container bundle + // rest of --mode dev-container bundle require.Equal(st, "true", rpArgs.SeastarFlags["overprovisioned"]) require.Equal(st, "true", rpArgs.SeastarFlags["unsafe-bypass-fsync"]) conf, err := new(config.Params).Load(fs) @@ -1432,7 +1432,7 @@ func TestStartCommand(t *testing.T) { require.Equal(st, true, conf.Redpanda.DeveloperMode) }, }, { - name: "redpanda.developer_mode: true behaves like --mode container", + name: "redpanda.developer_mode: true behaves like --mode dev-container", args: []string{"--install-dir", "/var/lib/redpanda"}, before: func(fs afero.Fs) error { conf, _ := new(config.Params).Load(fs) @@ -1462,10 +1462,10 @@ func TestStartCommand(t *testing.T) { require.Equal(st, expectedClusterFields, conf.Redpanda.Other) }, }, { - name: "--set overrides cluster configs set by --mode container", + name: "--set overrides cluster configs set by --mode dev-container", args: []string{ "--install-dir", "/var/lib/redpanda", - "--mode", "container", + "--mode", "dev-container", }, before: func(fs afero.Fs) error { // --set flags are parsed "outside" of Cobra, directly from @@ -1493,7 +1493,7 @@ func TestStartCommand(t *testing.T) { // set by --set flag "auto_create_topics_enabled": false, "group_topic_partitions": 1, - // rest of --mode container cfg fields + // rest of --mode dev-container cfg fields "storage_min_free_bytes": 10485760, "topic_partitions_per_shard": 1000, }