Skip to content

Commit

Permalink
pgwire: properly support unix socket clients with authentication
Browse files Browse the repository at this point in the history
tldr: this patch makes unix sockets more production-ready,
by enabling clients to use unix sockets in secure mode
and enabling authentication over unix sockets.

**Motivation:**

[Unix domain
sockets](https://en.wikipedia.org/wiki/Unix_domain_socket) are a way
for a server process to accept direct in-memory connections from
processes running on the same machine as the server. They are simpler
and faster as they avoid the TCP/IP stack entirely.

Unix sockets are used both to provide a local client interface
for administrator users operating the system; as well as
setting up more complex authentication systems using the following
topology:

```
         client
           ^
           |
     (non-standard protocol)
           |
.----------|--------------(server machine)--------------------------.
|          v                                                        |
| ,----------------------.                   ,--------------------. |
| |  connection proxy    |                   | server process     | |
| |  and transport-level |<--(unix socket)-->| and authentication | |
| |   security           |                   | (e.g. crdb)        | |
| `----------------------'                   `--------------------' |
`-------------------------------------------------------------------'
```

**Description of this change:**

CockroachDB already supports setting up a unix socket for use by
clients running on the same machine, subject to regular Unix
permission checks.

Prior to this patch, support for unix sockets was incomplete:

- it would work properly for insecure nodes/clusters; however, ...
- ... in secure mode, it would also require a TLS handshake over
  the unix socket, which is neither supported by pg clients
  nor meaningful: unix domain sockets have transport-level
  security already.

This patch extends/fixes support for unix sockets as follows:

- it properly accepts client connections without TLS over
  unix sockets;
- it subjects incoming unix socket connections to the standard HBA
  rule-based authentication selection (via the cluster setting
  `server.host_based_authentication.configuration`);
- it changes the default HBA configuration to contain
  a default `local` rule that requires password
  authentication, in a way compatible with PostgreSQL;
- it un-hides the `--socket` parameter from the output of
  `cockroach start --help`.

Release note (cli change): Connections using Unix sockets are now
accepted even when the server is running in secure more.
(Consult `cockroach start --help` for details about the `--socket`
parameter.)

Release note (security): Connections using unix sockets are now
subject to the HBA rules defined via the setting
`server.host_based_authentication.configuration`, in a way compatible
with PostgreSQL: incoming unix connections match `local` rules,
whereas incoming TCP connections match `host` rules.
The default HBA configuration used when the cluster
setting is empty is now:

    host      all root all cert
    host      all all  all cert-password
    local     all all      password
  • Loading branch information
knz committed Jan 9, 2020
1 parent 0f90885 commit c18eb14
Show file tree
Hide file tree
Showing 15 changed files with 379 additions and 107 deletions.
22 changes: 22 additions & 0 deletions pkg/acceptance/cli_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,3 +90,25 @@ func TestDockerCLI(t *testing.T) {
})
}
}

// TestDockerUnixSocket verifies that CockroachDB initializes a unix
// socket useable by 'psql', even when the server runs insecurely.
func TestDockerUnixSocket(t *testing.T) {
s := log.Scope(t)
defer s.Close(t)

containerConfig := defaultContainerConfig()
containerConfig.Cmd = []string{"stat", cluster.CockroachBinaryInContainer}
ctx := context.Background()

if err := testDockerOneShot(ctx, t, "cli_test", containerConfig); err != nil {
t.Skipf(`TODO(dt): No binary in one-shot container, see #6086: %s`, err)
}

containerConfig.Env = []string{fmt.Sprintf("PGUSER=%s", security.RootUser)}
containerConfig.Cmd = append(cmdBase,
"/mnt/data/psql/test-psql-unix.sh "+cluster.CockroachBinaryInContainer)
if err := testDockerOneShot(ctx, t, "unix_socket_test", containerConfig); err != nil {
t.Error(err)
}
}
52 changes: 52 additions & 0 deletions pkg/acceptance/testdata/psql/test-psql-unix.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
#!/usr/bin/env bash

CERTS_DIR=${CERTS_DIR:-/certs}
crdb=$1
trap "set -x; killall cockroach cockroachshort" EXIT HUP

set -euo pipefail

# Disable automatic network access by psql.
unset PGHOST
unset PGPORT
# Use root access.
export PGUSER=root

echo "Testing Unix socket connection via insecure server."
set -x

# Start an insecure CockroachDB server.
# We use a different port number from standard for an extra guarantee that
# "psql" is not going to find it.
"$crdb" start-single-node --background --insecure \
--socket=/tmp/.s.PGSQL.1111 \
--listen-addr=:12345

# Wait for server ready.
"$crdb" sql --insecure -e "select 1" -p 12345

# Verify that psql can connect to the server.
psql -h /tmp -p 1111 -c "select 1" | grep "1 row"

# It worked.
"$crdb" quit --insecure -p 12345
sleep 1; killall -9 cockroach cockroachshort || true

set +x
echo "Testing Unix socket connection via secure server."
set -x

# Restart the server in secure mode.
"$crdb" start-single-node --background \
--certs-dir="$CERTS_DIR" --socket=/tmp/.s.PGSQL.1111 \
--listen-addr=:12345

# Wait for server ready; also create a user that can log in.
"$crdb" sql --certs-dir="$CERTS_DIR" -e "create user foo with password 'pass'" -p 12345

# Also verify that psql can connect to the server.
env PGPASSWORD=pass psql -U foo -h /tmp -p 1111 -c "select 1" | grep "1 row"

set +x
# Done.
"$crdb" quit --certs-dir="$CERTS_DIR" -p 12345
3 changes: 1 addition & 2 deletions pkg/ccl/gssapiccl/gssapi.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@ const (
func authGSS(
c pgwire.AuthConn,
tlsState tls.ConnectionState,
insecure bool,
hashedPassword []byte,
execCfg *sql.ExecutorConfig,
entry *hba.Entry,
Expand Down Expand Up @@ -189,5 +188,5 @@ func checkEntry(entry hba.Entry) error {
}

func init() {
pgwire.RegisterAuthMethod("gss", authGSS, cluster.Version19_1, checkEntry)
pgwire.RegisterAuthMethod("gss", authGSS, cluster.Version19_1, hba.ConnHostSSL, checkEntry)
}
14 changes: 11 additions & 3 deletions pkg/cli/cliflags/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -535,9 +535,17 @@ write its process ID to the specified file.`,
Name: "socket",
EnvVar: "COCKROACH_SOCKET",
Description: `
Unix socket file, postgresql protocol only.
Note: when given a path to a unix socket, most postgres clients will
open "<given path>/.s.PGSQL.<server port>"`,
Accept client connections using a Unix domain socket with the
given name.
Note: for compatibility with PostgreSQL clients and drivers,
ensure that the socket name has the form "/path/to/.s.PGSQL.NNNN",
where NNNN is a number. PostgreSQL clients only take a port
number and directory as input and construct the socket name
programmatically.
To use, for example: psql -h /path/to -p NNNN ...
`,
}

ClientInsecure = FlagInfo{
Expand Down
5 changes: 0 additions & 5 deletions pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -350,12 +350,7 @@ func init() {
VarFlag(f, &serverCfg.StorageEngine, cliflags.StorageEngine)
VarFlag(f, &serverCfg.MaxOffset, cliflags.MaxOffset)

// Usage for the unix socket is odd as we use a real file, whereas
// postgresql and clients consider it a directory and build a filename
// inside it using the port.
// Thus, we keep it hidden and use it for testing only.
StringFlag(f, &serverCfg.SocketFile, cliflags.Socket, serverCfg.SocketFile)
_ = f.MarkHidden(cliflags.Socket.Name)

StringFlag(f, &startCtx.listeningURLFile, cliflags.ListeningURLFile, startCtx.listeningURLFile)

Expand Down
6 changes: 3 additions & 3 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1917,8 +1917,8 @@ func (s *Server) startServeSQL(
connCtx := logtags.AddTag(pgCtx, "client", conn.RemoteAddr().String())
tcpKeepAlive.configure(connCtx, conn)

if err := s.pgServer.ServeConn(connCtx, conn); err != nil {
log.Error(connCtx, err)
if err := s.pgServer.ServeConn(connCtx, conn, pgwire.SocketTCP); err != nil {
log.Errorf(connCtx, "serving SQL client conn: %v", err)
}
}))
})
Expand All @@ -1943,7 +1943,7 @@ func (s *Server) startServeSQL(
s.stopper.RunWorker(pgCtx, func(pgCtx context.Context) {
netutil.FatalIfUnexpected(connManager.ServeWith(pgCtx, s.stopper, unixLn, func(conn net.Conn) {
connCtx := logtags.AddTag(pgCtx, "client", conn.RemoteAddr().String())
if err := s.pgServer.ServeConn(connCtx, conn); err != nil {
if err := s.pgServer.ServeConn(connCtx, conn, pgwire.SocketUnix); err != nil {
log.Error(connCtx, err)
}
}))
Expand Down
102 changes: 74 additions & 28 deletions pkg/sql/pgwire/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,9 @@ type authOptions struct {
// be allowed to go through. A password, if presented, must be
// accepted.
insecure bool
// connType is the actual type of client connection (e.g. local,
// hostssl, hostnossl).
connType hba.ConnType
// auth is the current HBA configuration as returned by
// (*Server).GetAuthenticationConfiguration().
auth *hba.Conf
Expand Down Expand Up @@ -89,61 +92,104 @@ func (c *conn) handleAuthentication(
return sendError(errors.Errorf(security.ErrPasswordUserAuthFailed, c.sessionArgs.User))
}

if tlsConn, ok := c.conn.(*readTimeoutConn).Conn.(*tls.Conn); ok {
tlsState := tlsConn.ConnectionState()

methodFn, hbaEntry, err := c.lookupAuthenticationMethod(authOpt.auth)
if err != nil {
return sendError(err)
}
// Retrieve the authentication method.
tlsState, hbaEntry, methodFn, err := c.findAuthenticationMethod(authOpt)
if err != nil {
return sendError(err)
}

authenticationHook, err := methodFn(ac, tlsState, authOpt.insecure, hashedPassword, execCfg, hbaEntry)
if err != nil {
return sendError(err)
}
if err := authenticationHook(c.sessionArgs.User, true /* public */); err != nil {
return sendError(err)
}
// Ask the method to authenticate.
authenticationHook, err := methodFn(ac, tlsState, 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)
}

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())
func (c *conn) findAuthenticationMethod(
authOpt authOptions,
) (tlsState tls.ConnectionState, hbaEntry *hba.Entry, methodFn AuthMethod, err error) {
if authOpt.insecure {
// Insecure connections always use "trust" no matter what, and the
// remaining of the configuration is ignored.
methodFn = authTrust
return
}

// Look up the method from the HBA configuration.
var mi methodInfo
mi, hbaEntry, err = c.lookupAuthenticationMethodUsingRules(authOpt.connType, authOpt.auth)
if err != nil {
return
}
methodFn = mi.fn

// Check that this method can be used over this connection type.
if authOpt.connType&mi.validConnTypes == 0 {
err = errors.Newf("method %q required for this user, but unusable over this connection type",
hbaEntry.Method.Value)
return
}

// If the client is using SSL, retrieve the TLS state to provide as
// input to the method.
if authOpt.connType == hba.ConnHostSSL {
tlsConn, ok := c.conn.(*readTimeoutConn).Conn.(*tls.Conn)
if !ok {
err = errors.AssertionFailedf("server reports hostssl conn without TLS state")
return
}
tlsState = tlsConn.ConnectionState()
}

return
}

func (c *conn) lookupAuthenticationMethodUsingRules(
connType hba.ConnType, auth *hba.Conf,
) (mi methodInfo, entry *hba.Entry, err error) {
var ip net.IP
if connType != hba.ConnLocal {
// Extract the IP address of the client.
tcpAddr, ok := c.conn.RemoteAddr().(*net.TCPAddr)
if !ok {
err = errors.AssertionFailedf("client address type %T unsupported", c.conn.RemoteAddr())
return
}
ip = tcpAddr.IP
}
ip := tcpAddr.IP

// Look up the method.
for i := range auth.Entries {
entry := &auth.Entries[i]
addrMatch, err := entry.AddressMatches(ip)
entry = &auth.Entries[i]
var connMatch bool
connMatch, err = entry.ConnMatches(connType, 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
return
}
if !addrMatch {
if !connMatch {
// The address does not match.
continue
}
if !entry.UserMatches(c.sessionArgs.User) {
// The user does not match.
continue
}
return entry.MethodFn.(AuthMethod), entry, nil
return entry.MethodFn.(methodInfo), entry, nil
}

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

// authenticatorIO is the interface used by the connection to pass password data
Expand Down
24 changes: 10 additions & 14 deletions pkg/sql/pgwire/auth_methods.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,34 +39,33 @@ func loadDefaultMethods() {
//
// Care should be taken by administrators to only accept this auth
// method over secure connections, e.g. those encrypted using SSL.
RegisterAuthMethod("password", authPassword, cluster.Version19_1, nil)
RegisterAuthMethod("password", authPassword, cluster.Version19_1, hba.ConnAny, 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, cluster.Version19_1, nil)
RegisterAuthMethod("cert", authCert, cluster.Version19_1, hba.ConnHostSSL, 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, cluster.Version19_1, nil)
RegisterAuthMethod("cert-password", authCertPassword, cluster.Version19_1, hba.ConnAny, nil)

// The "reject" method rejects any connection attempt that matches
// the current rule.
RegisterAuthMethod("reject", authReject, cluster.VersionAuthLocalAndTrustRejectMethods, nil)
RegisterAuthMethod("reject", authReject, cluster.VersionAuthLocalAndTrustRejectMethods, hba.ConnAny, nil)

// The "trust" method accepts any connection attempt that matches
// the current rule.
RegisterAuthMethod("trust", authTrust, cluster.VersionAuthLocalAndTrustRejectMethods, nil)
RegisterAuthMethod("trust", authTrust, cluster.VersionAuthLocalAndTrustRejectMethods, hba.ConnAny, 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,
Expand All @@ -75,7 +74,6 @@ type AuthMethod func(
func authPassword(
c AuthConn,
tlsState tls.ConnectionState,
insecure bool,
hashedPassword []byte,
execCfg *sql.ExecutorConfig,
entry *hba.Entry,
Expand All @@ -92,7 +90,7 @@ func authPassword(
return nil, err
}
return security.UserAuthPasswordHook(
insecure, password, hashedPassword,
false /*insecure*/, password, hashedPassword,
), nil
}

Expand All @@ -107,7 +105,6 @@ func passwordString(pwdData []byte) (string, error) {
func authCert(
_ AuthConn,
tlsState tls.ConnectionState,
insecure bool,
hashedPassword []byte,
execCfg *sql.ExecutorConfig,
entry *hba.Entry,
Expand All @@ -119,13 +116,12 @@ func authCert(
tlsState.PeerCertificates[0].Subject.CommonName = tree.Name(
tlsState.PeerCertificates[0].Subject.CommonName,
).Normalize()
return security.UserAuthCertHook(insecure, &tlsState)
return security.UserAuthCertHook(false /*insecure*/, &tlsState)
}

func authCertPassword(
c AuthConn,
tlsState tls.ConnectionState,
insecure bool,
hashedPassword []byte,
execCfg *sql.ExecutorConfig,
entry *hba.Entry,
Expand All @@ -136,17 +132,17 @@ func authCertPassword(
} else {
fn = authCert
}
return fn(c, tlsState, insecure, hashedPassword, execCfg, entry)
return fn(c, tlsState, hashedPassword, execCfg, entry)
}

func authTrust(
_ AuthConn, _ tls.ConnectionState, _ bool, _ []byte, _ *sql.ExecutorConfig, _ *hba.Entry,
_ AuthConn, _ tls.ConnectionState, _ []byte, _ *sql.ExecutorConfig, _ *hba.Entry,
) (security.UserAuthHook, error) {
return func(_ string, _ bool) error { return nil }, nil
}

func authReject(
_ AuthConn, _ tls.ConnectionState, _ bool, _ []byte, _ *sql.ExecutorConfig, _ *hba.Entry,
_ AuthConn, _ tls.ConnectionState, _ []byte, _ *sql.ExecutorConfig, _ *hba.Entry,
) (security.UserAuthHook, error) {
return func(_ string, _ bool) error {
return errors.New("authentication rejected by configuration")
Expand Down
Loading

0 comments on commit c18eb14

Please sign in to comment.