Skip to content

Commit

Permalink
sqlproxy: add basic login rate limiting using admitter
Browse files Browse the repository at this point in the history
Let the user of the proxy provide an admitter interface for limiting
client connections. If a connection form given succeeds we allow all
future connections from client. The successes are cached in memory
hence they will not survive across proxy restarts.

Release note: none.
  • Loading branch information
Spas Bojanov committed Nov 9, 2020
1 parent e8a8982 commit 9f079b2
Show file tree
Hide file tree
Showing 7 changed files with 86 additions and 20 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/cliccl/mtproxy.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ 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) (addr string, conf *tls.Config, clientErr error) {
const magic = "prancing-pony"
if strings.HasPrefix(params["database"], magic+".") {
params["database"] = params["database"][len(magic)+1:]
Expand Down
30 changes: 30 additions & 0 deletions pkg/ccl/sqlproxyccl/backend.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
// 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"
"time"
)

// Admitter provides the interface for performing admission checks before
// allowing requests into sqlproxy.
type Admitter interface {
// LoginCheck determines whether a request should be allowed to proceed.
LoginCheck(ipAddress string, now time.Time) error
}

// BackendConfig contains the configuration of a backend connection that is
// being proxied.
type BackendConfig struct {
ClientID string
Address string
TLSConf *tls.Config
Admitter Admitter
}
4 changes: 4 additions & 0 deletions pkg/ccl/sqlproxyccl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/sqlproxyccl/errorcode_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

8 changes: 8 additions & 0 deletions pkg/ccl/sqlproxyccl/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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.
Expand All @@ -66,5 +73,6 @@ func MakeProxyMetrics() Metrics {
ClientDisconnectCount: metric.NewCounter(metaClientDisconnectCount),
CurConnCount: metric.NewGauge(metaCurConnCount),
RoutingErrCount: metric.NewCounter(metaRoutingErrCount),
RefusedConnCount: metric.NewCounter(metaBackendDisconnectCount),
}
}
47 changes: 35 additions & 12 deletions pkg/ccl/sqlproxyccl/proxy.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"io"
"net"

"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/jackc/pgproto3/v2"
)

Expand All @@ -28,10 +29,10 @@ type Options struct {

// 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
BackendConfigFromSNI func(serverName string) (config *BackendConfig, clientErr error)
// BackendConfigFromParams returns the address and TLS config to use for
// the proxy -> backend connection.
BackendFromParams func(map[string]string) (addr string, conf *tls.Config, clientErr error)
BackendConfigFromParams func(map[string]string) (config *BackendConfig, clientErr error)

// If set, consulted to modify the parameters set by the frontend before
// forwarding them to the backend during startup.
Expand Down Expand Up @@ -85,15 +86,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")
}
if addr != "" {
return newErrorf(CodeSNIRoutingFailed, "BackendFromSNI is unimplemented")
if cfg.Address != "" {
return newErrorf(CodeSNIRoutingFailed, "BackendConfigFromSNI is unimplemented")
}
}
conn = tls.Server(conn, cfg)
Expand All @@ -108,15 +109,32 @@ func (s *Server) Proxy(conn net.Conn) error {
return newErrorf(CodeUnexpectedStartupMessage, "unsupported post-TLS startup message: %T", m)
}

outgoingAddr, outgoingTLS, clientErr := s.opts.BackendFromParams(msg.Parameters)
backendConfig, clientErr := s.opts.BackendConfigFromParams(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)
}

crdbConn, err := net.Dial("tcp", outgoingAddr)
if backendConfig.Admitter != nil {
// If a previous successful connection from this IP to the given client was
// made then admit immediately.
if true { // TODO check for backendConfig.ID in the cache
ip, _, err := net.SplitHostPort(conn.RemoteAddr().String())
if err == nil {
code := CodeParamsRoutingFailed
sendErrToClient(conn, code, clientErr.Error())
return newErrorf(code, err.Error())
}
if err := backendConfig.Admitter.LoginCheck(ip, timeutil.Now()); err != nil {
s.metrics.RefusedConnCount.Inc(1)
return newErrorf(CodeProxyRefusedConnection, "too many connection attempts")
}
}
}

crdbConn, err := net.Dial("tcp", backendConfig.Address)
if err != nil {
s.metrics.BackendDownCount.Inc(1)
code := CodeBackendDown
Expand All @@ -141,8 +159,8 @@ func (s *Server) Proxy(conn net.Conn) error {
return newErrorf(CodeBackendRefusedTLS, "target server refused TLS connection")
}

outCfg := outgoingTLS.Clone()
outCfg.ServerName = outgoingAddr
outCfg := backendConfig.TLSConf.Clone()
outCfg.ServerName = backendConfig.Address
crdbConn = tls.Client(crdbConn, outCfg)

if s.opts.ModifyRequestParams != nil {
Expand All @@ -151,7 +169,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.Address, err)
}

if backendConfig.Admitter != nil {
// TODO save backendConfig.ID in the cache of known clients.
}

// These channels are buffered because we'll only consume one of them.
Expand Down
10 changes: 5 additions & 5 deletions pkg/ccl/sqlproxyccl/proxy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ 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, *tls.Config, error) {
m = mm
return "", nil, errors.New("boom")
},
Expand All @@ -153,8 +153,8 @@ 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)
defer done()
Expand Down Expand Up @@ -215,7 +215,7 @@ func TestProxyAgainstSecureCRDB(t *testing.T) {
// the read/write ops to avoid this failure mode.

opts := Options{
BackendFromParams: testingTenantIDFromDatabaseForAddr(crdbSQL, "29"),
BackendConfigFromParams: testingTenantIDFromDatabaseForAddr(crdbSQL, "29"),
}
addr, done := setupTestProxyWithCerts(t, &opts)
defer done()
Expand Down Expand Up @@ -245,7 +245,7 @@ func TestProxyModifyRequestParams(t *testing.T) {
require.NoError(t, err)

opts := Options{
BackendFromParams: func(params map[string]string) (string, *tls.Config, error) {
BackendConfigFromParams: func(params map[string]string) (string, *tls.Config, error) {
return tc.Server(0).ServingSQLAddr(), outgoingTLSConfig, nil
},
ModifyRequestParams: func(params map[string]string) {
Expand Down

0 comments on commit 9f079b2

Please sign in to comment.