-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
57849: ccl/sqlproxyccl: frontend admitter r=darinpp a=darinpp 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 cc: @cockroachdb/sqlproxy-prs Co-authored-by: Darin Peshev <[email protected]>
- Loading branch information
Showing
7 changed files
with
273 additions
and
52 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,76 @@ | ||
// 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 the default implementation of a frontend admitter. It can | ||
// upgrade to an optional SSL connection, and will handle and verify | ||
// the startup message received from the PG SQL client. | ||
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 | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,149 @@ | ||
// 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 ( | ||
"context" | ||
"crypto/tls" | ||
"fmt" | ||
"net" | ||
"testing" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/util/leaktest" | ||
"github.com/cockroachdb/cockroach/pkg/util/timeutil" | ||
"github.com/jackc/pgconn" | ||
"github.com/jackc/pgproto3/v2" | ||
"github.com/stretchr/testify/require" | ||
) | ||
|
||
func tlsConfig() (*tls.Config, error) { | ||
cer, err := tls.LoadX509KeyPair("testserver.crt", "testserver.key") | ||
if err != nil { | ||
return nil, err | ||
} | ||
return &tls.Config{ | ||
Certificates: []tls.Certificate{cer}, | ||
ServerName: "localhost", | ||
}, nil | ||
} | ||
|
||
func TestFrontendAdmitWithClientSSLDisableAndCustomParam(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
|
||
cli, srv := net.Pipe() | ||
require.NoError(t, srv.SetReadDeadline(timeutil.Now().Add(3e9))) | ||
require.NoError(t, cli.SetReadDeadline(timeutil.Now().Add(3e9))) | ||
|
||
ctx, cancel := context.WithCancel(context.Background()) | ||
defer cancel() | ||
|
||
go func() { | ||
cfg, err := pgconn.ParseConfig( | ||
"postgres://localhost?sslmode=disable&p1=a", | ||
) | ||
require.NoError(t, err) | ||
require.NotNil(t, cfg) | ||
cfg.DialFunc = func( | ||
ctx context.Context, network, addr string, | ||
) (net.Conn, error) { | ||
return cli, nil | ||
} | ||
_, _ = pgconn.ConnectConfig(ctx, cfg) | ||
fmt.Printf("Done\n") | ||
}() | ||
|
||
frontendCon, msg, err := FrontendAdmit(srv, nil) | ||
require.NoError(t, err) | ||
require.Equal(t, srv, frontendCon) | ||
require.NotNil(t, msg) | ||
require.Contains(t, msg.Parameters, "p1") | ||
require.Equal(t, msg.Parameters["p1"], "a") | ||
} | ||
|
||
func TestFrontendAdmitWithClientSSLRequire(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
|
||
cli, srv := net.Pipe() | ||
require.NoError(t, srv.SetReadDeadline(timeutil.Now().Add(3e9))) | ||
require.NoError(t, cli.SetReadDeadline(timeutil.Now().Add(3e9))) | ||
|
||
ctx, cancel := context.WithCancel(context.Background()) | ||
defer cancel() | ||
|
||
go func() { | ||
cfg, err := pgconn.ParseConfig("postgres://localhost?sslmode=require") | ||
require.NoError(t, err) | ||
require.NotNil(t, cfg) | ||
cfg.DialFunc = func(ctx context.Context, network, addr string) (net.Conn, error) { | ||
return cli, nil | ||
} | ||
_, _ = pgconn.ConnectConfig(ctx, cfg) | ||
}() | ||
|
||
tlsConfig, err := tlsConfig() | ||
require.NoError(t, err) | ||
frontendCon, msg, err := FrontendAdmit(srv, tlsConfig) | ||
require.NoError(t, err) | ||
require.NotEqual(t, srv, frontendCon) // The connection was replaced by SSL | ||
require.NotNil(t, msg) | ||
} | ||
|
||
func TestFrontendAdmitWithCancel(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
|
||
cli, srv := net.Pipe() | ||
require.NoError(t, srv.SetReadDeadline(timeutil.Now().Add(3e9))) | ||
require.NoError(t, cli.SetReadDeadline(timeutil.Now().Add(3e9))) | ||
|
||
go func() { | ||
cancelRequest := pgproto3.CancelRequest{ProcessID: 1, SecretKey: 2} | ||
_, err := cli.Write(cancelRequest.Encode([]byte{})) | ||
require.NoError(t, err) | ||
}() | ||
|
||
frontendCon, msg, err := FrontendAdmit(srv, nil) | ||
require.EqualError(t, err, | ||
"CodeUnexpectedStartupMessage: "+ | ||
"unsupported post-TLS startup message: *pgproto3.CancelRequest", | ||
) | ||
require.Nil(t, frontendCon) | ||
require.Nil(t, msg) | ||
} | ||
|
||
func TestFrontendAdmitWithSSLAndCancel(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
|
||
cli, srv := net.Pipe() | ||
require.NoError(t, srv.SetReadDeadline(timeutil.Now().Add(3e9))) | ||
require.NoError(t, cli.SetReadDeadline(timeutil.Now().Add(3e9))) | ||
|
||
go func() { | ||
sslRequest := pgproto3.SSLRequest{} | ||
_, err := cli.Write(sslRequest.Encode([]byte{})) | ||
require.NoError(t, err) | ||
b := []byte{0} | ||
n, err := cli.Read(b) | ||
require.Equal(t, n, 1) | ||
require.NoError(t, err) | ||
cli = tls.Client(cli, &tls.Config{InsecureSkipVerify: true}) | ||
cancelRequest := pgproto3.CancelRequest{ProcessID: 1, SecretKey: 2} | ||
_, err = cli.Write(cancelRequest.Encode([]byte{})) | ||
require.NoError(t, err) | ||
}() | ||
|
||
tlsConfig, err := tlsConfig() | ||
require.NoError(t, err) | ||
frontendCon, msg, err := FrontendAdmit(srv, tlsConfig) | ||
require.EqualError(t, err, | ||
"CodeUnexpectedStartupMessage: "+ | ||
"unsupported post-TLS startup message: *pgproto3.CancelRequest", | ||
) | ||
require.Nil(t, frontendCon) | ||
require.Nil(t, msg) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters