Skip to content

Commit

Permalink
pgwire: pre-normalize the HBA rules upon cluster setting changes
Browse files Browse the repository at this point in the history
Prior to this patch, the authentication code was relatively complex to
understand:

- the code path to use the HBA configuration would only be activated
  if the cluster setting was set. There was a special case if it
  wasn't. The special case was in the per-conn auth logic.
- the special case for the root login escape was also present in the
  per-conn auth logic.
- every time the cluster setting would be updated, the result of
  parsing the setting was cached as-is, and the textual strings
  re-interpreted over and over upon every connection.

This complexity can be interpreted as a defect, for the main reason
that **the per-conn auth code should be as simple as possible** to
facilitate security audits and future maintainance. Additionally, an
argument could be made that the per-conn re-interpretation of the
result was a performance mishap.

This patch aims to simplify the per-conn auth logic by folding all the
special casing as pre-defined rules in the HBA configuration:

- the default logic (when the cluster setting is empty or invalid)
  becomes a predefined HBA configuration with just two rules:

         host all root all cert
         host all all  all cert-password

- each time the config is loaded from a cluster setting, the
  root escape is implemented by force-inserting `host all root all
  cert` at the start of the configuration.

With this in place, the auth logic can be simplified to always
and exclusively use the HBA rules.

This special casing can also be inspected in the output of
`/debug/hba_conf`.

Additionally, this patch optimizes the code by pre-normalizing upfront
when the setting is updated. Normalizing includes:

- unicode-normalizing and case-folding usernames, since the username
  upon new connection is also normalized and case-folded.
- expanding lists of multiple usernames into multiple rules, so
  that the checking code can be simplified to only check one username
  per rule.
- name resolution of the authentication method into its function
  pointer.

Release note (security): The authentication code for new SQL
connections has been simplified to always use the HBA configuration
defined per `server.host_based_authentication.configuration`.

The format of this file generally follows that of `pg_hba.conf` as
defined here:
https://www.postgresql.org/docs/current/auth-pg-hba-conf.html.

Upon each configuration change, CockroachDB auto-magically inserts the
entry `host all root all cert` as a first rule, to ensure the root
user can always log in with a valid client certificate.

If the configuration is set to empty, or found to be invalid in the
cluster setting, the following default configuration is automatically
used:

         host all root all cert
         host all all  all cert-password

At any moment the current configuration on each node can be inspected
using the `/debug/hba_conf` URL on the HTTP endpoint.

The list of valid authentication methods is currently:

- `cert`, for certificate-based authentication over a SSL connection
  exclusively;
- `cert-password`, which allows either cert-based or password-based
  authentication over a SSL connection;
- `password` for password-based authentication over a SSL connection;
- `gss` for Kerberos-based authentication over a SSL connection,
  enabled when running a CCL binary and an Enterprise license.

In effect CockroachDB treats all the `host` rules as `hostssl`, and
behaves as per a default of `hostnossl all all all reject`.

It is not currently possible to define authentication rules over
non-SSL connections: as of this writing, non-SSL connections are only
possible when running with `--insecure`, and on insecure nodes all the
authentication logic is entirely disabled.

This behavior remains equivalent to previous CockroachDB versions,
and this change is only discussed here for clarity.
  • Loading branch information
knz committed Jan 7, 2020
1 parent 87e150d commit 76a74db
Show file tree
Hide file tree
Showing 17 changed files with 688 additions and 212 deletions.
91 changes: 38 additions & 53 deletions pkg/sql/pgwire/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,60 +72,10 @@ func (c *conn) handleAuthentication(

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.IsKeyword("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.IsKeyword("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))
}
methodFn, hbaEntry, err := c.lookupAuthenticationMethod(auth)
if err != nil {
return sendError(err)
}

authenticationHook, err := methodFn(ac, tlsState, insecure, hashedPassword, execCfg, hbaEntry)
Expand All @@ -142,6 +92,41 @@ func (c *conn) handleAuthentication(
return c.msgBuilder.finishMsg(c.conn)
}

func (c *conn) lookupAuthenticationMethod(
auth *hba.Conf,
) (methodFn AuthMethod, entry *hba.Entry, err error) {
// Extract the IP address of the client.
tcpAddr, ok := c.conn.RemoteAddr().(*net.TCPAddr)
if !ok {
return nil, nil, errors.AssertionFailedf("client address type %T unsupported", c.conn.RemoteAddr())
}
ip := tcpAddr.IP

// Look up the method.
for i := range auth.Entries {
entry := &auth.Entries[i]
addrMatch, err := entry.AddressMatches(ip)
if err != nil {
// TODO(knz): Determine if an error should be reported
// upon unknown address formats.
// See: https://github.com/cockroachdb/cockroach/issues/43716
return nil, nil, err
}
if !addrMatch {
// The address does not match.
continue
}
if !entry.UserMatches(c.sessionArgs.User) {
// The user does not match.
continue
}
return entry.MethodFn.(AuthMethod), entry, nil
}

// No match.
return nil, nil, errors.Errorf("no %s entry for host %q, user %q", serverHBAConfSetting, ip, c.sessionArgs.User)
}

// 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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/auth_methods.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import (
// e.g. in the CCL modules to add support for GSS authentication using
// Kerberos.

func init() {
func loadDefaultMethods() {
// The "password" method requires a clear text password.
//
// Care should be taken by administrators to only accept this auth
Expand Down
17 changes: 8 additions & 9 deletions pkg/sql/pgwire/auth_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,26 +13,24 @@ package pgwire_test
import (
"context"
gosql "database/sql"
"errors"
"fmt"
"io/ioutil"
"net"
"net/url"
"reflect"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/stdstrings"
"github.com/lib/pq"
)
Expand Down Expand Up @@ -148,19 +146,20 @@ func hbaRunTest(t *testing.T, insecure bool) {
// Wait until the configuration has propagated back to the
// test client. We need to wait because the cluster setting
// change propagates asynchronously.
var expConf *hba.Conf
expConf := pgwire.DefaultHBAConfig
if td.Input != "" {
expConf, err = hba.Parse(td.Input)
expConf, err = pgwire.ParseAndNormalize(td.Input)
if err != nil {
// The SET above succeeded so we don't expect a problem here.
t.Fatal(err)
}
pgwire.NormalizeHBAEntries(expConf)
}
testutils.SucceedsSoon(t, func() error {
curConf := pgServer.TestingGetHBAConf()
if !reflect.DeepEqual(expConf, curConf) {
return errors.New("HBA config not yet loaded")
curConf := pgServer.GetAuthenticationConfiguration()
if expConf.String() != curConf.String() {
return errors.Newf(
"HBA config not yet loaded\ngot:\n%s\nexpected:\n%s",
curConf, expConf)
}
return nil
})
Expand Down
Loading

0 comments on commit 76a74db

Please sign in to comment.