Skip to content

Commit

Permalink
ccl/sqlproxyccl: frontend admitter
Browse files Browse the repository at this point in the history
This replaces `IncomingTLSConfig` with a function similar to
`BackendDialer` that will be called to setup the frontend
connection/pipeline of the proxy.
It requires that all clients replace the calls like this one
```
  server := sqlproxyccl.NewServer(
    sqlproxyccl.Options{
      IncomingTLSConfig: &tlsConfig
    }
  }
```
with the analogous code
```
  server := sqlproxyccl.NewServer(
    sqlproxyccl.Options{
      FrontendAdmitter: func(
        incoming net.Conn,
      ) (net.Conn, *pgproto3.StartupMessage, error) {
        return sqlproxyccl.FrontendAdmit(
          incoming,
          &tls.Config,
        )
      }
    }
  }

```
While a bit more verbose, this makes possible for the proxy
library users to implement a custom logic to admit or
reject connections based on the client's IP address.

Release note: none
  • Loading branch information
darinpp committed Dec 12, 2020
1 parent f43005e commit be50759
Show file tree
Hide file tree
Showing 5 changed files with 119 additions and 51 deletions.
9 changes: 7 additions & 2 deletions pkg/ccl/cliccl/mtproxy.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,8 +146,13 @@ Uuwb2FVdh76ZK0AVd3Jh3KJs4+hr2u9syHaa7UPKXTcZsFWlGwZuu6X5A+0SO0S2
InsecureSkipVerify: true,
}
server := sqlproxyccl.NewServer(sqlproxyccl.Options{
IncomingTLSConfig: &tls.Config{
Certificates: []tls.Certificate{cer},
FrontendAdmitter: func(incoming net.Conn) (net.Conn, *pgproto3.StartupMessage, error) {
return sqlproxyccl.FrontendAdmit(
incoming,
&tls.Config{
Certificates: []tls.Certificate{cer},
},
)
},
BackendDialer: func(msg *pgproto3.StartupMessage) (net.Conn, error) {
params := msg.Parameters
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/sqlproxyccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
"backend_dialer.go",
"error.go",
"errorcode_string.go",
"frontend_admitter.go",
"idle_disconnect_connection.go",
"metrics.go",
"proxy.go",
Expand Down
74 changes: 74 additions & 0 deletions pkg/ccl/sqlproxyccl/frontend_admitter.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
// Copyright 2020 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package sqlproxyccl

import (
"crypto/tls"
"net"

"github.com/jackc/pgproto3/v2"
)

// FrontendAdmit is an example frontend admitter
func FrontendAdmit(
conn net.Conn, incomingTLSConfig *tls.Config,
) (net.Conn, *pgproto3.StartupMessage, error) {
// `conn` could be replaced by `conn` embedded in a `tls.Conn` connection,
// hence it's important to close `conn` rather than `proxyConn` since closing
// the latter will not call `Close` method of `tls.Conn`.
var sniServerName string
// If we have an incoming TLS Config, require that the client initiates
// with a TLS connection.
if incomingTLSConfig != nil {
m, err := pgproto3.NewBackend(pgproto3.NewChunkReader(conn), conn).ReceiveStartupMessage()
if err != nil {
return nil, nil, NewErrorf(CodeClientReadFailed, "while receiving startup message")
}
switch m.(type) {
case *pgproto3.SSLRequest:
case *pgproto3.CancelRequest:
// Ignore CancelRequest explicitly. We don't need to do this but it makes
// testing easier by avoiding a call to sendErrToClient on this path
// (which would confuse assertCtx).
return nil, nil, nil
default:
code := CodeUnexpectedInsecureStartupMessage
return nil, nil, NewErrorf(code, "unsupported startup message: %T", m)
}

_, err = conn.Write([]byte{pgAcceptSSLRequest})
if err != nil {
return nil, nil, NewErrorf(CodeClientWriteFailed, "acking SSLRequest: %v", err)
}

cfg := incomingTLSConfig.Clone()

cfg.GetConfigForClient = func(h *tls.ClientHelloInfo) (*tls.Config, error) {
sniServerName = h.ServerName
return nil, nil
}
conn = tls.Server(conn, cfg)
}

m, err := pgproto3.NewBackend(pgproto3.NewChunkReader(conn), conn).ReceiveStartupMessage()
if err != nil {
return nil, nil, NewErrorf(CodeClientReadFailed, "receiving post-TLS startup message: %v", err)
}
msg, ok := m.(*pgproto3.StartupMessage)
if !ok {
return nil, nil, NewErrorf(CodeUnexpectedStartupMessage, "unsupported post-TLS startup message: %T", m)
}

// Add the sniServerName (if used) as parameter
if sniServerName != "" {
msg.Parameters["sni-server"] = sniServerName
}

return conn, msg, nil
}
75 changes: 29 additions & 46 deletions pkg/ccl/sqlproxyccl/proxy.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@ type BackendConfig struct {

// Options are the options to the Proxy method.
type Options struct {
// Deprecated: construct FrontendAdmitter, passing this information in case
// that SSL is desired.
IncomingTLSConfig *tls.Config // config used for client -> proxy connection

// BackendFromParams returns the config to use for the proxy -> backend
Expand All @@ -75,6 +77,11 @@ type Options struct {
// The argument is the startup message received from the frontend. It
// contains the protocol version and params sent by the client.
BackendDialer func(msg *pgproto3.StartupMessage) (net.Conn, error)

// If set, will be called immediately after a new incoming connection
// is accepted. It can optionally negotiate SSL, provide admittance control or
// other types of frontend connection filtering.
FrontendAdmitter func(incoming net.Conn) (net.Conn, *pgproto3.StartupMessage, error)
}

// Proxy takes an incoming client connection and relays it to a backend SQL
Expand All @@ -98,59 +105,35 @@ func (s *Server) Proxy(proxyConn *Conn) error {
}).Encode(nil))
}

var conn net.Conn = proxyConn
// `conn` could be replaced by `conn` embedded in a `tls.Conn` connection,
// hence it's important to close `conn` rather than `proxyConn` since closing
// the latter will not call `Close` method of `tls.Conn`.
defer func() { _ = conn.Close() }()
var sniServerName string
// If we have an incoming TLS Config, require that the client initiates
// with a TLS connection.
if s.opts.IncomingTLSConfig != nil {
m, err := pgproto3.NewBackend(pgproto3.NewChunkReader(conn), conn).ReceiveStartupMessage()
if err != nil {
return NewErrorf(CodeClientReadFailed, "while receiving startup message")
}
switch m.(type) {
case *pgproto3.SSLRequest:
case *pgproto3.CancelRequest:
// Ignore CancelRequest explicitly. We don't need to do this but it makes
// testing easier by avoiding a call to sendErrToClient on this path
// (which would confuse assertCtx).
return nil
default:
code := CodeUnexpectedInsecureStartupMessage
sendErrToClient(conn, code, "server requires encryption")
return NewErrorf(code, "unsupported startup message: %T", m)
frontendAdmitter := s.opts.FrontendAdmitter
if frontendAdmitter == nil {
// Keep this until all clients are switched to provide FrontendAdmitter
// at what point we can also drop IncomingTLSConfig
frontendAdmitter = func(incoming net.Conn) (net.Conn, *pgproto3.StartupMessage, error) {
return FrontendAdmit(incoming, s.opts.IncomingTLSConfig)
}

_, err = conn.Write([]byte{pgAcceptSSLRequest})
if err != nil {
return NewErrorf(CodeClientWriteFailed, "acking SSLRequest: %v", err)
}

cfg := s.opts.IncomingTLSConfig.Clone()

cfg.GetConfigForClient = func(h *tls.ClientHelloInfo) (*tls.Config, error) {
sniServerName = h.ServerName
return nil, nil
}
conn = tls.Server(conn, cfg)
}

m, err := pgproto3.NewBackend(pgproto3.NewChunkReader(conn), conn).ReceiveStartupMessage()
conn, msg, err := frontendAdmitter(proxyConn)
if err != nil {
return NewErrorf(CodeClientReadFailed, "receiving post-TLS startup message: %v", err)
}
msg, ok := m.(*pgproto3.StartupMessage)
if !ok {
return NewErrorf(CodeUnexpectedStartupMessage, "unsupported post-TLS startup message: %T", m)
var codeErr *CodeError
if errors.As(err, &codeErr) && codeErr.code == CodeUnexpectedInsecureStartupMessage {
sendErrToClient(
proxyConn, // Do this on the TCP connection as it means denying SSL
CodeUnexpectedInsecureStartupMessage,
"server requires encryption",
)
}
return err
}

// Add the sniServerName (if used) as parameter
if sniServerName != "" {
msg.Parameters["sni-server"] = sniServerName
// This currently only happens for CancelRequest type of startup messages
// that we don't support
if conn == nil {
return nil

}
defer func() { _ = conn.Close() }()

backendDialer := s.opts.BackendDialer
if backendDialer == nil {
Expand Down
11 changes: 8 additions & 3 deletions pkg/ccl/sqlproxyccl/proxy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,9 +40,14 @@ openssl req -new -x509 -sha256 -key testserver.key -out testserver.crt \
`
cer, err := tls.LoadX509KeyPair("testserver.crt", "testserver.key")
require.NoError(t, err)
opts.IncomingTLSConfig = &tls.Config{
Certificates: []tls.Certificate{cer},
ServerName: "localhost",
opts.FrontendAdmitter = func(incoming net.Conn) (net.Conn, *pgproto3.StartupMessage, error) {
return FrontendAdmit(
incoming,
&tls.Config{
Certificates: []tls.Certificate{cer},
ServerName: "localhost",
},
)
}

const listenAddress = "127.0.0.1:0"
Expand Down

0 comments on commit be50759

Please sign in to comment.