Skip to content

Commit c93c886

Browse files
committed
fix(snapshots): Fix snapshot handling bugs.
We have observed at customer sites various hard-to-replicate cluster instabilities where followers will fail to resync with the rest of the cluster after a restart. In particular, we get a couple of different types of errors from Raft: failed to decode incoming command: error=msgpack decode error [pos 12207]: only encoded map or array can be decoded into a struct This one generally happens when the follower is far enough behind that the leader decides to send a snapshot instead. The second is one that should not be possible: failed to decode incoming command: error=unknown rpc type 125 where the RPC number seems to change almost at random. After many rounds of debugging at a remote customer site, it appears that the root cause of the issue is that the Raft library will send extra data if the file size of the snapshot on disk is larger than the snapshot size indicated in the snapshot metadata. In my particular usecase that can happen because the SnapshotStore we use prefers to recycle older snapshot files rather than deleting them and creating a new file. We do this to make the system more resistant to running out of disk space. On the leader side, alleviate this issue by clamping the maximum amount of snapshot data we will send to the size indicated by the snapshot metadata. On the follower side, rework the RPC event loop to exit after handling an installSnapshot RPC instead of attempting to process the rest of the bytes remaining in the conn. This mirrors what the leader has done for snapshot handling with pooled conns since forever. On the leader side, arrange for pooled connections to discard any outstanding data when a connection is returned to the pool, and arrange for the system to panic if someone ever attempts to use a conn after returning it to the pool.
1 parent c4fb904 commit c93c886

File tree

3 files changed

+80
-40
lines changed

3 files changed

+80
-40
lines changed

fuzzy/go.mod

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@ go 1.20
44

55
require (
66
github.com/hashicorp/go-hclog v1.6.2
7-
github.com/hashicorp/go-msgpack/v2 v2.1.1
7+
github.com/hashicorp/go-msgpack/v2 v2.1.2
88
github.com/hashicorp/raft v1.2.0
99
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea
1010
)

fuzzy/go.sum

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@ github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9
4040
github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM=
4141
github.com/hashicorp/go-msgpack/v2 v2.1.1 h1:xQEY9yB2wnHitoSzk/B9UjXWRQ67QKu5AOm8aFp8N3I=
4242
github.com/hashicorp/go-msgpack/v2 v2.1.1/go.mod h1:upybraOAblm4S7rx0+jeNy+CWWhzywQsSRV5033mMu4=
43+
github.com/hashicorp/go-msgpack/v2 v2.1.2/go.mod h1:upybraOAblm4S7rx0+jeNy+CWWhzywQsSRV5033mMu4=
4344
github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs=
4445
github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM=
4546
github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=

net_transport.go

Lines changed: 78 additions & 39 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ package raft
66
import (
77
"bufio"
88
"context"
9+
"encoding/base64"
910
"errors"
1011
"fmt"
1112
"io"
@@ -54,13 +55,24 @@ const (
5455
minInFlightForPipelining = 2
5556
)
5657

58+
type zs struct{}
59+
60+
func (zs) Read([]byte) (int, error) {
61+
panic("Read should never be called")
62+
}
63+
64+
func (zs) Write([]byte) (int, error) {
65+
panic("Write should never be called")
66+
}
67+
5768
var (
5869
// ErrTransportShutdown is returned when operations on a transport are
5970
// invoked after it's been terminated.
6071
ErrTransportShutdown = errors.New("transport shutdown")
6172

6273
// ErrPipelineShutdown is returned when the pipeline is closed.
6374
ErrPipelineShutdown = errors.New("append pipeline closed")
75+
xplod = zs{}
6476
)
6577

6678
// NetworkTransport provides a network based transport that can be
@@ -112,6 +124,12 @@ type NetworkTransport struct {
112124
msgpackUseNewTimeFormat bool
113125
}
114126

127+
func (n *NetworkTransport) encoder(to io.Writer) *codec.Encoder {
128+
mgsp := &codec.MsgpackHandle{}
129+
mgsp.TimeNotBuiltin = !n.msgpackUseNewTimeFormat
130+
return codec.NewEncoder(to, mgsp)
131+
}
132+
115133
// NetworkTransportConfig encapsulates configuration for the network transport layer.
116134
type NetworkTransportConfig struct {
117135
// ServerAddressProvider is used to override the target address when establishing a connection to invoke an RPC
@@ -186,6 +204,7 @@ type StreamLayer interface {
186204
type netConn struct {
187205
target ServerAddress
188206
conn net.Conn
207+
r *bufio.Reader
189208
w *bufio.Writer
190209
dec *codec.Decoder
191210
enc *codec.Encoder
@@ -403,6 +422,11 @@ func (n *NetworkTransport) getProviderAddressOrFallback(id ServerID, target Serv
403422
func (n *NetworkTransport) getConn(target ServerAddress) (*netConn, error) {
404423
// Check for a pooled conn
405424
if conn := n.getPooledConn(target); conn != nil {
425+
// Sanity reset. Discard data that might not have been consumed the last time the connection was used.
426+
conn.r.Reset(conn.conn)
427+
conn.w.Reset(conn.conn)
428+
conn.enc.Reset(conn.w)
429+
conn.dec.Reset(conn.r)
406430
return conn, nil
407431
}
408432

@@ -411,21 +435,12 @@ func (n *NetworkTransport) getConn(target ServerAddress) (*netConn, error) {
411435
if err != nil {
412436
return nil, err
413437
}
414-
415-
// Wrap the conn
416-
netConn := &netConn{
417-
target: target,
418-
conn: conn,
419-
dec: codec.NewDecoder(bufio.NewReader(conn), &codec.MsgpackHandle{}),
420-
w: bufio.NewWriterSize(conn, connSendBufferSize),
421-
}
422-
423-
mp := &codec.MsgpackHandle{}
424-
mp.TimeNotBuiltin = !n.msgpackUseNewTimeFormat
425-
netConn.enc = codec.NewEncoder(netConn.w, mp)
426-
427-
// Done
428-
return netConn, nil
438+
rBuf := bufio.NewReader(conn)
439+
wBuf := bufio.NewWriterSize(conn, connSendBufferSize)
440+
dec := codec.NewDecoder(rBuf, &codec.MsgpackHandle{})
441+
enc := n.encoder(wBuf)
442+
res := &netConn{target: target, conn: conn, dec: dec, enc: enc, r: rBuf, w: wBuf}
443+
return res, nil
429444
}
430445

431446
// returnConn returns a connection back to the pool.
@@ -437,6 +452,10 @@ func (n *NetworkTransport) returnConn(conn *netConn) {
437452
conns := n.connPool[key]
438453

439454
if !n.IsShutdown() && len(conns) < n.maxPool {
455+
conn.dec.Reset(xplod)
456+
conn.enc.Reset(xplod)
457+
conn.w.Reset(xplod)
458+
conn.r.Reset(xplod)
440459
n.connPool[key] = append(conns, conn)
441460
} else {
442461
_ = conn.Release()
@@ -527,7 +546,7 @@ func (n *NetworkTransport) InstallSnapshot(id ServerID, target ServerAddress, ar
527546
}
528547

529548
// Stream the state
530-
if _, err = io.Copy(conn.w, data); err != nil {
549+
if _, err = io.CopyN(conn.w, data, args.Size); err != nil {
531550
return err
532551
}
533552

@@ -606,40 +625,55 @@ func (n *NetworkTransport) handleConn(connCtx context.Context, conn net.Conn) {
606625
r := bufio.NewReaderSize(conn, connReceiveBufferSize)
607626
w := bufio.NewWriter(conn)
608627
dec := codec.NewDecoder(r, &codec.MsgpackHandle{})
609-
610-
mp := &codec.MsgpackHandle{}
611-
mp.TimeNotBuiltin = !n.msgpackUseNewTimeFormat
612-
enc := codec.NewEncoder(w, mp)
613-
614-
for {
628+
enc := n.encoder(w)
629+
var cmd, prevCmd byte
630+
var err error
631+
// Do not re-enter the loop once the conn has been used to install a snapshot.
632+
// The server side has pretty much always done this.
633+
for cmd != rpcInstallSnapshot {
634+
prevCmd = cmd
615635
select {
616636
case <-connCtx.Done():
617637
n.logger.Debug("stream layer is closed")
618638
return
619639
default:
620640
}
621641

622-
if err := n.handleCommand(r, dec, enc); err != nil {
642+
if cmd, err = n.handleCommand(r, dec, enc); err != nil {
623643
if err != io.EOF {
624-
n.logger.Error("failed to decode incoming command", "error", err)
644+
n.logger.Error("failed to decode incoming command", "error", err, "cmd", cmd, "prevCmd", prevCmd)
645+
}
646+
if r.Buffered() > 0 {
647+
unread := r.Buffered()
648+
snippet := [100]byte{}
649+
cnt, _ := r.Read(snippet[:])
650+
dst := make([]byte, base64.StdEncoding.EncodedLen(cnt))
651+
base64.StdEncoding.Encode(dst, snippet[:cnt])
652+
n.logger.Error("remaining read buffer", "sz", unread, "snippet", string(dst), "cmd", cmd, "prevCmd", prevCmd)
625653
}
626654
return
627655
}
628-
if err := w.Flush(); err != nil {
629-
n.logger.Error("failed to flush response", "error", err)
656+
if err = w.Flush(); err != nil {
657+
n.logger.Error("failed to flush response", "error", err, "cmd", cmd, "prevCmd", prevCmd)
630658
return
631659
}
632660
}
661+
// If we close conn too soon, the leader may get stuck in an infinite loop
662+
// attempting to re-install the snapshot we just finished processing. Avert that
663+
// by waiting a bit for the server-side to close things, and close them ourselves
664+
// of we wait longer than 5 seconds.
665+
_ = conn.SetReadDeadline(time.Now().Add(5 * time.Second))
666+
_, _ = io.Copy(io.Discard, r)
633667
}
634668

635669
// handleCommand is used to decode and dispatch a single command.
636-
func (n *NetworkTransport) handleCommand(r *bufio.Reader, dec *codec.Decoder, enc *codec.Encoder) error {
670+
func (n *NetworkTransport) handleCommand(r *bufio.Reader, dec *codec.Decoder, enc *codec.Encoder) (byte, error) {
637671
getTypeStart := time.Now()
638672

639673
// Get the rpc type
640674
rpcType, err := r.ReadByte()
641675
if err != nil {
642-
return err
676+
return 255, err
643677
}
644678

645679
// measuring the time to get the first byte separately because the heartbeat conn will hang out here
@@ -655,12 +689,13 @@ func (n *NetworkTransport) handleCommand(r *bufio.Reader, dec *codec.Decoder, en
655689

656690
// Decode the command
657691
isHeartbeat := false
692+
wantFlush := false
658693
var labels []metrics.Label
659694
switch rpcType {
660695
case rpcAppendEntries:
661696
var req AppendEntriesRequest
662697
if err := dec.Decode(&req); err != nil {
663-
return err
698+
return rpcType, err
664699
}
665700
rpc.Command = &req
666701

@@ -684,34 +719,35 @@ func (n *NetworkTransport) handleCommand(r *bufio.Reader, dec *codec.Decoder, en
684719
case rpcRequestVote:
685720
var req RequestVoteRequest
686721
if err := dec.Decode(&req); err != nil {
687-
return err
722+
return rpcType, err
688723
}
689724
rpc.Command = &req
690725
labels = []metrics.Label{{Name: "rpcType", Value: "RequestVote"}}
691726
case rpcRequestPreVote:
692727
var req RequestPreVoteRequest
693728
if err := dec.Decode(&req); err != nil {
694-
return err
729+
return rpcType, err
695730
}
696731
rpc.Command = &req
697732
labels = []metrics.Label{{Name: "rpcType", Value: "RequestPreVote"}}
698733
case rpcInstallSnapshot:
699734
var req InstallSnapshotRequest
700735
if err := dec.Decode(&req); err != nil {
701-
return err
736+
return rpcType, err
702737
}
703738
rpc.Command = &req
704739
rpc.Reader = io.LimitReader(r, req.Size)
740+
wantFlush = true
705741
labels = []metrics.Label{{Name: "rpcType", Value: "InstallSnapshot"}}
706742
case rpcTimeoutNow:
707743
var req TimeoutNowRequest
708744
if err := dec.Decode(&req); err != nil {
709-
return err
745+
return rpcType, err
710746
}
711747
rpc.Command = &req
712748
labels = []metrics.Label{{Name: "rpcType", Value: "TimeoutNow"}}
713749
default:
714-
return fmt.Errorf("unknown rpc type %d", rpcType)
750+
return rpcType, fmt.Errorf("unknown rpc type %d", rpcType)
715751
}
716752

717753
metrics.MeasureSinceWithLabels([]string{"raft", "net", "rpcDecode"}, decodeStart, labels)
@@ -733,7 +769,7 @@ func (n *NetworkTransport) handleCommand(r *bufio.Reader, dec *codec.Decoder, en
733769
select {
734770
case n.consumeCh <- rpc:
735771
case <-n.shutdownCh:
736-
return ErrTransportShutdown
772+
return rpcType, ErrTransportShutdown
737773
}
738774

739775
// Wait for response
@@ -744,23 +780,26 @@ RESP:
744780
select {
745781
case resp := <-respCh:
746782
defer metrics.MeasureSinceWithLabels([]string{"raft", "net", "rpcRespond"}, respWaitStart, labels)
783+
for wantFlush && r.Buffered() > 0 {
784+
_, _ = io.CopyN(io.Discard, r, int64(r.Buffered()))
785+
}
747786
// Send the error first
748787
respErr := ""
749788
if resp.Error != nil {
750789
respErr = resp.Error.Error()
751790
}
752791
if err := enc.Encode(respErr); err != nil {
753-
return err
792+
return rpcType, err
754793
}
755794

756795
// Send the response
757796
if err := enc.Encode(resp.Response); err != nil {
758-
return err
797+
return rpcType, err
759798
}
760799
case <-n.shutdownCh:
761-
return ErrTransportShutdown
800+
return rpcType, ErrTransportShutdown
762801
}
763-
return nil
802+
return rpcType, nil
764803
}
765804

766805
// decodeResponse is used to decode an RPC response and reports whether

0 commit comments

Comments
 (0)