Skip to content

Commit

Permalink
Merge #104235
Browse files Browse the repository at this point in the history
104235: pgwire: add gauge for connections throttled by semaphore r=rafiss a=rafiss

fixes #82900

Release note (ops change): Added a gauge metric named sql.conns_waiting_to_hash. It counts the number of connection attempts that are being throttled in order to limit the amount of concurrent password hashing operations. The throttling behavior has been present since v21.2, and was added in order to prevent password hashing from using up too much CPU. The metric is the only new addition in this change.

The metric is expected to be 0 or close to 0 in a healthy setup. If the metric is consistently high and connection latencies are high, then an operator should do one or more of the following:

- Make sure applications using the cluster have properly configured connection pools.
- Add more vCPU or more nodes to the cluster.
- Increase the password hashing concurrency using the COCKROACH_MAX_PW_HASH_COMPUTE_CONCURRENCY environment variable.

Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
  • Loading branch information
craig[bot] and rafiss committed Jun 6, 2023
2 parents 530e2ef + 9f619e3 commit b15f6f5
Show file tree
Hide file tree
Showing 5 changed files with 45 additions and 10 deletions.
5 changes: 3 additions & 2 deletions pkg/security/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/password"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -232,7 +233,7 @@ func IsTenantCertificate(cert *x509.Certificate) bool {
// UserAuthPasswordHook builds an authentication hook based on the security
// mode, password, and its potentially matching hash.
func UserAuthPasswordHook(
insecureMode bool, passwordStr string, hashedPassword password.PasswordHash,
insecureMode bool, passwordStr string, hashedPassword password.PasswordHash, gauge *metric.Gauge,
) UserAuthHook {
return func(ctx context.Context, systemIdentity username.SQLUsername, clientConnection bool) error {
if systemIdentity.Undefined() {
Expand All @@ -252,7 +253,7 @@ func UserAuthPasswordHook(
return NewErrPasswordUserAuthFailed(systemIdentity)
}
ok, err := password.CompareHashAndCleartextPassword(ctx,
hashedPassword, passwordStr, GetExpensiveHashComputeSem(ctx))
hashedPassword, passwordStr, GetExpensiveHashComputeSemWithGauge(ctx, gauge))
if err != nil {
return err
}
Expand Down
24 changes: 21 additions & 3 deletions pkg/security/password.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/errors"
"golang.org/x/crypto/bcrypt"
Expand Down Expand Up @@ -209,8 +210,7 @@ var expensiveHashComputeSemOnce struct {
// Otherwise, a default is computed from the configure GOMAXPROCS.
var envMaxHashComputeConcurrency = envutil.EnvOrDefaultInt("COCKROACH_MAX_PW_HASH_COMPUTE_CONCURRENCY", 0)

// GetExpensiveHashComputeSem retrieves the hashing semaphore.
func GetExpensiveHashComputeSem(ctx context.Context) password.HashSemaphore {
func maybeInitializeSem(ctx context.Context) {
expensiveHashComputeSemOnce.once.Do(func() {
var n int
if envMaxHashComputeConcurrency >= 1 {
Expand All @@ -228,7 +228,25 @@ func GetExpensiveHashComputeSem(ctx context.Context) password.HashSemaphore {
log.VInfof(ctx, 1, "configured maximum hashing concurrency: %d", n)
expensiveHashComputeSemOnce.sem = quotapool.NewIntPool("password_hashes", uint64(n))
})
return func(ctx context.Context) (func(), error) {
}

// GetExpensiveHashComputeSem retrieves the hashing semaphore.
func GetExpensiveHashComputeSem(ctx context.Context) password.HashSemaphore {
return GetExpensiveHashComputeSemWithGauge(ctx, nil /* gauge */)
}

// GetExpensiveHashComputeSemWithGauge retrieves the hashing semaphore and
// will make the callback update a gauge to track the number of goroutines
// waiting for the semaphore.
func GetExpensiveHashComputeSemWithGauge(
ctx context.Context, gauge *metric.Gauge,
) password.HashSemaphore {
maybeInitializeSem(ctx)
return func(ctx context.Context) (cleanup func(), retErr error) {
if gauge != nil {
gauge.Inc(1)
defer gauge.Dec(1)
}
alloc, err := expensiveHashComputeSemOnce.sem.Acquire(ctx, 1)
if err != nil {
return nil, err
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/pgwire/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -401,6 +401,8 @@ type AuthConn interface {
LogAuthFailed(ctx context.Context, reason eventpb.AuthFailReason, err error)
// LogAuthOK logs when the authentication handshake has completed.
LogAuthOK(ctx context.Context)
// GetTenantSpecificMetrics returns the tenant-specific metrics for the connection.
GetTenantSpecificMetrics() *tenantSpecificMetrics
}

// authPipe is the implementation for the authenticator and AuthConn interfaces.
Expand Down Expand Up @@ -560,3 +562,8 @@ func (p *authPipe) SendAuthRequest(authType int32, data []byte) error {
c.msgBuilder.write(data)
return c.msgBuilder.finishMsg(c.conn)
}

// GetTenantSpecificMetrics is part of the AuthConn interface.
func (p *authPipe) GetTenantSpecificMetrics() *tenantSpecificMetrics {
return p.c.metrics
}
3 changes: 2 additions & 1 deletion pkg/sql/pgwire/auth_methods.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,9 +190,10 @@ func passwordAuthenticator(
// in auth.go (and report CREDENTIALS_INVALID).
}

metrics := c.GetTenantSpecificMetrics()
// Now check the cleartext password against the retrieved credentials.
if err := security.UserAuthPasswordHook(
false /*insecure*/, passwordStr, hashedPassword,
false, passwordStr, hashedPassword, metrics.ConnsWaitingToHash,
)(ctx, systemIdentity, clientConnection); err != nil {
if errors.HasType(err, &security.PasswordUserAuthError{}) {
c.LogAuthFailed(ctx, eventpb.AuthFailReason_CREDENTIALS_INVALID, err)
Expand Down
16 changes: 12 additions & 4 deletions pkg/sql/pgwire/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,12 @@ var (
Measurement: "Connections",
Unit: metric.Unit_COUNT,
}
MetaConnsWaitingToHash = metric.Metadata{
Name: "sql.conns_waiting_to_hash",
Help: "Number of SQL connection attempts that are being throttled in order to limit password hashing concurrency",
Measurement: "Connections",
Unit: metric.Unit_COUNT,
}
MetaBytesIn = metric.Metadata{
Name: "sql.bytesin",
Help: "Number of SQL bytes received",
Expand Down Expand Up @@ -284,6 +290,7 @@ type tenantSpecificMetrics struct {
BytesOutCount *metric.Counter
Conns *metric.Gauge
NewConns *metric.Counter
ConnsWaitingToHash *metric.Gauge
ConnLatency metric.IHistogram
ConnFailures *metric.Counter
PGWireCancelTotalCount *metric.Counter
Expand All @@ -297,10 +304,11 @@ func makeTenantSpecificMetrics(
sqlMemMetrics sql.MemoryMetrics, histogramWindow time.Duration,
) tenantSpecificMetrics {
return tenantSpecificMetrics{
BytesInCount: metric.NewCounter(MetaBytesIn),
BytesOutCount: metric.NewCounter(MetaBytesOut),
Conns: metric.NewGauge(MetaConns),
NewConns: metric.NewCounter(MetaNewConns),
BytesInCount: metric.NewCounter(MetaBytesIn),
BytesOutCount: metric.NewCounter(MetaBytesOut),
Conns: metric.NewGauge(MetaConns),
NewConns: metric.NewCounter(MetaNewConns),
ConnsWaitingToHash: metric.NewGauge(MetaConnsWaitingToHash),
ConnLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePreferHdrLatency,
Metadata: MetaConnLatency,
Expand Down

0 comments on commit b15f6f5

Please sign in to comment.