Skip to content

Commit

Permalink
Merge #28200
Browse files Browse the repository at this point in the history
28200: cli: enhance errors reported upon conn failures r=knz a=knz

Fixes #24788.

Prior to this patch, the various CLI commands were making
a halfhearted attempt at decorating the underlying error with some
troubleshooting instructions.

Unfortunately, this was mixing up situations where the TCP connection
failed outright, those where the security settings were incorrect, and
connections dropped later, after the initial handshake was
successful.

In practice, we've noticed that the troubleshooting steps are rather
different for both kinds of situations. So this patch enhances the
code to clarify what is going on. It attempts to distinguish:

- TCP connection problem ("check --host vs --advertise-host").
- secure conn to insecure server ("use --insecure?")
- invalid TLS settings / TLS auth error ("check credentials").
- timeouts.
- connection lost (e.g. conn closed by server).

Release note (cli change): The various `cockroach` client comments
now better attempt to inform the user about why a connection is failing.

Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
craig[bot] and knz committed Aug 7, 2018
2 parents c559495 + 0894cd6 commit bb66aae
Show file tree
Hide file tree
Showing 8 changed files with 385 additions and 109 deletions.
30 changes: 17 additions & 13 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,6 @@ import (
"sync"
"time"

"github.com/pkg/errors"

"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
Expand Down Expand Up @@ -152,8 +150,14 @@ type Config struct {
HistogramWindowInterval time.Duration
}

func didYouMeanInsecureError(err error) error {
return errors.Wrap(err, "problem using security settings, did you mean to use --insecure?")
func wrapError(err error) error {
if _, ok := err.(*security.Error); !ok {
return &security.Error{
Message: "problem using security settings",
Err: err,
}
}
return err
}

// InitDefaults sets up the default values for a config.
Expand Down Expand Up @@ -212,7 +216,7 @@ func (cfg *Config) LoadSecurityOptions(options url.Values, username string) erro
// Fetch CA cert. This is required.
caCertPath, err := cfg.GetCACertPath()
if err != nil {
return didYouMeanInsecureError(err)
return wrapError(err)
}
options.Add("sslmode", "verify-full")
options.Add("sslrootcert", caCertPath)
Expand Down Expand Up @@ -294,12 +298,12 @@ func (cfg *Config) GetClientTLSConfig() (*tls.Config, error) {

cm, err := cfg.GetCertificateManager()
if err != nil {
return nil, didYouMeanInsecureError(err)
return nil, wrapError(err)
}

tlsCfg, err := cm.GetClientTLSConfig(cfg.User)
if err != nil {
return nil, didYouMeanInsecureError(err)
return nil, wrapError(err)
}
return tlsCfg, nil
}
Expand All @@ -316,12 +320,12 @@ func (cfg *Config) GetUIClientTLSConfig() (*tls.Config, error) {

cm, err := cfg.GetCertificateManager()
if err != nil {
return nil, didYouMeanInsecureError(err)
return nil, wrapError(err)
}

tlsCfg, err := cm.GetUIClientTLSConfig()
if err != nil {
return nil, didYouMeanInsecureError(err)
return nil, wrapError(err)
}
return tlsCfg, nil
}
Expand All @@ -337,12 +341,12 @@ func (cfg *Config) GetServerTLSConfig() (*tls.Config, error) {

cm, err := cfg.GetCertificateManager()
if err != nil {
return nil, didYouMeanInsecureError(err)
return nil, wrapError(err)
}

tlsCfg, err := cm.GetServerTLSConfig()
if err != nil {
return nil, didYouMeanInsecureError(err)
return nil, wrapError(err)
}
return tlsCfg, nil
}
Expand All @@ -358,12 +362,12 @@ func (cfg *Config) GetUIServerTLSConfig() (*tls.Config, error) {

cm, err := cfg.GetCertificateManager()
if err != nil {
return nil, didYouMeanInsecureError(err)
return nil, wrapError(err)
}

tlsCfg, err := cm.GetUIServerTLSConfig()
if err != nil {
return nil, didYouMeanInsecureError(err)
return nil, wrapError(err)
}
return tlsCfg, nil
}
Expand Down
56 changes: 0 additions & 56 deletions pkg/cli/cli_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -339,62 +339,6 @@ func TestQuit(t *testing.T) {
c.Run("quit")
// Wait until this async command cleanups the server.
<-c.Stopper().IsStopped()

// NB: if this test is ever flaky due to port reuse, we could run against
// :0 (which however changes some of the errors we get).
// One way of getting that is:
// c.Cfg.AdvertiseAddr = "127.0.0.1:0"

styled := func(s string) string {
const preamble = `unable to connect or connection lost.
Please check the address and credentials such as certificates \(if attempting to
communicate with a secure cluster\).
`
return preamble + s
}

for _, test := range []struct {
cmd, expOutPattern string
}{
// Error returned from GRPC to internal/client (which has to pass it
// up the stack as a roachpb.NewError(roachpb.NewSendError(.)).
// Error returned directly from GRPC.
{`quit`, styled(
`Failed to connect to the node: initial connection heartbeat failed: rpc ` +
`error: code = Unavailable desc = all SubConns are in TransientFailure, ` +
`latest connection error: connection error: desc = "transport: Error while dialing dial tcp .*: ` +
`connect: connection refused"`),
},
// Going through the SQL client libraries gives a *net.OpError which
// we also handle.
//
// On *nix, this error is:
//
// dial tcp 127.0.0.1:65054: getsockopt: connection refused
//
// On Windows, this error is:
//
// dial tcp 127.0.0.1:59951: connectex: No connection could be made because the target machine actively refused it.
//
// So we look for the common bit.
{`zone ls`, styled(
`dial tcp .*: .* refused`),
},
} {
t.Run(test.cmd, func(t *testing.T) {
out, err := c.RunWithCapture(test.cmd)
if err != nil {
t.Fatal(err)
}
exp := test.cmd + "\n" + test.expOutPattern
re := regexp.MustCompile(exp)
if !re.MatchString(out) {
t.Errorf("expected '%s' to match pattern:\n%s\ngot:\n%s", test.cmd, exp, out)
}
})
}
}

func Example_logging() {
Expand Down
147 changes: 129 additions & 18 deletions pkg/cli/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,21 +16,51 @@ package cli

import (
"context"
"crypto/x509"
"fmt"
"net"
"regexp"
"strings"

"github.com/lib/pq"
"github.com/pkg/errors"
"github.com/spf13/cobra"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/netutil"
)

// reConnRefused is a regular expression that can be applied
// to the details of a GRPC connection failure.
//
// On *nix, a connect error looks like:
// dial tcp <addr>: <syscall>: connection refused
// On Windows, it looks like:
// dial tcp <addr>: <syscall>: No connection could be made because the target machine actively refused it.
// So we look for the common bit.
var reGRPCConnRefused = regexp.MustCompile(`Error while dialing dial tcp .*: connection.* refused`)

// reGRPCNoTLS is a regular expression that can be applied to the
// details of a GRPC auth failure when the server is insecure.
var reGRPCNoTLS = regexp.MustCompile(`authentication handshake failed: tls: first record does not look like a TLS handshake`)

// reGRPCAuthFailure is a regular expression that can be applied to
// the details of a GRPC auth failure when the SSL handshake fails.
var reGRPCAuthFailure = regexp.MustCompile(`authentication handshake failed: x509`)

// reGRPCConnFailed is a regular expression that can be applied
// to the details of a GRPC connection failure when, perhaps,
// the server was expecting a TLS handshake but the client didn't
// provide one (i.e. the client was started with --insecure).
// Note however in that case it's not certain what the problem is,
// as the same error could be raised for other reasons.
var reGRPCConnFailed = regexp.MustCompile(`desc = transport is closing`)

// MaybeDecorateGRPCError catches grpc errors and provides a more helpful error
// message to the user.
func MaybeDecorateGRPCError(
Expand All @@ -43,34 +73,115 @@ func MaybeDecorateGRPCError(
return nil
}

connDropped := func() error {
const format = `unable to connect or connection lost.
Please check the address and credentials such as certificates (if attempting to
communicate with a secure cluster).
extraInsecureHint := func() string {
extra := ""
if baseCfg.Insecure {
extra = "\nIf the node is configured to require secure connections,\n" +
"remove --insecure and configure secure credentials instead.\n"
}
return extra
}

%s`
connFailed := func() error {
const format = "cannot dial server.\n" +
"Is the server running?\n" +
"If the server is running, check --host client-side and --advertise server-side.\n\n%v"
return errors.Errorf(format, err)
}
opTimeout := func() error {
const format = `operation timed out.

%s`
return errors.Errorf(format, err)
connSecurityHint := func() error {
const format = "SSL authentication error while connecting.\n%s\n%v"
return errors.Errorf(format, extraInsecureHint(), err)
}

connInsecureHint := func() error {
return errors.Errorf("cannot establish secure connection to insecure server.\n"+
"Maybe use --insecure?\n\n%v", err)
}

connRefused := func() error {
extra := extraInsecureHint()
return errors.Errorf("server closed the connection.\n"+
"Is this a CockroachDB node?\n"+extra+"\n%v", err)
}

// Is this an "unable to connect" type of error?
unwrappedErr := errors.Cause(err)
switch unwrappedErr.(type) {
case *roachpb.SendError:
return connDropped()

if unwrappedErr == pq.ErrSSLNotSupported {
// SQL command failed after establishing a TCP connection
// successfully, but discovering that it cannot use TLS while it
// expected the server supports TLS.
return connInsecureHint()
}

switch wErr := unwrappedErr.(type) {
case *security.Error:
return errors.Errorf("cannot load certificates.\n"+
"Check your certificate settings, set --certs-dir, or use --insecure for insecure clusters.\n\n%v",
unwrappedErr)

case *x509.UnknownAuthorityError:
// A SQL connection was attempted with an incorrect CA.
return connSecurityHint()

case *initialSQLConnectionError:
// SQL handshake failed after establishing a TCP connection
// successfully, something else than CockroachDB responded, was
// confused and closed the door on us.
return connRefused()

case *pq.Error:
// SQL commands will fail with a pq error but only after
// establishing a TCP connection successfully. So if we got
// here, there was a TCP connection already.

// Did we fail due to security settings?
if wErr.Code == pgerror.CodeProtocolViolationError {
return connSecurityHint()
}
// Otherwise, there was a regular SQL error. Just report that.
return wErr

case *net.OpError:
return connDropped()
// A non-RPC client command was used (e.g. a SQL command) and an
// error occurred early while establishing the TCP connection.

// Is this a TLS error?
if msg := wErr.Err.Error(); strings.HasPrefix(msg, "tls: ") {
// Error during the SSL handshake: a provided client
// certificate was invalid, but the CA was OK. (If the CA was
// not OK, we'd get a x509 error, see case above.)
return connSecurityHint()
}
return connFailed()

case *netutil.InitialHeartbeatFailedError:
return connDropped()
// A GRPC TCP connection was established but there was an early failure.
// Try to distinguish the cases.
msg := wErr.Error()
if reGRPCConnRefused.MatchString(msg) {
return connFailed()
}
if reGRPCNoTLS.MatchString(msg) {
return connInsecureHint()
}
if reGRPCAuthFailure.MatchString(msg) {
return connSecurityHint()
}
if reGRPCConnFailed.MatchString(msg) {
return connRefused()
}

// Other cases may be timeouts or other situations, these
// will be handled below.
}

opTimeout := func() error {
return errors.Errorf("operation timed out.\n\n%v", err)
}

// No, it's not. Is it a plain context cancellation (i.e. timeout)?
// Is it a plain context cancellation (i.e. timeout)?
switch unwrappedErr {
case context.DeadlineExceeded:
return opTimeout()
Expand All @@ -88,7 +199,7 @@ communicate with a secure cluster).
return fmt.Errorf(
"incompatible client and server versions (likely server version: v1.0, required: >=v1.1)")
} else if grpcutil.IsClosedConnection(unwrappedErr) {
return connDropped()
return errors.Errorf("connection lost.\n\n%v", err)
}

// Nothing we can special case, just return what we have.
Expand Down
15 changes: 15 additions & 0 deletions pkg/cli/interactive_tests/netcat.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
import socket
import sys

server = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
server.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1)
server.bind(("0.0.0.0", 26257))
server.listen(1)
print "ready"
client_socket, addr = server.accept()
print "connected"

while True:
c = client_socket.recv(1)
sys.stdout.write("%c" % c)
sys.stdout.flush()
Loading

0 comments on commit bb66aae

Please sign in to comment.