From 36baefcd097b11aeea87eeefa371958d1c86e208 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Fri, 3 Jan 2020 13:13:23 +0100 Subject: [PATCH] pgwire: split the authentication code in its own files This patch splits the pgwire authentication in its own files and adds missing explanatory comments. Release note: None --- pkg/sql/pgwire/auth.go | 271 ++++++++++++++++++++++ pkg/sql/pgwire/auth_methods.go | 130 +++++++++++ pkg/sql/pgwire/conn.go | 400 +-------------------------------- pkg/sql/pgwire/hba_conf.go | 168 ++++++++++++++ pkg/sql/pgwire/server.go | 28 +-- 5 files changed, 575 insertions(+), 422 deletions(-) create mode 100644 pkg/sql/pgwire/auth.go create mode 100644 pkg/sql/pgwire/auth_methods.go create mode 100644 pkg/sql/pgwire/hba_conf.go diff --git a/pkg/sql/pgwire/auth.go b/pkg/sql/pgwire/auth.go new file mode 100644 index 000000000000..75dd984d8e96 --- /dev/null +++ b/pkg/sql/pgwire/auth.go @@ -0,0 +1,271 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package pgwire + +import ( + "context" + "crypto/tls" + "net" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" + "github.com/cockroachdb/errors" +) + +const ( + // authOK is the pgwire auth response code for successful authentication + // during the connection handshake. + authOK int32 = 0 + // authCleartextPassword is the pgwire auth response code to request + // a plaintext password during the connection handshake. + authCleartextPassword int32 = 3 +) + +type authOptions struct { + skipAuth bool // test-only + authHook func(ctx context.Context) error // test-only + insecure bool + auth *hba.Conf + ie *sql.InternalExecutor +} + +// handleAuthentication checks the connection's user. Errors are sent to the +// client and also returned. +// +// TODO(knz): handleAuthentication should discuss with the client to arrange +// authentication and update c.sessionArgs with the authenticated user's name, +// if different from the one given initially. +func (c *conn) handleAuthentication( + ctx context.Context, + ac AuthConn, + insecure bool, + ie *sql.InternalExecutor, + auth *hba.Conf, + execCfg *sql.ExecutorConfig, +) error { + sendError := func(err error) error { + _ /* err */ = writeErr(ctx, &execCfg.Settings.SV, err, &c.msgBuilder, c.conn) + return err + } + + // Check that the requested user exists and retrieve the hashed + // password in case password authentication is needed. + exists, hashedPassword, err := sql.GetUserHashedPassword( + ctx, ie, &c.metrics.SQLMemMetrics, c.sessionArgs.User, + ) + if err != nil { + return sendError(err) + } + if !exists { + return sendError(errors.Errorf(security.ErrPasswordUserAuthFailed, c.sessionArgs.User)) + } + + if tlsConn, ok := c.conn.(*readTimeoutConn).Conn.(*tls.Conn); ok { + tlsState := tlsConn.ConnectionState() + var methodFn AuthMethod + var hbaEntry *hba.Entry + + if auth == nil { + methodFn = authCertPassword + } else if c.sessionArgs.User == security.RootUser { + // If a hba.conf file is specified, hard code the root user to always use + // cert auth. This prevents users from shooting themselves in the foot and + // making root not able to login, thus disallowing anyone from fixing the + // hba.conf file. + methodFn = authCert + } else { + addr, _, err := net.SplitHostPort(c.conn.RemoteAddr().String()) + if err != nil { + return sendError(err) + } + ip := net.ParseIP(addr) + for _, entry := range auth.Entries { + switch a := entry.Address.(type) { + case *net.IPNet: + if !a.Contains(ip) { + continue + } + case hba.String: + if !a.IsSpecial("all") { + return sendError(errors.Errorf("unexpected %s address: %q", serverHBAConfSetting, a.Value)) + } + default: + return sendError(errors.Errorf("unexpected address type %T", a)) + } + match := false + for _, u := range entry.User { + if u.IsSpecial("all") { + match = true + break + } + if u.Value == c.sessionArgs.User { + match = true + break + } + } + if !match { + continue + } + methodFn = hbaAuthMethods[entry.Method] + if methodFn == nil { + return sendError(errors.Errorf("unknown auth method %s", entry.Method)) + } + hbaEntry = &entry + break + } + if methodFn == nil { + return sendError(errors.Errorf("no %s entry for host %q, user %q", serverHBAConfSetting, addr, c.sessionArgs.User)) + } + } + + authenticationHook, err := methodFn(ac, tlsState, insecure, hashedPassword, execCfg, hbaEntry) + if err != nil { + return sendError(err) + } + if err := authenticationHook(c.sessionArgs.User, true /* public */); err != nil { + return sendError(err) + } + } + + c.msgBuilder.initMsg(pgwirebase.ServerMsgAuth) + c.msgBuilder.putInt32(authOK) + return c.msgBuilder.finishMsg(c.conn) +} + +// authenticatorIO is the interface used by the connection to pass password data +// to the authenticator and expect an authentication decision from it. +type authenticatorIO interface { + // sendPwdData is used to push authentication data into the authenticator. + // This call is blocking; authenticators are supposed to consume data hastily + // once they've requested it. + sendPwdData(data []byte) error + // noMorePwdData is used to inform the authenticator that the client is not + // sending any more authentication data. This method can be called multiple + // times. + noMorePwdData() + // authResult blocks for an authentication decision. This call also informs + // the authenticator that no more auth data is coming from the client; + // noMorePwdData() is called internally. + // + // The auth result is either an unqualifiedIntSizer (in case the auth + // succeeded) or an auth error. + authResult() (unqualifiedIntSizer, error) +} + +// AuthConn is the interface used by the authenticator for interacting with the +// pgwire connection. +type AuthConn interface { + // SendAuthRequest send a request for authentication information. After + // calling this, the authenticator needs to call GetPwdData() quickly, as the + // connection's goroutine will be blocked on providing us the requested data. + SendAuthRequest(authType int32, data []byte) error + // GetPwdData returns authentication info that was previously requested with + // SendAuthRequest. The call blocks until such data is available. + // An error is returned if the client connection dropped or if the client + // didn't respect the protocol. After an error has been returned, GetPwdData() + // cannot be called any more. + GetPwdData() ([]byte, error) + // AuthOK declares that authentication succeeded and provides a + // unqualifiedIntSizer, to be returned by authenticator.authResult(). Future + // authenticator.sendPwdData() calls fail. + AuthOK(unqualifiedIntSizer) + // AuthFail declares that authentication has failed and provides an error to + // be returned by authenticator.authResult(). Future + // authenticator.sendPwdData() calls fail. The error has already been written + // to the client connection. + AuthFail(err error) +} + +// authPipe is the implementation for the authenticator and AuthConn interfaces. +// A single authPipe will serve as both an AuthConn and an authenticator; the +// two represent the two "ends" of the pipe and we'll pass data between them. +type authPipe struct { + c *conn // Only used for writing, not for reading. + + ch chan []byte + // writerDone is a channel closed by noMorePwdData(). + // Nil if noMorePwdData(). + writerDone chan struct{} + readerDone chan authRes +} + +type authRes struct { + intSizer unqualifiedIntSizer + err error +} + +func newAuthPipe(c *conn) *authPipe { + ap := &authPipe{ + c: c, + ch: make(chan []byte), + writerDone: make(chan struct{}), + readerDone: make(chan authRes, 1), + } + return ap +} + +var _ authenticatorIO = &authPipe{} +var _ AuthConn = &authPipe{} + +func (p *authPipe) sendPwdData(data []byte) error { + select { + case p.ch <- data: + return nil + case <-p.readerDone: + return pgwirebase.NewProtocolViolationErrorf("unexpected auth data") + } +} + +func (p *authPipe) noMorePwdData() { + if p.writerDone == nil { + return + } + // A reader blocked in GetPwdData() gets unblocked with an error. + close(p.writerDone) + p.writerDone = nil +} + +// GetPwdData is part of the AuthConn interface. +func (p *authPipe) GetPwdData() ([]byte, error) { + select { + case data := <-p.ch: + return data, nil + case <-p.writerDone: + return nil, pgwirebase.NewProtocolViolationErrorf("client didn't send required auth data") + } +} + +// AuthOK is part of the AuthConn interface. +func (p *authPipe) AuthOK(intSizer unqualifiedIntSizer) { + p.readerDone <- authRes{intSizer: intSizer} +} + +func (p *authPipe) AuthFail(err error) { + p.readerDone <- authRes{err: err} +} + +// authResult is part of the authenticator interface. +func (p *authPipe) authResult() (unqualifiedIntSizer, error) { + p.noMorePwdData() + res := <-p.readerDone + return res.intSizer, res.err +} + +// SendAuthRequest is part of the AuthConn interface. +func (p *authPipe) SendAuthRequest(authType int32, data []byte) error { + c := p.c + c.msgBuilder.initMsg(pgwirebase.ServerMsgAuth) + c.msgBuilder.putInt32(authType) + c.msgBuilder.write(data) + return c.msgBuilder.finishMsg(c.conn) +} diff --git a/pkg/sql/pgwire/auth_methods.go b/pkg/sql/pgwire/auth_methods.go new file mode 100644 index 000000000000..1f81a219285c --- /dev/null +++ b/pkg/sql/pgwire/auth_methods.go @@ -0,0 +1,130 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package pgwire + +import ( + "bytes" + "crypto/tls" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/errors" +) + +// This file contains the methods that are accepted to perform +// authentication of users during the pgwire connection handshake. +// +// Which method are accepted for which user is selected using +// the HBA config loaded into the cluster setting +// server.host_based_authentication.configuration. +// +// Other methods can be added using RegisterAuthMethod(). This is done +// e.g. in the CCL modules to add support for GSS authentication using +// Kerberos. + +func init() { + // The "password" method requires a clear text password. + // + // Care should be taken by administrators to only accept this auth + // method over secure connections, e.g. those encrypted using SSL. + RegisterAuthMethod("password", authPassword, nil) + + // The "cert" method requires a valid client certificate for the + // user attempting to connect. + // + // This method is only usable over SSL connections. + RegisterAuthMethod("cert", authCert, nil) + + // The "cert-password" method requires either a valid client + // certificate for the connecting user, or, if no cert is provided, + // a cleartext password. + RegisterAuthMethod("cert-password", authCertPassword, nil) +} + +// AuthMethod defines a method for authentication of a connection. +type AuthMethod func( + c AuthConn, + tlsState tls.ConnectionState, + insecure bool, + hashedPassword []byte, + execCfg *sql.ExecutorConfig, + entry *hba.Entry, +) (security.UserAuthHook, error) + +func authPassword( + c AuthConn, + tlsState tls.ConnectionState, + insecure bool, + hashedPassword []byte, + execCfg *sql.ExecutorConfig, + entry *hba.Entry, +) (security.UserAuthHook, error) { + if err := c.SendAuthRequest(authCleartextPassword, nil /* data */); err != nil { + return nil, err + } + pwdData, err := c.GetPwdData() + if err != nil { + return nil, err + } + password, err := passwordString(pwdData) + if err != nil { + return nil, err + } + return security.UserAuthPasswordHook( + insecure, password, hashedPassword, + ), nil +} + +func passwordString(pwdData []byte) (string, error) { + // Make a string out of the byte array. + if bytes.IndexByte(pwdData, 0) != len(pwdData)-1 { + return "", fmt.Errorf("expected 0-terminated byte array") + } + return string(pwdData[:len(pwdData)-1]), nil +} + +func authCert( + _ AuthConn, + tlsState tls.ConnectionState, + insecure bool, + hashedPassword []byte, + execCfg *sql.ExecutorConfig, + entry *hba.Entry, +) (security.UserAuthHook, error) { + if len(tlsState.PeerCertificates) == 0 { + return nil, errors.New("no TLS peer certificates, but required for auth") + } + // Normalize the username contained in the certificate. + tlsState.PeerCertificates[0].Subject.CommonName = tree.Name( + tlsState.PeerCertificates[0].Subject.CommonName, + ).Normalize() + return security.UserAuthCertHook(insecure, &tlsState) +} + +func authCertPassword( + c AuthConn, + tlsState tls.ConnectionState, + insecure bool, + hashedPassword []byte, + execCfg *sql.ExecutorConfig, + entry *hba.Entry, +) (security.UserAuthHook, error) { + var fn AuthMethod + if len(tlsState.PeerCertificates) == 0 { + fn = authPassword + } else { + fn = authCert + } + return fn(c, tlsState, insecure, hashedPassword, execCfg, entry) +} diff --git a/pkg/sql/pgwire/conn.go b/pkg/sql/pgwire/conn.go index c717315a0ddb..ce343959a316 100644 --- a/pkg/sql/pgwire/conn.go +++ b/pkg/sql/pgwire/conn.go @@ -14,13 +14,10 @@ import ( "bufio" "bytes" "context" - "crypto/tls" "fmt" "io" "net" - "sort" "strconv" - "strings" "sync" "sync/atomic" "time" @@ -29,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/parser" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" @@ -38,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -49,11 +44,6 @@ import ( "github.com/lib/pq/oid" ) -const ( - authOK int32 = 0 - authCleartextPassword int32 = 3 -) - // conn implements a pgwire network connection (version 3 of the protocol, // implemented by Postgres v7.4 and later). conn.serve() reads protocol // messages, transforms them into commands that it pushes onto a StmtBuf (where @@ -102,14 +92,6 @@ type conn struct { sv *settings.Values } -type authOptions struct { - skipAuth bool // test-only - authHook func(ctx context.Context) error // test-only - insecure bool - auth *hba.Conf - ie *sql.InternalExecutor -} - // serveConn creates a conn that will serve the netConn. It returns once the // network connection is closed. // @@ -256,7 +238,7 @@ func (c *conn) serveImpl( // We'll build an authPipe to communicate with the authentication process. authPipe := newAuthPipe(c) - var authenticator authenticator = authPipe + var authenticator authenticatorIO = authPipe // procCh is the channel on which we'll receive the termination signal from // the command processor. @@ -1485,386 +1467,6 @@ func (r *pgwireReader) ReadByte() (byte, error) { return b, err } -// handleAuthentication checks the connection's user. Errors are sent to the -// client and also returned. -// -// TODO(knz): handleAuthentication should discuss with the client to arrange -// authentication and update c.sessionArgs with the authenticated user's name, -// if different from the one given initially. -func (c *conn) handleAuthentication( - ctx context.Context, - ac AuthConn, - insecure bool, - ie *sql.InternalExecutor, - auth *hba.Conf, - execCfg *sql.ExecutorConfig, -) error { - sendError := func(err error) error { - _ /* err */ = writeErr(ctx, &execCfg.Settings.SV, err, &c.msgBuilder, c.conn) - return err - } - - // Check that the requested user exists and retrieve the hashed - // password in case password authentication is needed. - exists, hashedPassword, err := sql.GetUserHashedPassword( - ctx, ie, &c.metrics.SQLMemMetrics, c.sessionArgs.User, - ) - if err != nil { - return sendError(err) - } - if !exists { - return sendError(errors.Errorf(security.ErrPasswordUserAuthFailed, c.sessionArgs.User)) - } - - if tlsConn, ok := c.conn.(*readTimeoutConn).Conn.(*tls.Conn); ok { - tlsState := tlsConn.ConnectionState() - var methodFn AuthMethod - var hbaEntry *hba.Entry - - if auth == nil { - methodFn = authCertPassword - } else if c.sessionArgs.User == security.RootUser { - // If a hba.conf file is specified, hard code the root user to always use - // cert auth. This prevents users from shooting themselves in the foot and - // making root not able to login, thus disallowing anyone from fixing the - // hba.conf file. - methodFn = authCert - } else { - addr, _, err := net.SplitHostPort(c.conn.RemoteAddr().String()) - if err != nil { - return sendError(err) - } - ip := net.ParseIP(addr) - for _, entry := range auth.Entries { - switch a := entry.Address.(type) { - case *net.IPNet: - if !a.Contains(ip) { - continue - } - case hba.String: - if !a.IsSpecial("all") { - return sendError(errors.Errorf("unexpected %s address: %q", serverHBAConfSetting, a.Value)) - } - default: - return sendError(errors.Errorf("unexpected address type %T", a)) - } - match := false - for _, u := range entry.User { - if u.IsSpecial("all") { - match = true - break - } - if u.Value == c.sessionArgs.User { - match = true - break - } - } - if !match { - continue - } - methodFn = hbaAuthMethods[entry.Method] - if methodFn == nil { - return sendError(errors.Errorf("unknown auth method %s", entry.Method)) - } - hbaEntry = &entry - break - } - if methodFn == nil { - return sendError(errors.Errorf("no %s entry for host %q, user %q", serverHBAConfSetting, addr, c.sessionArgs.User)) - } - } - - authenticationHook, err := methodFn(ac, tlsState, insecure, hashedPassword, execCfg, hbaEntry) - if err != nil { - return sendError(err) - } - if err := authenticationHook(c.sessionArgs.User, true /* public */); err != nil { - return sendError(err) - } - } - - c.msgBuilder.initMsg(pgwirebase.ServerMsgAuth) - c.msgBuilder.putInt32(authOK) - return c.msgBuilder.finishMsg(c.conn) -} - -const serverHBAConfSetting = "server.host_based_authentication.configuration" - -var connAuthConf = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - serverHBAConfSetting, - "host-based authentication configuration to use during connection authentication", - "", - func(values *settings.Values, s string) error { - if s == "" { - return nil - } - conf, err := hba.Parse(s) - if err != nil { - return err - } - for _, entry := range conf.Entries { - for _, db := range entry.Database { - if !db.IsSpecial("all") { - return errors.WithHint( - unimplemented.New("hba-per-db", "per-database HBA rules are not supported"), - "Use the special value 'all' (without quotes) to match all databases.") - } - } - if addr, ok := entry.Address.(hba.String); ok && !addr.IsSpecial("all") { - return errors.WithHint( - unimplemented.New("hba-hostnames", "hostname-based HBA rules are not supported"), - "List the numeric CIDR notation instead, for example: 127.0.0.1/8.") - } - if hbaAuthMethods[entry.Method] == nil { - return errors.WithHintf(unimplemented.Newf("hba-method-"+entry.Method, - "unknown auth method %q", entry.Method), - "Supported methods: %s", listSupportedMethods()) - } - if check := hbaCheckHBAEntries[entry.Method]; check != nil { - if err := check(entry); err != nil { - return err - } - } - } - return nil - }, - ) - s.SetVisibility(settings.Public) - return s -}() - -// authenticator is the interface used by the connection to pass password data -// to the authenticator and expect an authentication decision from it. -type authenticator interface { - // sendPwdData is used to push authentication data into the authenticator. - // This call is blocking; authenticators are supposed to consume data hastily - // once they've requested it. - sendPwdData(data []byte) error - // noMorePwdData is used to inform the authenticator that the client is not - // sending any more authentication data. This method can be called multiple - // times. - noMorePwdData() - // authResult blocks for an authentication decision. This call also informs - // the authenticator that no more auth data is coming from the client; - // noMorePwdData() is called internally. - // - // The auth result is either an unqualifiedIntSizer (in case the auth - // succeeded) or an auth error. - authResult() (unqualifiedIntSizer, error) -} - -// AuthConn is the interface used by the authenticator for interacting with the -// pgwire connection. -type AuthConn interface { - // SendAuthRequest send a request for authentication information. After - // calling this, the authenticator needs to call GetPwdData() quickly, as the - // connection's goroutine will be blocked on providing us the requested data. - SendAuthRequest(authType int32, data []byte) error - // GetPwdData returns authentication info that was previously requested with - // SendAuthRequest. The call blocks until such data is available. - // An error is returned if the client connection dropped or if the client - // didn't respect the protocol. After an error has been returned, GetPwdData() - // cannot be called any more. - GetPwdData() ([]byte, error) - // AuthOK declares that authentication succeeded and provides a - // unqualifiedIntSizer, to be returned by authenticator.authResult(). Future - // authenticator.sendPwdData() calls fail. - AuthOK(unqualifiedIntSizer) - // AuthFail declares that authentication has failed and provides an error to - // be returned by authenticator.authResult(). Future - // authenticator.sendPwdData() calls fail. The error has already been written - // to the client connection. - AuthFail(err error) -} - -// authPipe is the implementation for the authenticator and AuthConn interfaces. -// A single authPipe will serve as both an AuthConn and an authenticator; the -// two represent the two "ends" of the pipe and we'll pass data between them. -type authPipe struct { - c *conn // Only used for writing, not for reading. - - ch chan []byte - // writerDone is a channel closed by noMorePwdData(). - // Nil if noMorePwdData(). - writerDone chan struct{} - readerDone chan authRes -} - -type authRes struct { - intSizer unqualifiedIntSizer - err error -} - -func newAuthPipe(c *conn) *authPipe { - ap := &authPipe{ - c: c, - ch: make(chan []byte), - writerDone: make(chan struct{}), - readerDone: make(chan authRes, 1), - } - return ap -} - -var _ authenticator = &authPipe{} -var _ AuthConn = &authPipe{} - -func (p *authPipe) sendPwdData(data []byte) error { - select { - case p.ch <- data: - return nil - case <-p.readerDone: - return pgwirebase.NewProtocolViolationErrorf("unexpected auth data") - } -} - -func (p *authPipe) noMorePwdData() { - if p.writerDone == nil { - return - } - // A reader blocked in GetPwdData() gets unblocked with an error. - close(p.writerDone) - p.writerDone = nil -} - -// GetPwdData is part of the AuthConn interface. -func (p *authPipe) GetPwdData() ([]byte, error) { - select { - case data := <-p.ch: - return data, nil - case <-p.writerDone: - return nil, pgwirebase.NewProtocolViolationErrorf("client didn't send required auth data") - } -} - -// AuthOK is part of the AuthConn interface. -func (p *authPipe) AuthOK(intSizer unqualifiedIntSizer) { - p.readerDone <- authRes{intSizer: intSizer} -} - -func (p *authPipe) AuthFail(err error) { - p.readerDone <- authRes{err: err} -} - -// authResult is part of the authenticator interface. -func (p *authPipe) authResult() (unqualifiedIntSizer, error) { - p.noMorePwdData() - res := <-p.readerDone - return res.intSizer, res.err -} - -// SendAuthRequest is part of the AuthConn interface. -func (p *authPipe) SendAuthRequest(authType int32, data []byte) error { - c := p.c - c.msgBuilder.initMsg(pgwirebase.ServerMsgAuth) - c.msgBuilder.putInt32(authType) - c.msgBuilder.write(data) - return c.msgBuilder.finishMsg(c.conn) -} - -type ( - // AuthMethod defines a method for authentication of a connection. - AuthMethod func(c AuthConn, tlsState tls.ConnectionState, insecure bool, hashedPassword []byte, execCfg *sql.ExecutorConfig, entry *hba.Entry) (security.UserAuthHook, error) - - // CheckHBAEntry defines a method for error checking an hba Entry. - CheckHBAEntry func(hba.Entry) error -) - -var ( - hbaAuthMethods = map[string]AuthMethod{} - hbaCheckHBAEntries = map[string]CheckHBAEntry{} -) - -// RegisterAuthMethod registers an AuthMethod for pgwire authentication. -func RegisterAuthMethod(method string, fn AuthMethod, checkEntry CheckHBAEntry) { - hbaAuthMethods[method] = fn - if checkEntry != nil { - hbaCheckHBAEntries[method] = checkEntry - } -} - -func listSupportedMethods() string { - methods := make([]string, 0, len(hbaAuthMethods)) - for method := range hbaAuthMethods { - methods = append(methods, method) - } - sort.Strings(methods) - return strings.Join(methods, ", ") -} - -func passwordString(pwdData []byte) (string, error) { - // Make a string out of the byte array. - if bytes.IndexByte(pwdData, 0) != len(pwdData)-1 { - return "", fmt.Errorf("expected 0-terminated byte array") - } - return string(pwdData[:len(pwdData)-1]), nil -} - -func authPassword( - c AuthConn, - tlsState tls.ConnectionState, - insecure bool, - hashedPassword []byte, - execCfg *sql.ExecutorConfig, - entry *hba.Entry, -) (security.UserAuthHook, error) { - if err := c.SendAuthRequest(authCleartextPassword, nil /* data */); err != nil { - return nil, err - } - pwdData, err := c.GetPwdData() - if err != nil { - return nil, err - } - password, err := passwordString(pwdData) - if err != nil { - return nil, err - } - return security.UserAuthPasswordHook( - insecure, password, hashedPassword, - ), nil -} - -func authCert( - _ AuthConn, - tlsState tls.ConnectionState, - insecure bool, - hashedPassword []byte, - execCfg *sql.ExecutorConfig, - entry *hba.Entry, -) (security.UserAuthHook, error) { - if len(tlsState.PeerCertificates) == 0 { - return nil, errors.New("no TLS peer certificates, but required for auth") - } - // Normalize the username contained in the certificate. - tlsState.PeerCertificates[0].Subject.CommonName = tree.Name( - tlsState.PeerCertificates[0].Subject.CommonName, - ).Normalize() - return security.UserAuthCertHook(insecure, &tlsState) -} - -func authCertPassword( - c AuthConn, - tlsState tls.ConnectionState, - insecure bool, - hashedPassword []byte, - execCfg *sql.ExecutorConfig, - entry *hba.Entry, -) (security.UserAuthHook, error) { - var fn AuthMethod - if len(tlsState.PeerCertificates) == 0 { - fn = authPassword - } else { - fn = authCert - } - return fn(c, tlsState, insecure, hashedPassword, execCfg, entry) -} - -func init() { - RegisterAuthMethod("password", authPassword, nil) - RegisterAuthMethod("cert", authCert, nil) - RegisterAuthMethod("cert-password", authCertPassword, nil) -} - // statusReportParams is a list of session variables that are also // reported as server run-time parameters in the pgwire connection // initialization. diff --git a/pkg/sql/pgwire/hba_conf.go b/pkg/sql/pgwire/hba_conf.go new file mode 100644 index 000000000000..a7a2c83083e2 --- /dev/null +++ b/pkg/sql/pgwire/hba_conf.go @@ -0,0 +1,168 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package pgwire + +import ( + "context" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" +) + +// This file contains the logic for the configuration of HBA rules. +// +// In a nutshell, administrators customize the cluster setting +// `server.host_based_authentication.configuration`; each time they +// do so, all the nodes parse this configuration and re-initialize +// their authentication rules (a list of entries) from the setting. +// +// The HBA configuration is an ordered list of rules. Each time +// a client attempts to connect, the server scans the +// rules from the beginning of the list. The first rule that +// matches the connection decides how to authenticate. +// +// The syntax is inspired/derived from that of PostgreSQL's pg_hba.conf: +// https://www.postgresql.org/docs/12/auth-pg-hba-conf.html +// +// For now, CockroachDB only supports the following syntax: +// +// host all +// +// The matching rules are as follows: +// - A rule matches if the connecting username matches either of the +// usernames listed in the rule, or if the pseudo-user 'all' is +// present in the user column. +// - A rule matches if the connecting client's IP address is included +// in the network address specified in the CIDR notation. +// + +// serverHBAConfSetting is the name of the cluster setting that holds +// the HBA configuration. +const serverHBAConfSetting = "server.host_based_authentication.configuration" + +// connAuthConf is the cluster setting that holds the HBA configuration. +var connAuthConf = func() *settings.StringSetting { + s := settings.RegisterValidatedStringSetting( + serverHBAConfSetting, + "host-based authentication configuration to use during connection authentication", + "", + checkHBASyntaxBeforeUpdatingSetting, + ) + s.SetVisibility(settings.Public) + return s +}() + +// loadLocalAuthConfigUponRemoteSettingChange initializes the local node's cache +// of the auth configuration each time the cluster setting is updated. +func loadLocalAuthConfigUponRemoteSettingChange( + ctx context.Context, server *Server, st *cluster.Settings, +) { + val := connAuthConf.Get(&st.SV) + server.auth.Lock() + defer server.auth.Unlock() + if val == "" { + server.auth.conf = nil + return + } + conf, err := hba.Parse(val) + if err != nil { + log.Warningf(ctx, "invalid %s: %v", serverHBAConfSetting, err) + conf = nil + } + // Usernames are normalized during session init. Normalize the HBA usernames + // in the same way. + for _, entry := range conf.Entries { + for iu := range entry.User { + user := &entry.User[iu] + user.Value = tree.Name(user.Value).Normalize() + } + } + server.auth.conf = conf +} + +// checkHBASyntaxBeforeUpdatingSetting is run by the SQL gateway each time +// a client attempts to update the cluster setting. +func checkHBASyntaxBeforeUpdatingSetting(values *settings.Values, s string) error { + if s == "" { + // An empty configuration is always valid. + return nil + } + conf, err := hba.Parse(s) + if err != nil { + return err + } + for _, entry := range conf.Entries { + for _, db := range entry.Database { + if !db.IsSpecial("all") { + return errors.WithHint( + unimplemented.New("hba-per-db", "per-database HBA rules are not supported"), + "Use the special value 'all' (without quotes) to match all databases.") + } + } + if addr, ok := entry.Address.(hba.String); ok && !addr.IsSpecial("all") { + return errors.WithHint( + unimplemented.New("hba-hostnames", "hostname-based HBA rules are not supported"), + "List the numeric CIDR notation instead, for example: 127.0.0.1/8.") + } + if hbaAuthMethods[entry.Method] == nil { + return errors.WithHintf(unimplemented.Newf("hba-method-"+entry.Method, + "unknown auth method %q", entry.Method), + "Supported methods: %s", listRegisteredMethods()) + } + if check := hbaCheckHBAEntries[entry.Method]; check != nil { + if err := check(entry); err != nil { + return err + } + } + } + return nil +} + +// RegisterAuthMethod registers an AuthMethod for pgwire +// authentication and for use in HBA configuration. +// +// The checkEntry method, if provided, is called upon configuration +// the cluster setting in the SQL client which attempts to change the +// configuration. It can block the configuration if e.g. the syntax is +// invalid. +func RegisterAuthMethod(method string, fn AuthMethod, checkEntry CheckHBAEntry) { + hbaAuthMethods[method] = fn + if checkEntry != nil { + hbaCheckHBAEntries[method] = checkEntry + } +} + +// listsupportedMethods returns a sorted, comma-delimited list +// of registered AuthMethods. +func listRegisteredMethods() string { + methods := make([]string, 0, len(hbaAuthMethods)) + for method := range hbaAuthMethods { + methods = append(methods, method) + } + sort.Strings(methods) + return strings.Join(methods, ", ") +} + +var ( + hbaAuthMethods = map[string]AuthMethod{} + hbaCheckHBAEntries = map[string]CheckHBAEntry{} +) + +// CheckHBAEntry defines a method for validating an hba Entry upon +// configuration of the cluster setting by a SQL client. +type CheckHBAEntry func(hba.Entry) error diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index c8e9540cd437..d904562aa7f7 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -241,29 +241,11 @@ func MakeServer( server.mu.connCancelMap = make(cancelChanMap) server.mu.Unlock() - connAuthConf.SetOnChange(&st.SV, func() { - val := connAuthConf.Get(&st.SV) - server.auth.Lock() - defer server.auth.Unlock() - if val == "" { - server.auth.conf = nil - return - } - conf, err := hba.Parse(val) - if err != nil { - log.Warningf(ambientCtx.AnnotateCtx(context.Background()), "invalid %s: %v", serverHBAConfSetting, err) - conf = nil - } - // Usernames are normalized during session init. Normalize the HBA usernames - // in the same way. - for _, entry := range conf.Entries { - for iu := range entry.User { - user := &entry.User[iu] - user.Value = tree.Name(user.Value).Normalize() - } - } - server.auth.conf = conf - }) + connAuthConf.SetOnChange(&st.SV, + func() { + loadLocalAuthConfigUponRemoteSettingChange( + ambientCtx.AnnotateCtx(context.Background()), server, st) + }) return server }