Skip to content

Commit

Permalink
sql: make the stored hash method configurable
Browse files Browse the repository at this point in the history
Release note (security update): The hash method used to encode
cleartext passwords before storing them is now configurable, via the
new cluster setting `server.user_login.password_encryption`. Its
supported values are `crdb-bcrypt` and `scram-sha-256`.

The cluster setting only becomes effective and its default value
is `scram-sha-256` after all cluster nodes have been upgraded. Prior
to completion of the upgrade, the cluster behaves as if the cluster
setting is fixed to `crdb-bcrypt` (for backward compatibility)

Note that the preferred way to populate password credentials
for SQL user accounts is to pre-compute the hash client-side, and
pass the precomputed hash via CREATE/ALTER USER/ROLE WITH PASSWORD.
This ensures that the server never sees the cleartext password.

Release note (security update): The cost of the hashing function for
`scram-sha-256` is now configurable via the new cluster setting
`server.user_login.password_hashes.default_cost.scram_sha_256`.  Its
default value is 119680, which corresponds to an approximate password
check latency of 50-100ms on modern hardware.

This value should be increased over time to reflect improvements to
CPU performance: the latency should not become so small that it
becomes feasible to bruteforce passwords via repeated login attempts.

Future versions of CockroachDB will likely update the default accordingly.

Release note (sql change): The session variable `password_encryption`
is now exposed to SQL clients. Note that SQL clients cannot modify
its value directly; it is configurable via a cluster setting.
  • Loading branch information
knz committed Jan 20, 2022
1 parent 86fb393 commit 16db78a
Show file tree
Hide file tree
Showing 13 changed files with 408 additions and 35 deletions.
2 changes: 2 additions & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,9 @@ server.shutdown.lease_transfer_wait duration 5s the amount of time a server wait
server.shutdown.query_wait duration 10s the server will wait for at least this amount of time for active queries to finish (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
server.time_until_store_dead duration 5m0s the time after which if there is no new gossiped information about a store, it is considered dead
server.user_login.min_password_length integer 1 the minimum length accepted for passwords set in cleartext via SQL. Note that a value lower than 1 is ignored: passwords cannot be empty in any case.
server.user_login.password_encryption enumeration scram-sha-256 which hash method to use to encode cleartext passwords passed via ALTER/CREATE USER/ROLE WITH PASSWORD [crdb-bcrypt = 2, scram-sha-256 = 3]
server.user_login.password_hashes.default_cost.crdb_bcrypt integer 10 the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method crdb-bcrypt (allowed range: 4-31)
server.user_login.password_hashes.default_cost.scram_sha_256 integer 119680 the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method scram-sha-256 (allowed range: 4096-240000000000)
server.user_login.store_client_pre_hashed_passwords.enabled boolean true whether the server accepts to store passwords pre-hashed by clients
server.user_login.timeout duration 10s timeout after which client authentication times out if some system range is unavailable (0 = no timeout)
server.web_session.auto_logout.timeout duration 168h0m0s the duration that web sessions will survive before being periodically purged, since they were last used
Expand Down
2 changes: 2 additions & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,9 @@
<tr><td><code>server.shutdown.query_wait</code></td><td>duration</td><td><code>10s</code></td><td>the server will wait for at least this amount of time for active queries to finish (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td></tr>
<tr><td><code>server.time_until_store_dead</code></td><td>duration</td><td><code>5m0s</code></td><td>the time after which if there is no new gossiped information about a store, it is considered dead</td></tr>
<tr><td><code>server.user_login.min_password_length</code></td><td>integer</td><td><code>1</code></td><td>the minimum length accepted for passwords set in cleartext via SQL. Note that a value lower than 1 is ignored: passwords cannot be empty in any case.</td></tr>
<tr><td><code>server.user_login.password_encryption</code></td><td>enumeration</td><td><code>scram-sha-256</code></td><td>which hash method to use to encode cleartext passwords passed via ALTER/CREATE USER/ROLE WITH PASSWORD [crdb-bcrypt = 2, scram-sha-256 = 3]</td></tr>
<tr><td><code>server.user_login.password_hashes.default_cost.crdb_bcrypt</code></td><td>integer</td><td><code>10</code></td><td>the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method crdb-bcrypt (allowed range: 4-31)</td></tr>
<tr><td><code>server.user_login.password_hashes.default_cost.scram_sha_256</code></td><td>integer</td><td><code>119680</code></td><td>the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method scram-sha-256 (allowed range: 4096-240000000000)</td></tr>
<tr><td><code>server.user_login.store_client_pre_hashed_passwords.enabled</code></td><td>boolean</td><td><code>true</code></td><td>whether the server accepts to store passwords pre-hashed by clients</td></tr>
<tr><td><code>server.user_login.timeout</code></td><td>duration</td><td><code>10s</code></td><td>timeout after which client authentication times out if some system range is unavailable (0 = no timeout)</td></tr>
<tr><td><code>server.web_session.auto_logout.timeout</code></td><td>duration</td><td><code>168h0m0s</code></td><td>the duration that web sessions will survive before being periodically purged, since they were last used</td></tr>
Expand Down
1 change: 1 addition & 0 deletions pkg/security/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/security",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
Expand Down
199 changes: 186 additions & 13 deletions pkg/security/password.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,15 +14,18 @@ import (
"bytes"
"context"
"crypto/hmac"
"crypto/rand"
"crypto/sha256"
"encoding/base64"
"fmt"
"io"
"regexp"
"runtime"
"strconv"
"sync"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -63,8 +66,37 @@ var BcryptCost = settings.RegisterIntSetting(
// BcryptCostSettingName is the name of the cluster setting BcryptCost.
const BcryptCostSettingName = "server.user_login.password_hashes.default_cost.crdb_bcrypt"

// SCRAMCost is the cost to use in SCRAM exchanges.
// The value of 4096 is the minimum value recommended by RFC 5802.
// It should be increased along with computation power.
var SCRAMCost = settings.RegisterIntSetting(
settings.TenantWritable,
SCRAMCostSettingName,
fmt.Sprintf(
"the hashing cost to use when storing passwords supplied as cleartext by SQL clients "+
"with the hashing method scram-sha-256 (allowed range: %d-%d)",
scramMinCost, scramMaxCost),
// The minimum value 4096 incurs a password check latency of ~2ms on AMD 3950X 3.7GHz.
//
// The default value 119680 incurs ~60ms latency on the same hw.
// This default was calibrated to incur a similar check latency as the
// default value for BCryptCost above.
//
// For reference, value 250000 incurs ~125ms latency on the same hw,
// value 1000000 incurs ~500ms.
119680,
func(i int64) error {
if i < scramMinCost || i > scramMaxCost {
return errors.Newf("cost not in allowed range (%d,%d)", scramMinCost, scramMaxCost)
}
return nil
}).WithPublic()

const scramMinCost = 4096 // as per RFC 5802.
const scramMaxCost = 240000000000 // arbitrary value to prevent unreasonably long logins.
const scramMaxCost = 240000000000 // arbitrary value to prevent unreasonably long logins

// SCRAMCostSettingName is the name of the cluster setting SCRAMCost.
const SCRAMCostSettingName = "server.user_login.password_hashes.default_cost.scram_sha_256"

// ErrEmptyPassword indicates that an empty password was attempted to be set.
var ErrEmptyPassword = errors.New("empty passwords are not permitted")
Expand All @@ -83,19 +115,38 @@ type HashMethod int8
const (
// HashInvalidMethod represents invalid hashes.
// This always fails authentication.
HashInvalidMethod HashMethod = iota
HashInvalidMethod HashMethod = 0
// HashMissingPassword represents a virtual hash when there was
// no password. This too always fails authentication.
// We need a different method here than HashInvalidMethod because
// the authentication code distinguishes the two cases when reporting
// why authentication fails in audit logs.
HashMissingPassword
HashMissingPassword HashMethod = 1
// HashBCrypt indicates CockroachDB's bespoke bcrypt-based method.
HashBCrypt
// NB: Do not renumber this constant; it is used as value
// in cluster setting enums.
HashBCrypt HashMethod = 2
// HashSCRAMSHA256 indicates SCRAM-SHA-256.
HashSCRAMSHA256
// NB: Do not renumber this constant; it is used as value
// in cluster setting enums.
HashSCRAMSHA256 HashMethod = 3
)

func (h HashMethod) String() string {
switch h {
case HashInvalidMethod:
return "<invalid>"
case HashMissingPassword:
return "<missing password>"
case HashBCrypt:
return "crdb-bcrypt"
case HashSCRAMSHA256:
return "scram-sha-256"
default:
panic(errors.AssertionFailedf("programming errof: unknown hash method %d", int(h)))
}
}

// PasswordHash represents the type of a password hash loaded from a credential store.
type PasswordHash interface {
fmt.Stringer
Expand Down Expand Up @@ -308,15 +359,135 @@ func computeHMAC(hg scram.HashGeneratorFcn, key, data []byte) []byte {
return mac.Sum(nil)
}

// HashPassword takes a raw password and returns a bcrypt hashed password.
// PasswordHashMethod is the cluster setting that configures which
// hash method to use when clients request to store a cleartext password.
//
// It is exported for use in tests. Do not use this setting directly
// to read the current hash method. Instead use the
// GetConfiguredHashMethod() function.
var PasswordHashMethod = settings.RegisterEnumSetting(
settings.TenantWritable,
"server.user_login.password_encryption",
"which hash method to use to encode cleartext passwords passed via ALTER/CREATE USER/ROLE WITH PASSWORD",
// Note: the default is initially SCRAM, even in mixed-version clusters where
// previous-version nodes do not know anything about SCRAM. This is handled
// in the GetConfiguredPasswordHashMethod() function.
"scram-sha-256",
map[int64]string{
int64(HashBCrypt): HashBCrypt.String(),
int64(HashSCRAMSHA256): HashSCRAMSHA256.String(),
},
).WithPublic()

// hasClusterVersion verifies that all nodes have been upgraded to
// support the given target version key.
func hasClusterVersion(
ctx context.Context, values *settings.Values, versionkey clusterversion.Key,
) bool {
var vh clusterversion.Handle
if values != nil {
vh = values.Opaque().(clusterversion.Handle)
}
return vh != nil && vh.IsActive(ctx, versionkey)
}

// GetConfiguredPasswordHashMethod returns the configured hash method
// to use before storing passwords provided in cleartext from clients.
func GetConfiguredPasswordHashMethod(ctx context.Context, sv *settings.Values) (method HashMethod) {
method = HashMethod(PasswordHashMethod.Get(sv))
if method == HashSCRAMSHA256 && !hasClusterVersion(ctx, sv, clusterversion.SCRAMAuthentication) {
// Not all nodes are upgraded to understand SCRAM yet. Force
// Bcrypt for now, otherwise previous-version nodes will get confused.
method = HashBCrypt
}
return method
}

// HashPassword takes a raw password and returns a hashed password, hashed
// using the currently configured method.
func HashPassword(ctx context.Context, sv *settings.Values, password string) ([]byte, error) {
sem := getExpensiveHashComputeSem(ctx)
alloc, err := sem.Acquire(ctx, 1)
if err != nil {
return nil, err
method := GetConfiguredPasswordHashMethod(ctx, sv)
switch method {
case HashBCrypt:
sem := getExpensiveHashComputeSem(ctx)
alloc, err := sem.Acquire(ctx, 1)
if err != nil {
return nil, err
}
defer alloc.Release()
return bcrypt.GenerateFromPassword(appendEmptySha256(password), int(BcryptCost.Get(sv)))

case HashSCRAMSHA256:
prepared, err := stringprep.SASLprep.Prepare(password)
if err != nil {
// Special PostgreSQL case, quoth comment at the top of
// auth-scram.c:
//
// * - If the password isn't valid UTF-8, or contains characters prohibited
// * by the SASLprep profile, we skip the SASLprep pre-processing and use
// * the raw bytes in calculating the hash.
prepared = password
}

// The computation of ServerKey and StoredKey is conveniently provided
// to us by xdg/scram in the Client method GetStoredCredentials().
// To use it, we need a client.
client, err := scram.SHA256.NewClientUnprepped("" /* username: unused */, prepared, "" /* authzID: unused */)
if err != nil {
return nil, errors.AssertionFailedf("programming error: client construction should never fail")
}

// We also need to generate a random salt ourselves.
const scramSaltSize = 16 // postgres: SCRAM_DEFAULT_SALT_LEN.
salt := make([]byte, scramSaltSize)
if _, err := io.ReadFull(rand.Reader, salt); err != nil {
return nil, errors.Wrap(err, "generating random salt")
}

// The computation of the SCRAM hash is expensive. Use the shared
// semaphore for it. We reuse the same pattern as the bcrypt case above.
sem := getExpensiveHashComputeSem(ctx)
alloc, err := sem.Acquire(ctx, 1)
if err != nil {
return nil, err
}
defer alloc.Release()
// Compute the credentials.
cost := int(SCRAMCost.Get(sv))
creds := client.GetStoredCredentials(scram.KeyFactors{Iters: cost, Salt: string(salt)})
// Encode them in our standard hash format.
return encodeScramHash(salt, creds), nil

default:
return nil, errors.Newf("unsupported hash method: %v", method)
}
defer alloc.Release()
return bcrypt.GenerateFromPassword(appendEmptySha256(password), int(BcryptCost.Get(sv)))
}

// encodeScramHash encodes the provided SCRAM credentials using the
// standard PostgreSQL / RFC5802 representation.
func encodeScramHash(saltBytes []byte, sc scram.StoredCredentials) []byte {
b64enc := base64.StdEncoding
saltLen := b64enc.EncodedLen(len(saltBytes))
storedKeyLen := b64enc.EncodedLen(len(sc.StoredKey))
serverKeyLen := b64enc.EncodedLen(len(sc.ServerKey))
// The representation is:
// SCRAM-SHA-256$<iters>:<salt>$<stored key>:<server key>
// We use a capacity-based slice extension instead of a size-based fill
// so as to automatically support iteration counts with more than 4 digits.
res := make([]byte, 0, len(scramPrefix)+1+4 /*iters*/ +1+saltLen+1+storedKeyLen+1+serverKeyLen)
res = append(res, scramPrefix...)
res = append(res, '$')
res = strconv.AppendInt(res, int64(sc.Iters), 10)
res = append(res, ':')
res = append(res, make([]byte, saltLen)...)
b64enc.Encode(res[len(res)-saltLen:], saltBytes)
res = append(res, '$')
res = append(res, make([]byte, storedKeyLen)...)
b64enc.Encode(res[len(res)-storedKeyLen:], sc.StoredKey)
res = append(res, ':')
res = append(res, make([]byte, serverKeyLen)...)
b64enc.Encode(res[len(res)-serverKeyLen:], sc.ServerKey)
return res
}

// AutoDetectPasswordHashes is the cluster setting that configures whether
Expand Down Expand Up @@ -363,6 +534,8 @@ func checkBcryptHash(inputPassword []byte) (ok bool, hashedPassword []byte, err
return true, hashedPassword, err
}

const scramPrefix = "SCRAM-SHA-256"

// scramHashRe matches the lexical structure of PostgreSQL's
// pre-computed SCRAM hashes.
//
Expand All @@ -371,7 +544,7 @@ func checkBcryptHash(inputPassword []byte) (ok bool, hashedPassword []byte, err
// "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"
// The salt must have size >0; the server key pair is two times 32 bytes,
// which always encode to 44 base64 characters.
var scramHashRe = regexp.MustCompile(`^SCRAM-SHA-256\$(\d+):([A-Za-z0-9+/]+=*)\$([A-Za-z0-9+/]{43}=):([A-Za-z0-9+/]{43}=)$`)
var scramHashRe = regexp.MustCompile(`^` + scramPrefix + `\$(\d+):([A-Za-z0-9+/]+=*)\$([A-Za-z0-9+/]{43}=):([A-Za-z0-9+/]{43}=)$`)

// scramParts is an intermediate type to connect the output of
// isSCRAMHash() to makeSCRAMHash(), so that the latter cannot be
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -4691,6 +4691,7 @@ optimizer_use_histograms on
optimizer_use_multicol_stats on
override_multi_region_zone_config off
parallelize_multi_key_lookup_joins_enabled off
password_encryption scram-sha-256
prefer_lookup_joins_for_fks off
propagate_input_ordering off
reorder_joins_limit 8
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -4097,6 +4097,7 @@ optimizer_use_histograms on NULL
optimizer_use_multicol_stats on NULL NULL NULL string
override_multi_region_zone_config off NULL NULL NULL string
parallelize_multi_key_lookup_joins_enabled off NULL NULL NULL string
password_encryption scram-sha-256 NULL NULL NULL string
prefer_lookup_joins_for_fks off NULL NULL NULL string
propagate_input_ordering off NULL NULL NULL string
reorder_joins_limit 8 NULL NULL NULL string
Expand Down Expand Up @@ -4209,6 +4210,7 @@ optimizer_use_histograms on NULL
optimizer_use_multicol_stats on NULL user NULL on on
override_multi_region_zone_config off NULL user NULL off off
parallelize_multi_key_lookup_joins_enabled off NULL user NULL false false
password_encryption scram-sha-256 NULL user NULL scram-sha-256 scram-sha-256
prefer_lookup_joins_for_fks off NULL user NULL off off
propagate_input_ordering off NULL user NULL off off
reorder_joins_limit 8 NULL user NULL 8 8
Expand Down Expand Up @@ -4317,6 +4319,7 @@ optimizer_use_histograms NULL NULL NULL
optimizer_use_multicol_stats NULL NULL NULL NULL NULL
override_multi_region_zone_config NULL NULL NULL NULL NULL
parallelize_multi_key_lookup_joins_enabled NULL NULL NULL NULL NULL
password_encryption NULL NULL NULL NULL NULL
prefer_lookup_joins_for_fks NULL NULL NULL NULL NULL
propagate_input_ordering NULL NULL NULL NULL NULL
reorder_joins_limit NULL NULL NULL NULL NULL
Expand Down
46 changes: 44 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/role
Original file line number Diff line number Diff line change
Expand Up @@ -1415,6 +1415,9 @@ subtest pw_hashes

user root

statement ok
SET CLUSTER SETTING server.user_login.password_encryption = 'crdb-bcrypt'

let $bcrypt_pw
SELECT 'CRDB-BCRYPT$'||'2a$'||'10$'||'vcmoIBvgeHjgScVHWRMWI.Z3v03WMixAw2bBS6qZihljSUuwi88Yq'

Expand Down Expand Up @@ -1463,7 +1466,7 @@ RESET CLUSTER SETTING server.user_login.store_client_pre_hashed_passwords.enable
subtest bcrypt_cost

statement ok
SET CLUSTER SETTING server.user_login.password_hashes.default_cost.crdb_bcrypt = 20
SET CLUSTER SETTING server.user_login.password_hashes.default_cost.crdb_bcrypt = 8

statement ok
CREATE USER hash7 WITH PASSWORD 'hello'
Expand All @@ -1472,7 +1475,46 @@ CREATE USER hash7 WITH PASSWORD 'hello'
query TT
SELECT username, substr("hashedPassword", 1, 7) FROM system.users WHERE username = 'hash7'
----
hash7 $2a$20$
hash7 $2a$08$

statement ok
RESET CLUSTER SETTING server.user_login.password_hashes.default_cost.crdb_bcrypt;

subtest scram_gen_hash

statement ok
SET CLUSTER SETTING server.user_login.password_encryption = 'scram-sha-256'

statement ok
CREATE USER hash8 WITH PASSWORD 'hello world'

query TT
SELECT username, substr("hashedPassword", 1, 20) FROM system.users WHERE username = 'hash8'
----
hash8 SCRAM-SHA-256$119680

statement ok
ALTER USER hash8 WITH PASSWORD 'hello universe'

query TT
SELECT username, substr("hashedPassword", 1, 20) FROM system.users WHERE username = 'hash8'
----
hash8 SCRAM-SHA-256$119680

subtest scram_cost

statement ok
SET CLUSTER SETTING server.user_login.password_hashes.default_cost.scram_sha_256 = 200000

statement ok
ALTER USER hash8 WITH PASSWORD 'hai'

query TT
SELECT username, substr("hashedPassword", 1, 20) FROM system.users WHERE username = 'hash8'
----
hash8 SCRAM-SHA-256$200000

# Reset cluster setting after test completion.
statement ok
RESET CLUSTER SETTING server.user_login.password_encryption;
RESET CLUSTER SETTING server.user_login.password_hashes.default_cost.scram_sha_256
Loading

0 comments on commit 16db78a

Please sign in to comment.