Password rotation in secrets (#1749)

* password rotation in K8s secrets
* add db connection to syncSecrets
* add user retention
* add e2e test
* cleanup on username mismatch if rotation was switched off
* add unit test for syncSecrets + new updateSecret func
This commit is contained in:
Felix Kunde 2022-02-18 11:54:47 +01:00 committed by GitHub
parent 95301c102e
commit 658923d10d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 674 additions and 62 deletions

View File

@ -122,6 +122,15 @@ spec:
users:
type: object
properties:
enable_password_rotation:
type: boolean
default: false
password_rotation_interval:
type: integer
default: 90
password_rotation_user_retention:
type: integer
default: 180
replication_username:
type: string
default: standby

View File

@ -551,6 +551,16 @@ spec:
- SUPERUSER
- nosuperuser
- NOSUPERUSER
usersWithPasswordRotation:
type: array
nullable: true
items:
type: string
usersWithInPlacePasswordRotation:
type: array
nullable: true
items:
type: string
volume:
type: object
required:

View File

@ -293,6 +293,84 @@ that are aggregated into the K8s [default roles](https://kubernetes.io/docs/refe
For Helm deployments setting `rbac.createAggregateClusterRoles: true` adds these clusterroles to the deployment.
## Password rotation in K8s secrets
The operator regularly updates credentials in the K8s secrets if the
`enable_password_rotation` option is set to `true` in the configuration.
It happens only for `LOGIN` roles with an associated secret (manifest roles,
default users from `preparedDatabases`). Furthermore, there are the following
exceptions:
1. Infrastructure role secrets since rotation should happen by the infrastructure.
2. Team API roles that connect via OAuth2 and JWT token (no secrets to these roles anyway).
3. Database owners since ownership on database objects can not be inherited.
4. System users such as `postgres`, `standby` and `pooler` user.
The interval of days can be set with `password_rotation_interval` (default
`90` = 90 days, minimum 1). On each rotation the user name and password values
are replaced in the K8s secret. They belong to a newly created user named after
the original role plus rotation date in YYMMDD format. All priviliges are
inherited meaning that migration scripts should still grant and revoke rights
against the original role. The timestamp of the next rotation is written to the
secret as well. Note, if the rotation interval is decreased it is reflected in
the secrets only if the next rotation date is more days away than the new
length of the interval.
Pods still using the previous secret values which they keep in memory continue
to connect to the database since the password of the corresponding user is not
replaced. However, a retention policy can be configured for users created by
the password rotation feature with `password_rotation_user_retention`. The
operator will ensure that this period is at least twice as long as the
configured rotation interval, hence the default of `180` = 180 days. When
the creation date of a rotated user is older than the retention period it
might not get removed immediately. Only on the next user rotation it is checked
if users can get removed. Therefore, you might want to configure the retention
to be a multiple of the rotation interval.
### Password rotation for single users
From the configuration, password rotation is enabled for all secrets with the
mentioned exceptions. If you wish to first test rotation for a single user (or
just have it enabled only for a few secrets) you can specify it in the cluster
manifest. The rotation and retention intervals can only be configured globally.
```
spec:
usersWithSecretRotation:
- foo_user
- bar_reader_user
```
### Password replacement without extra users
For some use cases where the secret is only used rarely - think of a `flyway`
user running a migration script on pod start - we do not need to create extra
database users but can replace only the password in the K8s secret. This type
of rotation cannot be configured globally but specified in the cluster
manifest:
```
spec:
usersWithInPlaceSecretRotation:
- flyway
- bar_owner_user
```
This would be the recommended option to enable rotation in secrets of database
owners, but only if they are not used as application users for regular read
and write operations.
### Turning off password rotation
When password rotation is turned off again the operator will check if the
`username` value in the secret matches the original username and replace it
with the latter. A new password is assigned and the `nextRotation` field is
cleared. A final lookup for child (rotation) users to be removed is done but
they will only be dropped if the retention policy allows for it. This is to
avoid sudden connection issues in pods which still use credentials of these
users in memory. You have to remove these child users manually or re-enable
password rotation with smaller interval so they get cleaned up.
## Use taints and tolerations for dedicated PostgreSQL nodes
To ensure Postgres pods are running on nodes without any other application pods,

View File

@ -115,6 +115,22 @@ These parameters are grouped directly under the `spec` key in the manifest.
create the K8s secret in that namespace. The part after the first `.` is
considered to be the user name. Optional.
* **usersWithSecretRotation**
list of users to enable credential rotation in K8s secrets. The rotation
interval can only be configured globally. On each rotation a new user will
be added in the database replacing the `username` value in the secret of
the listed user. Although, rotation users inherit all rights from the
original role, keep in mind that ownership is not transferred. See more
details in the [administrator docs](https://github.com/zalando/postgres-operator/blob/master/docs/administrator.md#password-rotation-in-k8s-secrets).
* **usersWithInPlaceSecretRotation**
list of users to enable in-place password rotation in K8s secrets. The
rotation interval can only be configured globally. On each rotation the
password value will be replaced in the secrets which the operator reflects
in the database, too. List only users here that rarely connect to the
database, like a flyway user running a migration on Pod start. See more
details in the [administrator docs](https://github.com/zalando/postgres-operator/blob/master/docs/administrator.md#password-replacement-without-extra-users).
* **databases**
a map of database names to database owners for the databases that should be
created by the operator. The owner users should already exist on the cluster

View File

@ -174,6 +174,28 @@ under the `users` key.
Postgres username used for replication between instances. The default is
`standby`.
* **enable_password_rotation**
For all `LOGIN` roles that are not database owners the operator can rotate
credentials in the corresponding K8s secrets by replacing the username and
password. This means, new users will be added on each rotation inheriting
all priviliges from the original roles. The rotation date (in YYMMDD format)
is appended to the names of the new user. The timestamp of the next rotation
is written to the secret. The default is `false`.
* **password_rotation_interval**
If password rotation is enabled (either from config or cluster manifest) the
interval can be configured with this parameter. The measure is in days which
means daily rotation (`1`) is the most frequent interval possible.
Default is `90`.
* **password_rotation_user_retention**
To avoid an ever growing amount of new users due to password rotation the
operator will remove the created users again after a certain amount of days
has passed. The number can be configured with this parameter. However, the
operator will check that the retention policy is at least twice as long as
the rotation interval and update to this minimum in case it is not.
Default is `180`.
## Major version upgrades
Parameters configuring automatic major version upgrades. In a

View File

@ -321,6 +321,9 @@ class K8s:
def get_cluster_replica_pod(self, labels='application=spilo,cluster-name=acid-minimal-cluster', namespace='default'):
return self.get_cluster_pod('replica', labels, namespace)
def get_secret_data(self, username, clustername='acid-minimal-cluster', namespace='default'):
return self.api.core_v1.read_namespaced_secret(
"{}.{}.credentials.postgresql.acid.zalan.do".format(username.replace("_","-"), clustername), namespace).data
class K8sBase:
'''

View File

@ -4,8 +4,9 @@ import time
import timeout_decorator
import os
import yaml
import base64
from datetime import datetime
from datetime import datetime, date, timedelta
from kubernetes import client
from tests.k8s_api import K8s
@ -579,6 +580,7 @@ class EndToEndTestCase(unittest.TestCase):
"Parameters": None,
"AdminRole": "",
"Origin": 2,
"IsDbOwner": False,
"Deleted": False
})
return True
@ -600,7 +602,6 @@ class EndToEndTestCase(unittest.TestCase):
but lets pods run with the old image until they are recreated for
reasons other than operator's activity. That works because the operator
configures stateful sets to use "onDelete" pod update policy.
The test covers:
1) enabling lazy upgrade in existing operator deployment
2) forcing the normal rolling upgrade by changing the operator
@ -695,7 +696,6 @@ class EndToEndTestCase(unittest.TestCase):
Ensure we can (a) create the cron job at user request for a specific PG cluster
(b) update the cluster-wide image for the logical backup pod
(c) delete the job at user request
Limitations:
(a) Does not run the actual batch job because there is no S3 mock to upload backups to
(b) Assumes 'acid-minimal-cluster' exists as defined in setUp
@ -1074,6 +1074,122 @@ class EndToEndTestCase(unittest.TestCase):
self.eventuallyEqual(lambda: k8s.count_running_pods("connection-pooler=acid-minimal-cluster-pooler"),
0, "Pooler pods not scaled down")
@timeout_decorator.timeout(TEST_TIMEOUT_SEC)
def test_password_rotation(self):
'''
Test password rotation and removal of users due to retention policy
'''
k8s = self.k8s
leader = k8s.get_cluster_leader_pod()
today = date.today()
# enable password rotation for owner of foo database
pg_patch_inplace_rotation_for_owner = {
"spec": {
"usersWithInPlaceSecretRotation": [
"zalando"
]
}
}
k8s.api.custom_objects_api.patch_namespaced_custom_object(
"acid.zalan.do", "v1", "default", "postgresqls", "acid-minimal-cluster", pg_patch_inplace_rotation_for_owner)
self.eventuallyEqual(lambda: k8s.get_operator_state(), {"0": "idle"}, "Operator does not get in sync")
# check if next rotation date was set in secret
secret_data = k8s.get_secret_data("zalando")
next_rotation_timestamp = datetime.fromisoformat(str(base64.b64decode(secret_data["nextRotation"]), 'utf-8'))
today90days = today+timedelta(days=90)
self.assertEqual(today90days, next_rotation_timestamp.date(),
"Unexpected rotation date in secret of zalando user: expected {}, got {}".format(today90days, next_rotation_timestamp.date()))
# create fake rotation users that should be removed by operator
# but have one that would still fit into the retention period
create_fake_rotation_user = """
CREATE ROLE foo_user201031 IN ROLE foo_user;
CREATE ROLE foo_user211031 IN ROLE foo_user;
CREATE ROLE foo_user"""+(today-timedelta(days=40)).strftime("%y%m%d")+""" IN ROLE foo_user;
"""
self.query_database(leader.metadata.name, "postgres", create_fake_rotation_user)
# patch foo_user secret with outdated rotation date
fake_rotation_date = today.isoformat() + ' 00:00:00'
fake_rotation_date_encoded = base64.b64encode(fake_rotation_date.encode('utf-8'))
secret_fake_rotation = {
"data": {
"nextRotation": str(fake_rotation_date_encoded, 'utf-8'),
},
}
k8s.api.core_v1.patch_namespaced_secret(
name="foo-user.acid-minimal-cluster.credentials.postgresql.acid.zalan.do",
namespace="default",
body=secret_fake_rotation)
# enable password rotation for all other users (foo_user)
# this will force a sync of secrets for further assertions
enable_password_rotation = {
"data": {
"enable_password_rotation": "true",
"password_rotation_interval": "30",
"password_rotation_user_retention": "30", # should be set to 60
},
}
k8s.update_config(enable_password_rotation)
self.eventuallyEqual(lambda: k8s.get_operator_state(), {"0": "idle"},
"Operator does not get in sync")
# check if next rotation date and username have been replaced
secret_data = k8s.get_secret_data("foo_user")
secret_username = str(base64.b64decode(secret_data["username"]), 'utf-8')
next_rotation_timestamp = datetime.fromisoformat(str(base64.b64decode(secret_data["nextRotation"]), 'utf-8'))
rotation_user = "foo_user"+today.strftime("%y%m%d")
today30days = today+timedelta(days=30)
self.assertEqual(rotation_user, secret_username,
"Unexpected username in secret of foo_user: expected {}, got {}".format(rotation_user, secret_username))
self.assertEqual(today30days, next_rotation_timestamp.date(),
"Unexpected rotation date in secret of foo_user: expected {}, got {}".format(today30days, next_rotation_timestamp.date()))
# check if oldest fake rotation users were deleted
# there should only be foo_user, foo_user+today and foo_user+today-40days
user_query = """
SELECT rolname
FROM pg_catalog.pg_roles
WHERE rolname LIKE 'foo_user%';
"""
self.eventuallyEqual(lambda: len(self.query_database(leader.metadata.name, "postgres", user_query)), 3,
"Found incorrect number of rotation users", 10, 5)
# disable password rotation for all other users (foo_user)
# and pick smaller intervals to see if the third fake rotation user is dropped
enable_password_rotation = {
"data": {
"enable_password_rotation": "false",
"password_rotation_interval": "15",
"password_rotation_user_retention": "30", # 2 * rotation interval
},
}
k8s.update_config(enable_password_rotation)
self.eventuallyEqual(lambda: k8s.get_operator_state(), {"0": "idle"},
"Operator does not get in sync")
# check if username in foo_user secret is reset
secret_data = k8s.get_secret_data("foo_user")
secret_username = str(base64.b64decode(secret_data["username"]), 'utf-8')
next_rotation_timestamp = str(base64.b64decode(secret_data["nextRotation"]), 'utf-8')
self.assertEqual("foo_user", secret_username,
"Unexpected username in secret of foo_user: expected {}, got {}".format("foo_user", secret_username))
self.assertEqual('', next_rotation_timestamp,
"Unexpected rotation date in secret of foo_user: expected empty string, got {}".format(next_rotation_timestamp))
# check roles again, there should only be foo_user and foo_user+today
user_query = """
SELECT rolname
FROM pg_catalog.pg_roles
WHERE rolname LIKE 'foo_user%';
"""
self.eventuallyEqual(lambda: len(self.query_database(leader.metadata.name, "postgres", user_query)), 2,
"Found incorrect number of rotation users", 10, 5)
@timeout_decorator.timeout(TEST_TIMEOUT_SEC)
def test_patroni_config_update(self):
'''

View File

@ -16,6 +16,9 @@ spec:
zalando:
- superuser
- createdb
foo_user: []
# usersWithSecretRotation: "foo_user"
# usersWithInPlaceSecretRotation: "flyway,bar_owner_user"
enableMasterLoadBalancer: false
enableReplicaLoadBalancer: false
enableConnectionPooler: false # enable/disable connection pooler deployment

View File

@ -44,6 +44,7 @@ data:
# enable_init_containers: "true"
# enable_lazy_spilo_upgrade: "false"
enable_master_load_balancer: "false"
enable_password_rotation: "false"
enable_pgversion_env_var: "true"
# enable_pod_antiaffinity: "false"
# enable_pod_disruption_budget: "true"
@ -92,6 +93,8 @@ data:
# pam_configuration: |
# https://info.example.com/oauth2/tokeninfo?access_token= uid realm=/employees
# pam_role_name: zalandos
# password_rotation_interval: "90"
# password_rotation_user_retention: "180"
pdb_name_format: "postgres-{cluster}-pdb"
# pod_antiaffinity_topology_key: "kubernetes.io/hostname"
pod_deletion_wait_timeout: 10m

View File

@ -120,6 +120,15 @@ spec:
users:
type: object
properties:
enable_password_rotation:
type: boolean
default: false
password_rotation_interval:
type: integer
default: 90
password_rotation_user_retention:
type: integer
default: 180
replication_username:
type: string
default: standby

View File

@ -25,6 +25,9 @@ configuration:
# protocol: TCP
workers: 8
users:
enable_password_rotation: false
password_rotation_interval: 90
password_rotation_user_retention: 180
replication_username: standby
super_username: postgres
major_version_upgrade:

View File

@ -549,6 +549,16 @@ spec:
- SUPERUSER
- nosuperuser
- NOSUPERUSER
usersWithPasswordRotation:
type: array
nullable: true
items:
type: string
usersWithInPlacePasswordRotation:
type: array
nullable: true
items:
type: string
volume:
type: object
required:

View File

@ -833,6 +833,24 @@ var PostgresCRDResourceValidation = apiextv1.CustomResourceValidation{
},
},
},
"usersWithSecretRotation": {
Type: "array",
Nullable: true,
Items: &apiextv1.JSONSchemaPropsOrArray{
Schema: &apiextv1.JSONSchemaProps{
Type: "string",
},
},
},
"usersWithInPlaceSecretRotation": {
Type: "array",
Nullable: true,
Items: &apiextv1.JSONSchemaPropsOrArray{
Schema: &apiextv1.JSONSchemaProps{
Type: "string",
},
},
},
"volume": {
Type: "object",
Required: []string{"size"},

View File

@ -37,8 +37,11 @@ type OperatorConfigurationList struct {
// PostgresUsersConfiguration defines the system users of Postgres.
type PostgresUsersConfiguration struct {
SuperUsername string `json:"super_username,omitempty"`
ReplicationUsername string `json:"replication_username,omitempty"`
SuperUsername string `json:"super_username,omitempty"`
ReplicationUsername string `json:"replication_username,omitempty"`
EnablePasswordRotation bool `json:"enable_password_rotation,omitempty"`
PasswordRotationInterval uint32 `json:"password_rotation_interval,omitempty"`
PasswordRotationUserRetention uint32 `json:"password_rotation_user_retention,omitempty"`
}
// MajorVersionUpgradeConfiguration defines how to execute major version upgrades of Postgres.

View File

@ -53,8 +53,11 @@ type PostgresSpec struct {
// load balancers' source ranges are the same for master and replica services
AllowedSourceRanges []string `json:"allowedSourceRanges"`
Users map[string]UserFlags `json:"users,omitempty"`
UsersWithSecretRotation []string `json:"usersWithSecretRotation,omitempty"`
UsersWithInPlaceSecretRotation []string `json:"usersWithInPlaceSecretRotation,omitempty"`
NumberOfInstances int32 `json:"numberOfInstances"`
Users map[string]UserFlags `json:"users,omitempty"`
MaintenanceWindows []MaintenanceWindow `json:"maintenanceWindows,omitempty"`
Clone *CloneDescription `json:"clone,omitempty"`
ClusterName string `json:"-"`

View File

@ -641,6 +641,16 @@ func (in *PostgresSpec) DeepCopyInto(out *PostgresSpec) {
(*out)[key] = outVal
}
}
if in.UsersWithSecretRotation != nil {
in, out := &in.UsersWithSecretRotation, &out.UsersWithSecretRotation
*out = make([]string, len(*in))
copy(*out, *in)
}
if in.UsersWithInPlaceSecretRotation != nil {
in, out := &in.UsersWithInPlaceSecretRotation, &out.UsersWithInPlaceSecretRotation
*out = make([]string, len(*in))
copy(*out, *in)
}
if in.MaintenanceWindows != nil {
in, out := &in.MaintenanceWindows, &out.MaintenanceWindows
*out = make([]MaintenanceWindow, len(*in))

View File

@ -711,13 +711,18 @@ func (c *Cluster) Update(oldSpec, newSpec *acidv1.Postgresql) error {
}
}
// connection pooler needs one system user created, which is done in
// initUsers. Check if it needs to be called.
// check if users need to be synced
sameUsers := reflect.DeepEqual(oldSpec.Spec.Users, newSpec.Spec.Users) &&
reflect.DeepEqual(oldSpec.Spec.PreparedDatabases, newSpec.Spec.PreparedDatabases)
sameRotatedUsers := reflect.DeepEqual(oldSpec.Spec.UsersWithSecretRotation, newSpec.Spec.UsersWithSecretRotation) &&
reflect.DeepEqual(oldSpec.Spec.UsersWithInPlaceSecretRotation, newSpec.Spec.UsersWithInPlaceSecretRotation)
// connection pooler needs one system user created, which is done in
// initUsers. Check if it needs to be called.
needConnectionPooler := needMasterConnectionPoolerWorker(&newSpec.Spec) ||
needReplicaConnectionPoolerWorker(&newSpec.Spec)
if !sameUsers || needConnectionPooler {
if !sameUsers || !sameRotatedUsers || needConnectionPooler {
c.logger.Debugf("initialize users")
if err := c.initUsers(); err != nil {
c.logger.Errorf("could not init users: %v", err)
@ -1001,6 +1006,7 @@ func (c *Cluster) initSystemUsers() {
Origin: spec.RoleOriginSystem,
Name: c.OpConfig.ReplicationUsername,
Namespace: c.Namespace,
Flags: []string{constants.RoleFlagLogin},
Password: util.RandomPassword(constants.PasswordLength),
}
@ -1113,7 +1119,6 @@ func (c *Cluster) initPreparedDatabaseRoles() error {
func (c *Cluster) initDefaultRoles(defaultRoles map[string]string, admin, prefix, searchPath, secretNamespace string) error {
for defaultRole, inherits := range defaultRoles {
namespace := c.Namespace
//if namespaced secrets are allowed
if secretNamespace != "" {
@ -1136,8 +1141,10 @@ func (c *Cluster) initDefaultRoles(defaultRoles map[string]string, admin, prefix
}
adminRole := ""
isOwner := false
if strings.Contains(defaultRole, constants.OwnerRoleNameSuffix) {
adminRole = admin
isOwner = true
} else {
adminRole = prefix + constants.OwnerRoleNameSuffix
}
@ -1151,6 +1158,7 @@ func (c *Cluster) initDefaultRoles(defaultRoles map[string]string, admin, prefix
MemberOf: memberOf,
Parameters: map[string]string{"search_path": searchPath},
AdminRole: adminRole,
IsDbOwner: isOwner,
}
if currentRole, present := c.pgUsers[roleName]; present {
c.pgUsers[roleName] = c.resolveNameConflict(&currentRole, &newRole)
@ -1172,6 +1180,14 @@ func (c *Cluster) initRobotUsers() error {
}
namespace := c.Namespace
// check if role is specified as database owner
isOwner := false
for _, owner := range c.Spec.Databases {
if username == owner {
isOwner = true
}
}
//if namespaced secrets are allowed
if c.Config.OpConfig.EnableCrossNamespaceSecret {
if strings.Contains(username, ".") {
@ -1196,6 +1212,7 @@ func (c *Cluster) initRobotUsers() error {
Password: util.RandomPassword(constants.PasswordLength),
Flags: flags,
AdminRole: adminRole,
IsDbOwner: isOwner,
}
if currentRole, present := c.pgUsers[username]; present {
c.pgUsers[username] = c.resolveNameConflict(&currentRole, &newRole)

View File

@ -14,18 +14,25 @@ import (
"github.com/zalando/postgres-operator/pkg/spec"
"github.com/zalando/postgres-operator/pkg/util/constants"
"github.com/zalando/postgres-operator/pkg/util/retryutil"
"github.com/zalando/postgres-operator/pkg/util/users"
)
const (
getUserSQL = `SELECT a.rolname, COALESCE(a.rolpassword, ''), a.rolsuper, a.rolinherit,
a.rolcreaterole, a.rolcreatedb, a.rolcanlogin, s.setconfig,
ARRAY(SELECT b.rolname
FROM pg_catalog.pg_auth_members m
JOIN pg_catalog.pg_authid b ON (m.roleid = b.oid)
WHERE m.member = a.oid) as memberof
FROM pg_catalog.pg_authid a LEFT JOIN pg_db_role_setting s ON (a.oid = s.setrole AND s.setdatabase = 0::oid)
WHERE a.rolname = ANY($1)
ORDER BY 1;`
a.rolcreaterole, a.rolcreatedb, a.rolcanlogin, s.setconfig,
ARRAY(SELECT b.rolname
FROM pg_catalog.pg_auth_members m
JOIN pg_catalog.pg_authid b ON (m.roleid = b.oid)
WHERE m.member = a.oid) as memberof
FROM pg_catalog.pg_authid a LEFT JOIN pg_db_role_setting s ON (a.oid = s.setrole AND s.setdatabase = 0::oid)
WHERE a.rolname = ANY($1)
ORDER BY 1;`
getUsersForRetention = `SELECT r.rolname, right(r.rolname, 6) AS roldatesuffix
FROM pg_roles r
JOIN unnest($1::text[]) AS u(name) ON r.rolname LIKE u.name || '%'
AND right(r.rolname, 6) ~ '^[0-9\.]+$'
ORDER BY 1;`
getDatabasesSQL = `SELECT datname, pg_get_userbyid(datdba) AS owner FROM pg_database;`
getSchemasSQL = `SELECT n.nspname AS dbschema FROM pg_catalog.pg_namespace n
@ -227,6 +234,65 @@ func (c *Cluster) readPgUsersFromDatabase(userNames []string) (users spec.PgUser
return users, nil
}
func findUsersFromRotation(rotatedUsers []string, db *sql.DB) (map[string]string, error) {
extraUsers := make(map[string]string, 0)
rows, err := db.Query(getUsersForRetention, pq.Array(rotatedUsers))
if err != nil {
return nil, fmt.Errorf("query failed: %v", err)
}
defer func() {
if err2 := rows.Close(); err2 != nil {
err = fmt.Errorf("error when closing query cursor: %v", err2)
}
}()
for rows.Next() {
var (
rolname, roldatesuffix string
)
err := rows.Scan(&rolname, &roldatesuffix)
if err != nil {
return nil, fmt.Errorf("error when processing rows of deprecated users: %v", err)
}
extraUsers[rolname] = roldatesuffix
}
return extraUsers, nil
}
func (c *Cluster) cleanupRotatedUsers(rotatedUsers []string, db *sql.DB) error {
c.setProcessName("checking for rotated users to remove from the database due to configured retention")
extraUsers, err := findUsersFromRotation(rotatedUsers, db)
if err != nil {
return fmt.Errorf("error when querying for deprecated users from password rotation: %v", err)
}
// make sure user retention policy aligns with rotation interval
retenionDays := c.OpConfig.PasswordRotationUserRetention
if retenionDays < 2*c.OpConfig.PasswordRotationInterval {
retenionDays = 2 * c.OpConfig.PasswordRotationInterval
c.logger.Warnf("user retention days too few compared to rotation interval %d - setting it to %d", c.OpConfig.PasswordRotationInterval, retenionDays)
}
retentionDate := time.Now().AddDate(0, 0, int(retenionDays)*-1)
for rotatedUser, dateSuffix := range extraUsers {
userCreationDate, err := time.Parse("060102", dateSuffix)
if err != nil {
c.logger.Errorf("could not parse creation date suffix of user %q: %v", rotatedUser, err)
continue
}
if retentionDate.After(userCreationDate) {
c.logger.Infof("dropping user %q due to configured days in password_rotation_user_retention", rotatedUser)
if err = users.DropPgUser(rotatedUser, db); err != nil {
c.logger.Errorf("could not drop role %q: %v", rotatedUser, err)
continue
}
}
}
return nil
}
// getDatabases returns the map of current databases with owners
// The caller is responsible for opening and closing the database connection
func (c *Cluster) getDatabases() (dbs map[string]string, err error) {

View File

@ -611,60 +611,180 @@ func (c *Cluster) checkAndSetGlobalPostgreSQLConfiguration(pod *v1.Pod, patroniC
return requiresMasterRestart, nil
}
func (c *Cluster) getNextRotationDate(currentDate time.Time) (time.Time, string) {
nextRotationDate := currentDate.AddDate(0, 0, int(c.OpConfig.PasswordRotationInterval))
return nextRotationDate, nextRotationDate.Format("2006-01-02 15:04:05")
}
func (c *Cluster) syncSecrets() error {
var (
err error
secret *v1.Secret
)
c.logger.Info("syncing secrets")
c.setProcessName("syncing secrets")
secrets := c.generateUserSecrets()
generatedSecrets := c.generateUserSecrets()
rotationUsers := make(spec.PgUserMap)
retentionUsers := make([]string, 0)
currentTime := time.Now()
for secretUsername, secretSpec := range secrets {
if secret, err = c.KubeClient.Secrets(secretSpec.Namespace).Create(context.TODO(), secretSpec, metav1.CreateOptions{}); err == nil {
for secretUsername, generatedSecret := range generatedSecrets {
secret, err := c.KubeClient.Secrets(generatedSecret.Namespace).Create(context.TODO(), generatedSecret, metav1.CreateOptions{})
if err == nil {
c.Secrets[secret.UID] = secret
c.logger.Debugf("created new secret %s, namespace: %s, uid: %s", util.NameFromMeta(secret.ObjectMeta), secretSpec.Namespace, secret.UID)
c.logger.Debugf("created new secret %s, namespace: %s, uid: %s", util.NameFromMeta(secret.ObjectMeta), generatedSecret.Namespace, secret.UID)
continue
}
if k8sutil.ResourceAlreadyExists(err) {
var userMap map[string]spec.PgUser
if secret, err = c.KubeClient.Secrets(secretSpec.Namespace).Get(context.TODO(), secretSpec.Name, metav1.GetOptions{}); err != nil {
return fmt.Errorf("could not get current secret: %v", err)
}
if secretUsername != string(secret.Data["username"]) {
c.logger.Errorf("secret %s does not contain the role %s", secretSpec.Name, secretUsername)
continue
}
c.Secrets[secret.UID] = secret
c.logger.Debugf("secret %s already exists, fetching its password", util.NameFromMeta(secret.ObjectMeta))
if secretUsername == c.systemUsers[constants.SuperuserKeyName].Name {
secretUsername = constants.SuperuserKeyName
userMap = c.systemUsers
} else if secretUsername == c.systemUsers[constants.ReplicationUserKeyName].Name {
secretUsername = constants.ReplicationUserKeyName
userMap = c.systemUsers
} else {
userMap = c.pgUsers
}
pwdUser := userMap[secretUsername]
// if this secret belongs to the infrastructure role and the password has changed - replace it in the secret
if pwdUser.Password != string(secret.Data["password"]) &&
pwdUser.Origin == spec.RoleOriginInfrastructure {
c.logger.Debugf("updating the secret %s from the infrastructure roles", secretSpec.Name)
if _, err = c.KubeClient.Secrets(secretSpec.Namespace).Update(context.TODO(), secretSpec, metav1.UpdateOptions{}); err != nil {
return fmt.Errorf("could not update infrastructure role secret for role %q: %v", secretUsername, err)
}
} else {
// for non-infrastructure role - update the role with the password from the secret
pwdUser.Password = string(secret.Data["password"])
userMap[secretUsername] = pwdUser
if err = c.updateSecret(secretUsername, generatedSecret, &rotationUsers, &retentionUsers, currentTime); err != nil {
c.logger.Warningf("syncing secret %s failed: %v", util.NameFromMeta(secret.ObjectMeta), err)
}
} else {
return fmt.Errorf("could not create secret for user %s: in namespace %s: %v", secretUsername, secretSpec.Namespace, err)
return fmt.Errorf("could not create secret for user %s: in namespace %s: %v", secretUsername, generatedSecret.Namespace, err)
}
}
// add new user with date suffix and use it in the secret of the original user
if len(rotationUsers) > 0 {
err := c.initDbConn()
if err != nil {
return fmt.Errorf("could not init db connection: %v", err)
}
pgSyncRequests := c.userSyncStrategy.ProduceSyncRequests(spec.PgUserMap{}, rotationUsers)
if err = c.userSyncStrategy.ExecuteSyncRequests(pgSyncRequests, c.pgDb); err != nil {
return fmt.Errorf("error creating database roles for password rotation: %v", err)
}
if err := c.closeDbConn(); err != nil {
c.logger.Errorf("could not close database connection after creating users for password rotation: %v", err)
}
}
// remove rotation users that exceed the retention interval
if len(retentionUsers) > 0 {
err := c.initDbConn()
if err != nil {
return fmt.Errorf("could not init db connection: %v", err)
}
if err = c.cleanupRotatedUsers(retentionUsers, c.pgDb); err != nil {
return fmt.Errorf("error removing users exceeding configured retention interval: %v", err)
}
if err := c.closeDbConn(); err != nil {
c.logger.Errorf("could not close database connection after removing users exceeding configured retention interval: %v", err)
}
}
return nil
}
func (c *Cluster) updateSecret(
secretUsername string,
generatedSecret *v1.Secret,
rotationUsers *spec.PgUserMap,
retentionUsers *[]string,
currentTime time.Time) error {
var (
secret *v1.Secret
err error
updateSecret bool
updateSecretMsg string
nextRotationDate time.Time
nextRotationDateStr string
)
// get the secret first
if secret, err = c.KubeClient.Secrets(generatedSecret.Namespace).Get(context.TODO(), generatedSecret.Name, metav1.GetOptions{}); err != nil {
return fmt.Errorf("could not get current secret: %v", err)
}
c.Secrets[secret.UID] = secret
// fetch user map to update later
var userMap map[string]spec.PgUser
var userKey string
if secretUsername == c.systemUsers[constants.SuperuserKeyName].Name {
userKey = constants.SuperuserKeyName
userMap = c.systemUsers
} else if secretUsername == c.systemUsers[constants.ReplicationUserKeyName].Name {
userKey = constants.ReplicationUserKeyName
userMap = c.systemUsers
} else {
userKey = secretUsername
userMap = c.pgUsers
}
pwdUser := userMap[userKey]
secretName := util.NameFromMeta(secret.ObjectMeta)
// if password rotation is enabled update password and username if rotation interval has been passed
if (c.OpConfig.EnablePasswordRotation && !pwdUser.IsDbOwner &&
pwdUser.Origin != spec.RoleOriginInfrastructure && pwdUser.Origin != spec.RoleOriginSystem) ||
util.SliceContains(c.Spec.UsersWithSecretRotation, secretUsername) ||
util.SliceContains(c.Spec.UsersWithInPlaceSecretRotation, secretUsername) {
// initialize password rotation setting first rotation date
nextRotationDateStr = string(secret.Data["nextRotation"])
if nextRotationDate, err = time.ParseInLocation("2006-01-02 15:04:05", nextRotationDateStr, time.Local); err != nil {
nextRotationDate, nextRotationDateStr = c.getNextRotationDate(currentTime)
secret.Data["nextRotation"] = []byte(nextRotationDateStr)
updateSecret = true
updateSecretMsg = fmt.Sprintf("rotation date not found in secret %q. Setting it to %s", secretName, nextRotationDateStr)
}
// check if next rotation can happen sooner
// if rotation interval has been decreased
currentRotationDate, _ := c.getNextRotationDate(currentTime)
if nextRotationDate.After(currentRotationDate) {
nextRotationDate = currentRotationDate
}
// update password and next rotation date if configured interval has passed
if currentTime.After(nextRotationDate) {
// create rotation user if role is not listed for in-place password update
if !util.SliceContains(c.Spec.UsersWithInPlaceSecretRotation, secretUsername) {
rotationUser := pwdUser
newRotationUsername := secretUsername + currentTime.Format("060102")
rotationUser.Name = newRotationUsername
rotationUser.MemberOf = []string{secretUsername}
(*rotationUsers)[newRotationUsername] = rotationUser
secret.Data["username"] = []byte(newRotationUsername)
// whenever there is a rotation, check if old rotation users can be deleted
*retentionUsers = append(*retentionUsers, secretUsername)
}
secret.Data["password"] = []byte(util.RandomPassword(constants.PasswordLength))
_, nextRotationDateStr = c.getNextRotationDate(nextRotationDate)
secret.Data["nextRotation"] = []byte(nextRotationDateStr)
updateSecret = true
updateSecretMsg = fmt.Sprintf("updating secret %q due to password rotation - next rotation date: %s", secretName, nextRotationDateStr)
}
} else {
// username might not match if password rotation has been disabled again
if secretUsername != string(secret.Data["username"]) {
*retentionUsers = append(*retentionUsers, secretUsername)
secret.Data["username"] = []byte(secretUsername)
secret.Data["password"] = []byte(util.RandomPassword(constants.PasswordLength))
secret.Data["nextRotation"] = []byte{}
updateSecret = true
updateSecretMsg = fmt.Sprintf("secret %s does not contain the role %s - updating username and resetting password", secretName, secretUsername)
}
}
// if this secret belongs to the infrastructure role and the password has changed - replace it in the secret
if pwdUser.Password != string(secret.Data["password"]) && pwdUser.Origin == spec.RoleOriginInfrastructure {
secret = generatedSecret
updateSecret = true
updateSecretMsg = fmt.Sprintf("updating the secret %s from the infrastructure roles", secretName)
} else {
// for non-infrastructure role - update the role with the password from the secret
pwdUser.Password = string(secret.Data["password"])
userMap[userKey] = pwdUser
}
if updateSecret {
c.logger.Debugln(updateSecretMsg)
if _, err = c.KubeClient.Secrets(secret.Namespace).Update(context.TODO(), secret, metav1.UpdateOptions{}); err != nil {
return fmt.Errorf("could not update secret %q: %v", secretName, err)
}
c.Secrets[secret.UID] = secret
}
return nil
}

View File

@ -19,6 +19,7 @@ import (
"github.com/zalando/postgres-operator/mocks"
acidv1 "github.com/zalando/postgres-operator/pkg/apis/acid.zalan.do/v1"
fakeacidv1 "github.com/zalando/postgres-operator/pkg/generated/clientset/versioned/fake"
"github.com/zalando/postgres-operator/pkg/spec"
"github.com/zalando/postgres-operator/pkg/util/config"
"github.com/zalando/postgres-operator/pkg/util/k8sutil"
"github.com/zalando/postgres-operator/pkg/util/patroni"
@ -26,6 +27,8 @@ import (
)
var patroniLogger = logrus.New().WithField("test", "patroni")
var acidClientSet = fakeacidv1.NewSimpleClientset()
var clientSet = fake.NewSimpleClientset()
func newMockPod(ip string) *v1.Pod {
return &v1.Pod{
@ -36,9 +39,6 @@ func newMockPod(ip string) *v1.Pod {
}
func newFakeK8sSyncClient() (k8sutil.KubernetesClient, *fake.Clientset) {
acidClientSet := fakeacidv1.NewSimpleClientset()
clientSet := fake.NewSimpleClientset()
return k8sutil.KubernetesClient{
PodsGetter: clientSet.CoreV1(),
PostgresqlsGetter: acidClientSet.AcidV1(),
@ -46,6 +46,12 @@ func newFakeK8sSyncClient() (k8sutil.KubernetesClient, *fake.Clientset) {
}, clientSet
}
func newFakeK8sSyncSecretsClient() (k8sutil.KubernetesClient, *fake.Clientset) {
return k8sutil.KubernetesClient{
SecretsGetter: clientSet.CoreV1(),
}, clientSet
}
func TestSyncStatefulSetsAnnotations(t *testing.T) {
testName := "test syncing statefulsets annotations"
client, _ := newFakeK8sSyncClient()
@ -257,3 +263,72 @@ func TestCheckAndSetGlobalPostgreSQLConfiguration(t *testing.T) {
}
}
}
func TestUpdateSecret(t *testing.T) {
testName := "test syncing secrets"
client, _ := newFakeK8sSyncSecretsClient()
clusterName := "acid-test-cluster"
namespace := "default"
username := "foo"
secretTemplate := config.StringTemplate("{username}.{cluster}.credentials")
rotationUsers := make(spec.PgUserMap)
retentionUsers := make([]string, 0)
yesterday := time.Now().AddDate(0, 0, -1)
// new cluster with pvc storage resize mode and configured labels
var cluster = New(
Config{
OpConfig: config.Config{
Auth: config.Auth{
SecretNameTemplate: secretTemplate,
EnablePasswordRotation: true,
PasswordRotationInterval: 1,
PasswordRotationUserRetention: 3,
},
Resources: config.Resources{
ClusterLabels: map[string]string{"application": "spilo"},
ClusterNameLabel: "cluster-name",
},
},
}, client, acidv1.Postgresql{}, logger, eventRecorder)
cluster.Name = clusterName
cluster.Namespace = namespace
cluster.pgUsers = map[string]spec.PgUser{}
cluster.Spec.Users = map[string]acidv1.UserFlags{username: {}}
cluster.initRobotUsers()
// create a secret for user foo
cluster.syncSecrets()
secret, err := cluster.KubeClient.Secrets(namespace).Get(context.TODO(), secretTemplate.Format("username", username, "cluster", clusterName), metav1.GetOptions{})
assert.NoError(t, err)
generatedSecret := cluster.Secrets[secret.UID]
// now update the secret setting next rotation date (yesterday + interval)
cluster.updateSecret(username, generatedSecret, &rotationUsers, &retentionUsers, yesterday)
updatedSecret, err := cluster.KubeClient.Secrets(namespace).Get(context.TODO(), secretTemplate.Format("username", username, "cluster", clusterName), metav1.GetOptions{})
assert.NoError(t, err)
nextRotation := string(updatedSecret.Data["nextRotation"])
_, nextRotationDate := cluster.getNextRotationDate(yesterday)
if nextRotation != nextRotationDate {
t.Errorf("%s: updated secret does not contain correct rotation date: expected %s, got %s", testName, nextRotationDate, nextRotation)
}
// update secret again but use current time to trigger rotation
cluster.updateSecret(username, generatedSecret, &rotationUsers, &retentionUsers, time.Now())
updatedSecret, err = cluster.KubeClient.Secrets(namespace).Get(context.TODO(), secretTemplate.Format("username", username, "cluster", clusterName), metav1.GetOptions{})
assert.NoError(t, err)
if len(rotationUsers) != 1 && len(retentionUsers) != 1 {
t.Errorf("%s: unexpected number of users to rotate - expected only foo, found %d", testName, len(rotationUsers))
}
secretUsername := string(updatedSecret.Data["username"])
rotatedUsername := username + time.Now().Format("060102")
if secretUsername != rotatedUsername {
t.Errorf("%s: updated secret does not contain correct username: expected %s, got %s", testName, rotatedUsername, secretUsername)
}
}

View File

@ -54,6 +54,9 @@ func (c *Controller) importConfigurationFromCRD(fromCRD *acidv1.OperatorConfigur
// user config
result.SuperUsername = util.Coalesce(fromCRD.PostgresUsersConfiguration.SuperUsername, "postgres")
result.ReplicationUsername = util.Coalesce(fromCRD.PostgresUsersConfiguration.ReplicationUsername, "standby")
result.EnablePasswordRotation = fromCRD.PostgresUsersConfiguration.EnablePasswordRotation
result.PasswordRotationInterval = util.CoalesceUInt32(fromCRD.PostgresUsersConfiguration.PasswordRotationInterval, 90)
result.PasswordRotationUserRetention = util.CoalesceUInt32(fromCRD.PostgresUsersConfiguration.DeepCopy().PasswordRotationUserRetention, 180)
// major version upgrade config
result.MajorVersionUpgradeMode = util.Coalesce(fromCRD.MajorVersionUpgrade.MajorVersionUpgradeMode, "off")

View File

@ -55,6 +55,7 @@ type PgUser struct {
MemberOf []string `yaml:"inrole"`
Parameters map[string]string `yaml:"db_parameters"`
AdminRole string `yaml:"admin_role"`
IsDbOwner bool `yaml:"is_db_owner"`
Deleted bool `yaml:"deleted"`
}

View File

@ -100,6 +100,9 @@ type Auth struct {
InfrastructureRolesDefs string `name:"infrastructure_roles_secrets"`
SuperUsername string `name:"super_username" default:"postgres"`
ReplicationUsername string `name:"replication_username" default:"standby"`
EnablePasswordRotation bool `name:"enable_password_rotation" default:"false"`
PasswordRotationInterval uint32 `name:"password_rotation_interval" default:"90"`
PasswordRotationUserRetention uint32 `name:"password_rotation_user_retention" default:"180"`
}
// Scalyr holds the configuration for the Scalyr Agent sidecar for log shipping:

View File

@ -18,6 +18,7 @@ const (
alterUserRenameSQL = `ALTER ROLE "%s" RENAME TO "%s%s"`
alterRoleResetAllSQL = `ALTER ROLE "%s" RESET ALL`
alterRoleSetSQL = `ALTER ROLE "%s" SET %s TO %s`
dropUserSQL = `SET LOCAL synchronous_commit = 'local'; DROP ROLE "%s";`
grantToUserSQL = `GRANT %s TO "%s"`
doBlockStmt = `SET LOCAL synchronous_commit = 'local'; DO $$ BEGIN %s; END;$$;`
passwordTemplate = "ENCRYPTED PASSWORD '%s'"
@ -288,3 +289,13 @@ func quoteParameterValue(name, val string) string {
}
return fmt.Sprintf(`'%s'`, strings.Trim(val, " "))
}
// DropPgUser to remove user created by the operator e.g. for password rotation
func DropPgUser(user string, db *sql.DB) error {
query := fmt.Sprintf(dropUserSQL, user)
if _, err := db.Exec(query); err != nil { // TODO: Try several times
return err
}
return nil
}