Skip to content

Commit

Permalink
sql,pgwire,server: factor the password expiry code
Browse files Browse the repository at this point in the history
To understand this commit, it is perhaps useful to remember that
password credentials, in Postgres's SQL dialect, have their own
expiration timestamp, set via the VALID UNTIL role option.  The
timestamp is attached to the password, not the user account, such that
a user can still log in using a separate authentication method even
when their password is expired.

Additionally, it is perhaps useful to understand that throughout the
authentication stack, we distinguish the phase where we determine
whether a principal *exists* and is allowed to log in (i.e. it has the
LOGIN privilege); and the phase where we load its password credentials
from storage (or from an in-RAM cache). In particular, we are keen to
skip the loading of the password credentials in the case where the
client is authenticating using another method than passwords (for
example TLS client certs, which is still pretty common).

In this context, this commit performs two changes.

The first change is an API cleanup.

Prior to this commit, we were loading the password expiration
timestamp in the first phase (checking that the principal exists),
rather than the second phase (loading the credentials). This was a
misdesign because, as explained above, the expiry timestamp is a
property of the password, and thus needs not be looked at when
passwords are not used.

We fix this in this commit by making the expiration timestamp part of
the second phase, as it should have been from the start.

The second change in this commit is a reduction of security risk.

Prior to this commit, the password expiration timestamp was checked
(compared to the current time) again and again in each of the call
points for the password retrieval function. It was the responsibility
of each caller to perform this check. This constituted a security risk
because a new caller could forget to add the expiry check and bypass
the expiration mechanism entirely.

We fix this by performing the expiry validation in a central location,
in the common function that retrieves the credentials from storage (or
cache). If the password is expired, the function also hides the
password hash by returning the new "password missing" pseudo-hash, so
that the caller cannot any more mistakenly attempt to validate a
password despite the expiration.

Release note: None
  • Loading branch information
knz committed Jan 19, 2022
1 parent cf5d36d commit d92f393
Show file tree
Hide file tree
Showing 6 changed files with 61 additions and 62 deletions.
32 changes: 12 additions & 20 deletions pkg/ccl/serverccl/role_authentication_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ func TestVerifyPassword(t *testing.T) {
t.Run("", func(t *testing.T) {
execCfg := s.ExecutorConfig().(sql.ExecutorConfig)
username := security.MakeSQLUsernameFromPreNormalizedString(tc.username)
exists, canLogin, isSuperuser, validUntil, _, pwRetrieveFn, err := sql.GetUserSessionInitInfo(
exists, canLogin, isSuperuser, _, pwRetrieveFn, err := sql.GetUserSessionInitInfo(
context.Background(), &execCfg, &ie, username, "", /* databaseName */
)

Expand All @@ -142,35 +142,27 @@ func TestVerifyPassword(t *testing.T) {

valid := true
expired := false

if !exists || !canLogin {
valid = false
}

hashedPassword, err := pwRetrieveFn(ctx)
if err != nil {
t.Errorf(
"credentials %s/%s failed with error %s, wanted no error",
tc.username,
tc.password,
err,
)
}
} else {
var hashedPassword security.PasswordHash
expired, hashedPassword, err = pwRetrieveFn(ctx)
if err != nil {
t.Errorf(
"credentials %s/%s failed with error %s, wanted no error",
tc.username,
tc.password,
err,
)
}

if valid {
valid, err = security.CompareHashAndCleartextPassword(ctx, hashedPassword, tc.password)
if err != nil {
t.Error(err)
valid = false
}
}

if validUntil != nil {
if validUntil.Time.Sub(timeutil.Now()) < 0 {
expired = true
}
}

if valid && !expired != tc.shouldAuthenticate {
t.Errorf(
"credentials %s/%s valid = %t, wanted %t",
Expand Down
13 changes: 5 additions & 8 deletions pkg/server/authentication.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/ui"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
Expand Down Expand Up @@ -255,7 +254,7 @@ func (s *authenticationServer) UserLoginFromSSO(
// without further normalization.
username, _ := security.MakeSQLUsernameFromUserInput(reqUsername, security.UsernameValidation)

exists, canLogin, _, _, _, _, err := sql.GetUserSessionInitInfo(
exists, canLogin, _, _, _, err := sql.GetUserSessionInitInfo(
ctx,
s.server.sqlServer.execCfg,
s.server.sqlServer.execCfg.InternalExecutor,
Expand Down Expand Up @@ -418,7 +417,7 @@ WHERE id = $1`
func (s *authenticationServer) verifyPassword(
ctx context.Context, username security.SQLUsername, password string,
) (valid bool, expired bool, err error) {
exists, canLogin, _, validUntil, _, pwRetrieveFn, err := sql.GetUserSessionInitInfo(
exists, canLogin, _, _, pwRetrieveFn, err := sql.GetUserSessionInitInfo(
ctx,
s.server.sqlServer.execCfg,
s.server.sqlServer.execCfg.InternalExecutor,
Expand All @@ -431,15 +430,13 @@ func (s *authenticationServer) verifyPassword(
if !exists || !canLogin {
return false, false, nil
}
hashedPassword, err := pwRetrieveFn(ctx)
expired, hashedPassword, err := pwRetrieveFn(ctx)
if err != nil {
return false, false, err
}

if validUntil != nil {
if validUntil.Time.Sub(timeutil.Now()) < 0 {
return false, true, nil
}
if expired {
return false, true, nil
}

ok, err := security.CompareHashAndCleartextPassword(ctx, hashedPassword, password)
Expand Down
11 changes: 2 additions & 9 deletions pkg/sql/pgwire/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -146,7 +145,7 @@ func (c *conn) handleAuthentication(

// Check that the requested user exists and retrieve the hashed
// password in case password authentication is needed.
exists, canLogin, isSuperuser, validUntil, defaultSettings, pwRetrievalFn, err :=
exists, canLogin, isSuperuser, defaultSettings, pwRetrievalFn, err :=
sql.GetUserSessionInitInfo(
ctx,
execCfg,
Expand Down Expand Up @@ -178,16 +177,10 @@ func (c *conn) handleAuthentication(
))
}

// Set up lazy provider for password or cert-password methods.
pwDataFn := func(ctx context.Context) (security.PasswordHash, *tree.DTimestamp, error) {
pwHash, err := pwRetrievalFn(ctx)
return pwHash, validUntil, err
}

// At this point, we know that the requested user exists and is
// allowed to log in. Now we can delegate to the selected AuthMethod
// implementation to complete the authentication.
if err := behaviors.Authenticate(ctx, systemIdentity, true /* public */, pwDataFn); err != nil {
if err := behaviors.Authenticate(ctx, systemIdentity, true /* public */, pwRetrievalFn); err != nil {
ac.LogAuthFailed(ctx, eventpb.AuthFailReason_CREDENTIALS_INVALID, err)
return connClose, sendError(pgerror.WithCandidateCode(err, pgcode.InvalidAuthorizationSpecification))
}
Expand Down
27 changes: 15 additions & 12 deletions pkg/sql/pgwire/auth_methods.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/xdg-go/scram"
)
Expand Down Expand Up @@ -123,7 +122,7 @@ func authPassword(
// Note: if this fails, we can't return the error right away,
// because we need to consume the client response first. This
// will be handled below.
hashedPassword, pwValidUntil, pwRetrievalErr := pwRetrieveFn(ctx)
expired, hashedPassword, pwRetrievalErr := pwRetrieveFn(ctx)

// Wait for the password response from the client.
pwdData, err := c.GetPwdData()
Expand All @@ -146,21 +145,25 @@ func authPassword(
c.LogAuthFailed(ctx, eventpb.AuthFailReason_PRE_HOOK_ERROR, err)
return err
}
// Inform operators looking at logs if there's something amiss.
if hashedPassword.Method() == security.HashMissingPassword {

// Expiration check.
//
// NB: This check is advisory and could be omitted; the retrieval
// function ensures that the returned hashedPassword is
// security.MissingPasswordHash when the credentials have expired,
// so the credential check below would fail anyway.
if expired {
c.LogAuthFailed(ctx, eventpb.AuthFailReason_CREDENTIALS_EXPIRED, nil)
// TODO(knz): Consider omitting the error return here, and let the
// code fallback, to hide the reason for the failure from the
// client.
return errors.New("password is expired")
} else if hashedPassword.Method() == security.HashMissingPassword {
c.LogAuthInfof(ctx, "user has no password defined")
// NB: the failure reason will be automatically handled by the fallback
// in auth.go (and report CREDENTIALS_INVALID).
}

// Expiration check.
if pwValidUntil != nil {
if pwValidUntil.Sub(timeutil.Now()) < 0 {
c.LogAuthFailed(ctx, eventpb.AuthFailReason_CREDENTIALS_EXPIRED, nil)
return errors.New("password is expired")
}
}

// Now check the cleartext password against the retrieved credentials.
return security.UserAuthPasswordHook(
false /*insecure*/, password, hashedPassword,
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/pgwire/authenticator.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"

"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

// Authenticator is a component of an AuthMethod that determines if the
Expand All @@ -31,7 +30,7 @@ type Authenticator = func(
// and expiration time for a user logging in with password-based
// authentication.
type PasswordRetrievalFn = func(context.Context) (
expired bool,
pwHash security.PasswordHash,
pwExpiration *tree.DTimestamp,
_ error,
)
37 changes: 26 additions & 11 deletions pkg/sql/user.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,9 +78,8 @@ func GetUserSessionInitInfo(
exists bool,
canLogin bool,
isSuperuser bool,
validUntil *tree.DTimestamp,
defaultSettings []sessioninit.SettingsCacheEntry,
pwRetrieveFn func(ctx context.Context) (hashedPassword security.PasswordHash, err error),
pwRetrieveFn func(ctx context.Context) (expired bool, hashedPassword security.PasswordHash, err error),
err error,
) {
// We may be operating with a timeout.
Expand All @@ -103,25 +102,32 @@ func GetUserSessionInitInfo(
// As explained above, for root we report that the user exists
// immediately, and delay retrieving the password until strictly
// necessary.
rootFn := func(ctx context.Context) (security.PasswordHash, error) {
var ret security.PasswordHash
err := runFn(func(ctx context.Context) error {
rootFn := func(ctx context.Context) (expired bool, ret security.PasswordHash, err error) {
var validUntil *tree.DTimestamp
err = runFn(func(ctx context.Context) error {
authInfo, _, err := retrieveSessionInitInfoWithCache(ctx, execCfg, ie, username, databaseName)
if err != nil {
return err
}
validUntil = authInfo.ValidUntil
ret = authInfo.HashedPassword
return nil
})
if validUntil != nil {
if validUntil.Time.Sub(timeutil.Now()) < 0 {
expired = true
ret = nil
}
}
if ret == nil {
ret = security.MissingPasswordHash
}
return ret, err
return expired, ret, err
}

// Root user cannot have password expiry and must have login.
// It also never has default settings applied to it.
return true, true, true, nil, nil, rootFn, nil
return true, true, true, nil, rootFn, nil
}

var authInfo sessioninit.AuthInfo
Expand Down Expand Up @@ -166,14 +172,23 @@ func GetUserSessionInitInfo(
return authInfo.UserExists,
authInfo.CanLogin,
isSuperuser,
authInfo.ValidUntil,
settingsEntries,
func(ctx context.Context) (security.PasswordHash, error) {
ret := authInfo.HashedPassword
func(ctx context.Context) (expired bool, ret security.PasswordHash, err error) {
ret = authInfo.HashedPassword
if authInfo.ValidUntil != nil {
// NB: we compute the expiration as late as possible,
// to ensure that we determine the expiration relative
// to the time at which the client presents the password
// to the server (and not earlier).
if authInfo.ValidUntil.Time.Sub(timeutil.Now()) < 0 {
expired = true
ret = nil
}
}
if ret == nil {
ret = security.MissingPasswordHash
}
return ret, nil
return expired, ret, nil
},
err
}
Expand Down

0 comments on commit d92f393

Please sign in to comment.