From 2cc8f0fa0740fd64ca1c71d976a8831aa41ec531 Mon Sep 17 00:00:00 2001 From: djshow832 <873581766@qq.com> Date: Tue, 21 Nov 2023 12:38:07 +0800 Subject: [PATCH] add more error sources --- pkg/proxy/backend/authenticator.go | 18 ++-- pkg/proxy/backend/backend_conn_mgr.go | 46 ++++------ pkg/proxy/backend/error.go | 115 ++++++++++++++++++++++++- pkg/proxy/backend/handshake_handler.go | 50 ++--------- pkg/proxy/client/client_conn.go | 6 +- pkg/proxy/net/mysql.go | 4 +- pkg/proxy/net/packetio.go | 12 +-- pkg/proxy/net/packetio_test.go | 4 +- pkg/proxy/net/tls.go | 2 +- 9 files changed, 159 insertions(+), 98 deletions(-) diff --git a/pkg/proxy/backend/authenticator.go b/pkg/proxy/backend/authenticator.go index f5acb6e2..6ae1ab3b 100644 --- a/pkg/proxy/backend/authenticator.go +++ b/pkg/proxy/backend/authenticator.go @@ -76,10 +76,10 @@ func (auth *Authenticator) verifyBackendCaps(logger *zap.Logger, backendCapabili // The error cannot be sent to the client because the client only expects an initial handshake packet. // The only way is to log it and disconnect. logger.Error("require backend capabilities", zap.Stringer("common", commonCaps), zap.Stringer("required", requiredBackendCaps^commonCaps)) - return errors.Wrapf(ErrCapabilityNegotiation, "require %s from backend", requiredBackendCaps^commonCaps) + return pnet.WrapUserError(errors.Wrap(ErrBackendHandshake, errors.Wrapf(ErrCapabilityNegotiation, "require %s from backend", requiredBackendCaps^commonCaps)), capabilityErrMsg) } if auth.requireBackendTLS && (backendCapability&pnet.ClientSSL == 0) { - return pnet.WrapUserError(errors.New("backend doesn't enable TLS"), requireTiDBTLSErrMsg) + return pnet.WrapUserError(errors.Wrap(ErrBackendHandshake, errors.New("backend doesn't enable TLS")), requireTiDBTLSErrMsg) } return nil } @@ -125,7 +125,7 @@ func (auth *Authenticator) handshakeFirstTime(logger *zap.Logger, cctx ConnConte if writeErr := clientIO.WriteErrPacket(mysql.NewDefaultError(mysql.ER_NOT_SUPPORTED_AUTH_MODE)); writeErr != nil { return writeErr } - return errors.Wrapf(ErrCapabilityNegotiation, "require %s from frontend", requiredFrontendCaps&^commonCaps) + return errors.Wrap(ErrClientHandshake, errors.Wrapf(ErrCapabilityNegotiation, "require %s from frontend", requiredFrontendCaps&^commonCaps)) } commonCaps := frontendCapability & proxyCapability if frontendCapability^commonCaps != 0 { @@ -185,7 +185,7 @@ RECONNECT: } if err := auth.verifyBackendCaps(logger, backendCapability); err != nil { - return pnet.WrapUserError(err, capabilityErrMsg) + return err } if common := proxyCapability & backendCapability; (proxyCapability^common)&^pnet.ClientSSL != 0 { @@ -224,12 +224,14 @@ loop: } return err } - var packetErr error + var packetErr *mysql.MyError if serverPkt[0] == pnet.ErrHeader.Byte() { packetErr = pnet.ParseErrorPacket(serverPkt) - if handshakeHandler.HandleHandshakeErr(cctx, packetErr.(*mysql.MyError)) { - logger.Warn("handle handshake error, start reconnect", zap.Error(err)) - backendIO.Close() + if handshakeHandler.HandleHandshakeErr(cctx, packetErr) { + logger.Warn("handle handshake error, start reconnect", zap.Error(packetErr)) + if closeErr := backendIO.Close(); closeErr != nil { + logger.Warn("close backend error", zap.Error(closeErr)) + } goto RECONNECT } } diff --git a/pkg/proxy/backend/backend_conn_mgr.go b/pkg/proxy/backend/backend_conn_mgr.go index 5123734d..85862c65 100644 --- a/pkg/proxy/backend/backend_conn_mgr.go +++ b/pkg/proxy/backend/backend_conn_mgr.go @@ -148,7 +148,7 @@ func NewBackendConnManager(logger *zap.Logger, handshakeHandler HandshakeHandler // There are 2 types of signals, which may be sent concurrently. signalReceived: make(chan signalType, signalTypeNums), redirectResCh: make(chan *redirectResult, 1), - quitSource: SrcClientQuit, + quitSource: SrcNone, } mgr.SetValue(ConnContextKeyConnID, connectionID) return mgr @@ -170,13 +170,11 @@ func (mgr *BackendConnManager) Connect(ctx context.Context, clientIO *pnet.Packe mgr.clientIO = clientIO err := mgr.authenticator.handshakeFirstTime(mgr.logger.Named("authenticator"), mgr, clientIO, mgr.handshakeHandler, mgr.getBackendIO, frontendTLSConfig, backendTLSConfig) if err != nil { - mgr.setQuitSourceByErr(err) - mgr.handshakeHandler.OnHandshake(mgr, mgr.ServerAddr(), err) + mgr.handshakeHandler.OnHandshake(mgr, mgr.ServerAddr(), err, Error2Source(err)) clientIO.WriteUserError(err) return err } - mgr.resetQuitSource() - mgr.handshakeHandler.OnHandshake(mgr, mgr.ServerAddr(), nil) + mgr.handshakeHandler.OnHandshake(mgr, mgr.ServerAddr(), nil, SrcNone) mgr.cmdProcessor.capability = mgr.authenticator.capability childCtx, cancelFunc := context.WithCancel(ctx) @@ -235,8 +233,7 @@ func (mgr *BackendConnManager) getBackendIO(cctx ConnContext, auth *Authenticato backoff.WithContext(backoff.NewConstantBackOff(200*time.Millisecond), bctx), func(err error, d time.Duration) { origErr = err - mgr.setQuitSourceByErr(err) - mgr.handshakeHandler.OnHandshake(cctx, addr, err) + mgr.handshakeHandler.OnHandshake(cctx, addr, err, Error2Source(err)) }, ) cancel() @@ -431,7 +428,7 @@ func (mgr *BackendConnManager) tryRedirect(ctx context.Context) { // If the backend connection is closed, also close the client connection. // Otherwise, if the client is idle, the mgr will keep retrying. if errors.Is(rs.err, net.ErrClosed) || pnet.IsDisconnectError(rs.err) || errors.Is(rs.err, os.ErrDeadlineExceeded) { - mgr.quitSource = SrcBackendQuit + mgr.quitSource = SrcBackendNetwork if ignoredErr := mgr.clientIO.GracefulClose(); ignoredErr != nil { mgr.logger.Warn("graceful close client IO error", zap.Stringer("client_addr", mgr.clientIO.RemoteAddr()), zap.Error(ignoredErr)) } @@ -442,12 +439,10 @@ func (mgr *BackendConnManager) tryRedirect(ctx context.Context) { return } - defer mgr.resetQuitSource() var cn net.Conn cn, rs.err = net.DialTimeout("tcp", rs.to, DialTimeout) if rs.err != nil { - mgr.quitSource = SrcBackendQuit - mgr.handshakeHandler.OnHandshake(mgr, rs.to, rs.err) + mgr.handshakeHandler.OnHandshake(mgr, rs.to, rs.err, SrcBackendNetwork) return } newBackendIO := pnet.NewPacketIO(cn, mgr.logger, mgr.config.ConnBufferSize, pnet.WithRemoteAddr(rs.to, cn.RemoteAddr()), pnet.WithWrapError(ErrBackendConn)) @@ -455,8 +450,7 @@ func (mgr *BackendConnManager) tryRedirect(ctx context.Context) { if rs.err = mgr.authenticator.handshakeSecondTime(mgr.logger, mgr.clientIO, newBackendIO, mgr.backendTLS, sessionToken); rs.err == nil { rs.err = mgr.initSessionStates(newBackendIO, sessionStates) } else { - mgr.setQuitSourceByErr(rs.err) - mgr.handshakeHandler.OnHandshake(mgr, newBackendIO.RemoteAddr().String(), rs.err) + mgr.handshakeHandler.OnHandshake(mgr, newBackendIO.RemoteAddr().String(), rs.err, Error2Source(rs.err)) } if rs.err != nil { if ignoredErr := newBackendIO.Close(); ignoredErr != nil && !pnet.IsDisconnectError(ignoredErr) { @@ -469,7 +463,7 @@ func (mgr *BackendConnManager) tryRedirect(ctx context.Context) { } mgr.backendIO.Store(newBackendIO) mgr.setKeepAlive(mgr.config.HealthyKeepAlive) - mgr.handshakeHandler.OnHandshake(mgr, mgr.ServerAddr(), nil) + mgr.handshakeHandler.OnHandshake(mgr, mgr.ServerAddr(), nil, SrcNone) } // The original db in the auth info may be dropped during the session, so we need to authenticate with the current db. @@ -553,7 +547,7 @@ func (mgr *BackendConnManager) checkBackendActive() { if !backendIO.IsPeerActive() { mgr.logger.Info("backend connection is closed, close client connection", zap.Stringer("client_addr", mgr.clientIO.RemoteAddr()), zap.Stringer("backend_addr", backendIO.RemoteAddr())) - mgr.quitSource = SrcBackendQuit + mgr.quitSource = SrcBackendNetwork if err := mgr.clientIO.GracefulClose(); err != nil { mgr.logger.Warn("graceful close client IO error", zap.Stringer("client_addr", mgr.clientIO.RemoteAddr()), zap.Error(err)) } @@ -627,7 +621,7 @@ func (mgr *BackendConnManager) Close() error { } mgr.wg.Wait() - handErr := mgr.handshakeHandler.OnConnClose(mgr) + handErr := mgr.handshakeHandler.OnConnClose(mgr, mgr.quitSource) var connErr error var addr string @@ -677,26 +671,16 @@ func (mgr *BackendConnManager) setKeepAlive(cfg config.KeepAlive) { } } -// quitSource will be read by OnHandshake and OnConnClose, so setQuitSourceByErr should be called before them. func (mgr *BackendConnManager) setQuitSourceByErr(err error) { - // Do not update the source if err is nil. It may be already be set. if err == nil { return } - if errors.Is(err, ErrBackendConn) { - mgr.quitSource = SrcBackendQuit - } else if IsMySQLError(err) { - mgr.quitSource = SrcClientErr - } else if !errors.Is(err, ErrClientConn) { - mgr.quitSource = SrcProxyErr + // The source may be already be set. + // E.g. quitSource is set before TiProxy shuts down and client connection error is caused by shutdown instead of network. + if mgr.quitSource != SrcNone { + return } -} - -func (mgr *BackendConnManager) resetQuitSource() { - // SrcClientQuit is by default. - // Sometimes ErrClientConn is caused by GracefulClose and the quitSource is already set. - // Error maybe set during handshake for OnHandshake. If handshake finally succeeds, we reset it. - mgr.quitSource = SrcClientQuit + mgr.quitSource = Error2Source(err) } func (mgr *BackendConnManager) UpdateLogger(fields ...zap.Field) { diff --git a/pkg/proxy/backend/error.go b/pkg/proxy/backend/error.go index 4097126a..38e44fa6 100644 --- a/pkg/proxy/backend/error.go +++ b/pkg/proxy/backend/error.go @@ -4,7 +4,9 @@ package backend import ( + gomysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/pingcap/tiproxy/lib/util/errors" + pnet "github.com/pingcap/tiproxy/pkg/proxy/net" ) const ( @@ -18,6 +20,115 @@ const ( ) var ( - ErrClientConn = errors.New("this is an error from client") - ErrBackendConn = errors.New("this is an error from backend") + ErrClientConn = errors.New("read or write client connection fail") + ErrClientHandshake = errors.New("handshake with client fail") + ErrBackendConn = errors.New("read or write backend connection fail") + ErrBackendHandshake = errors.New("handshake with backend fail") ) + +type SourceComp int + +const ( + CompNone SourceComp = iota + CompClient + CompProxy + CompBackend +) + +type ErrorSource int + +const ( + // SrcNone includes: succeed for OnHandshake; client normally quit for OnConnClose + SrcNone ErrorSource = iota + // SrcClientNetwork includes: EOF; reset by peer; connection refused; TLS handshake fails + SrcClientNetwork + // SrcClientHandshake includes: client capability unsupported + SrcClientHandshake + // SrcClientSQLErr includes: backend returns auth fail; SQL error + SrcClientSQLErr + // SrcProxyQuit includes: proxy graceful shutdown + SrcProxyQuit + // SrcProxyMalformed includes: malformed packet; invalid sequence + SrcProxyMalformed + // SrcProxyGetBackend includes: no backends + SrcProxyGetBackend + // SrcProxyErr includes: HandshakeHandler returns error; proxy disables TLS + SrcProxyErr + // SrcBackendNetwork includes: EOF; reset by peer; connection refused; TLS handshake fails + SrcBackendNetwork + // SrcBackendHandshake includes: backend capability unsupported; backend disables TLS + SrcBackendHandshake +) + +// Error2Source returns the ErrorSource by the error. +func Error2Source(err error) ErrorSource { + switch { + case err == nil: + return SrcNone + case errors.Is(err, pnet.ErrInvalidSequence) || errors.Is(err, gomysql.ErrMalformPacket): + // We assume the clients and TiDB are right and treat it as TiProxy bugs. + // ErrInvalidSequence may be wrapped with ErrClientConn or ErrBackendConn, so put it before other conditions. + return SrcProxyErr + case errors.Is(err, ErrClientConn): + return SrcClientNetwork + case errors.Is(err, ErrBackendConn): + return SrcBackendNetwork + case errors.Is(err, ErrClientHandshake): + return SrcClientHandshake + case errors.Is(err, ErrBackendHandshake): + return SrcBackendHandshake + case IsMySQLError(err): + return SrcClientSQLErr + default: + return SrcProxyErr + } +} + +func (es ErrorSource) String() string { + switch es { + case SrcNone: + return "ok" + case SrcClientNetwork: + return "client network break" + case SrcClientHandshake: + return "client handshake fail" + case SrcClientSQLErr: + return "client SQL error" + case SrcProxyQuit: + return "proxy shutdown" + case SrcProxyMalformed: + return "malformed packet" + case SrcProxyGetBackend: + return "proxy get backend fail" + case SrcProxyErr: + return "proxy error" + case SrcBackendNetwork: + return "backend network break" + case SrcBackendHandshake: + return "backend handshake fail" + } + return "unknown" +} + +// GetSourceComp returns which component does this error belong to. +func (es ErrorSource) GetSourceComp() SourceComp { + switch es { + case SrcClientNetwork, SrcClientHandshake, SrcClientSQLErr: + return CompClient + case SrcProxyQuit, SrcProxyMalformed, SrcProxyGetBackend, SrcProxyErr: + return CompProxy + case SrcBackendNetwork, SrcBackendHandshake: + return CompBackend + default: + return CompNone + } +} + +// Normal returns whether this error source is expected. +func (es ErrorSource) Normal() bool { + switch es { + case SrcNone, SrcProxyQuit: + return true + } + return false +} diff --git a/pkg/proxy/backend/handshake_handler.go b/pkg/proxy/backend/handshake_handler.go index cb2665ac..9cc8e4e0 100644 --- a/pkg/proxy/backend/handshake_handler.go +++ b/pkg/proxy/backend/handshake_handler.go @@ -12,6 +12,8 @@ import ( "go.uber.org/zap" ) +// Interfaces in this file are used for the serverless tier. + // Context keys. type ConnContextKey string @@ -21,41 +23,6 @@ const ( ConnContextKeyConnAddr ConnContextKey = "conn-addr" ) -type ErrorSource int - -const ( - // SrcClientQuit includes: client quit; bad client conn - SrcClientQuit ErrorSource = iota - // SrcClientErr includes: wrong password; mal format packet - SrcClientErr - // SrcProxyQuit includes: proxy graceful shutdown - SrcProxyQuit - // SrcProxyErr includes: cannot get backend list; capability negotiation - SrcProxyErr - // SrcBackendQuit includes: backend quit - SrcBackendQuit - // SrcBackendErr is reserved - SrcBackendErr -) - -func (es ErrorSource) String() string { - switch es { - case SrcClientQuit: - return "client quit" - case SrcClientErr: - return "client error" - case SrcProxyQuit: - return "proxy shutdown" - case SrcProxyErr: - return "proxy error" - case SrcBackendQuit: - return "backend quit" - case SrcBackendErr: - return "backend error" - } - return "unknown" -} - var _ HandshakeHandler = (*DefaultHandshakeHandler)(nil) var _ HandshakeHandler = (*CustomHandshakeHandler)(nil) @@ -64,7 +31,6 @@ type ConnContext interface { ServerAddr() string ClientInBytes() uint64 ClientOutBytes() uint64 - QuitSource() ErrorSource UpdateLogger(fields ...zap.Field) SetValue(key, val any) Value(key any) any @@ -74,8 +40,8 @@ type HandshakeHandler interface { HandleHandshakeResp(ctx ConnContext, resp *pnet.HandshakeResp) error HandleHandshakeErr(ctx ConnContext, err *gomysql.MyError) bool // return true means retry connect GetRouter(ctx ConnContext, resp *pnet.HandshakeResp) (router.Router, error) - OnHandshake(ctx ConnContext, to string, err error) - OnConnClose(ctx ConnContext) error + OnHandshake(ctx ConnContext, to string, err error, src ErrorSource) + OnConnClose(ctx ConnContext, src ErrorSource) error OnTraffic(ctx ConnContext) GetCapability() pnet.Capability GetServerVersion() string @@ -111,13 +77,13 @@ func (handler *DefaultHandshakeHandler) GetRouter(ctx ConnContext, resp *pnet.Ha return ns.GetRouter(), nil } -func (handler *DefaultHandshakeHandler) OnHandshake(ConnContext, string, error) { +func (handler *DefaultHandshakeHandler) OnHandshake(ConnContext, string, error, ErrorSource) { } func (handler *DefaultHandshakeHandler) OnTraffic(ConnContext) { } -func (handler *DefaultHandshakeHandler) OnConnClose(ConnContext) error { +func (handler *DefaultHandshakeHandler) OnConnClose(ConnContext, ErrorSource) error { return nil } @@ -156,7 +122,7 @@ func (h *CustomHandshakeHandler) GetRouter(ctx ConnContext, resp *pnet.Handshake return nil, errors.New("no router") } -func (h *CustomHandshakeHandler) OnHandshake(ctx ConnContext, addr string, err error) { +func (h *CustomHandshakeHandler) OnHandshake(ctx ConnContext, addr string, err error, src ErrorSource) { if h.onHandshake != nil { h.onHandshake(ctx, addr, err) } @@ -168,7 +134,7 @@ func (h *CustomHandshakeHandler) OnTraffic(ctx ConnContext) { } } -func (h *CustomHandshakeHandler) OnConnClose(ctx ConnContext) error { +func (h *CustomHandshakeHandler) OnConnClose(ctx ConnContext, src ErrorSource) error { if h.onConnClose != nil { return h.onConnClose(ctx) } diff --git a/pkg/proxy/client/client_conn.go b/pkg/proxy/client/client_conn.go index a2fc4f50..b66afe16 100644 --- a/pkg/proxy/client/client_conn.go +++ b/pkg/proxy/client/client_conn.go @@ -57,11 +57,9 @@ func (cc *ClientConnection) Run(ctx context.Context) { clean: src := cc.connMgr.QuitSource() - switch src { - case backend.SrcClientQuit, backend.SrcClientErr, backend.SrcProxyQuit: - default: + if !src.Normal() { fields := cc.connMgr.ConnInfo() - fields = append(fields, zap.Stringer("quit source", src), zap.Error(err)) + fields = append(fields, zap.Stringer("quit_source", src), zap.Error(err)) cc.logger.Warn(msg, fields...) } } diff --git a/pkg/proxy/net/mysql.go b/pkg/proxy/net/mysql.go index d9b56f56..74a98ccd 100644 --- a/pkg/proxy/net/mysql.go +++ b/pkg/proxy/net/mysql.go @@ -32,7 +32,7 @@ func ParseInitialHandshake(data []byte) (Capability, uint64, string) { // skip min version serverVersion := string(data[1 : 1+bytes.IndexByte(data[1:], 0)]) pos := 1 + len(serverVersion) + 1 - connid := uint32(binary.LittleEndian.Uint32(data[pos : pos+4])) + connid := binary.LittleEndian.Uint32(data[pos : pos+4]) // skip salt first part // skip filter pos += 4 + 8 + 1 @@ -398,7 +398,7 @@ func ParseOKPacket(data []byte) uint16 { } // ParseErrorPacket transforms an error packet into a MyError object. -func ParseErrorPacket(data []byte) error { +func ParseErrorPacket(data []byte) *gomysql.MyError { e := new(gomysql.MyError) pos := 1 e.Code = binary.LittleEndian.Uint16(data[pos:]) diff --git a/pkg/proxy/net/packetio.go b/pkg/proxy/net/packetio.go index 8b5013b1..201a6b33 100644 --- a/pkg/proxy/net/packetio.go +++ b/pkg/proxy/net/packetio.go @@ -333,7 +333,7 @@ func (p *PacketIO) ForwardUntil(dest *PacketIO, isEnd func(firstByte byte, first for { header, err := p.readWriter.Peek(5) if err != nil { - return errors.Wrap(ErrReadConn, err) + return p.wrapErr(errors.Wrap(ErrReadConn, err)) } length := int(header[0]) | int(header[1])<<8 | int(header[2])<<16 end, needData := isEnd(header[4], length) @@ -343,30 +343,30 @@ func (p *PacketIO) ForwardUntil(dest *PacketIO, isEnd func(firstByte byte, first // TODO: allocate a buffer from pool and return the buffer after `process`. data, err = p.ReadPacket() if err != nil { - return errors.Wrap(ErrReadConn, err) + return p.wrapErr(errors.Wrap(ErrReadConn, err)) } if err := dest.WritePacket(data, false); err != nil { - return errors.Wrap(ErrWriteConn, err) + return p.wrapErr(errors.Wrap(ErrWriteConn, err)) } } else { for { sequence, pktSequence := header[3], p.readWriter.Sequence() if sequence != pktSequence { - return ErrInvalidSequence.GenWithStack("invalid sequence, expected %d, actual %d", pktSequence, sequence) + return p.wrapErr(ErrInvalidSequence.GenWithStack("invalid sequence, expected %d, actual %d", pktSequence, sequence)) } p.readWriter.SetSequence(sequence + 1) // Sequence may be different (e.g. with compression) so we can't just copy the data to the destination. dest.readWriter.SetSequence(dest.readWriter.Sequence() + 1) p.limitReader.N = int64(length + 4) if _, err := dest.readWriter.ReadFrom(&p.limitReader); err != nil { - return errors.Wrap(ErrRelayConn, err) + return p.wrapErr(errors.Wrap(ErrRelayConn, err)) } // For large packets, continue. if length < MaxPayloadLen { break } if header, err = p.readWriter.Peek(4); err != nil { - return errors.Wrap(ErrReadConn, err) + return p.wrapErr(errors.Wrap(ErrReadConn, err)) } length = int(header[0]) | int(header[1])<<8 | int(header[2])<<16 } diff --git a/pkg/proxy/net/packetio_test.go b/pkg/proxy/net/packetio_test.go index 100ae4d1..4c092437 100644 --- a/pkg/proxy/net/packetio_test.go +++ b/pkg/proxy/net/packetio_test.go @@ -265,13 +265,13 @@ func TestPredefinedPacket(t *testing.T) { func(t *testing.T, cli *PacketIO) { data, err := cli.ReadPacket() require.NoError(t, err) - merr := ParseErrorPacket(data).(*mysql.MyError) + merr := ParseErrorPacket(data) require.Equal(t, uint16(mysql.ER_UNKNOWN_ERROR), merr.Code) require.Equal(t, "Unknown error", merr.Message) data, err = cli.ReadPacket() require.NoError(t, err) - merr = ParseErrorPacket(data).(*mysql.MyError) + merr = ParseErrorPacket(data) require.Equal(t, uint16(mysql.ER_UNKNOWN_ERROR), merr.Code) require.Equal(t, "test error", merr.Message) diff --git a/pkg/proxy/net/tls.go b/pkg/proxy/net/tls.go index 4ad416ec..9303ecf3 100644 --- a/pkg/proxy/net/tls.go +++ b/pkg/proxy/net/tls.go @@ -39,7 +39,7 @@ func (p *PacketIO) ClientTLSHandshake(tlsConfig *tls.Config) error { conn := &tlsInternalConn{p.readWriter} tlsConn := tls.Client(conn, tlsConfig) if err := tlsConn.Handshake(); err != nil { - return errors.WithStack(errors.Wrap(ErrHandshakeTLS, err)) + return p.wrapErr(errors.Wrap(ErrHandshakeTLS, err)) } p.readWriter = newTLSReadWriter(p.readWriter, tlsConn) return nil