Skip to content

Commit

Permalink
Merge pull request #5709 from r-vasquez/remove-config_file
Browse files Browse the repository at this point in the history
rpk: remove config_file from redpanda.yaml
  • Loading branch information
r-vasquez committed Aug 2, 2022
2 parents 8911f57 + 0a425b7 commit 5d623ae
Show file tree
Hide file tree
Showing 16 changed files with 97 additions and 131 deletions.
6 changes: 3 additions & 3 deletions src/go/k8s/pkg/resources/configmap_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ func TestEnsureConfigMap_AdditionalConfig(t *testing.T) {
name: "Primitive object in additional configuration",
additionalConfiguration: map[string]string{"redpanda.transactional_id_expiration_ms": "25920000000"},
expectedStrings: []string{"transactional_id_expiration_ms: 25920000000"},
expectedHash: "2bcfa4a6609253c35503587411b445cb",
expectedHash: "62bdc61185e7b81b8f96b400db98ed2f",
},
{
name: "Complex struct in additional configuration",
Expand All @@ -114,15 +114,15 @@ func TestEnsureConfigMap_AdditionalConfig(t *testing.T) {
- address: 0.0.0.0
port: 8081
name: external`},
expectedHash: "42a92bbbe2e3092ac6bc86d705477ec0",
expectedHash: "979cb39eb1bebf515348713f4cd2d6da",
},
{
name: "shadow index cache directory",
expectedStrings: []string{
`cloud_storage_cache_directory: /var/lib/shadow-index-cache`,
`cloud_storage_cache_size: "10737418240"`,
},
expectedHash: "49ddb404391b63a9b604aa57e655406e",
expectedHash: "fad668348b9d5c88dae6f4db3ffe4041",
},
}
for _, tc := range testcases {
Expand Down
2 changes: 1 addition & 1 deletion src/go/rpk/pkg/cli/cmd/iotune.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ func NewIoTuneCmd(fs afero.Fs) *cobra.Command {
command.Flags().StringVar(
&outputFile,
"out",
filepath.Join(filepath.Dir(config.Default().ConfigFile), "io-config.yaml"),
filepath.Join(filepath.Dir(config.DefaultPath), "io-config.yaml"),
"The file path where the IO config will be written",
)
command.Flags().StringSliceVar(&directories,
Expand Down
13 changes: 5 additions & 8 deletions src/go/rpk/pkg/cli/cmd/redpanda/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ func TestBootstrap(t *testing.T) {
return
}
require.NoError(t, err)
_, err = fs.Stat(config.Default().ConfigFile)
_, err = fs.Stat(config.DefaultPath)
require.NoError(t, err)
conf, err := new(config.Params).Load(fs)
require.NoError(t, err)
Expand Down Expand Up @@ -166,7 +166,7 @@ func TestInitNode(t *testing.T) {

bs, err := yaml.Marshal(c)
require.NoError(t, err)
err = afero.WriteFile(fs, c.ConfigFile, bs, 0o644)
err = afero.WriteFile(fs, config.DefaultPath, bs, 0o644)
require.NoError(t, err)

cmd := initNode(fs)
Expand Down Expand Up @@ -196,8 +196,7 @@ func TestSetCommand(t *testing.T) {
}{
{
name: "set without config file on disk",
exp: `config_file: /etc/redpanda/redpanda.yaml
redpanda:
exp: `redpanda:
data_directory: /var/lib/redpanda/data
node_id: 0
rack: redpanda-rack
Expand All @@ -221,8 +220,7 @@ schema_registry: {}
},
{
name: "set with loaded config",
cfgFile: `config_file: /etc/redpanda/redpanda.yaml
redpanda:
cfgFile: `redpanda:
data_directory: data/dir
node_id: 0
rack: redpanda-rack
Expand All @@ -241,8 +239,7 @@ rpk:
tune_network: true
tune_disk_scheduler: true
`,
exp: `config_file: /etc/redpanda/redpanda.yaml
redpanda:
exp: `redpanda:
data_directory: data/dir
node_id: 0
rack: redpanda-rack
Expand Down
2 changes: 1 addition & 1 deletion src/go/rpk/pkg/cli/cmd/redpanda/mode.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ func executeMode(fs afero.Fs, cmd *cobra.Command, mode string) error {
return err
}

fmt.Printf("Writing %q mode defaults to %q\n", mode, cfg.ConfigFile)
fmt.Printf("Writing %q mode defaults to %q\n", mode, cfg.FileLocation())
err = cfg.Write(fs)
if err != nil {
return err
Expand Down
9 changes: 3 additions & 6 deletions src/go/rpk/pkg/cli/cmd/redpanda/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,9 +85,6 @@ const (
)

func updateConfigWithFlags(conf *config.Config, flags *pflag.FlagSet) {
if flags.Changed(configFlag) {
conf.ConfigFile, _ = flags.GetString(configFlag)
}
if flags.Changed(lockMemoryFlag) {
conf.Rpk.EnableMemoryLocking, _ = flags.GetBool(lockMemoryFlag)
}
Expand Down Expand Up @@ -579,7 +576,7 @@ func buildRedpandaFlags(
// If --io-properties-file and --io-properties weren't set, try
// finding an IO props file in the default location.
sFlags.ioPropertiesFile = rp.GetIOConfigPath(
filepath.Dir(conf.ConfigFile),
filepath.Dir(conf.FileLocation()),
)
if exists, _ := afero.Exists(fs, sFlags.ioPropertiesFile); !exists {
sFlags.ioPropertiesFile = ""
Expand Down Expand Up @@ -619,14 +616,14 @@ func buildRedpandaFlags(
" remove it and pass '--%s' directly"+
" to `rpk start`.",
n,
conf.ConfigFile,
conf.FileLocation(),
n,
)
}
finalFlags[n] = fmt.Sprint(v)
}
return &rp.RedpandaArgs{
ConfigFilePath: conf.ConfigFile,
ConfigFilePath: conf.FileLocation(),
SeastarFlags: finalFlags,
}, nil
}
Expand Down
55 changes: 29 additions & 26 deletions src/go/rpk/pkg/cli/cmd/redpanda/start_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,11 +211,11 @@ func TestStartCommand(t *testing.T) {
expectedErrMsg string
}{{
name: "should fail if the config at the given path is corrupt",
args: []string{"--config", config.Default().ConfigFile},
args: []string{"--config", config.DefaultPath},
before: func(fs afero.Fs) error {
return afero.WriteFile(
fs,
config.Default().ConfigFile,
config.DefaultPath,
[]byte("^&notyaml"),
0o755,
)
Expand All @@ -224,11 +224,11 @@ func TestStartCommand(t *testing.T) {
}, {
name: "should generate the config at the given path if it doesn't exist",
args: []string{
"--config", config.Default().ConfigFile,
"--config", config.DefaultPath,
"--install-dir", "/var/lib/redpanda",
},
postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) {
path := config.Default().ConfigFile
path := config.DefaultPath
exists, err := afero.Exists(
fs,
path,
Expand Down Expand Up @@ -260,7 +260,10 @@ func TestStartCommand(t *testing.T) {
p := &config.Params{ConfigPath: "/arbitrary/path/redpanda.yaml"} // In command execution this will be done by with ParamsFromCommand
conf, err := p.Load(fs)
require.NoError(st, err)
require.Exactly(st, path, conf.ConfigFile)
require.Exactly(st, path, conf.FileLocation())
exists, err := afero.Exists(fs, testConfigPath)
require.NoError(st, err)
require.True(st, exists)
},
}, {
name: "it should allow passing arbitrary config values and write them to the config file",
Expand Down Expand Up @@ -444,7 +447,7 @@ func TestStartCommand(t *testing.T) {
postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) {
conf, err := new(config.Params).Load(fs)
require.NoError(st, err)
require.Exactly(st, config.Default().ConfigFile, conf.ConfigFile)
require.Exactly(st, config.DefaultPath, conf.FileLocation())
},
}, {
name: "it should leave config_file untouched if --config wasn't passed",
Expand All @@ -458,13 +461,13 @@ func TestStartCommand(t *testing.T) {
postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) {
conf, err := new(config.Params).Load(fs)
require.NoError(st, err)
require.Exactly(st, config.Default().ConfigFile, conf.ConfigFile)
require.Exactly(st, config.DefaultPath, conf.FileLocation())
},
}, {
name: "it should write the given node ID",
args: []string{
"--node-id", "34",
"--config", config.Default().ConfigFile,
"--config", config.DefaultPath,
"--install-dir", "/var/lib/redpanda",
},
postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) {
Expand All @@ -475,7 +478,7 @@ func TestStartCommand(t *testing.T) {
}, {
name: "it should write the default node ID if --node-id isn't passed and the config file doesn't exist",
args: []string{
"--config", config.Default().ConfigFile,
"--config", config.DefaultPath,
"--install-dir", "/var/lib/redpanda",
},
postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) {
Expand All @@ -487,7 +490,7 @@ func TestStartCommand(t *testing.T) {
}, {
name: "it should write default data_directory if loaded config doesn't have one",
args: []string{
"--config", config.Default().ConfigFile,
"--config", config.DefaultPath,
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
Expand All @@ -511,7 +514,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Redpanda.ID = 98
return conf.Write(fs)
},
Expand All @@ -528,7 +531,7 @@ func TestStartCommand(t *testing.T) {
name: "--well-known-io should override rpk.well_known_io",
args: []string{
"--well-known-io", "aws:i3xlarge:default",
"--config", config.Default().ConfigFile,
"--config", config.DefaultPath,
"--install-dir", "/var/lib/redpanda",
},
postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) {
Expand All @@ -543,7 +546,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Rpk.WellKnownIo = "gcp:n2standard:ssd"
return conf.Write(fs)
},
Expand All @@ -562,7 +565,7 @@ func TestStartCommand(t *testing.T) {
// Bool flags will be true by just being present. Therefore, to
// change their value, <flag>=<value> needs to be used
"--overprovisioned=false",
"--config", config.Default().ConfigFile,
"--config", config.DefaultPath,
"--install-dir", "/var/lib/redpanda",
},
postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) {
Expand Down Expand Up @@ -594,7 +597,7 @@ func TestStartCommand(t *testing.T) {
name: "--lock-memory should override the default value for rpk.enable_memory_locking",
args: []string{
"--lock-memory",
"--config", config.Default().ConfigFile,
"--config", config.DefaultPath,
"--install-dir", "/var/lib/redpanda",
},
postCheck: func(fs afero.Fs, _ *redpanda.RedpandaArgs, st *testing.T) {
Expand All @@ -610,7 +613,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Rpk.EnableMemoryLocking = true
return conf.Write(fs)
},
Expand Down Expand Up @@ -728,7 +731,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Redpanda.SeedServers = []config.SeedServer{{
Host: config.SocketAddress{
Address: "10.23.12.5",
Expand Down Expand Up @@ -844,7 +847,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Redpanda.RPCServer = config.SocketAddress{
Address: "192.168.33.33",
Port: 9892,
Expand Down Expand Up @@ -994,7 +997,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Redpanda.KafkaAPI = []config.NamedAuthNSocketAddress{{
Address: "192.168.33.33",
Port: 9892,
Expand Down Expand Up @@ -1094,7 +1097,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Redpanda.AdvertisedKafkaAPI = []config.NamedSocketAddress{{
Address: "192.168.33.33",
Port: 9892,
Expand Down Expand Up @@ -1240,7 +1243,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Redpanda.AdvertisedRPCAPI = &config.SocketAddress{
Address: "192.168.33.33",
Port: 9892,
Expand All @@ -1267,7 +1270,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda", "--overprovisioned",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Rpk.AdditionalStartFlags = []string{"--overprovisioned"}
return conf.Write(fs)
},
Expand All @@ -1278,7 +1281,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda", "--smp", "1",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Rpk.AdditionalStartFlags = []string{"--smp=1"}
return conf.Write(fs)
},
Expand All @@ -1289,7 +1292,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda", "--memory", "2G",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Rpk.AdditionalStartFlags = []string{"--memory=1G"}
return conf.Write(fs)
},
Expand All @@ -1300,7 +1303,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Rpk.AdditionalStartFlags = []string{
"--smp=3", "--smp=55",
}
Expand All @@ -1319,7 +1322,7 @@ func TestStartCommand(t *testing.T) {
"--install-dir", "/var/lib/redpanda",
},
before: func(fs afero.Fs) error {
conf := config.Default()
conf, _ := new(config.Params).Load(fs)
conf.Rpk.AdditionalStartFlags = []string{
"--logger-log-level=archival=debug:cloud_storage=debug",
}
Expand Down
2 changes: 1 addition & 1 deletion src/go/rpk/pkg/cli/cmd/redpanda/stop_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ func TestStopCommand(t *testing.T) {

var out bytes.Buffer
c := cmd.NewStopCommand(fs)
args := append([]string{"--config", conf.ConfigFile}, tt.args...)
args := append([]string{"--config", conf.FileLocation()}, tt.args...)
c.SetArgs(args)

logrus.SetOutput(&out)
Expand Down
12 changes: 6 additions & 6 deletions src/go/rpk/pkg/config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ const (

func Default() *Config {
return &Config{
ConfigFile: "/etc/redpanda/redpanda.yaml",
fileLocation: DefaultPath,
Redpanda: RedpandaConfig{
Directory: "/var/lib/redpanda/data",
RPCServer: SocketAddress{
Expand Down Expand Up @@ -147,13 +147,13 @@ func AvailableModes() []string {
// 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
return c.File()
} else {
cfg := Default()
cfg.ConfigFile = c.ConfigFile
// --config set but the file doesn't exist yet:
if c.fileLocation != "" {
cfg.fileLocation = c.fileLocation
}
return cfg // no file, write the defaults
}
}
Expand Down
Loading

0 comments on commit 5d623ae

Please sign in to comment.