Skip to content

Commit

Permalink
Add ability to merge users.txt file into bootstrap.yaml (#265)
Browse files Browse the repository at this point in the history
This adds a routine to merge the supuser secrets into the bootstrap.yaml read when performing post install/update jobs. This is needed due to the bootstrap user additions added in [this commit](redpanda-data/helm-charts@081c08b#diff-84c6555620e4e5f79262384a9fa3e8f4876b36bb3a64748cbd8fbdcb66e8c1b9R247) causing redpanda-data/helm-charts#1566.

What is essentially occuring is this:

1. When creating a bootstrap user via the `REDPANDA_BOOTSTRAP_USER` environment variable said user isn't marked automatically as a superuser
2. If you want to use something like `admin_api_require_auth` then the _only_ way you can actually run management operations on your installation is to pass in the above user in to the nodes' `bootstrap.yaml` so that the user is immediately marked as a superuser and all of the config-watcher scripts that manage other specified users can leverage it to create the rest of the users.
3. The bootstrap user changes that added the user to `bootstrap.yaml` uncovered that setting any sort of `superusers` values in the bootstrap.yaml is incompatible with the users created by a pre-existing users secret. This is due to the `superusers` entry found in the bootstrap.yaml not containing them. When an upgrade finishes the jobs reset the configuration to only contain what is found in `superusers` without regard to anything managed by the pre-existing secret/config-watcher.
4. The above manifested in all users from a pre-existing secret getting unmarked as superusers when an upgrade occurred. Restarting any StatefulSet pod clears this up.

Since we _must_ still set the bootstrap user in the `superusers` section of `bootstrap.yaml` in order for `admin_api_require_auth` to function correctly on installations, this makes the config synchronization code aware of our users secret.

It needs to be coupled with a change in the `helm-charts` code to add in a `--users-directory` flag as needed and an additional secrets mount to mount the users secret into our job containers.
  • Loading branch information
andrewstucki authored Oct 16, 2024
1 parent 60373ad commit deb4a85
Show file tree
Hide file tree
Showing 2 changed files with 143 additions and 6 deletions.
119 changes: 117 additions & 2 deletions operator/cmd/syncclusterconfig/sync.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,15 @@
package syncclusterconfig

import (
"bufio"
"bytes"
"fmt"
"os"
"path/filepath"
"sort"
"strings"

"github.com/go-logr/logr"
"github.com/redpanda-data/common-go/rpadmin"
rpkadminapi "github.com/redpanda-data/redpanda/src/go/rpk/pkg/adminapi"
rpkconfig "github.com/redpanda-data/redpanda/src/go/rpk/pkg/config"
Expand All @@ -31,15 +36,17 @@ import (
)

const (
licenseEnvvar = "REDPANDA_LICENSE"
licenseEnvvar = "REDPANDA_LICENSE"
superusersEntry = "superusers"
)

func Command() *cobra.Command {
var usersDirectoryPath string
var redpandaYAMLPath string
var bootstrapYAMLPath string

cmd := &cobra.Command{
Use: "sync-cluster-config [--bootstrap-yaml file] [--redpanda-yaml file]",
Use: "sync-cluster-config [--bootstrap-yaml file] [--redpanda-yaml file] [--users-directory file]",
Long: fmt.Sprintf(`sync-cluster-config patches a cluster's configuration with values from the provided bootstrap.yaml.
If present and not empty, the $%s environment variable will be set as the cluster's license.
`, licenseEnvvar),
Expand Down Expand Up @@ -67,6 +74,9 @@ If present and not empty, the $%s environment variable will be set as the cluste
return err
}

// do conditional merge of users.txt and bootstrap.yaml
maybeMergeSuperusers(logger, clusterConfig, usersDirectoryPath)

// NB: remove must be an empty slice NOT nil.
result, err := client.PatchClusterConfig(ctx, clusterConfig, []string{})
if err != nil {
Expand All @@ -88,6 +98,7 @@ If present and not empty, the $%s environment variable will be set as the cluste
},
}

cmd.Flags().StringVar(&usersDirectoryPath, "users-directory", "/etc/secrets/users/", "Path to users directory where secrets are mounted")
cmd.Flags().StringVar(&redpandaYAMLPath, "redpanda-yaml", "/etc/redpanda/redpanda.yaml", "Path to redpanda.yaml")
cmd.Flags().StringVar(&bootstrapYAMLPath, "bootstrap-yaml", "/etc/redpanda/.bootstrap.yaml", "Path to .bootstrap.yaml")

Expand Down Expand Up @@ -125,3 +136,107 @@ func loadBoostrapYAML(path string) (map[string]any, error) {

return config, nil
}

// maybeMergeSuperusers merges the values found in users.txt into our superusers
// list found in our bootstrap.yaml. This only occurs if an entry for superusers
// actually exists in the bootstrap.yaml.
func maybeMergeSuperusers(logger logr.Logger, clusterConfig map[string]any, path string) {
if path == "" {
// we have no path to a users directory, so don't do anything
logger.Info("--users-directory not specified. Skipping superusers merge.")
return
}

superusersConfig, ok := clusterConfig[superusersEntry]
if !ok {
// we have no superusers configuration, so don't do anything
logger.Info("Configuration does not contain a 'superusers' entry. Skipping superusers merge.")
return
}

superusers, err := loadUsersFiles(logger, path)
if err != nil {
logger.Info(fmt.Sprintf("Error reading users directory %q: %v. Skipping superusers merge.", path, err))
return
}

superusersAny, ok := superusersConfig.([]any)
if !ok {
logger.Info(fmt.Sprintf("Unable to cast superusers entry to array. Skipping superusers merge. Type is: %T", superusersConfig))
return
}

clusterConfig[superusersEntry] = normalizeSuperusers(append(superusers, mapConvertibleTo[string](logger, superusersAny)...))
}

func loadUsersFiles(logger logr.Logger, path string) ([]string, error) {
files, err := os.ReadDir(path)
if err != nil {
return nil, err
}

users := []string{}

for _, file := range files {
if file.IsDir() {
continue
}

filename := filepath.Join(path, file.Name())

usersFile, err := os.ReadFile(filename)
if err != nil {
logger.Info(fmt.Sprintf("Cannot read user file %q: %v. Skipping.", filename, err))
continue
}

scanner := bufio.NewScanner(bytes.NewReader(usersFile))

i := 0
for scanner.Scan() {
i++

line := scanner.Text()
tokens := strings.Split(line, ":")
if len(tokens) != 2 && len(tokens) != 3 {
logger.Info(fmt.Sprintf("Skipping malformatted line number %d in file %q", i, filename))
continue
}
users = append(users, tokens[0])
}
}

return users, nil
}

// normalizeSuperusers de-duplicates and sorts the superusers
func normalizeSuperusers(entries []string) []string {
var sorted sort.StringSlice

unique := make(map[string]struct{})
for _, value := range entries {
if _, ok := unique[value]; !ok {
sorted = append(sorted, value)
}
unique[value] = struct{}{}
}

sorted.Sort()

return sorted
}

func mapConvertibleTo[T any](logger logr.Logger, array []any) []T {
var v T

converted := []T{}
for _, value := range array {
if cast, ok := value.(T); ok {
converted = append(converted, cast)
} else {
logger.Info("Unable to cast value from %T to %T, skipping.", value, v)
}
}

return converted
}
30 changes: 26 additions & 4 deletions operator/cmd/syncclusterconfig/sync_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@ import (
"context"
"encoding/json"
"fmt"
"path/filepath"
"strings"
"testing"

"github.com/go-logr/logr/testr"
Expand All @@ -27,6 +29,7 @@ import (

func TestSync(t *testing.T) {
const user = "syncer"
const admin = "admin"
const password = "password"
const saslMechanism = "SCRAM-SHA-256"

Expand Down Expand Up @@ -81,18 +84,36 @@ func TestSync(t *testing.T) {
require.NoError(t, err)

redpandaYAMLPath := testutils.WriteFile(t, "redpanda-*.yaml", rpkConfigBytes)
usersTxtYAMLPath := testutils.WriteFile(t, "users-*.txt", []byte(strings.Join([]string{admin, password, saslMechanism}, ":")))

cases := []struct {
Config map[string]any
Expected map[string]any
}{
{
// No superusers entry, the value just gets pulled from
// what we initialized in our redpanda.Run call above.
Config: map[string]any{},
Expected: map[string]any{
"admin_api_require_auth": true,
"superusers": []any{user},
},
},
{
// Passing a superusers entry to show that the value from
// the users.txt gets merged.
//
// Note that all subsequent runs build off of previous runs
// in this test, so all subsequent test cases will have both
// superusers.
Config: map[string]any{
"superusers": []string{user},
},
Expected: map[string]any{
"admin_api_require_auth": true,
"superusers": []any{admin, user},
},
},
{
Config: map[string]any{
"abort_index_segment_size": 10,
Expand All @@ -102,7 +123,7 @@ func TestSync(t *testing.T) {
"abort_index_segment_size": 10,
"admin_api_require_auth": true,
"audit_queue_drain_interval_ms": 60,
"superusers": []any{user},
"superusers": []any{admin, user},
},
},
{
Expand All @@ -116,7 +137,7 @@ func TestSync(t *testing.T) {
"abort_index_segment_size": 10,
"admin_api_require_auth": true,
"audit_queue_drain_interval_ms": 60,
"superusers": []any{user},
"superusers": []any{admin, user},
},
},
{
Expand All @@ -127,7 +148,7 @@ func TestSync(t *testing.T) {
"abort_index_segment_size": 10,
"admin_api_require_auth": true,
"audit_queue_drain_interval_ms": 70,
"superusers": []any{user},
"superusers": []any{admin, user},
},
},
{
Expand All @@ -139,7 +160,7 @@ func TestSync(t *testing.T) {
Expected: map[string]any{
"abort_index_segment_size": 10,
"audit_queue_drain_interval_ms": 70,
"superusers": []any{user},
"superusers": []any{admin, user},
},
},
}
Expand All @@ -150,6 +171,7 @@ func TestSync(t *testing.T) {

cmd := Command()
cmd.SetArgs([]string{
"--users-directory", filepath.Dir(usersTxtYAMLPath),
"--redpanda-yaml", redpandaYAMLPath,
"--bootstrap-yaml", testutils.WriteFile(t, "bootstrap-*.yaml", configBytes),
})
Expand Down

0 comments on commit deb4a85

Please sign in to comment.