From 9349bd405a095ab8eefe9320b513ed892db69725 Mon Sep 17 00:00:00 2001 From: Spas Bojanov Date: Tue, 10 Nov 2020 12:36:25 -0500 Subject: [PATCH] [CC-3025] sqlproxy: add hook to rate limit connections We need a mechanism for sqlproxy users to provide admission control to clients that is enforced in the proxy. To that end we expand `BackendConfigFromParams` to provide a signal whether a connection should be refused and a callback to record the success of a connection. The callback can implement caching of known connections for example so that they are not rate limited on further connection requests. Release note: none. --- pkg/ccl/cliccl/mtproxy.go | 14 ++- pkg/ccl/sqlproxyccl/BUILD.bazel | 1 - pkg/ccl/sqlproxyccl/error.go | 7 +- pkg/ccl/sqlproxyccl/errorcode_string.go | 5 +- pkg/ccl/sqlproxyccl/metrics.go | 8 ++ pkg/ccl/sqlproxyccl/proxy.go | 99 ++++++++++----- pkg/ccl/sqlproxyccl/proxy_test.go | 114 +++++++++++++----- .../lint/passes/fmtsafe/functions.go | 2 +- 8 files changed, 180 insertions(+), 70 deletions(-) diff --git a/pkg/ccl/cliccl/mtproxy.go b/pkg/ccl/cliccl/mtproxy.go index 6b09f706cfe4..81b2f6be2944 100644 --- a/pkg/ccl/cliccl/mtproxy.go +++ b/pkg/ccl/cliccl/mtproxy.go @@ -148,16 +148,22 @@ Uuwb2FVdh76ZK0AVd3Jh3KJs4+hr2u9syHaa7UPKXTcZsFWlGwZuu6X5A+0SO0S2 IncomingTLSConfig: &tls.Config{ Certificates: []tls.Certificate{cer}, }, - BackendFromParams: func(params map[string]string) (addr string, conf *tls.Config, clientErr error) { + BackendConfigFromParams: func( + params map[string]string, ipAddress string, + ) (config *sqlproxyccl.BackendConfig, clientErr error) { const magic = "prancing-pony" + cfg := &sqlproxyccl.BackendConfig{ + OutgoingAddress: sqlProxyTargetAddr, + TLSConf: outgoingConf, + } if strings.HasPrefix(params["database"], magic+".") { params["database"] = params["database"][len(magic)+1:] - return sqlProxyTargetAddr, outgoingConf, nil + return cfg, nil } if params["options"] == "--cluster="+magic { - return sqlProxyTargetAddr, outgoingConf, nil + return cfg, nil } - return "", nil, errors.Errorf("client failed to pass '%s' via database or options", magic) + return nil, errors.Errorf("client failed to pass '%s' via database or options", magic) }, }) diff --git a/pkg/ccl/sqlproxyccl/BUILD.bazel b/pkg/ccl/sqlproxyccl/BUILD.bazel index 5d9454efef59..63677fba25d2 100644 --- a/pkg/ccl/sqlproxyccl/BUILD.bazel +++ b/pkg/ccl/sqlproxyccl/BUILD.bazel @@ -38,7 +38,6 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/testutils/serverutils", - "//pkg/testutils/skip", "//pkg/testutils/testcluster", "//pkg/util/leaktest", "//pkg/util/randutil", diff --git a/pkg/ccl/sqlproxyccl/error.go b/pkg/ccl/sqlproxyccl/error.go index e1bf9c6d9288..f46b85b698e0 100644 --- a/pkg/ccl/sqlproxyccl/error.go +++ b/pkg/ccl/sqlproxyccl/error.go @@ -58,6 +58,10 @@ const ( // CodeClientDisconnected indicates that the client disconnected unexpectedly // (with a connection error) while in a session with backend SQL server. CodeClientDisconnected + + // CodeProxyRefusedConnection indicates that the proxy refused the connection + // request due to high load or too many connection attempts. + CodeProxyRefusedConnection ) type codeError struct { @@ -69,7 +73,8 @@ func (e *codeError) Error() string { return fmt.Sprintf("%s: %s", e.code, e.err) } -func newErrorf(code ErrorCode, format string, args ...interface{}) error { +// NewErrorf returns a new codeError out of the supplied args. +func NewErrorf(code ErrorCode, format string, args ...interface{}) error { return &codeError{ code: code, err: errors.Errorf(format, args...), diff --git a/pkg/ccl/sqlproxyccl/errorcode_string.go b/pkg/ccl/sqlproxyccl/errorcode_string.go index 45292b2181f3..81f980eb5fa4 100644 --- a/pkg/ccl/sqlproxyccl/errorcode_string.go +++ b/pkg/ccl/sqlproxyccl/errorcode_string.go @@ -18,11 +18,12 @@ func _() { _ = x[CodeBackendRefusedTLS-8] _ = x[CodeBackendDisconnected-9] _ = x[CodeClientDisconnected-10] + _ = x[CodeProxyRefusedConnection-11] } -const _ErrorCode_name = "CodeClientReadFailedCodeClientWriteFailedCodeUnexpectedInsecureStartupMessageCodeSNIRoutingFailedCodeUnexpectedStartupMessageCodeParamsRoutingFailedCodeBackendDownCodeBackendRefusedTLSCodeBackendDisconnectedCodeClientDisconnected" +const _ErrorCode_name = "CodeClientReadFailedCodeClientWriteFailedCodeUnexpectedInsecureStartupMessageCodeSNIRoutingFailedCodeUnexpectedStartupMessageCodeParamsRoutingFailedCodeBackendDownCodeBackendRefusedTLSCodeBackendDisconnectedCodeClientDisconnectedCodeProxyRefusedConnection" -var _ErrorCode_index = [...]uint8{0, 20, 41, 77, 97, 125, 148, 163, 184, 207, 229} +var _ErrorCode_index = [...]uint8{0, 20, 41, 77, 97, 125, 148, 163, 184, 207, 229, 255} func (i ErrorCode) String() string { i -= 1 diff --git a/pkg/ccl/sqlproxyccl/metrics.go b/pkg/ccl/sqlproxyccl/metrics.go index 7f7c87c72ea0..98270c88c033 100644 --- a/pkg/ccl/sqlproxyccl/metrics.go +++ b/pkg/ccl/sqlproxyccl/metrics.go @@ -18,6 +18,7 @@ type Metrics struct { ClientDisconnectCount *metric.Counter CurConnCount *metric.Gauge RoutingErrCount *metric.Counter + RefusedConnCount *metric.Counter } // MetricStruct implements the metrics.Struct interface. @@ -56,6 +57,12 @@ var ( Measurement: "Disconnects", Unit: metric.Unit_COUNT, } + metaRefusedConnCount = metric.Metadata{ + Name: "proxy.err.refused_conn", + Help: "Number of refused connections initiated by a given IP", + Measurement: "Refused", + Unit: metric.Unit_COUNT, + } ) // MakeProxyMetrics instantiates the metrics holder for proxy monitoring. @@ -66,5 +73,6 @@ func MakeProxyMetrics() Metrics { ClientDisconnectCount: metric.NewCounter(metaClientDisconnectCount), CurConnCount: metric.NewGauge(metaCurConnCount), RoutingErrCount: metric.NewCounter(metaRoutingErrCount), + RefusedConnCount: metric.NewCounter(metaRefusedConnCount), } } diff --git a/pkg/ccl/sqlproxyccl/proxy.go b/pkg/ccl/sqlproxyccl/proxy.go index 01efd134a541..b53e14da32f2 100644 --- a/pkg/ccl/sqlproxyccl/proxy.go +++ b/pkg/ccl/sqlproxyccl/proxy.go @@ -14,6 +14,7 @@ import ( "io" "net" + "github.com/cockroachdb/errors" "github.com/jackc/pgproto3/v2" ) @@ -22,17 +23,30 @@ const pgAcceptSSLRequest = 'S' // See https://www.postgresql.org/docs/9.1/protocol-message-formats.html. var pgSSLRequest = []int32{8, 80877103} +// BackendConfig contains the configuration of a backend connection that is +// being proxied. +type BackendConfig struct { + // The address to which the connection is forwarded. + OutgoingAddress string + // TLS settings to use when connecting to OutgoingAddress. + TLSConf *tls.Config + // Called after successfully connecting to OutgoingAddr. + OnConnectionSuccess func() +} + // Options are the options to the Proxy method. type Options struct { IncomingTLSConfig *tls.Config // config used for client -> proxy connection // TODO(tbg): this is unimplemented and exists only to check which clients // allow use of SNI. Should always return ("", nil). - BackendFromSNI func(serverName string) (addr string, conf *tls.Config, clientErr error) - // BackendFromParams returns the address and TLS config to use for - // the proxy -> backend connection. The returned config must have - // an appropriate ServerName for the remote backend. - BackendFromParams func(map[string]string) (addr string, conf *tls.Config, clientErr error) + BackendConfigFromSNI func(serverName string) (config *BackendConfig, clientErr error) + // BackendFromParams returns the config to use for the proxy -> backend + // connection. The TLS config is in it and it must have an appropriate + // ServerName for the remote backend. + BackendConfigFromParams func( + params map[string]string, ipAddress string, + ) (config *BackendConfig, clientErr error) // If set, consulted to modify the parameters set by the frontend before // forwarding them to the backend during startup. @@ -60,7 +74,7 @@ func (s *Server) Proxy(conn net.Conn) error { { m, err := pgproto3.NewBackend(pgproto3.NewChunkReader(conn), conn).ReceiveStartupMessage() if err != nil { - return newErrorf(CodeClientReadFailed, "while receiving startup message") + return NewErrorf(CodeClientReadFailed, "while receiving startup message") } switch m.(type) { case *pgproto3.SSLRequest: @@ -72,12 +86,12 @@ func (s *Server) Proxy(conn net.Conn) error { default: code := CodeUnexpectedInsecureStartupMessage sendErrToClient(conn, code, "server requires encryption") - return newErrorf(code, "unsupported startup message: %T", m) + return NewErrorf(code, "unsupported startup message: %T", m) } _, err = conn.Write([]byte{pgAcceptSSLRequest}) if err != nil { - return newErrorf(CodeClientWriteFailed, "acking SSLRequest: %v", err) + return NewErrorf(CodeClientWriteFailed, "acking SSLRequest: %v", err) } cfg := s.opts.IncomingTLSConfig.Clone() @@ -86,15 +100,15 @@ func (s *Server) Proxy(conn net.Conn) error { sniServerName = h.ServerName return nil, nil } - if s.opts.BackendFromSNI != nil { - addr, _, clientErr := s.opts.BackendFromSNI(sniServerName) + if s.opts.BackendConfigFromSNI != nil { + cfg, clientErr := s.opts.BackendConfigFromSNI(sniServerName) if clientErr != nil { code := CodeSNIRoutingFailed sendErrToClient(conn, code, clientErr.Error()) // won't actually be shown by most clients - return newErrorf(code, "rejected by OutgoingAddrFromSNI") + return NewErrorf(code, "rejected by OutgoingAddrFromSNI") } - if addr != "" { - return newErrorf(CodeSNIRoutingFailed, "BackendFromSNI is unimplemented") + if cfg.OutgoingAddress != "" { + return NewErrorf(CodeSNIRoutingFailed, "BackendConfigFromSNI is unimplemented") } } conn = tls.Server(conn, cfg) @@ -102,47 +116,67 @@ func (s *Server) Proxy(conn net.Conn) error { m, err := pgproto3.NewBackend(pgproto3.NewChunkReader(conn), conn).ReceiveStartupMessage() if err != nil { - return newErrorf(CodeClientReadFailed, "receiving post-TLS startup message: %v", err) + 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) + return NewErrorf(CodeUnexpectedStartupMessage, "unsupported post-TLS startup message: %T", m) } - outgoingAddr, outgoingTLS, clientErr := s.opts.BackendFromParams(msg.Parameters) - if clientErr != nil { - s.metrics.RoutingErrCount.Inc(1) - code := CodeParamsRoutingFailed - sendErrToClient(conn, code, clientErr.Error()) - return newErrorf(code, "rejected by OutgoingAddrFromParams: %v", clientErr) + var backendConfig *BackendConfig + { + ip, _, err := net.SplitHostPort(conn.RemoteAddr().String()) + if err != nil { + return NewErrorf( + CodeParamsRoutingFailed, "could not parse address %s: %v", + conn.RemoteAddr().String(), err) + } + var clientErr error + backendConfig, clientErr = s.opts.BackendConfigFromParams(msg.Parameters, ip) + if clientErr != nil { + var codeErr *codeError + if !errors.As(clientErr, &codeErr) { + codeErr = &codeError{ + code: CodeParamsRoutingFailed, + err: errors.Errorf("rejected by BackendConfigFromParams: %v", clientErr), + } + } + if codeErr.code == CodeProxyRefusedConnection { + s.metrics.RefusedConnCount.Inc(1) + } else { + s.metrics.RoutingErrCount.Inc(1) + } + sendErrToClient(conn, codeErr.code, clientErr.Error()) + return codeErr + } } - crdbConn, err := net.Dial("tcp", outgoingAddr) + crdbConn, err := net.Dial("tcp", backendConfig.OutgoingAddress) if err != nil { s.metrics.BackendDownCount.Inc(1) code := CodeBackendDown sendErrToClient(conn, code, "unable to reach backend SQL server") - return newErrorf(code, "dialing backend server: %v", err) + return NewErrorf(code, "dialing backend server: %v", err) } // Send SSLRequest. if err := binary.Write(crdbConn, binary.BigEndian, pgSSLRequest); err != nil { s.metrics.BackendDownCount.Inc(1) - return newErrorf(CodeBackendDown, "sending SSLRequest to target server: %v", err) + return NewErrorf(CodeBackendDown, "sending SSLRequest to target server: %v", err) } response := make([]byte, 1) if _, err = io.ReadFull(crdbConn, response); err != nil { s.metrics.BackendDownCount.Inc(1) - return newErrorf(CodeBackendDown, "reading response to SSLRequest") + return NewErrorf(CodeBackendDown, "reading response to SSLRequest") } if response[0] != pgAcceptSSLRequest { s.metrics.BackendDownCount.Inc(1) - return newErrorf(CodeBackendRefusedTLS, "target server refused TLS connection") + return NewErrorf(CodeBackendRefusedTLS, "target server refused TLS connection") } - outCfg := outgoingTLS.Clone() + outCfg := backendConfig.TLSConf.Clone() crdbConn = tls.Client(crdbConn, outCfg) if s.opts.ModifyRequestParams != nil { @@ -151,7 +185,12 @@ func (s *Server) Proxy(conn net.Conn) error { if _, err := crdbConn.Write(msg.Encode(nil)); err != nil { s.metrics.BackendDownCount.Inc(1) - return newErrorf(CodeBackendDown, "relaying StartupMessage to target server %v: %v", outgoingAddr, err) + return NewErrorf(CodeBackendDown, "relaying StartupMessage to target server %v: %v", + backendConfig.OutgoingAddress, err) + } + + if backendConfig.OnConnectionSuccess != nil { + backendConfig.OnConnectionSuccess() } // These channels are buffered because we'll only consume one of them. @@ -177,14 +216,14 @@ func (s *Server) Proxy(conn net.Conn) error { case err := <-errIncoming: if err != nil { s.metrics.BackendDisconnectCount.Inc(1) - return newErrorf(CodeBackendDisconnected, "copying from target server to client: %s", err) + return NewErrorf(CodeBackendDisconnected, "copying from target server to client: %s", err) } return nil case err := <-errOutgoing: // The incoming connection got closed. if err != nil { s.metrics.ClientDisconnectCount.Inc(1) - return newErrorf(CodeClientDisconnected, "copying from target server to client: %v", err) + return NewErrorf(CodeClientDisconnected, "copying from target server to client: %v", err) } return nil } diff --git a/pkg/ccl/sqlproxyccl/proxy_test.go b/pkg/ccl/sqlproxyccl/proxy_test.go index 581bb7988939..c430e2c6baca 100644 --- a/pkg/ccl/sqlproxyccl/proxy_test.go +++ b/pkg/ccl/sqlproxyccl/proxy_test.go @@ -19,14 +19,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/errors" "github.com/jackc/pgx/v4" "github.com/stretchr/testify/require" ) -func setupTestProxyWithCerts(t *testing.T, opts *Options) (addr string, done func()) { +func setupTestProxyWithCerts( + t *testing.T, opts *Options, +) (server *Server, addr string, done func()) { // Created via: const _ = ` openssl genrsa -out testserver.key 2048 @@ -53,41 +54,43 @@ openssl req -new -x509 -sha256 -key testserver.key -out testserver.crt \ wg.Wait() } - server := NewServer(*opts) + server = NewServer(*opts) go func() { defer wg.Done() _ = server.Serve(ln) }() - return ln.Addr().String(), done + return server, ln.Addr().String(), done } func testingTenantIDFromDatabaseForAddr( addr string, validTenant string, -) func(map[string]string) (string, *tls.Config, error) { - return func(p map[string]string) (_ string, config *tls.Config, clientErr error) { +) func(map[string]string, string) (config *BackendConfig, clientErr error) { + return func(p map[string]string, _ string) (config *BackendConfig, clientErr error) { const dbKey = "database" db, ok := p[dbKey] if !ok { - return "", nil, errors.Newf("need to specify database") + return nil, errors.Newf("need to specify database") } sl := strings.SplitN(db, "_", 2) if len(sl) != 2 { - return "", nil, errors.Newf("malformed database name") + return nil, errors.Newf("malformed database name") } db, tenantID := sl[0], sl[1] if tenantID != validTenant { - return "", nil, errors.Newf("invalid tenantID") + return nil, errors.Newf("invalid tenantID") } p[dbKey] = db - config = &tls.Config{ - // NB: this would be false in production. - InsecureSkipVerify: true, - } - return addr, config, nil + return &BackendConfig{ + OutgoingAddress: addr, + TLSConf: &tls.Config{ + // NB: this would be false in production. + InsecureSkipVerify: true, + }, + }, nil } } @@ -131,19 +134,21 @@ func TestLongDBName(t *testing.T) { var m map[string]string opts := Options{ - BackendFromParams: func(mm map[string]string) (string, *tls.Config, error) { + BackendConfigFromParams: func( + mm map[string]string, _ string) (config *BackendConfig, clientErr error) { m = mm - return "", nil, errors.New("boom") + return nil, errors.New("boom") }, OnSendErrToClient: ac.onSendErrToClient, } - addr, done := setupTestProxyWithCerts(t, &opts) + s, addr, done := setupTestProxyWithCerts(t, &opts) defer done() longDB := strings.Repeat("x", 70) // 63 is limit pgurl := fmt.Sprintf("postgres://unused:unused@%s/%s", addr, longDB) ac.assertConnectErr(t, pgurl, "" /* suffix */, CodeParamsRoutingFailed, "boom") require.Equal(t, longDB, m["database"]) + require.Equal(t, int64(1), s.metrics.RoutingErrCount.Count()) } func TestFailedConnection(t *testing.T) { @@ -154,10 +159,10 @@ func TestFailedConnection(t *testing.T) { ac := makeAssertCtx() opts := Options{ - BackendFromParams: testingTenantIDFromDatabaseForAddr("undialable%$!@$", "29"), - OnSendErrToClient: ac.onSendErrToClient, + BackendConfigFromParams: testingTenantIDFromDatabaseForAddr("undialable%$!@$", "29"), + OnSendErrToClient: ac.onSendErrToClient, } - addr, done := setupTestProxyWithCerts(t, &opts) + s, addr, done := setupTestProxyWithCerts(t, &opts) defer done() _, p, err := net.SplitHostPort(addr) @@ -178,6 +183,7 @@ func TestFailedConnection(t *testing.T) { t, u, "defaultdb_29?sslmode=verify-full&sslrootcert=testserver.crt", CodeBackendDown, "unable to reach backend SQL server", ) + require.Equal(t, int64(4), s.metrics.BackendDownCount.Count()) // Unencrypted connections bounce. for _, sslmode := range []string{"disable", "allow"} { @@ -198,34 +204,45 @@ func TestFailedConnection(t *testing.T) { t, u, "defaultdb?sslmode=require", CodeParamsRoutingFailed, "malformed database name", ) + require.Equal(t, int64(2), s.metrics.RoutingErrCount.Count()) } func TestProxyAgainstSecureCRDB(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) - skip.IgnoreLint(t, "this test needs a running (secure) CockroachDB instance at the given address") - const crdbSQL = "127.0.0.1:52966" - // TODO(asubiotto): use an in-mem test server once this code lives in the CRDB - // repo. - // // TODO(tbg): if I use the https (!) port of ./cockroach demo, the // connection hangs instead of failing. Why? Probably both ends end up waiting // for the other side due to protocol mismatch. Should set deadlines on all // the read/write ops to avoid this failure mode. + outgoingTLSConfig, err := tc.Server(0).RPCContext().GetClientTLSConfig() + require.NoError(t, err) + outgoingTLSConfig.InsecureSkipVerify = true + + var connSuccess bool opts := Options{ - BackendFromParams: testingTenantIDFromDatabaseForAddr(crdbSQL, "29"), + BackendConfigFromParams: func(params map[string]string, _ string) (*BackendConfig, error) { + return &BackendConfig{ + OutgoingAddress: tc.Server(0).ServingSQLAddr(), + TLSConf: outgoingTLSConfig, + OnConnectionSuccess: func() { connSuccess = true }, + }, nil + }, } - addr, done := setupTestProxyWithCerts(t, &opts) + s, addr, done := setupTestProxyWithCerts(t, &opts) defer done() url := fmt.Sprintf("postgres://root:admin@%s/defaultdb_29?sslmode=require", addr) conn, err := pgx.Connect(context.Background(), url) require.NoError(t, err) + require.Equal(t, int64(1), s.metrics.CurConnCount.Value()) defer func() { require.NoError(t, conn.Close(ctx)) + require.True(t, connSuccess) }() var n int @@ -242,12 +259,15 @@ func TestProxyModifyRequestParams(t *testing.T) { defer tc.Stopper().Stop(ctx) outgoingTLSConfig, err := tc.Server(0).RPCContext().GetClientTLSConfig() - outgoingTLSConfig.InsecureSkipVerify = true require.NoError(t, err) + outgoingTLSConfig.InsecureSkipVerify = true opts := Options{ - BackendFromParams: func(params map[string]string) (string, *tls.Config, error) { - return tc.Server(0).ServingSQLAddr(), outgoingTLSConfig, nil + BackendConfigFromParams: func(params map[string]string, _ string) (*BackendConfig, error) { + return &BackendConfig{ + OutgoingAddress: tc.Server(0).ServingSQLAddr(), + TLSConf: outgoingTLSConfig, + }, nil }, ModifyRequestParams: func(params map[string]string) { require.EqualValues(t, map[string]string{ @@ -261,12 +281,13 @@ func TestProxyModifyRequestParams(t *testing.T) { params["user"] = "root" }, } - proxyAddr, done := setupTestProxyWithCerts(t, &opts) + s, proxyAddr, done := setupTestProxyWithCerts(t, &opts) defer done() u := fmt.Sprintf("postgres://bogususer@%s/?sslmode=require&authToken=abc123", proxyAddr) conn, err := pgx.Connect(ctx, u) require.NoError(t, err) + require.Equal(t, int64(1), s.metrics.CurConnCount.Value()) defer func() { require.NoError(t, conn.Close(ctx)) }() @@ -276,3 +297,34 @@ func TestProxyModifyRequestParams(t *testing.T) { require.NoError(t, err) require.EqualValues(t, 1, n) } + +func TestProxyRefuseConn(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + + outgoingTLSConfig, err := tc.Server(0).RPCContext().GetClientTLSConfig() + require.NoError(t, err) + outgoingTLSConfig.InsecureSkipVerify = true + + ac := makeAssertCtx() + opts := Options{ + BackendConfigFromParams: func(params map[string]string, _ string) (*BackendConfig, error) { + return &BackendConfig{ + OutgoingAddress: tc.Server(0).ServingSQLAddr(), + TLSConf: outgoingTLSConfig, + }, NewErrorf(CodeProxyRefusedConnection, "too many attempts") + }, + OnSendErrToClient: ac.onSendErrToClient, + } + s, addr, done := setupTestProxyWithCerts(t, &opts) + defer done() + + ac.assertConnectErr( + t, fmt.Sprintf("postgres://root:admin@%s/", addr), "defaultdb_29?sslmode=require", + CodeProxyRefusedConnection, "too many attempts", + ) + require.Equal(t, int64(1), s.metrics.RefusedConnCount.Count()) +} diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index 94ddabb21b7e..8d5b3757cc40 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -197,5 +197,5 @@ var requireConstFmt = map[string]bool{ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate.inputErrorf": true, - "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl.newErrorf": true, + "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl.NewErrorf": true, }