diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/config.go b/src/go/rpk/pkg/cli/cmd/redpanda/config.go index 3daa5575bee9..c5be52368df2 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/config.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/config.go @@ -67,6 +67,7 @@ partial json/yaml config objects: p := config.ParamsFromCommand(cmd) cfg, err := p.Load(fs) out.MaybeDie(err, "unable to load config: %v", err) + cfg = cfg.FileOrDefaults() // we set fields in the raw file without writing env / flag overrides if format == "single" { fmt.Println("'--format single' is deprecated, either remove it or use yaml/json") @@ -104,6 +105,7 @@ func bootstrap(fs afero.Fs) *cobra.Command { p := config.ParamsFromCommand(cmd) cfg, err := p.Load(fs) out.MaybeDie(err, "unable to load config: %v", err) + cfg = cfg.FileOrDefaults() // we modify fields in the raw file without writing env / flag overrides seeds, err := parseSeedIPs(ips) out.MaybeDieErr(err) @@ -167,6 +169,7 @@ func initNode(fs afero.Fs) *cobra.Command { p := config.ParamsFromCommand(cmd) cfg, err := p.Load(fs) out.MaybeDie(err, "unable to load config: %v", err) + cfg = cfg.FileOrDefaults() // we modify fields in the raw file without writing env / flag overrides // Don't reset the node's UUID if it has already been set. if cfg.NodeUUID == "" { diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/config_test.go b/src/go/rpk/pkg/cli/cmd/redpanda/config_test.go index 7891c9b3dfed..7f9d716684ae 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/config_test.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/config_test.go @@ -184,3 +184,151 @@ func TestInitNode(t *testing.T) { }) } } + +// This is a top level command test, individual cases for set are +// tested in 'rpk/pkg/config/config_test.go'. +func TestSetCommand(t *testing.T) { + for _, test := range []struct { + name string + cfgFile string + exp string + args []string + }{ + { + name: "set without config file on disk", + exp: `config_file: /etc/redpanda/redpanda.yaml +redpanda: + data_directory: /var/lib/redpanda/data + node_id: 0 + rack: redpanda-rack + seed_servers: [] + rpc_server: + address: 0.0.0.0 + port: 33145 + kafka_api: + - address: 0.0.0.0 + port: 9092 + admin: + - address: 0.0.0.0 + port: 9644 + developer_mode: true +rpk: + enable_usage_stats: false + tune_network: false + tune_disk_scheduler: false + tune_disk_nomerges: false + tune_disk_write_cache: false + tune_disk_irq: false + tune_fstrim: false + tune_cpu: false + tune_aio_events: false + tune_clocksource: false + tune_swappiness: false + tune_transparent_hugepages: false + enable_memory_locking: false + tune_coredump: false + coredump_dir: /var/lib/redpanda/coredump + tune_ballast_file: false + overprovisioned: false +pandaproxy: {} +schema_registry: {} +`, + args: []string{"redpanda.rack", "redpanda-rack"}, + }, + { + name: "set with loaded config", + cfgFile: `config_file: /etc/redpanda/redpanda.yaml +redpanda: + data_directory: "" + node_id: 0 + rack: redpanda-rack + seed_servers: [] + rpc_server: + address: 0.0.0.0 + port: 33145 + kafka_api: + - address: 0.0.0.0 + port: 9092 + admin: + - address: 0.0.0.0 + port: 9644 + developer_mode: true +rpk: + enable_usage_stats: false + tune_network: false + tune_disk_scheduler: false + tune_disk_nomerges: false + tune_disk_write_cache: false + tune_disk_irq: false + tune_fstrim: false + tune_cpu: false + tune_aio_events: false + tune_clocksource: false + tune_swappiness: false + tune_transparent_hugepages: false + enable_memory_locking: false + tune_coredump: false + tune_ballast_file: false + overprovisioned: false +`, + exp: `config_file: /etc/redpanda/redpanda.yaml +redpanda: + node_id: 0 + rack: redpanda-rack + seed_servers: [] + rpc_server: + address: 0.0.0.0 + port: 33145 + kafka_api: + - address: 0.0.0.0 + port: 9092 + admin: + - address: 0.0.0.0 + port: 9644 + developer_mode: true +rpk: + enable_usage_stats: true + tune_network: false + tune_disk_scheduler: false + tune_disk_nomerges: false + tune_disk_write_cache: false + tune_disk_irq: false + tune_fstrim: false + tune_cpu: false + tune_aio_events: false + tune_clocksource: false + tune_swappiness: false + tune_transparent_hugepages: false + enable_memory_locking: false + tune_coredump: false + tune_ballast_file: false + overprovisioned: false +`, + args: []string{"rpk.enable_usage_stats", "true"}, + }, + } { + fs := afero.NewMemMapFs() + + // We create a config file in default redpanda location + if test.cfgFile != "" { + err := afero.WriteFile(fs, "/etc/redpanda/redpanda.yaml", []byte(test.cfgFile), 0o644) + if err != nil { + t.Errorf("unexpected failure writing passed config file: %v", err) + } + } + + c := set(fs) + c.SetArgs(test.args) + err := c.Execute() + if err != nil { + t.Errorf("error during command execution: %v", err) + } + + // Read back from that default location and compare. + file, err := afero.ReadFile(fs, "/etc/redpanda/redpanda.yaml") + if err != nil { + t.Errorf("unexpected failure reading config file: %v", err) + } + require.Equal(t, test.exp, string(file)) + } +} diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/mode.go b/src/go/rpk/pkg/cli/cmd/redpanda/mode.go index 4433f0948390..a487737ecfd1 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/mode.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/mode.go @@ -56,6 +56,7 @@ func executeMode(fs afero.Fs, cmd *cobra.Command, mode string) error { if err != nil { return fmt.Errorf("unable to load config: %v", err) } + cfg = cfg.FileOrDefaults() // we modify fields in the raw file without writing env / flag overrides cfg, err = config.SetMode(mode, cfg) if err != nil { return err diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/mode_test.go b/src/go/rpk/pkg/cli/cmd/redpanda/mode_test.go index 5d556bffc132..85eec5c67738 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/mode_test.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/mode_test.go @@ -42,10 +42,6 @@ func fillRpkConfig(path, mode string) *config.Config { Overprovisioned: !val, TuneBallastFile: val, } - // Unset defaults that get added after command execution, needed to compare - // expected config with loaded config. - conf.Rpk.KafkaAPI = config.RpkKafkaAPI{Brokers: []string{"0.0.0.0:9092"}} - conf.Rpk.AdminAPI = config.RpkAdminAPI{Addresses: []string{"127.0.0.1:9644"}} return conf } diff --git a/src/go/rpk/pkg/cli/cmd/redpanda/start.go b/src/go/rpk/pkg/cli/cmd/redpanda/start.go index 565eacb4e21b..29bd6805541e 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/start.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/start.go @@ -152,6 +152,13 @@ func NewStartCommand(fs afero.Fs, launcher rp.Launcher) *cobra.Command { if err != nil { return fmt.Errorf("unable to load config file: %s", err) } + // We set fields in the raw file without writing rpk specific env + // or flag overrides. This command itself has all redpanda specific + // flags installed, and handles redpanda specific env vars itself. + // The magic `--set` flag is what modifies any redpanda.yaml fields. + // Thus, we can ignore any env / flags that would come from rpk + // configuration itself. + cfg = cfg.FileOrDefaults() if len(configKvs) > 0 { if err = setConfig(cfg, configKvs); err != nil { @@ -335,6 +342,11 @@ func NewStartCommand(fs afero.Fs, launcher rp.Launcher) *cobra.Command { sendEnv(fs, env, cfg, !prestartCfg.checkEnabled, err) return err } + + if cfg.Redpanda.Directory == "" { + cfg.Redpanda.Directory = config.Default().Redpanda.Directory + } + checkPayloads, tunerPayloads, err := prestart( fs, rpArgs, 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 ac7a3b536760..cfcfde0a3556 100644 --- a/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go +++ b/src/go/rpk/pkg/cli/cmd/redpanda/start_test.go @@ -14,9 +14,7 @@ package redpanda import ( "bytes" - "net" "os" - "strconv" "testing" "github.com/redpanda-data/redpanda/src/go/rpk/pkg/config" @@ -194,10 +192,6 @@ func TestStartCommand(t *testing.T) { path, ) c := config.Default() - // Adding unset default that get added on first load. - b0 := c.Redpanda.KafkaAPI[0] - c.Rpk.KafkaAPI.Brokers = []string{net.JoinHostPort(b0.Address, strconv.Itoa(b0.Port))} - c.Rpk.AdminAPI.Addresses = []string{"127.0.0.1:9644"} conf, err := new(config.Params).Load(fs) require.NoError(st, err) @@ -441,6 +435,27 @@ func TestStartCommand(t *testing.T) { // Check that the generated config is as expected. require.Exactly(st, config.Default().Redpanda.ID, conf.Redpanda.ID) }, + }, { + name: "it should write default data_directory if loaded config doesn't have one", + args: []string{ + "--config", config.Default().ConfigFile, + "--install-dir", "/var/lib/redpanda", + }, + before: func(fs afero.Fs) error { + conf := config.Default() + conf.Redpanda.Directory = "" + return conf.Write(fs) + }, + postCheck: func( + fs afero.Fs, + _ *redpanda.RedpandaArgs, + st *testing.T, + ) { + conf, err := new(config.Params).Load(fs) + require.NoError(st, err) + // Check that the generated config is as expected. + require.Exactly(st, config.Default().Redpanda.Directory, conf.Redpanda.Directory) + }, }, { name: "it should leave redpanda.node_id untouched if --node-id wasn't passed", args: []string{ diff --git a/src/go/rpk/pkg/config/config.go b/src/go/rpk/pkg/config/config.go index 0881da098d61..06d439df26ca 100644 --- a/src/go/rpk/pkg/config/config.go +++ b/src/go/rpk/pkg/config/config.go @@ -143,6 +143,21 @@ func AvailableModes() []string { } } +// FileOrDefaults return the configuration as read from the file or +// the default configuration if there is no file loaded. +func (c *Config) FileOrDefaults() *Config { + if c.File() != nil { + cfg := c.File() + cfg.loadedPath = c.loadedPath + cfg.ConfigFile = c.ConfigFile // preserve loaded ConfigFile property. + return cfg + } else { + cfg := Default() + cfg.ConfigFile = c.ConfigFile + return cfg // no file, write the defaults + } +} + // Check checks if the redpanda and rpk configuration is valid before running // the tuners. See: redpanda_checkers. func (c *Config) Check() (bool, []error) { diff --git a/src/go/rpk/pkg/config/params_test.go b/src/go/rpk/pkg/config/params_test.go index 111a1ebaf189..e3de9f09430f 100644 --- a/src/go/rpk/pkg/config/params_test.go +++ b/src/go/rpk/pkg/config/params_test.go @@ -56,7 +56,6 @@ redpanda: }, exp: `config_file: /etc/redpanda/redpanda.yaml redpanda: - data_directory: "" node_id: 6 rack: my_rack `, diff --git a/src/go/rpk/pkg/config/schema.go b/src/go/rpk/pkg/config/schema.go index c3440460a4f1..f3e7ebfdd5be 100644 --- a/src/go/rpk/pkg/config/schema.go +++ b/src/go/rpk/pkg/config/schema.go @@ -44,7 +44,7 @@ func (c *Config) File() *Config { } type RedpandaConfig struct { - Directory string `yaml:"data_directory" json:"data_directory"` + Directory string `yaml:"data_directory,omitempty" json:"data_directory"` ID int `yaml:"node_id" json:"node_id"` Rack string `yaml:"rack,omitempty" json:"rack"` SeedServers []SeedServer `yaml:"seed_servers" json:"seed_servers"` diff --git a/tests/rptest/tests/rpk_config_test.py b/tests/rptest/tests/rpk_config_test.py index 78711b6f8c4a..ab42007b1431 100644 --- a/tests/rptest/tests/rpk_config_test.py +++ b/tests/rptest/tests/rpk_config_test.py @@ -53,15 +53,9 @@ def test_config_init(self): port: 9644 developer_mode: true rpk: - admin_api: - addresses: - - 127.0.0.1:9644 coredump_dir: /var/lib/redpanda/coredump enable_memory_locking: false enable_usage_stats: false - kafka_api: - brokers: - - 0.0.0.0:9092 overprovisioned: false tune_aio_events: false tune_ballast_file: false @@ -182,9 +176,6 @@ def test_config_set_json(self): rpk.config_set(key, value, format='json') expected_config = yaml.full_load(''' -admin_api: - addresses: - - 127.0.0.1:9644 coredump_dir: /var/lib/redpanda/coredump enable_memory_locking: false enable_usage_stats: false @@ -210,12 +201,6 @@ def test_config_set_json(self): with open(os.path.join(d, 'redpanda.yaml')) as f: actual_config = yaml.full_load(f.read()) - assert actual_config['rpk']['kafka_api'] is not None - - # Delete 'kafka_api' so they can be compared since the - # brokers change depending on the container it's running - del actual_config['rpk']['kafka_api'] - if actual_config['rpk'] != expected_config: self.logger.error("Configs differ") self.logger.error( @@ -269,12 +254,6 @@ def test_config_change_mode_prod(self): with open(os.path.join(d, 'redpanda.yaml')) as f: actual_config = yaml.full_load(f.read()) - # Delete 'admin_api' and 'kafka_api' since they are not - # needed for this test and the brokers change depending - # on the container it's running. - del actual_config['rpk']['kafka_api'] - del actual_config['rpk']['admin_api'] - if actual_config['rpk'] != expected_config: self.logger.error("Configs differ") self.logger.error(