Skip to content

Commit

Permalink
Merge #57431
Browse files Browse the repository at this point in the history
57431: ccl/sqlproxyccl: idle connection timeout support r=darinpp a=darinpp

Previusly, the connection from the end user to the backend
can be idle for any period of time without disconnecting.
In certain cases, we want the ability for idle connections
to disconnect when the idle time exceeds redefined timeout.
To address this, this patch adds tracking of how long
the connection to the backend has been idle. If there is
a timeout specified, the connection will be disconnected
and an error message will be sent back to the end user.

Release note: None

Co-authored-by: Darin Peshev <[email protected]>
  • Loading branch information
craig[bot] and darinpp committed Dec 11, 2020
2 parents b325be3 + 0a61e77 commit 05a192a
Show file tree
Hide file tree
Showing 11 changed files with 536 additions and 139 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/cliccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ go_library(
"//vendor/github.com/cockroachdb/cmux",
"//vendor/github.com/cockroachdb/errors",
"//vendor/github.com/cockroachdb/errors/oserror",
"//vendor/github.com/jackc/pgproto3/v2:pgproto3",
"//vendor/github.com/spf13/cobra",
"//vendor/golang.org/x/sync/errgroup",
],
Expand Down
20 changes: 9 additions & 11 deletions pkg/ccl/cliccl/mtproxy.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl"
"github.com/cockroachdb/cockroach/pkg/cli"
"github.com/cockroachdb/errors"
"github.com/jackc/pgproto3/v2"
"github.com/spf13/cobra"
"golang.org/x/sync/errgroup"
)
Expand Down Expand Up @@ -148,22 +149,19 @@ Uuwb2FVdh76ZK0AVd3Jh3KJs4+hr2u9syHaa7UPKXTcZsFWlGwZuu6X5A+0SO0S2
IncomingTLSConfig: &tls.Config{
Certificates: []tls.Certificate{cer},
},
BackendConfigFromParams: func(
params map[string]string, _ *sqlproxyccl.Conn,
) (config *sqlproxyccl.BackendConfig, clientErr error) {
BackendDialer: func(msg *pgproto3.StartupMessage) (net.Conn, error) {
params := msg.Parameters
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 cfg, nil
} else if params["options"] != "--cluster="+magic {
return nil, errors.Errorf("client failed to pass '%s' via database or options", magic)
}
if params["options"] == "--cluster="+magic {
return cfg, nil
conn, err := sqlproxyccl.BackendDial(msg, sqlProxyTargetAddr, outgoingConf)
if err != nil {
return nil, err
}
return nil, errors.Errorf("client failed to pass '%s' via database or options", magic)
return conn, nil
},
})

Expand Down
4 changes: 4 additions & 0 deletions pkg/ccl/sqlproxyccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "sqlproxyccl",
srcs = [
"backend_dialer.go",
"error.go",
"errorcode_string.go",
"idle_disconnect_connection.go",
"metrics.go",
"proxy.go",
"server.go",
Expand All @@ -26,6 +28,7 @@ go_library(
go_test(
name = "sqlproxyccl_test",
srcs = [
"idle_disconnect_connection_test.go",
"main_test.go",
"proxy_test.go",
"server_test.go",
Expand All @@ -43,6 +46,7 @@ go_test(
"//pkg/util/randutil",
"//pkg/util/timeutil",
"//vendor/github.com/cockroachdb/errors",
"//vendor/github.com/jackc/pgproto3/v2:pgproto3",
"//vendor/github.com/jackc/pgx/v4:pgx",
"//vendor/github.com/stretchr/testify/require",
],
Expand Down
79 changes: 79 additions & 0 deletions pkg/ccl/sqlproxyccl/backend_dialer.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
// 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"
"encoding/binary"
"io"
"net"

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

// BackendDial is an example backend dialer that does a TCP/IP connection
// to a backend, SSL and forwards the start message.
func BackendDial(
msg *pgproto3.StartupMessage, outgoingAddress string, tlsConfig *tls.Config,
) (net.Conn, error) {
conn, err := net.Dial("tcp", outgoingAddress)
if err != nil {
return nil, NewErrorf(
CodeBackendDown, "unable to reach backend SQL server: %v", err,
)
}
conn, err = SSLOverlay(conn, tlsConfig)
if err != nil {
return nil, err
}
err = RelayStartupMsg(conn, msg)
if err != nil {
return nil, NewErrorf(
CodeBackendDown, "relaying StartupMessage to target server %v: %v",
outgoingAddress, err)
}
return conn, nil
}

// SSLOverlay attempts to upgrade the PG connection to use SSL
// if a tls.Config is specified..
func SSLOverlay(conn net.Conn, tlsConfig *tls.Config) (net.Conn, error) {
if tlsConfig == nil {
return conn, nil
}

var err error
// Send SSLRequest.
if err := binary.Write(conn, binary.BigEndian, pgSSLRequest); err != nil {
return nil, NewErrorf(
CodeBackendDown, "sending SSLRequest to target server: %v", err,
)
}

response := make([]byte, 1)
if _, err = io.ReadFull(conn, response); err != nil {
return nil,
NewErrorf(CodeBackendDown, "reading response to SSLRequest")
}

if response[0] != pgAcceptSSLRequest {
return nil, NewErrorf(
CodeBackendRefusedTLS, "target server refused TLS connection",
)
}

outCfg := tlsConfig.Clone()
return tls.Client(conn, outCfg), nil
}

// RelayStartupMsg forwards the start message on the backend connection.
func RelayStartupMsg(conn net.Conn, msg *pgproto3.StartupMessage) (err error) {
_, err = conn.Write(msg.Encode(nil))
return
}
14 changes: 10 additions & 4 deletions pkg/ccl/sqlproxyccl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,20 +66,26 @@ const (
// CodeExpiredClientConnection indicates that proxy connection to the client
// has expired and should be closed.
CodeExpiredClientConnection

// CodeIdleDisconnect indicates that the connection was disconnected for
// being idle for longer than the specified timeout.
CodeIdleDisconnect
)

type codeError struct {
// CodeError is combines an error with one of the above codes to ease
// the processing of the errors.
type CodeError struct {
code ErrorCode
err error
}

func (e *codeError) Error() string {
func (e *CodeError) Error() string {
return fmt.Sprintf("%s: %s", e.code, e.err)
}

// NewErrorf returns a new codeError out of the supplied args.
// NewErrorf returns a new CodeError out of the supplied args.
func NewErrorf(code ErrorCode, format string, args ...interface{}) error {
return &codeError{
return &CodeError{
code: code,
err: errors.Errorf(format, args...),
}
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.

97 changes: 97 additions & 0 deletions pkg/ccl/sqlproxyccl/idle_disconnect_connection.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
// 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 (
"net"
"time"

"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

// IdleDisconnectConnection is a wrapper around net.Conn that disconnects if
// connection is idle. The idle time is only counted while the client is
// waiting, blocked on Read.
type IdleDisconnectConnection struct {
net.Conn
timeout time.Duration
mu struct {
syncutil.Mutex
lastDeadlineSetAt time.Time
}
}

var errNotSupported = errors.Errorf(
"Not supported for IdleDisconnectConnection",
)

func (c *IdleDisconnectConnection) updateDeadline() error {
now := timeutil.Now()
// If it has been more than 1% of the timeout duration - advance the deadline.
c.mu.Lock()
defer c.mu.Unlock()
if now.Sub(c.mu.lastDeadlineSetAt) > c.timeout/100 {
c.mu.lastDeadlineSetAt = now

if err := c.Conn.SetReadDeadline(now.Add(c.timeout)); err != nil {
return err
}
}
return nil
}

// Read reads data from the connection with timeout.
func (c *IdleDisconnectConnection) Read(b []byte) (n int, err error) {
if err := c.updateDeadline(); err != nil {
return 0, err
}
return c.Conn.Read(b)
}

// Write writes data to the connection and sets the read timeout.
func (c *IdleDisconnectConnection) Write(b []byte) (n int, err error) {
// The Write for the connection is not blocking (or can block only temporary
// in case of flow control). For idle connections, the Read will be the call
// that will block and stay blocked until the backend doesn't send something.
// However, it is theoretically possible, that the traffic is only going in
// one direction - from the proxy to the backend, in which case we will call
// repeatedly Write but stay blocked on the Read. For that specific case - the
// write pushes further out the read deadline so the read doesn't timeout.
if err := c.updateDeadline(); err != nil {
return 0, err
}
return c.Conn.Write(b)
}

// SetDeadline is unsupported as it will interfere with the reads.
func (c *IdleDisconnectConnection) SetDeadline(t time.Time) error {
return errNotSupported
}

// SetReadDeadline is unsupported as it will interfere with the reads.
func (c *IdleDisconnectConnection) SetReadDeadline(t time.Time) error {
return errNotSupported
}

// SetWriteDeadline is unsupported as it will interfere with the reads.
func (c *IdleDisconnectConnection) SetWriteDeadline(t time.Time) error {
return errNotSupported
}

// IdleDisconnectOverlay upgrades the connection to one that closes when
// idle for more than timeout duration. Timeout of zero will turn off
// the idle disconnect code.
func IdleDisconnectOverlay(conn net.Conn, timeout time.Duration) net.Conn {
if timeout != 0 {
return &IdleDisconnectConnection{Conn: conn, timeout: timeout}
}
return conn
}
Loading

0 comments on commit 05a192a

Please sign in to comment.