get.pme.sh/pnats@v0.0.0-20240304004023-26bb5a137ed0/server/raft.go (about)

     1  // Copyright 2020-2023 The NATS Authors
     2  // Licensed under the Apache License, Version 2.0 (the "License");
     3  // you may not use this file except in compliance with the License.
     4  // You may obtain a copy of the License at
     5  //
     6  // http://www.apache.org/licenses/LICENSE-2.0
     7  //
     8  // Unless required by applicable law or agreed to in writing, software
     9  // distributed under the License is distributed on an "AS IS" BASIS,
    10  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package server
    15  
    16  import (
    17  	"bytes"
    18  	"crypto/sha256"
    19  	"encoding/binary"
    20  	"errors"
    21  	"fmt"
    22  	"hash"
    23  	"math"
    24  	"math/rand"
    25  	"net"
    26  	"os"
    27  	"path/filepath"
    28  	"runtime"
    29  	"strings"
    30  	"sync"
    31  	"sync/atomic"
    32  	"time"
    33  
    34  	"get.pme.sh/pnats/internal/fastrand"
    35  
    36  	"github.com/minio/highwayhash"
    37  )
    38  
    39  type RaftNode interface {
    40  	Propose(entry []byte) error
    41  	ProposeDirect(entries []*Entry) error
    42  	ForwardProposal(entry []byte) error
    43  	InstallSnapshot(snap []byte) error
    44  	SendSnapshot(snap []byte) error
    45  	NeedSnapshot() bool
    46  	Applied(index uint64) (entries uint64, bytes uint64)
    47  	State() RaftState
    48  	Size() (entries, bytes uint64)
    49  	Progress() (index, commit, applied uint64)
    50  	Leader() bool
    51  	Quorum() bool
    52  	Current() bool
    53  	Healthy() bool
    54  	Term() uint64
    55  	GroupLeader() string
    56  	HadPreviousLeader() bool
    57  	StepDown(preferred ...string) error
    58  	SetObserver(isObserver bool)
    59  	IsObserver() bool
    60  	Campaign() error
    61  	ID() string
    62  	Group() string
    63  	Peers() []*Peer
    64  	UpdateKnownPeers(knownPeers []string)
    65  	ProposeAddPeer(peer string) error
    66  	ProposeRemovePeer(peer string) error
    67  	AdjustClusterSize(csz int) error
    68  	AdjustBootClusterSize(csz int) error
    69  	ClusterSize() int
    70  	ApplyQ() *ipQueue[*CommittedEntry]
    71  	PauseApply() error
    72  	ResumeApply()
    73  	LeadChangeC() <-chan bool
    74  	QuitC() <-chan struct{}
    75  	Created() time.Time
    76  	Stop()
    77  	Delete()
    78  	Wipe()
    79  }
    80  
    81  type WAL interface {
    82  	Type() StorageType
    83  	StoreMsg(subj string, hdr, msg []byte) (uint64, int64, error)
    84  	LoadMsg(index uint64, sm *StoreMsg) (*StoreMsg, error)
    85  	RemoveMsg(index uint64) (bool, error)
    86  	Compact(index uint64) (uint64, error)
    87  	Purge() (uint64, error)
    88  	Truncate(seq uint64) error
    89  	State() StreamState
    90  	FastState(*StreamState)
    91  	Stop() error
    92  	Delete() error
    93  }
    94  
    95  type Peer struct {
    96  	ID      string
    97  	Current bool
    98  	Last    time.Time
    99  	Lag     uint64
   100  }
   101  
   102  type RaftState uint8
   103  
   104  // Allowable states for a NATS Consensus Group.
   105  const (
   106  	Follower RaftState = iota
   107  	Leader
   108  	Candidate
   109  	Closed
   110  )
   111  
   112  func (state RaftState) String() string {
   113  	switch state {
   114  	case Follower:
   115  		return "FOLLOWER"
   116  	case Candidate:
   117  		return "CANDIDATE"
   118  	case Leader:
   119  		return "LEADER"
   120  	case Closed:
   121  		return "CLOSED"
   122  	}
   123  	return "UNKNOWN"
   124  }
   125  
   126  type raft struct {
   127  	sync.RWMutex
   128  
   129  	created time.Time // Time that the group was created
   130  	accName string    // Account name of the asset this raft group is for
   131  	group   string    // Raft group
   132  	sd      string    // Store directory
   133  	id      string    // Node ID
   134  
   135  	wal   WAL         // WAL store (filestore or memstore)
   136  	wtype StorageType // WAL type, e.g. FileStorage or MemoryStorage
   137  	track bool        //
   138  	werr  error       // Last write error
   139  
   140  	state    atomic.Int32 // RaftState
   141  	hh       hash.Hash64  // Highwayhash, used for snapshots
   142  	snapfile string       // Snapshot filename
   143  
   144  	csz   int             // Cluster size
   145  	qn    int             // Number of nodes needed to establish quorum
   146  	peers map[string]*lps // Other peers in the Raft group
   147  	ipeer bool            // Implicit leadership when quorum is impossible
   148  
   149  	removed map[string]struct{}            // Peers that were removed from the group
   150  	acks    map[uint64]map[string]struct{} // Append entry responses/acks, map of entry index -> peer ID
   151  	pae     map[uint64]*appendEntry        // Pending append entries
   152  
   153  	elect  *time.Timer // Election timer, normally accessed via electTimer
   154  	active time.Time   // Last activity time, i.e. for heartbeats
   155  	llqrt  time.Time   // Last quorum lost time
   156  	lsut   time.Time   // Last scale-up time
   157  
   158  	term     uint64 // The current vote term
   159  	pterm    uint64 // Previous term from the last snapshot
   160  	pindex   uint64 // Previous index from the last snapshot
   161  	commit   uint64 // Sequence number of the most recent commit
   162  	applied  uint64 // Sequence number of the most recently applied commit
   163  	hcbehind bool   // Were we falling behind at the last health check? (see: isCurrent)
   164  
   165  	leader string // The ID of the leader
   166  	vote   string // Our current vote state
   167  	lxfer  bool   // Are we doing a leadership transfer?
   168  
   169  	s  *Server    // Reference to top-level server
   170  	c  *client    // Internal client for subscriptions
   171  	js *jetStream // JetStream, if running, to see if we are out of resources
   172  
   173  	dflag    bool           // Debug flag
   174  	pleader  bool           // Has the group ever had a leader?
   175  	observer bool           // The node is observing, i.e. not participating in voting
   176  	extSt    extensionState // Extension state
   177  
   178  	psubj  string // Proposals subject
   179  	rpsubj string // Remove peers subject
   180  	vsubj  string // Vote requests subject
   181  	vreply string // Vote responses subject
   182  	asubj  string // Append entries subject
   183  	areply string // Append entries responses subject
   184  
   185  	sq    *sendq        // Send queue for outbound RPC messages
   186  	aesub *subscription // Subscription for handleAppendEntry callbacks
   187  
   188  	wtv []byte // Term and vote to be written
   189  	wps []byte // Peer state to be written
   190  
   191  	catchup  *catchupState               // For when we need to catch up as a follower.
   192  	progress map[string]*ipQueue[uint64] // For leader or server catching up a follower.
   193  
   194  	paused    bool   // Whether or not applies are paused
   195  	hcommit   uint64 // The commit at the time that applies were paused
   196  	pobserver bool   // Whether we were an observer at the time that applies were paused
   197  
   198  	prop  *ipQueue[*Entry]               // Proposals
   199  	entry *ipQueue[*appendEntry]         // Append entries
   200  	resp  *ipQueue[*appendEntryResponse] // Append entries responses
   201  	apply *ipQueue[*CommittedEntry]      // Apply queue (committed entries to be passed to upper layer)
   202  	reqs  *ipQueue[*voteRequest]         // Vote requests
   203  	votes *ipQueue[*voteResponse]        // Vote responses
   204  	leadc chan bool                      // Leader changes
   205  	quit  chan struct{}                  // Raft group shutdown
   206  }
   207  
   208  // cacthupState structure that holds our subscription, and catchup term and index
   209  // as well as starting term and index and how many updates we have seen.
   210  type catchupState struct {
   211  	sub    *subscription // Subscription that catchup messages will arrive on
   212  	cterm  uint64        // Catchup term
   213  	cindex uint64        // Catchup index
   214  	pterm  uint64        // Starting term
   215  	pindex uint64        // Starting index
   216  	active time.Time     // Last time we received a message for this catchup
   217  }
   218  
   219  // lps holds peer state of last time and last index replicated.
   220  type lps struct {
   221  	ts int64  // Last timestamp
   222  	li uint64 // Last index replicated
   223  	kp bool   // Known peer
   224  }
   225  
   226  const (
   227  	minElectionTimeoutDefault      = 4 * time.Second
   228  	maxElectionTimeoutDefault      = 9 * time.Second
   229  	minCampaignTimeoutDefault      = 100 * time.Millisecond
   230  	maxCampaignTimeoutDefault      = 8 * minCampaignTimeoutDefault
   231  	hbIntervalDefault              = 1 * time.Second
   232  	lostQuorumIntervalDefault      = hbIntervalDefault * 10 // 10 seconds
   233  	lostQuorumCheckIntervalDefault = hbIntervalDefault * 10 // 10 seconds
   234  )
   235  
   236  var (
   237  	minElectionTimeout = minElectionTimeoutDefault
   238  	maxElectionTimeout = maxElectionTimeoutDefault
   239  	minCampaignTimeout = minCampaignTimeoutDefault
   240  	maxCampaignTimeout = maxCampaignTimeoutDefault
   241  	hbInterval         = hbIntervalDefault
   242  	lostQuorumInterval = lostQuorumIntervalDefault
   243  	lostQuorumCheck    = lostQuorumCheckIntervalDefault
   244  )
   245  
   246  type RaftConfig struct {
   247  	Name     string
   248  	Store    string
   249  	Log      WAL
   250  	Track    bool
   251  	Observer bool
   252  }
   253  
   254  var (
   255  	errNotLeader         = errors.New("raft: not leader")
   256  	errAlreadyLeader     = errors.New("raft: already leader")
   257  	errNilCfg            = errors.New("raft: no config given")
   258  	errCorruptPeers      = errors.New("raft: corrupt peer state")
   259  	errEntryLoadFailed   = errors.New("raft: could not load entry from WAL")
   260  	errEntryStoreFailed  = errors.New("raft: could not store entry to WAL")
   261  	errNodeClosed        = errors.New("raft: node is closed")
   262  	errBadSnapName       = errors.New("raft: snapshot name could not be parsed")
   263  	errNoSnapAvailable   = errors.New("raft: no snapshot available")
   264  	errCatchupsRunning   = errors.New("raft: snapshot can not be installed while catchups running")
   265  	errSnapshotCorrupt   = errors.New("raft: snapshot corrupt")
   266  	errTooManyPrefs      = errors.New("raft: stepdown requires at most one preferred new leader")
   267  	errNoPeerState       = errors.New("raft: no peerstate")
   268  	errAdjustBootCluster = errors.New("raft: can not adjust boot peer size on established group")
   269  	errLeaderLen         = fmt.Errorf("raft: leader should be exactly %d bytes", idLen)
   270  	errTooManyEntries    = errors.New("raft: append entry can contain a max of 64k entries")
   271  	errBadAppendEntry    = errors.New("raft: append entry corrupt")
   272  )
   273  
   274  // This will bootstrap a raftNode by writing its config into the store directory.
   275  func (s *Server) bootstrapRaftNode(cfg *RaftConfig, knownPeers []string, allPeersKnown bool) error {
   276  	if cfg == nil {
   277  		return errNilCfg
   278  	}
   279  	// Check validity of peers if presented.
   280  	for _, p := range knownPeers {
   281  		if len(p) != idLen {
   282  			return fmt.Errorf("raft: illegal peer: %q", p)
   283  		}
   284  	}
   285  	expected := len(knownPeers)
   286  	// We need to adjust this is all peers are not known.
   287  	if !allPeersKnown {
   288  		s.Debugf("Determining expected peer size for JetStream meta group")
   289  		opts := s.getOpts()
   290  		nrs := len(opts.Routes)
   291  
   292  		cn := s.ClusterName()
   293  		ngwps := 0
   294  		for _, gw := range opts.Gateway.Gateways {
   295  			// Ignore our own cluster if specified.
   296  			if gw.Name == cn {
   297  				continue
   298  			}
   299  			for _, u := range gw.URLs {
   300  				host := u.Hostname()
   301  				// If this is an IP just add one.
   302  				if net.ParseIP(host) != nil {
   303  					ngwps++
   304  				} else {
   305  					addrs, _ := net.LookupHost(host)
   306  					ngwps += len(addrs)
   307  				}
   308  			}
   309  		}
   310  
   311  		if expected < nrs+ngwps {
   312  			expected = nrs + ngwps
   313  			s.Debugf("Adjusting expected peer set size to %d with %d known", expected, len(knownPeers))
   314  		}
   315  	}
   316  
   317  	// Check the store directory. If we have a memory based WAL we need to make sure the directory is setup.
   318  	if stat, err := os.Stat(cfg.Store); os.IsNotExist(err) {
   319  		if err := os.MkdirAll(cfg.Store, 0750); err != nil {
   320  			return fmt.Errorf("raft: could not create storage directory - %v", err)
   321  		}
   322  	} else if stat == nil || !stat.IsDir() {
   323  		return fmt.Errorf("raft: storage directory is not a directory")
   324  	}
   325  	tmpfile, err := os.CreateTemp(cfg.Store, "_test_")
   326  	if err != nil {
   327  		return fmt.Errorf("raft: storage directory is not writable")
   328  	}
   329  	tmpfile.Close()
   330  	os.Remove(tmpfile.Name())
   331  	if expected <= 1 {
   332  		os.WriteFile(filepath.Join(cfg.Store, ipeerMarkerFile), []byte{1}, 0644)
   333  	}
   334  	return writePeerState(cfg.Store, &peerState{knownPeers, expected, extUndetermined})
   335  }
   336  
   337  // startRaftNode will start the raft node.
   338  func (s *Server) startRaftNode(accName string, cfg *RaftConfig, labels pprofLabels) (RaftNode, error) {
   339  	if cfg == nil {
   340  		return nil, errNilCfg
   341  	}
   342  	s.mu.RLock()
   343  	if s.sys == nil {
   344  		s.mu.RUnlock()
   345  		return nil, ErrNoSysAccount
   346  	}
   347  	sq := s.sys.sq
   348  	sacc := s.sys.account
   349  	hash := s.sys.shash
   350  	s.mu.RUnlock()
   351  
   352  	// Do this here to process error quicker.
   353  	ps, err := readPeerState(cfg.Store)
   354  	if err != nil {
   355  		return nil, err
   356  	}
   357  	if ps == nil {
   358  		return nil, errNoPeerState
   359  	}
   360  
   361  	qpfx := fmt.Sprintf("[ACC:%s] RAFT '%s' ", accName, cfg.Name)
   362  	n := &raft{
   363  		created:  time.Now(),
   364  		id:       hash[:idLen],
   365  		group:    cfg.Name,
   366  		sd:       cfg.Store,
   367  		wal:      cfg.Log,
   368  		wtype:    cfg.Log.Type(),
   369  		track:    cfg.Track,
   370  		csz:      ps.clusterSize,
   371  		qn:       ps.clusterSize/2 + 1,
   372  		peers:    make(map[string]*lps),
   373  		acks:     make(map[uint64]map[string]struct{}),
   374  		pae:      make(map[uint64]*appendEntry),
   375  		s:        s,
   376  		c:        s.createInternalSystemClient(),
   377  		js:       s.getJetStream(),
   378  		sq:       sq,
   379  		quit:     make(chan struct{}),
   380  		reqs:     newIPQueue[*voteRequest](s, qpfx+"vreq"),
   381  		votes:    newIPQueue[*voteResponse](s, qpfx+"vresp"),
   382  		prop:     newIPQueue[*Entry](s, qpfx+"entry"),
   383  		entry:    newIPQueue[*appendEntry](s, qpfx+"appendEntry"),
   384  		resp:     newIPQueue[*appendEntryResponse](s, qpfx+"appendEntryResponse"),
   385  		apply:    newIPQueue[*CommittedEntry](s, qpfx+"committedEntry"),
   386  		accName:  accName,
   387  		leadc:    make(chan bool, 1),
   388  		observer: cfg.Observer,
   389  		extSt:    ps.domainExt,
   390  	}
   391  	n.c.registerWithAccount(sacc)
   392  	if _, err := os.Stat(filepath.Join(n.sd, ipeerMarkerFile)); err == nil {
   393  		n.ipeer = true
   394  	}
   395  
   396  	if atomic.LoadInt32(&s.logging.debug) > 0 {
   397  		n.dflag = true
   398  	}
   399  
   400  	// Set up the highwayhash for the snapshots.
   401  	key := sha256.Sum256([]byte(n.group))
   402  	n.hh, _ = highwayhash.New64(key[:])
   403  
   404  	// If we have a term and vote file (tav.idx on the filesystem) then read in
   405  	// what we think the term and vote was. It's possible these are out of date
   406  	// so a catch-up may be required.
   407  	if term, vote, err := n.readTermVote(); err == nil && term > 0 {
   408  		n.term = term
   409  		n.vote = vote
   410  	}
   411  
   412  	// Make sure that the snapshots directory exists.
   413  	if err := os.MkdirAll(filepath.Join(n.sd, snapshotsDir), 0750); err != nil {
   414  		return nil, fmt.Errorf("could not create snapshots directory - %v", err)
   415  	}
   416  
   417  	// Can't recover snapshots if memory based.
   418  	if _, ok := n.wal.(*memStore); ok {
   419  		os.Remove(filepath.Join(n.sd, snapshotsDir, "*"))
   420  	} else {
   421  		// See if we have any snapshots and if so load and process on startup.
   422  		n.setupLastSnapshot()
   423  	}
   424  
   425  	truncateAndErr := func(index uint64) {
   426  		if err := n.wal.Truncate(index); err != nil {
   427  			n.setWriteErr(err)
   428  		}
   429  	}
   430  
   431  	// Retrieve the stream state from the WAL. If there are pending append
   432  	// entries that were committed but not applied before we last shut down,
   433  	// we will try to replay them and process them here.
   434  	var state StreamState
   435  	n.wal.FastState(&state)
   436  	if state.Msgs > 0 {
   437  		n.debug("Replaying state of %d entries", state.Msgs)
   438  		if first, err := n.loadFirstEntry(); err == nil {
   439  			n.pterm, n.pindex = first.pterm, first.pindex
   440  			if first.commit > 0 && first.commit > n.commit {
   441  				n.commit = first.commit
   442  			}
   443  		}
   444  
   445  		// This process will queue up entries on our applied queue but prior to the upper
   446  		// state machine running. So we will monitor how much we have queued and if we
   447  		// reach a limit will pause the apply queue and resume inside of run() go routine.
   448  		const maxQsz = 32 * 1024 * 1024 // 32MB max
   449  
   450  		// It looks like there are entries we have committed but not applied
   451  		// yet. Replay them.
   452  		for index, qsz := state.FirstSeq, 0; index <= state.LastSeq; index++ {
   453  			ae, err := n.loadEntry(index)
   454  			if err != nil {
   455  				n.warn("Could not load %d from WAL [%+v]: %v", index, state, err)
   456  				truncateAndErr(index)
   457  				break
   458  			}
   459  			if ae.pindex != index-1 {
   460  				n.warn("Corrupt WAL, will truncate")
   461  				truncateAndErr(index)
   462  				break
   463  			}
   464  			n.processAppendEntry(ae, nil)
   465  			// Check how much we have queued up so far to determine if we should pause.
   466  			for _, e := range ae.entries {
   467  				qsz += len(e.Data)
   468  				if qsz > maxQsz && !n.paused {
   469  					n.PauseApply()
   470  				}
   471  			}
   472  		}
   473  	}
   474  
   475  	// Make sure to track ourselves.
   476  	n.peers[n.id] = &lps{time.Now().UnixNano(), 0, true}
   477  
   478  	// Track known peers
   479  	for _, peer := range ps.knownPeers {
   480  		if peer != n.id {
   481  			// Set these to 0 to start but mark as known peer.
   482  			n.peers[peer] = &lps{0, 0, true}
   483  		}
   484  	}
   485  
   486  	// Setup our internal subscriptions for proposals, votes and append entries.
   487  	// If we fail to do this for some reason then this is fatal — we cannot
   488  	// continue setting up or the Raft node may be partially/totally isolated.
   489  	if err := n.createInternalSubs(); err != nil {
   490  		n.shutdown(true)
   491  		return nil, err
   492  	}
   493  
   494  	n.debug("Started")
   495  
   496  	// Check if we need to start in observer mode due to lame duck status.
   497  	// This will stop us from taking on the leader role when we're about to
   498  	// shutdown anyway.
   499  	if s.isLameDuckMode() {
   500  		n.debug("Will start in observer mode due to lame duck status")
   501  		n.SetObserver(true)
   502  	}
   503  
   504  	// Set the election timer and lost quorum timers to now, so that we
   505  	// won't accidentally trigger either state without knowing the real state
   506  	// of the other nodes.
   507  	n.Lock()
   508  	n.resetElectionTimeout()
   509  	n.llqrt = time.Now()
   510  	n.Unlock()
   511  
   512  	// Register the Raft group.
   513  	labels["group"] = n.group
   514  	s.registerRaftNode(n.group, n)
   515  
   516  	// Start the run goroutine for the Raft state machine.
   517  	s.startGoRoutine(n.run, labels)
   518  
   519  	return n, nil
   520  }
   521  
   522  // outOfResources checks to see if we are out of resources.
   523  func (n *raft) outOfResources() bool {
   524  	js := n.js
   525  	if !n.track || js == nil {
   526  		return false
   527  	}
   528  	return js.limitsExceeded(n.wtype)
   529  }
   530  
   531  // Maps node names back to server names.
   532  func (s *Server) serverNameForNode(node string) string {
   533  	if si, ok := s.nodeToInfo.Load(node); ok && si != nil {
   534  		return si.(nodeInfo).name
   535  	}
   536  	return _EMPTY_
   537  }
   538  
   539  // Maps node names back to cluster names.
   540  func (s *Server) clusterNameForNode(node string) string {
   541  	if si, ok := s.nodeToInfo.Load(node); ok && si != nil {
   542  		return si.(nodeInfo).cluster
   543  	}
   544  	return _EMPTY_
   545  }
   546  
   547  // Registers the Raft node with the server, as it will track all of the Raft
   548  // nodes.
   549  func (s *Server) registerRaftNode(group string, n RaftNode) {
   550  	s.rnMu.Lock()
   551  	defer s.rnMu.Unlock()
   552  	if s.raftNodes == nil {
   553  		s.raftNodes = make(map[string]RaftNode)
   554  	}
   555  	s.raftNodes[group] = n
   556  }
   557  
   558  // Unregisters the Raft node from the server, i.e. at shutdown.
   559  func (s *Server) unregisterRaftNode(group string) {
   560  	s.rnMu.Lock()
   561  	defer s.rnMu.Unlock()
   562  	if s.raftNodes != nil {
   563  		delete(s.raftNodes, group)
   564  	}
   565  }
   566  
   567  // Returns how many Raft nodes are running in this server instance.
   568  func (s *Server) numRaftNodes() int {
   569  	s.rnMu.Lock()
   570  	defer s.rnMu.Unlock()
   571  	return len(s.raftNodes)
   572  }
   573  
   574  // Finds the Raft node for a given Raft group, if any. If there is no Raft node
   575  // running for this group then it can return nil.
   576  func (s *Server) lookupRaftNode(group string) RaftNode {
   577  	s.rnMu.RLock()
   578  	defer s.rnMu.RUnlock()
   579  	var n RaftNode
   580  	if s.raftNodes != nil {
   581  		n = s.raftNodes[group]
   582  	}
   583  	return n
   584  }
   585  
   586  // Reloads the debug state for all running Raft nodes. This is necessary when
   587  // the configuration has been reloaded and the debug log level has changed.
   588  func (s *Server) reloadDebugRaftNodes(debug bool) {
   589  	if s == nil {
   590  		return
   591  	}
   592  	s.rnMu.RLock()
   593  	for _, ni := range s.raftNodes {
   594  		n := ni.(*raft)
   595  		n.Lock()
   596  		n.dflag = debug
   597  		n.Unlock()
   598  	}
   599  	s.rnMu.RUnlock()
   600  }
   601  
   602  // Requests that all Raft nodes on this server step down and place them into
   603  // observer mode. This is called when the server is shutting down.
   604  func (s *Server) stepdownRaftNodes() {
   605  	if s == nil {
   606  		return
   607  	}
   608  	s.rnMu.RLock()
   609  	if len(s.raftNodes) == 0 {
   610  		s.rnMu.RUnlock()
   611  		return
   612  	}
   613  	s.Debugf("Stepping down all leader raft nodes")
   614  	nodes := make([]RaftNode, 0, len(s.raftNodes))
   615  	for _, n := range s.raftNodes {
   616  		nodes = append(nodes, n)
   617  	}
   618  	s.rnMu.RUnlock()
   619  
   620  	for _, node := range nodes {
   621  		if node.Leader() {
   622  			node.StepDown()
   623  		}
   624  		node.SetObserver(true)
   625  	}
   626  }
   627  
   628  // Shuts down all Raft nodes on this server. This is called either when the
   629  // server is either entering lame duck mode, shutting down or when JetStream
   630  // has been disabled.
   631  func (s *Server) shutdownRaftNodes() {
   632  	if s == nil {
   633  		return
   634  	}
   635  	s.rnMu.RLock()
   636  	if len(s.raftNodes) == 0 {
   637  		s.rnMu.RUnlock()
   638  		return
   639  	}
   640  	nodes := make([]RaftNode, 0, len(s.raftNodes))
   641  	s.Debugf("Shutting down all raft nodes")
   642  	for _, n := range s.raftNodes {
   643  		nodes = append(nodes, n)
   644  	}
   645  	s.rnMu.RUnlock()
   646  
   647  	for _, node := range nodes {
   648  		node.Stop()
   649  	}
   650  }
   651  
   652  // Used in lameduck mode to move off the leaders.
   653  // We also put all nodes in observer mode so new leaders
   654  // can not be placed on this server.
   655  func (s *Server) transferRaftLeaders() bool {
   656  	if s == nil {
   657  		return false
   658  	}
   659  	s.rnMu.RLock()
   660  	if len(s.raftNodes) == 0 {
   661  		s.rnMu.RUnlock()
   662  		return false
   663  	}
   664  	nodes := make([]RaftNode, 0, len(s.raftNodes))
   665  	for _, n := range s.raftNodes {
   666  		nodes = append(nodes, n)
   667  	}
   668  	s.rnMu.RUnlock()
   669  
   670  	var didTransfer bool
   671  	for _, node := range nodes {
   672  		if node.Leader() {
   673  			node.StepDown()
   674  			didTransfer = true
   675  		}
   676  		node.SetObserver(true)
   677  	}
   678  	return didTransfer
   679  }
   680  
   681  // Formal API
   682  
   683  // Propose will propose a new entry to the group.
   684  // This should only be called on the leader.
   685  func (n *raft) Propose(data []byte) error {
   686  	if state := n.State(); state != Leader {
   687  		n.debug("Proposal ignored, not leader (state: %v)", state)
   688  		return errNotLeader
   689  	}
   690  	n.RLock()
   691  	// Error if we had a previous write error.
   692  	if werr := n.werr; werr != nil {
   693  		n.RUnlock()
   694  		return werr
   695  	}
   696  	prop := n.prop
   697  	n.RUnlock()
   698  
   699  	prop.push(newEntry(EntryNormal, data))
   700  	return nil
   701  }
   702  
   703  // ProposeDirect will propose entries directly by skipping the Raft state
   704  // machine and sending them straight to the wire instead.
   705  // This should only be called on the leader.
   706  func (n *raft) ProposeDirect(entries []*Entry) error {
   707  	if state := n.State(); state != Leader {
   708  		n.debug("Direct proposal ignored, not leader (state: %v)", state)
   709  		return errNotLeader
   710  	}
   711  	n.RLock()
   712  	// Error if we had a previous write error.
   713  	if werr := n.werr; werr != nil {
   714  		n.RUnlock()
   715  		return werr
   716  	}
   717  	n.RUnlock()
   718  
   719  	n.sendAppendEntry(entries)
   720  	return nil
   721  }
   722  
   723  // ForwardProposal will forward the proposal to the leader if known.
   724  // If we are the leader this is the same as calling propose.
   725  // FIXME(dlc) - We could have a reply subject and wait for a response
   726  // for retries, but would need to not block and be in separate Go routine.
   727  func (n *raft) ForwardProposal(entry []byte) error {
   728  	if n.Leader() {
   729  		return n.Propose(entry)
   730  	}
   731  
   732  	n.sendRPC(n.psubj, _EMPTY_, entry)
   733  	return nil
   734  }
   735  
   736  // ProposeAddPeer is called to add a peer to the group.
   737  func (n *raft) ProposeAddPeer(peer string) error {
   738  	if n.State() != Leader {
   739  		return errNotLeader
   740  	}
   741  	n.RLock()
   742  	// Error if we had a previous write error.
   743  	if werr := n.werr; werr != nil {
   744  		n.RUnlock()
   745  		return werr
   746  	}
   747  	prop := n.prop
   748  	n.RUnlock()
   749  
   750  	prop.push(newEntry(EntryAddPeer, []byte(peer)))
   751  	return nil
   752  }
   753  
   754  // As a leader if we are proposing to remove a peer assume its already gone.
   755  func (n *raft) doRemovePeerAsLeader(peer string) {
   756  	n.Lock()
   757  	if n.removed == nil {
   758  		n.removed = map[string]struct{}{}
   759  	}
   760  	n.removed[peer] = struct{}{}
   761  	if _, ok := n.peers[peer]; ok {
   762  		delete(n.peers, peer)
   763  		// We should decrease our cluster size since we are tracking this peer and the peer is most likely already gone.
   764  		n.adjustClusterSizeAndQuorum()
   765  	}
   766  	n.Unlock()
   767  }
   768  
   769  // ProposeRemovePeer is called to remove a peer from the group.
   770  func (n *raft) ProposeRemovePeer(peer string) error {
   771  	n.RLock()
   772  	prop, subj := n.prop, n.rpsubj
   773  	isLeader := n.State() == Leader
   774  	werr := n.werr
   775  	n.RUnlock()
   776  
   777  	// Error if we had a previous write error.
   778  	if werr != nil {
   779  		return werr
   780  	}
   781  
   782  	// If we are the leader then we are responsible for processing the
   783  	// peer remove and then notifying the rest of the group that the
   784  	// peer was removed.
   785  	if isLeader {
   786  		prop.push(newEntry(EntryRemovePeer, []byte(peer)))
   787  		n.doRemovePeerAsLeader(peer)
   788  		return nil
   789  	}
   790  
   791  	// Otherwise we need to forward the proposal to the leader.
   792  	n.sendRPC(subj, _EMPTY_, []byte(peer))
   793  	return nil
   794  }
   795  
   796  // ClusterSize reports back the total cluster size.
   797  // This effects quorum etc.
   798  func (n *raft) ClusterSize() int {
   799  	n.Lock()
   800  	defer n.Unlock()
   801  	return n.csz
   802  }
   803  
   804  // AdjustBootClusterSize can be called to adjust the boot cluster size.
   805  // Will error if called on a group with a leader or a previous leader.
   806  // This can be helpful in mixed mode.
   807  func (n *raft) AdjustBootClusterSize(csz int) error {
   808  	n.Lock()
   809  	defer n.Unlock()
   810  
   811  	if n.leader != noLeader || n.pleader {
   812  		return errAdjustBootCluster
   813  	}
   814  	// Adjust the cluster size and the number of nodes needed to establish
   815  	// a quorum.
   816  	n.csz = csz
   817  	n.qn = n.csz/2 + 1
   818  
   819  	return nil
   820  }
   821  
   822  // AdjustClusterSize will change the cluster set size.
   823  // Must be the leader.
   824  func (n *raft) AdjustClusterSize(csz int) error {
   825  	if n.State() != Leader {
   826  		return errNotLeader
   827  	}
   828  	n.Lock()
   829  	// Adjust the cluster size and the number of nodes needed to establish
   830  	// a quorum.
   831  	n.csz = csz
   832  	n.qn = n.csz/2 + 1
   833  	n.Unlock()
   834  
   835  	n.sendPeerState()
   836  	return nil
   837  }
   838  
   839  // PauseApply will allow us to pause processing of append entries onto our
   840  // external apply queue. In effect this means that the upper layer will no longer
   841  // receive any new entries from the Raft group.
   842  func (n *raft) PauseApply() error {
   843  	if n.State() == Leader {
   844  		return errAlreadyLeader
   845  	}
   846  
   847  	n.Lock()
   848  	defer n.Unlock()
   849  
   850  	// If we are currently a candidate make sure we step down.
   851  	if n.State() == Candidate {
   852  		n.stepdownLocked(noLeader)
   853  	}
   854  
   855  	n.debug("Pausing our apply channel")
   856  	n.paused = true
   857  	n.hcommit = n.commit
   858  	// Also prevent us from trying to become a leader while paused and catching up.
   859  	n.pobserver, n.observer = n.observer, true
   860  	n.resetElect(48 * time.Hour)
   861  
   862  	return nil
   863  }
   864  
   865  // ResumeApply will resume sending applies to the external apply queue. This
   866  // means that we will start sending new entries to the upper layer.
   867  func (n *raft) ResumeApply() {
   868  	n.Lock()
   869  	defer n.Unlock()
   870  
   871  	if !n.paused {
   872  		return
   873  	}
   874  
   875  	n.debug("Resuming our apply channel")
   876  	n.observer, n.pobserver = n.pobserver, false
   877  	n.paused = false
   878  	// Run catchup..
   879  	if n.hcommit > n.commit {
   880  		n.debug("Resuming %d replays", n.hcommit+1-n.commit)
   881  		for index := n.commit + 1; index <= n.hcommit; index++ {
   882  			if err := n.applyCommit(index); err != nil {
   883  				n.warn("Got error on apply commit during replay: %v", err)
   884  				break
   885  			}
   886  			// We want to unlock here to allow the upper layers to call Applied() without blocking.
   887  			n.Unlock()
   888  			// Give hint to let other Go routines run.
   889  			// Might not be necessary but seems to make it more fine grained interleaving.
   890  			runtime.Gosched()
   891  			// Simply re-acquire
   892  			n.Lock()
   893  			// Need to check if we got closed or if we were paused again.
   894  			if n.State() == Closed || n.paused {
   895  				return
   896  			}
   897  		}
   898  	}
   899  	n.hcommit = 0
   900  
   901  	// If we had been selected to be the next leader campaign here now that we have resumed.
   902  	if n.lxfer {
   903  		n.xferCampaign()
   904  	} else {
   905  		n.resetElectionTimeout()
   906  	}
   907  }
   908  
   909  // Applied is a callback that must be be called by the upper layer when it
   910  // has successfully applied the committed entries that it received from the
   911  // apply queue. It will return the number of entries and an estimation of the
   912  // byte size that could be removed with a snapshot/compact.
   913  func (n *raft) Applied(index uint64) (entries uint64, bytes uint64) {
   914  	n.Lock()
   915  	defer n.Unlock()
   916  
   917  	// Ignore if not applicable. This can happen during a reset.
   918  	if index > n.commit {
   919  		return 0, 0
   920  	}
   921  
   922  	// Ignore if already applied.
   923  	if index > n.applied {
   924  		n.applied = index
   925  	}
   926  
   927  	// Calculate the number of entries and estimate the byte size that
   928  	// we can now remove with a compaction/snapshot.
   929  	var state StreamState
   930  	n.wal.FastState(&state)
   931  	if n.applied > state.FirstSeq {
   932  		entries = n.applied - state.FirstSeq
   933  	}
   934  	if state.Msgs > 0 {
   935  		bytes = entries * state.Bytes / state.Msgs
   936  	}
   937  	return entries, bytes
   938  }
   939  
   940  // For capturing data needed by snapshot.
   941  type snapshot struct {
   942  	lastTerm  uint64
   943  	lastIndex uint64
   944  	peerstate []byte
   945  	data      []byte
   946  }
   947  
   948  const minSnapshotLen = 28
   949  
   950  // Encodes a snapshot into a buffer for storage.
   951  // Lock should be held.
   952  func (n *raft) encodeSnapshot(snap *snapshot) []byte {
   953  	if snap == nil {
   954  		return nil
   955  	}
   956  	var le = binary.LittleEndian
   957  	buf := make([]byte, minSnapshotLen+len(snap.peerstate)+len(snap.data))
   958  	le.PutUint64(buf[0:], snap.lastTerm)
   959  	le.PutUint64(buf[8:], snap.lastIndex)
   960  	// Peer state
   961  	le.PutUint32(buf[16:], uint32(len(snap.peerstate)))
   962  	wi := 20
   963  	copy(buf[wi:], snap.peerstate)
   964  	wi += len(snap.peerstate)
   965  	// data itself.
   966  	copy(buf[wi:], snap.data)
   967  	wi += len(snap.data)
   968  
   969  	// Now do the hash for the end.
   970  	n.hh.Reset()
   971  	n.hh.Write(buf[:wi])
   972  	checksum := n.hh.Sum(nil)
   973  	copy(buf[wi:], checksum)
   974  	wi += len(checksum)
   975  	return buf[:wi]
   976  }
   977  
   978  // SendSnapshot will send the latest snapshot as a normal AE.
   979  // Should only be used when the upper layers know this is most recent.
   980  // Used when restoring streams, moving a stream from R1 to R>1, etc.
   981  func (n *raft) SendSnapshot(data []byte) error {
   982  	n.sendAppendEntry([]*Entry{{EntrySnapshot, data}})
   983  	return nil
   984  }
   985  
   986  // Used to install a snapshot for the given term and applied index. This will release
   987  // all of the log entries up to and including index. This should not be called with
   988  // entries that have been applied to the FSM but have not been applied to the raft state.
   989  func (n *raft) InstallSnapshot(data []byte) error {
   990  	if n.State() == Closed {
   991  		return errNodeClosed
   992  	}
   993  
   994  	n.Lock()
   995  
   996  	// If a write error has occurred already then stop here.
   997  	if werr := n.werr; werr != nil {
   998  		n.Unlock()
   999  		return werr
  1000  	}
  1001  
  1002  	// Check that a catchup isn't already taking place. If it is then we won't
  1003  	// allow installing snapshots until it is done.
  1004  	if len(n.progress) > 0 {
  1005  		n.Unlock()
  1006  		return errCatchupsRunning
  1007  	}
  1008  
  1009  	var state StreamState
  1010  	n.wal.FastState(&state)
  1011  
  1012  	if n.applied == 0 {
  1013  		n.Unlock()
  1014  		return errNoSnapAvailable
  1015  	}
  1016  
  1017  	n.debug("Installing snapshot of %d bytes", len(data))
  1018  
  1019  	var term uint64
  1020  	if ae, _ := n.loadEntry(n.applied); ae != nil {
  1021  		// Use the term from the most recently applied entry if possible.
  1022  		term = ae.term
  1023  	} else if ae, _ = n.loadFirstEntry(); ae != nil {
  1024  		// Otherwise see if we can find the term from the first entry.
  1025  		term = ae.term
  1026  	} else {
  1027  		// Last resort is to use the last pterm that we knew of.
  1028  		term = n.pterm
  1029  	}
  1030  
  1031  	snap := &snapshot{
  1032  		lastTerm:  term,
  1033  		lastIndex: n.applied,
  1034  		peerstate: encodePeerState(&peerState{n.peerNames(), n.csz, n.extSt}),
  1035  		data:      data,
  1036  	}
  1037  
  1038  	snapDir := filepath.Join(n.sd, snapshotsDir)
  1039  	sn := fmt.Sprintf(snapFileT, snap.lastTerm, snap.lastIndex)
  1040  	sfile := filepath.Join(snapDir, sn)
  1041  
  1042  	<-dios
  1043  	err := os.WriteFile(sfile, n.encodeSnapshot(snap), defaultFilePerms)
  1044  	dios <- struct{}{}
  1045  
  1046  	if err != nil {
  1047  		n.Unlock()
  1048  		// We could set write err here, but if this is a temporary situation, too many open files etc.
  1049  		// we want to retry and snapshots are not fatal.
  1050  		return err
  1051  	}
  1052  
  1053  	// Remember our latest snapshot file.
  1054  	n.snapfile = sfile
  1055  	if _, err := n.wal.Compact(snap.lastIndex + 1); err != nil {
  1056  		n.setWriteErrLocked(err)
  1057  		n.Unlock()
  1058  		return err
  1059  	}
  1060  	n.Unlock()
  1061  
  1062  	psnaps, _ := os.ReadDir(snapDir)
  1063  	// Remove any old snapshots.
  1064  	for _, fi := range psnaps {
  1065  		pn := fi.Name()
  1066  		if pn != sn {
  1067  			os.Remove(filepath.Join(snapDir, pn))
  1068  		}
  1069  	}
  1070  
  1071  	return nil
  1072  }
  1073  
  1074  // NeedSnapshot returns true if it is necessary to try to install a snapshot, i.e.
  1075  // after we have finished recovering/replaying at startup, on a regular interval or
  1076  // as a part of cleaning up when shutting down.
  1077  func (n *raft) NeedSnapshot() bool {
  1078  	n.RLock()
  1079  	defer n.RUnlock()
  1080  	return n.snapfile == _EMPTY_ && n.applied > 1
  1081  }
  1082  
  1083  const (
  1084  	snapshotsDir = "snapshots"
  1085  	snapFileT    = "snap.%d.%d"
  1086  )
  1087  
  1088  // termAndIndexFromSnapfile tries to load the snapshot file and returns the term
  1089  // and index from that snapshot.
  1090  func termAndIndexFromSnapFile(sn string) (term, index uint64, err error) {
  1091  	if sn == _EMPTY_ {
  1092  		return 0, 0, errBadSnapName
  1093  	}
  1094  	fn := filepath.Base(sn)
  1095  	if n, err := fmt.Sscanf(fn, snapFileT, &term, &index); err != nil || n != 2 {
  1096  		return 0, 0, errBadSnapName
  1097  	}
  1098  	return term, index, nil
  1099  }
  1100  
  1101  // setupLastSnapshot is called at startup to try and recover the last snapshot from
  1102  // the disk if possible. We will try to recover the term, index and commit/applied
  1103  // indices and then notify the upper layer what we found. Compacts the WAL if needed.
  1104  func (n *raft) setupLastSnapshot() {
  1105  	snapDir := filepath.Join(n.sd, snapshotsDir)
  1106  	psnaps, err := os.ReadDir(snapDir)
  1107  	if err != nil {
  1108  		return
  1109  	}
  1110  
  1111  	var lterm, lindex uint64
  1112  	var latest string
  1113  	for _, sf := range psnaps {
  1114  		sfile := filepath.Join(snapDir, sf.Name())
  1115  		var term, index uint64
  1116  		term, index, err := termAndIndexFromSnapFile(sf.Name())
  1117  		if err == nil {
  1118  			if term > lterm {
  1119  				lterm, lindex = term, index
  1120  				latest = sfile
  1121  			} else if term == lterm && index > lindex {
  1122  				lindex = index
  1123  				latest = sfile
  1124  			}
  1125  		} else {
  1126  			// Clean this up, can't parse the name.
  1127  			// TODO(dlc) - We could read in and check actual contents.
  1128  			n.debug("Removing snapshot, can't parse name: %q", sf.Name())
  1129  			os.Remove(sfile)
  1130  		}
  1131  	}
  1132  
  1133  	// Now cleanup any old entries
  1134  	for _, sf := range psnaps {
  1135  		sfile := filepath.Join(snapDir, sf.Name())
  1136  		if sfile != latest {
  1137  			n.debug("Removing old snapshot: %q", sfile)
  1138  			os.Remove(sfile)
  1139  		}
  1140  	}
  1141  
  1142  	if latest == _EMPTY_ {
  1143  		return
  1144  	}
  1145  
  1146  	// Set latest snapshot we have.
  1147  	n.Lock()
  1148  	defer n.Unlock()
  1149  
  1150  	n.snapfile = latest
  1151  	snap, err := n.loadLastSnapshot()
  1152  	if err != nil {
  1153  		// We failed to recover the last snapshot for some reason, so we will
  1154  		// assume it has been corrupted and will try to delete it.
  1155  		if n.snapfile != _EMPTY_ {
  1156  			os.Remove(n.snapfile)
  1157  			n.snapfile = _EMPTY_
  1158  		}
  1159  		return
  1160  	}
  1161  
  1162  	// We successfully recovered the last snapshot from the disk.
  1163  	// Recover state from the snapshot and then notify the upper layer.
  1164  	// Compact the WAL when we're done if needed.
  1165  	n.pindex = snap.lastIndex
  1166  	n.pterm = snap.lastTerm
  1167  	n.commit = snap.lastIndex
  1168  	n.applied = snap.lastIndex
  1169  	n.apply.push(newCommittedEntry(n.commit, []*Entry{{EntrySnapshot, snap.data}}))
  1170  	if _, err := n.wal.Compact(snap.lastIndex + 1); err != nil {
  1171  		n.setWriteErrLocked(err)
  1172  	}
  1173  }
  1174  
  1175  // loadLastSnapshot will load and return our last snapshot.
  1176  // Lock should be held.
  1177  func (n *raft) loadLastSnapshot() (*snapshot, error) {
  1178  	if n.snapfile == _EMPTY_ {
  1179  		return nil, errNoSnapAvailable
  1180  	}
  1181  
  1182  	<-dios
  1183  	buf, err := os.ReadFile(n.snapfile)
  1184  	dios <- struct{}{}
  1185  
  1186  	if err != nil {
  1187  		n.warn("Error reading snapshot: %v", err)
  1188  		os.Remove(n.snapfile)
  1189  		n.snapfile = _EMPTY_
  1190  		return nil, err
  1191  	}
  1192  	if len(buf) < minSnapshotLen {
  1193  		n.warn("Snapshot corrupt, too short")
  1194  		os.Remove(n.snapfile)
  1195  		n.snapfile = _EMPTY_
  1196  		return nil, errSnapshotCorrupt
  1197  	}
  1198  
  1199  	// Check to make sure hash is consistent.
  1200  	hoff := len(buf) - 8
  1201  	lchk := buf[hoff:]
  1202  	n.hh.Reset()
  1203  	n.hh.Write(buf[:hoff])
  1204  	if !bytes.Equal(lchk[:], n.hh.Sum(nil)) {
  1205  		n.warn("Snapshot corrupt, checksums did not match")
  1206  		os.Remove(n.snapfile)
  1207  		n.snapfile = _EMPTY_
  1208  		return nil, errSnapshotCorrupt
  1209  	}
  1210  
  1211  	var le = binary.LittleEndian
  1212  	lps := le.Uint32(buf[16:])
  1213  	snap := &snapshot{
  1214  		lastTerm:  le.Uint64(buf[0:]),
  1215  		lastIndex: le.Uint64(buf[8:]),
  1216  		peerstate: buf[20 : 20+lps],
  1217  		data:      buf[20+lps : hoff],
  1218  	}
  1219  
  1220  	// We had a bug in 2.9.12 that would allow snapshots on last index of 0.
  1221  	// Detect that here and return err.
  1222  	if snap.lastIndex == 0 {
  1223  		n.warn("Snapshot with last index 0 is invalid, cleaning up")
  1224  		os.Remove(n.snapfile)
  1225  		n.snapfile = _EMPTY_
  1226  		return nil, errSnapshotCorrupt
  1227  	}
  1228  
  1229  	return snap, nil
  1230  }
  1231  
  1232  // Leader returns if we are the leader for our group.
  1233  // We use an atomic here now vs acquiring the read lock.
  1234  func (n *raft) Leader() bool {
  1235  	if n == nil {
  1236  		return false
  1237  	}
  1238  	return n.State() == Leader
  1239  }
  1240  
  1241  // stepdown immediately steps down the Raft node to the
  1242  // follower state. This will take the lock itself.
  1243  func (n *raft) stepdown(newLeader string) {
  1244  	n.Lock()
  1245  	defer n.Unlock()
  1246  	n.stepdownLocked(newLeader)
  1247  }
  1248  
  1249  // stepdownLocked immediately steps down the Raft node to the
  1250  // follower state. This requires the lock is already held.
  1251  func (n *raft) stepdownLocked(newLeader string) {
  1252  	n.debug("Stepping down")
  1253  	n.switchToFollowerLocked(newLeader)
  1254  }
  1255  
  1256  // isCatchingUp returns true if a catchup is currently taking place.
  1257  func (n *raft) isCatchingUp() bool {
  1258  	n.RLock()
  1259  	defer n.RUnlock()
  1260  	return n.catchup != nil
  1261  }
  1262  
  1263  // isCurrent is called from the healthchecks and returns true if we believe
  1264  // that the upper layer is current with the Raft layer, i.e. that it has applied
  1265  // all of the commits that we have given it.
  1266  // Optionally we can also check whether or not we're making forward progress if we
  1267  // aren't current, in which case this function may block for up to ~10ms to find out.
  1268  // Lock should be held.
  1269  func (n *raft) isCurrent(includeForwardProgress bool) bool {
  1270  	// Check if we are closed.
  1271  	if n.State() == Closed {
  1272  		n.debug("Not current, node is closed")
  1273  		return false
  1274  	}
  1275  
  1276  	// Check whether we've made progress on any state, 0 is invalid so not healthy.
  1277  	if n.commit == 0 {
  1278  		n.debug("Not current, no commits")
  1279  		return false
  1280  	}
  1281  
  1282  	// If we were previously logging about falling behind, also log when the problem
  1283  	// was cleared.
  1284  	clearBehindState := func() {
  1285  		if n.hcbehind {
  1286  			n.warn("Health check OK, no longer falling behind")
  1287  			n.hcbehind = false
  1288  		}
  1289  	}
  1290  
  1291  	// Make sure we are the leader or we know we have heard from the leader recently.
  1292  	if n.State() == Leader {
  1293  		clearBehindState()
  1294  		return true
  1295  	}
  1296  
  1297  	// Check here on catchup status.
  1298  	if cs := n.catchup; cs != nil && n.pterm >= cs.cterm && n.pindex >= cs.cindex {
  1299  		n.cancelCatchup()
  1300  	}
  1301  
  1302  	// Check to see that we have heard from the current leader lately.
  1303  	if n.leader != noLeader && n.leader != n.id && n.catchup == nil {
  1304  		okInterval := int64(hbInterval) * 2
  1305  		ts := time.Now().UnixNano()
  1306  		if ps := n.peers[n.leader]; ps == nil || ps.ts == 0 && (ts-ps.ts) > okInterval {
  1307  			n.debug("Not current, no recent leader contact")
  1308  			return false
  1309  		}
  1310  	}
  1311  	if cs := n.catchup; cs != nil {
  1312  		n.debug("Not current, still catching up pindex=%d, cindex=%d", n.pindex, cs.cindex)
  1313  	}
  1314  
  1315  	if n.commit == n.applied {
  1316  		// At this point if we are current, we can return saying so.
  1317  		clearBehindState()
  1318  		return true
  1319  	} else if !includeForwardProgress {
  1320  		// Otherwise, if we aren't allowed to include forward progress
  1321  		// (i.e. we are checking "current" instead of "healthy") then
  1322  		// give up now.
  1323  		return false
  1324  	}
  1325  
  1326  	// Otherwise, wait for a short period of time and see if we are making any
  1327  	// forward progress.
  1328  	if startDelta := n.commit - n.applied; startDelta > 0 {
  1329  		for i := 0; i < 10; i++ { // 10ms, in 1ms increments
  1330  			n.Unlock()
  1331  			time.Sleep(time.Millisecond)
  1332  			n.Lock()
  1333  			if n.commit-n.applied < startDelta {
  1334  				// The gap is getting smaller, so we're making forward progress.
  1335  				clearBehindState()
  1336  				return true
  1337  			}
  1338  		}
  1339  	}
  1340  
  1341  	n.hcbehind = true
  1342  	n.warn("Falling behind in health check, commit %d != applied %d", n.commit, n.applied)
  1343  	return false
  1344  }
  1345  
  1346  // Current returns if we are the leader for our group or an up to date follower.
  1347  func (n *raft) Current() bool {
  1348  	if n == nil {
  1349  		return false
  1350  	}
  1351  	n.Lock()
  1352  	defer n.Unlock()
  1353  	return n.isCurrent(false)
  1354  }
  1355  
  1356  // Healthy returns if we are the leader for our group and nearly up-to-date.
  1357  func (n *raft) Healthy() bool {
  1358  	if n == nil {
  1359  		return false
  1360  	}
  1361  	n.Lock()
  1362  	defer n.Unlock()
  1363  	return n.isCurrent(true)
  1364  }
  1365  
  1366  // HadPreviousLeader indicates if this group ever had a leader.
  1367  func (n *raft) HadPreviousLeader() bool {
  1368  	n.RLock()
  1369  	defer n.RUnlock()
  1370  	return n.pleader
  1371  }
  1372  
  1373  // GroupLeader returns the current leader of the group.
  1374  func (n *raft) GroupLeader() string {
  1375  	if n == nil {
  1376  		return noLeader
  1377  	}
  1378  	n.RLock()
  1379  	defer n.RUnlock()
  1380  	return n.leader
  1381  }
  1382  
  1383  // Guess the best next leader. Stepdown will check more thoroughly.
  1384  // Lock should be held.
  1385  func (n *raft) selectNextLeader() string {
  1386  	nextLeader, hli := noLeader, uint64(0)
  1387  	for peer, ps := range n.peers {
  1388  		if peer == n.id || ps.li <= hli {
  1389  			continue
  1390  		}
  1391  		hli = ps.li
  1392  		nextLeader = peer
  1393  	}
  1394  	return nextLeader
  1395  }
  1396  
  1397  // StepDown will have a leader stepdown and optionally do a leader transfer.
  1398  func (n *raft) StepDown(preferred ...string) error {
  1399  	n.Lock()
  1400  
  1401  	if len(preferred) > 1 {
  1402  		n.Unlock()
  1403  		return errTooManyPrefs
  1404  	}
  1405  
  1406  	if n.State() != Leader {
  1407  		n.Unlock()
  1408  		return errNotLeader
  1409  	}
  1410  
  1411  	n.debug("Being asked to stepdown")
  1412  
  1413  	// See if we have up to date followers.
  1414  	maybeLeader := noLeader
  1415  	if len(preferred) > 0 {
  1416  		if preferred[0] != _EMPTY_ {
  1417  			maybeLeader = preferred[0]
  1418  		} else {
  1419  			preferred = nil
  1420  		}
  1421  	}
  1422  
  1423  	// Can't pick ourselves.
  1424  	if maybeLeader == n.id {
  1425  		maybeLeader = noLeader
  1426  		preferred = nil
  1427  	}
  1428  
  1429  	nowts := time.Now().UnixNano()
  1430  
  1431  	// If we have a preferred check it first.
  1432  	if maybeLeader != noLeader {
  1433  		var isHealthy bool
  1434  		if ps, ok := n.peers[maybeLeader]; ok {
  1435  			si, ok := n.s.nodeToInfo.Load(maybeLeader)
  1436  			isHealthy = ok && !si.(nodeInfo).offline && (nowts-ps.ts) < int64(hbInterval*3)
  1437  		}
  1438  		if !isHealthy {
  1439  			maybeLeader = noLeader
  1440  		}
  1441  	}
  1442  
  1443  	// If we do not have a preferred at this point pick the first healthy one.
  1444  	// Make sure not ourselves.
  1445  	if maybeLeader == noLeader {
  1446  		for peer, ps := range n.peers {
  1447  			if peer == n.id {
  1448  				continue
  1449  			}
  1450  			si, ok := n.s.nodeToInfo.Load(peer)
  1451  			isHealthy := ok && !si.(nodeInfo).offline && (nowts-ps.ts) < int64(hbInterval*3)
  1452  			if isHealthy {
  1453  				maybeLeader = peer
  1454  				break
  1455  			}
  1456  		}
  1457  	}
  1458  
  1459  	// Clear our vote state.
  1460  	n.vote = noVote
  1461  	n.writeTermVote()
  1462  
  1463  	prop := n.prop
  1464  	n.Unlock()
  1465  
  1466  	if len(preferred) > 0 && maybeLeader == noLeader {
  1467  		n.debug("Can not transfer to preferred peer %q", preferred[0])
  1468  	}
  1469  
  1470  	// If we have a new leader selected, transfer over to them.
  1471  	if maybeLeader != noLeader {
  1472  		n.debug("Selected %q for new leader", maybeLeader)
  1473  		prop.push(newEntry(EntryLeaderTransfer, []byte(maybeLeader)))
  1474  	} else {
  1475  		// Force us to stepdown here.
  1476  		n.stepdown(noLeader)
  1477  	}
  1478  
  1479  	return nil
  1480  }
  1481  
  1482  // Campaign will have our node start a leadership vote.
  1483  func (n *raft) Campaign() error {
  1484  	n.Lock()
  1485  	defer n.Unlock()
  1486  	return n.campaign()
  1487  }
  1488  
  1489  func randCampaignTimeout() time.Duration {
  1490  	delta := rand.Int63n(int64(maxCampaignTimeout - minCampaignTimeout))
  1491  	return (minCampaignTimeout + time.Duration(delta))
  1492  }
  1493  
  1494  // Campaign will have our node start a leadership vote.
  1495  // Lock should be held.
  1496  func (n *raft) campaign() error {
  1497  	n.debug("Starting campaign")
  1498  	if n.State() == Leader {
  1499  		return errAlreadyLeader
  1500  	}
  1501  	n.resetElect(randCampaignTimeout())
  1502  	return nil
  1503  }
  1504  
  1505  // xferCampaign will have our node start an immediate leadership vote.
  1506  // Lock should be held.
  1507  func (n *raft) xferCampaign() error {
  1508  	n.debug("Starting transfer campaign")
  1509  	if n.State() == Leader {
  1510  		n.lxfer = false
  1511  		return errAlreadyLeader
  1512  	}
  1513  	n.resetElect(10 * time.Millisecond)
  1514  	return nil
  1515  }
  1516  
  1517  // State returns the current state for this node.
  1518  func (n *raft) State() RaftState {
  1519  	return RaftState(n.state.Load())
  1520  }
  1521  
  1522  // Progress returns the current index, commit and applied values.
  1523  func (n *raft) Progress() (index, commit, applied uint64) {
  1524  	n.RLock()
  1525  	defer n.RUnlock()
  1526  	return n.pindex + 1, n.commit, n.applied
  1527  }
  1528  
  1529  // Size returns number of entries and total bytes for our WAL.
  1530  func (n *raft) Size() (uint64, uint64) {
  1531  	n.RLock()
  1532  	var state StreamState
  1533  	n.wal.FastState(&state)
  1534  	n.RUnlock()
  1535  	return state.Msgs, state.Bytes
  1536  }
  1537  
  1538  func (n *raft) ID() string {
  1539  	if n == nil {
  1540  		return _EMPTY_
  1541  	}
  1542  	n.RLock()
  1543  	defer n.RUnlock()
  1544  	return n.id
  1545  }
  1546  
  1547  func (n *raft) Group() string {
  1548  	n.RLock()
  1549  	defer n.RUnlock()
  1550  	return n.group
  1551  }
  1552  
  1553  func (n *raft) Peers() []*Peer {
  1554  	n.RLock()
  1555  	defer n.RUnlock()
  1556  
  1557  	var peers []*Peer
  1558  	for id, ps := range n.peers {
  1559  		var lag uint64
  1560  		if n.commit > ps.li {
  1561  			lag = n.commit - ps.li
  1562  		}
  1563  		p := &Peer{
  1564  			ID:      id,
  1565  			Current: id == n.leader || ps.li >= n.applied,
  1566  			Last:    time.Unix(0, ps.ts),
  1567  			Lag:     lag,
  1568  		}
  1569  		peers = append(peers, p)
  1570  	}
  1571  	return peers
  1572  }
  1573  
  1574  // Update our known set of peers.
  1575  func (n *raft) UpdateKnownPeers(knownPeers []string) {
  1576  	n.Lock()
  1577  	// Process like peer state update.
  1578  	ps := &peerState{knownPeers, len(knownPeers), n.extSt}
  1579  	n.processPeerState(ps)
  1580  	isLeader := n.State() == Leader
  1581  	n.Unlock()
  1582  
  1583  	// If we are the leader send this update out as well.
  1584  	if isLeader {
  1585  		n.sendPeerState()
  1586  	}
  1587  }
  1588  
  1589  // ApplyQ returns the apply queue that new commits will be sent to for the
  1590  // upper layer to apply.
  1591  func (n *raft) ApplyQ() *ipQueue[*CommittedEntry] { return n.apply }
  1592  
  1593  // LeadChangeC returns the leader change channel, notifying when the Raft
  1594  // leader role has moved.
  1595  func (n *raft) LeadChangeC() <-chan bool { return n.leadc }
  1596  
  1597  // QuitC returns the quit channel, notifying when the Raft group has shut down.
  1598  func (n *raft) QuitC() <-chan struct{} { return n.quit }
  1599  
  1600  func (n *raft) Created() time.Time {
  1601  	n.RLock()
  1602  	defer n.RUnlock()
  1603  	return n.created
  1604  }
  1605  
  1606  func (n *raft) Stop() {
  1607  	n.shutdown(false)
  1608  }
  1609  
  1610  func (n *raft) Delete() {
  1611  	n.shutdown(true)
  1612  }
  1613  
  1614  func (n *raft) shutdown(shouldDelete bool) {
  1615  	n.Lock()
  1616  
  1617  	// Returned swap value is the previous state. It looks counter-intuitive
  1618  	// to do this atomic operation with the lock held, but we have to do so in
  1619  	// order to make sure that switchState() is not already running. If it is
  1620  	// then it can potentially update the n.state back to a non-closed state,
  1621  	// allowing shutdown() to be called again. If that happens then the below
  1622  	// close(n.quit) will panic from trying to close an already-closed channel.
  1623  	if n.state.Swap(int32(Closed)) == int32(Closed) {
  1624  		n.Unlock()
  1625  		return
  1626  	}
  1627  
  1628  	close(n.quit)
  1629  	if c := n.c; c != nil {
  1630  		var subs []*subscription
  1631  		c.mu.Lock()
  1632  		for _, sub := range c.subs {
  1633  			subs = append(subs, sub)
  1634  		}
  1635  		c.mu.Unlock()
  1636  		for _, sub := range subs {
  1637  			n.unsubscribe(sub)
  1638  		}
  1639  		c.closeConnection(InternalClient)
  1640  	}
  1641  	s, g, wal := n.s, n.group, n.wal
  1642  
  1643  	// Delete our peer state and vote state and any snapshots.
  1644  	if shouldDelete {
  1645  		os.Remove(filepath.Join(n.sd, peerStateFile))
  1646  		os.Remove(filepath.Join(n.sd, termVoteFile))
  1647  		os.RemoveAll(filepath.Join(n.sd, snapshotsDir))
  1648  	}
  1649  	// Unregistering ipQueues do not prevent them from push/pop
  1650  	// just will remove them from the central monitoring map
  1651  	queues := []interface {
  1652  		unregister()
  1653  	}{n.reqs, n.votes, n.prop, n.entry, n.resp, n.apply}
  1654  	for _, q := range queues {
  1655  		q.unregister()
  1656  	}
  1657  	n.Unlock()
  1658  
  1659  	s.unregisterRaftNode(g)
  1660  	if shouldDelete {
  1661  		n.debug("Deleted")
  1662  	} else {
  1663  		n.debug("Shutdown")
  1664  	}
  1665  	if wal != nil {
  1666  		if shouldDelete {
  1667  			wal.Delete()
  1668  		} else {
  1669  			wal.Stop()
  1670  		}
  1671  	}
  1672  }
  1673  
  1674  // Wipe will force an on disk state reset and then call Delete().
  1675  // Useful in case we have been stopped before this point.
  1676  func (n *raft) Wipe() {
  1677  	n.RLock()
  1678  	wal := n.wal
  1679  	n.RUnlock()
  1680  	// Delete our underlying storage.
  1681  	if wal != nil {
  1682  		wal.Delete()
  1683  	}
  1684  	// Now call delete.
  1685  	n.Delete()
  1686  }
  1687  
  1688  const (
  1689  	raftAllSubj        = "$NRG.>"
  1690  	raftVoteSubj       = "$NRG.V.%s"
  1691  	raftAppendSubj     = "$NRG.AE.%s"
  1692  	raftPropSubj       = "$NRG.P.%s"
  1693  	raftRemovePeerSubj = "$NRG.RP.%s"
  1694  	raftReply          = "$NRG.R.%s"
  1695  	raftCatchupReply   = "$NRG.CR.%s"
  1696  )
  1697  
  1698  // Lock should be held (due to use of random generator)
  1699  func (n *raft) newCatchupInbox() string {
  1700  	var b [replySuffixLen]byte
  1701  	rn := fastrand.Uint64()
  1702  	for i, l := 0, rn; i < len(b); i++ {
  1703  		b[i] = digits[l%base]
  1704  		l /= base
  1705  	}
  1706  	return fmt.Sprintf(raftCatchupReply, b[:])
  1707  }
  1708  
  1709  func (n *raft) newInbox() string {
  1710  	var b [replySuffixLen]byte
  1711  	rn := fastrand.Uint64()
  1712  	for i, l := 0, rn; i < len(b); i++ {
  1713  		b[i] = digits[l%base]
  1714  		l /= base
  1715  	}
  1716  	return fmt.Sprintf(raftReply, b[:])
  1717  }
  1718  
  1719  // Our internal subscribe.
  1720  // Lock should be held.
  1721  func (n *raft) subscribe(subject string, cb msgHandler) (*subscription, error) {
  1722  	return n.s.systemSubscribe(subject, _EMPTY_, false, n.c, cb)
  1723  }
  1724  
  1725  // Lock should be held.
  1726  func (n *raft) unsubscribe(sub *subscription) {
  1727  	if sub != nil {
  1728  		n.c.processUnsub(sub.sid)
  1729  	}
  1730  }
  1731  
  1732  func (n *raft) createInternalSubs() error {
  1733  	n.Lock()
  1734  	defer n.Unlock()
  1735  	n.vsubj, n.vreply = fmt.Sprintf(raftVoteSubj, n.group), n.newInbox()
  1736  	n.asubj, n.areply = fmt.Sprintf(raftAppendSubj, n.group), n.newInbox()
  1737  	n.psubj = fmt.Sprintf(raftPropSubj, n.group)
  1738  	n.rpsubj = fmt.Sprintf(raftRemovePeerSubj, n.group)
  1739  
  1740  	// Votes
  1741  	if _, err := n.subscribe(n.vreply, n.handleVoteResponse); err != nil {
  1742  		return err
  1743  	}
  1744  	if _, err := n.subscribe(n.vsubj, n.handleVoteRequest); err != nil {
  1745  		return err
  1746  	}
  1747  	// AppendEntry
  1748  	if _, err := n.subscribe(n.areply, n.handleAppendEntryResponse); err != nil {
  1749  		return err
  1750  	}
  1751  	if sub, err := n.subscribe(n.asubj, n.handleAppendEntry); err != nil {
  1752  		return err
  1753  	} else {
  1754  		n.aesub = sub
  1755  	}
  1756  
  1757  	return nil
  1758  }
  1759  
  1760  func randElectionTimeout() time.Duration {
  1761  	delta := rand.Int63n(int64(maxElectionTimeout - minElectionTimeout))
  1762  	return (minElectionTimeout + time.Duration(delta))
  1763  }
  1764  
  1765  // Lock should be held.
  1766  func (n *raft) resetElectionTimeout() {
  1767  	n.resetElect(randElectionTimeout())
  1768  }
  1769  
  1770  func (n *raft) resetElectionTimeoutWithLock() {
  1771  	n.resetElectWithLock(randElectionTimeout())
  1772  }
  1773  
  1774  // Lock should be held.
  1775  func (n *raft) resetElect(et time.Duration) {
  1776  	if n.elect == nil {
  1777  		n.elect = time.NewTimer(et)
  1778  	} else {
  1779  		if !n.elect.Stop() {
  1780  			select {
  1781  			case <-n.elect.C:
  1782  			default:
  1783  			}
  1784  		}
  1785  		n.elect.Reset(et)
  1786  	}
  1787  }
  1788  
  1789  func (n *raft) resetElectWithLock(et time.Duration) {
  1790  	n.Lock()
  1791  	n.resetElect(et)
  1792  	n.Unlock()
  1793  }
  1794  
  1795  // run is the top-level runner for the Raft state machine. Depending on the
  1796  // state of the node (leader, follower, candidate, observer), this will call
  1797  // through to other functions. It is expected that this function will run for
  1798  // the entire life of the Raft node once started.
  1799  func (n *raft) run() {
  1800  	s := n.s
  1801  	defer s.grWG.Done()
  1802  
  1803  	// We want to wait for some routing to be enabled, so we will wait for
  1804  	// at least a route, leaf or gateway connection to be established before
  1805  	// starting the run loop.
  1806  	for gw := s.gateway; ; {
  1807  		s.mu.RLock()
  1808  		ready, gwEnabled := s.numRemotes()+len(s.leafs) > 0, gw.enabled
  1809  		s.mu.RUnlock()
  1810  		if !ready && gwEnabled {
  1811  			gw.RLock()
  1812  			ready = len(gw.out)+len(gw.in) > 0
  1813  			gw.RUnlock()
  1814  		}
  1815  		if n.csz == 1 || (n.csz == 2 && n.ipeer) {
  1816  			ready = true
  1817  			break
  1818  		}
  1819  		if !ready {
  1820  			select {
  1821  			case <-s.quitCh:
  1822  				return
  1823  			case <-time.After(100 * time.Millisecond):
  1824  				s.RateLimitWarnf("Waiting for routing to be established...")
  1825  			}
  1826  		} else {
  1827  			break
  1828  		}
  1829  	}
  1830  
  1831  	// We may have paused adding entries to apply queue, resume here.
  1832  	// No-op if not paused.
  1833  	n.ResumeApply()
  1834  
  1835  	// Send nil entry to signal the upper layers we are done doing replay/restore.
  1836  	n.apply.push(nil)
  1837  
  1838  	if n.implicitLeaderLocked() {
  1839  		n.switchToLeader()
  1840  	}
  1841  
  1842  	for s.isRunning() {
  1843  		switch n.State() {
  1844  		case Follower:
  1845  			n.runAsFollower()
  1846  		case Candidate:
  1847  			n.runAsCandidate()
  1848  		case Leader:
  1849  			n.runAsLeader()
  1850  		case Closed:
  1851  			return
  1852  		}
  1853  	}
  1854  }
  1855  
  1856  func (n *raft) debug(format string, args ...interface{}) {
  1857  	if n.dflag {
  1858  		nf := fmt.Sprintf("RAFT [%s - %s] %s", n.id, n.group, format)
  1859  		n.s.Debugf(nf, args...)
  1860  	}
  1861  }
  1862  
  1863  func (n *raft) warn(format string, args ...interface{}) {
  1864  	nf := fmt.Sprintf("RAFT [%s - %s] %s", n.id, n.group, format)
  1865  	n.s.RateLimitWarnf(nf, args...)
  1866  }
  1867  
  1868  func (n *raft) error(format string, args ...interface{}) {
  1869  	nf := fmt.Sprintf("RAFT [%s - %s] %s", n.id, n.group, format)
  1870  	n.s.Errorf(nf, args...)
  1871  }
  1872  
  1873  func (n *raft) electTimer() *time.Timer {
  1874  	n.RLock()
  1875  	defer n.RUnlock()
  1876  	return n.elect
  1877  }
  1878  
  1879  func (n *raft) IsObserver() bool {
  1880  	n.RLock()
  1881  	defer n.RUnlock()
  1882  	return n.observer
  1883  }
  1884  
  1885  // Sets the state to observer only.
  1886  func (n *raft) SetObserver(isObserver bool) {
  1887  	n.setObserver(isObserver, extUndetermined)
  1888  }
  1889  
  1890  func (n *raft) setObserver(isObserver bool, extSt extensionState) {
  1891  	n.Lock()
  1892  	defer n.Unlock()
  1893  	n.observer = isObserver
  1894  	n.extSt = extSt
  1895  }
  1896  
  1897  // processAppendEntries is called by the Raft state machine when there are
  1898  // new append entries to be committed and sent to the upper state machine.
  1899  func (n *raft) processAppendEntries() {
  1900  	canProcess := true
  1901  	if n.isClosed() {
  1902  		n.debug("AppendEntry not processing inbound, closed")
  1903  		canProcess = false
  1904  	}
  1905  	if n.outOfResources() {
  1906  		n.debug("AppendEntry not processing inbound, no resources")
  1907  		canProcess = false
  1908  	}
  1909  	// Always pop the entries, but check if we can process them. If we can't
  1910  	// then the entries are effectively dropped.
  1911  	aes := n.entry.pop()
  1912  	if canProcess {
  1913  		for _, ae := range aes {
  1914  			n.processAppendEntry(ae, ae.sub)
  1915  		}
  1916  	}
  1917  	n.entry.recycle(&aes)
  1918  }
  1919  
  1920  // runAsFollower is called by run and will block for as long as the node is
  1921  // running in the follower state.
  1922  func (n *raft) runAsFollower() {
  1923  	for n.State() == Follower {
  1924  		elect := n.electTimer()
  1925  
  1926  		select {
  1927  		case <-n.entry.ch:
  1928  			// New append entries have arrived over the network.
  1929  			n.processAppendEntries()
  1930  		case <-n.s.quitCh:
  1931  			// The server is shutting down.
  1932  			n.shutdown(false)
  1933  			return
  1934  		case <-n.quit:
  1935  			// The Raft node is shutting down.
  1936  			return
  1937  		case <-elect.C:
  1938  			// The election timer has fired so we think it's time to call an election.
  1939  			// If we are out of resources we just want to stay in this state for the moment.
  1940  			if n.outOfResources() {
  1941  				n.resetElectionTimeoutWithLock()
  1942  				n.debug("Not switching to candidate, no resources")
  1943  			} else if n.IsObserver() {
  1944  				n.resetElectWithLock(48 * time.Hour)
  1945  				n.debug("Not switching to candidate, observer only")
  1946  			} else if n.isCatchingUp() {
  1947  				n.debug("Not switching to candidate, catching up")
  1948  				// Check to see if our catchup has stalled.
  1949  				n.Lock()
  1950  				if n.catchupStalled() {
  1951  					n.cancelCatchup()
  1952  				}
  1953  				n.resetElectionTimeout()
  1954  				n.Unlock()
  1955  			} else {
  1956  				if n.implicitLeaderLocked() {
  1957  					n.switchToLeader()
  1958  				} else {
  1959  					n.switchToCandidate()
  1960  				}
  1961  				return
  1962  			}
  1963  		case <-n.votes.ch:
  1964  			// We're receiving votes from the network, probably because we have only
  1965  			// just stepped down and they were already in flight. Ignore them.
  1966  			n.debug("Ignoring old vote response, we have stepped down")
  1967  			n.votes.popOne()
  1968  		case <-n.resp.ch:
  1969  			// We're receiving append entry responses from the network, probably because
  1970  			// we have only just stepped down and they were already in flight. Ignore them.
  1971  			n.resp.popOne()
  1972  		case <-n.reqs.ch:
  1973  			// We've just received a vote request from the network.
  1974  			// Because of drain() it is possible that we get nil from popOne().
  1975  			if voteReq, ok := n.reqs.popOne(); ok {
  1976  				n.processVoteRequest(voteReq)
  1977  			}
  1978  		}
  1979  	}
  1980  }
  1981  
  1982  // Pool for CommittedEntry re-use.
  1983  var cePool = sync.Pool{
  1984  	New: func() any {
  1985  		return &CommittedEntry{}
  1986  	},
  1987  }
  1988  
  1989  // CommittedEntry is handed back to the user to apply a commit to their upper layer.
  1990  type CommittedEntry struct {
  1991  	Index   uint64
  1992  	Entries []*Entry
  1993  }
  1994  
  1995  // Create a new CommittedEntry. When the returned entry is no longer needed, it
  1996  // should be returned to the pool by calling ReturnToPool.
  1997  func newCommittedEntry(index uint64, entries []*Entry) *CommittedEntry {
  1998  	ce := cePool.Get().(*CommittedEntry)
  1999  	ce.Index, ce.Entries = index, entries
  2000  	return ce
  2001  }
  2002  
  2003  // ReturnToPool returns the CommittedEntry to the pool, after which point it is
  2004  // no longer safe to reuse.
  2005  func (ce *CommittedEntry) ReturnToPool() {
  2006  	if ce == nil {
  2007  		return
  2008  	}
  2009  	if len(ce.Entries) > 0 {
  2010  		for _, e := range ce.Entries {
  2011  			entryPool.Put(e)
  2012  		}
  2013  	}
  2014  	ce.Index, ce.Entries = 0, nil
  2015  	cePool.Put(ce)
  2016  }
  2017  
  2018  // Pool for Entry re-use.
  2019  var entryPool = sync.Pool{
  2020  	New: func() any {
  2021  		return &Entry{}
  2022  	},
  2023  }
  2024  
  2025  // Helper to create new entries. When the returned entry is no longer needed, it
  2026  // should be returned to the entryPool pool.
  2027  func newEntry(t EntryType, data []byte) *Entry {
  2028  	entry := entryPool.Get().(*Entry)
  2029  	entry.Type, entry.Data = t, data
  2030  	return entry
  2031  }
  2032  
  2033  // Pool for appendEntry re-use.
  2034  var aePool = sync.Pool{
  2035  	New: func() any {
  2036  		return &appendEntry{}
  2037  	},
  2038  }
  2039  
  2040  // appendEntry is the main struct that is used to sync raft peers.
  2041  type appendEntry struct {
  2042  	leader  string   // The leader that this append entry came from.
  2043  	term    uint64   // The current term, as the leader understands it.
  2044  	commit  uint64   // The commit index, as the leader understands it.
  2045  	pterm   uint64   // The previous term, for checking consistency.
  2046  	pindex  uint64   // The previous commit index, for checking consistency.
  2047  	entries []*Entry // Entries to process.
  2048  	// Below fields are for internal use only:
  2049  	reply string        // Reply subject to respond to once committed.
  2050  	sub   *subscription // The subscription that the append entry came in on.
  2051  	buf   []byte
  2052  }
  2053  
  2054  // Create a new appendEntry.
  2055  func newAppendEntry(leader string, term, commit, pterm, pindex uint64, entries []*Entry) *appendEntry {
  2056  	ae := aePool.Get().(*appendEntry)
  2057  	ae.leader, ae.term, ae.commit, ae.pterm, ae.pindex, ae.entries = leader, term, commit, pterm, pindex, entries
  2058  	ae.reply, ae.sub, ae.buf = _EMPTY_, nil, nil
  2059  	return ae
  2060  }
  2061  
  2062  // Will return this append entry, and its interior entries to their respective pools.
  2063  func (ae *appendEntry) returnToPool() {
  2064  	ae.entries, ae.buf, ae.sub, ae.reply = nil, nil, nil, _EMPTY_
  2065  	aePool.Put(ae)
  2066  }
  2067  
  2068  type EntryType uint8
  2069  
  2070  const (
  2071  	EntryNormal EntryType = iota
  2072  	EntryOldSnapshot
  2073  	EntryPeerState
  2074  	EntryAddPeer
  2075  	EntryRemovePeer
  2076  	EntryLeaderTransfer
  2077  	EntrySnapshot
  2078  )
  2079  
  2080  func (t EntryType) String() string {
  2081  	switch t {
  2082  	case EntryNormal:
  2083  		return "Normal"
  2084  	case EntryOldSnapshot:
  2085  		return "OldSnapshot"
  2086  	case EntryPeerState:
  2087  		return "PeerState"
  2088  	case EntryAddPeer:
  2089  		return "AddPeer"
  2090  	case EntryRemovePeer:
  2091  		return "RemovePeer"
  2092  	case EntryLeaderTransfer:
  2093  		return "LeaderTransfer"
  2094  	case EntrySnapshot:
  2095  		return "Snapshot"
  2096  	}
  2097  	return fmt.Sprintf("Unknown [%d]", uint8(t))
  2098  }
  2099  
  2100  type Entry struct {
  2101  	Type EntryType
  2102  	Data []byte
  2103  }
  2104  
  2105  func (ae *appendEntry) String() string {
  2106  	return fmt.Sprintf("&{leader:%s term:%d commit:%d pterm:%d pindex:%d entries: %d}",
  2107  		ae.leader, ae.term, ae.commit, ae.pterm, ae.pindex, len(ae.entries))
  2108  }
  2109  
  2110  const appendEntryBaseLen = idLen + 4*8 + 2
  2111  
  2112  func (ae *appendEntry) encode(b []byte) ([]byte, error) {
  2113  	if ll := len(ae.leader); ll != idLen && ll != 0 {
  2114  		return nil, errLeaderLen
  2115  	}
  2116  	if len(ae.entries) > math.MaxUint16 {
  2117  		return nil, errTooManyEntries
  2118  	}
  2119  
  2120  	var elen int
  2121  	for _, e := range ae.entries {
  2122  		elen += len(e.Data) + 1 + 4 // 1 is type, 4 is for size.
  2123  	}
  2124  	tlen := appendEntryBaseLen + elen + 1
  2125  
  2126  	var buf []byte
  2127  	if cap(b) >= tlen {
  2128  		buf = b[:tlen]
  2129  	} else {
  2130  		buf = make([]byte, tlen)
  2131  	}
  2132  
  2133  	var le = binary.LittleEndian
  2134  	copy(buf[:idLen], ae.leader)
  2135  	le.PutUint64(buf[8:], ae.term)
  2136  	le.PutUint64(buf[16:], ae.commit)
  2137  	le.PutUint64(buf[24:], ae.pterm)
  2138  	le.PutUint64(buf[32:], ae.pindex)
  2139  	le.PutUint16(buf[40:], uint16(len(ae.entries)))
  2140  	wi := 42
  2141  	for _, e := range ae.entries {
  2142  		le.PutUint32(buf[wi:], uint32(len(e.Data)+1))
  2143  		wi += 4
  2144  		buf[wi] = byte(e.Type)
  2145  		wi++
  2146  		copy(buf[wi:], e.Data)
  2147  		wi += len(e.Data)
  2148  	}
  2149  	return buf[:wi], nil
  2150  }
  2151  
  2152  // This can not be used post the wire level callback since we do not copy.
  2153  func (n *raft) decodeAppendEntry(msg []byte, sub *subscription, reply string) (*appendEntry, error) {
  2154  	if len(msg) < appendEntryBaseLen {
  2155  		return nil, errBadAppendEntry
  2156  	}
  2157  
  2158  	var le = binary.LittleEndian
  2159  
  2160  	ae := newAppendEntry(string(msg[:idLen]), le.Uint64(msg[8:]), le.Uint64(msg[16:]), le.Uint64(msg[24:]), le.Uint64(msg[32:]), nil)
  2161  	ae.reply, ae.sub = reply, sub
  2162  
  2163  	// Decode Entries.
  2164  	ne, ri := int(le.Uint16(msg[40:])), 42
  2165  	for i, max := 0, len(msg); i < ne; i++ {
  2166  		if ri >= max-1 {
  2167  			return nil, errBadAppendEntry
  2168  		}
  2169  		le := int(le.Uint32(msg[ri:]))
  2170  		ri += 4
  2171  		if le <= 0 || ri+le > max {
  2172  			return nil, errBadAppendEntry
  2173  		}
  2174  		entry := newEntry(EntryType(msg[ri]), msg[ri+1:ri+le])
  2175  		ae.entries = append(ae.entries, entry)
  2176  		ri += le
  2177  	}
  2178  	ae.buf = msg
  2179  	return ae, nil
  2180  }
  2181  
  2182  // Pool for appendEntryResponse re-use.
  2183  var arPool = sync.Pool{
  2184  	New: func() any {
  2185  		return &appendEntryResponse{}
  2186  	},
  2187  }
  2188  
  2189  // We want to make sure this does not change from system changing length of syshash.
  2190  const idLen = 8
  2191  const appendEntryResponseLen = 24 + 1
  2192  
  2193  // appendEntryResponse is our response to a received appendEntry.
  2194  type appendEntryResponse struct {
  2195  	term    uint64
  2196  	index   uint64
  2197  	peer    string
  2198  	reply   string // internal usage.
  2199  	success bool
  2200  }
  2201  
  2202  // Create a new appendEntryResponse.
  2203  func newAppendEntryResponse(term, index uint64, peer string, success bool) *appendEntryResponse {
  2204  	ar := arPool.Get().(*appendEntryResponse)
  2205  	ar.term, ar.index, ar.peer, ar.success = term, index, peer, success
  2206  	// Always empty out.
  2207  	ar.reply = _EMPTY_
  2208  	return ar
  2209  }
  2210  
  2211  func (ar *appendEntryResponse) encode(b []byte) []byte {
  2212  	var buf []byte
  2213  	if cap(b) >= appendEntryResponseLen {
  2214  		buf = b[:appendEntryResponseLen]
  2215  	} else {
  2216  		buf = make([]byte, appendEntryResponseLen)
  2217  	}
  2218  	var le = binary.LittleEndian
  2219  	le.PutUint64(buf[0:], ar.term)
  2220  	le.PutUint64(buf[8:], ar.index)
  2221  	copy(buf[16:16+idLen], ar.peer)
  2222  	if ar.success {
  2223  		buf[24] = 1
  2224  	} else {
  2225  		buf[24] = 0
  2226  	}
  2227  	return buf[:appendEntryResponseLen]
  2228  }
  2229  
  2230  // Track all peers we may have ever seen to use an string interns for appendEntryResponse decoding.
  2231  var peers sync.Map
  2232  
  2233  func (n *raft) decodeAppendEntryResponse(msg []byte) *appendEntryResponse {
  2234  	if len(msg) != appendEntryResponseLen {
  2235  		return nil
  2236  	}
  2237  	var le = binary.LittleEndian
  2238  	ar := arPool.Get().(*appendEntryResponse)
  2239  	ar.term = le.Uint64(msg[0:])
  2240  	ar.index = le.Uint64(msg[8:])
  2241  
  2242  	peer, ok := peers.Load(string(msg[16 : 16+idLen]))
  2243  	if !ok {
  2244  		// We missed so store inline here.
  2245  		peer = string(msg[16 : 16+idLen])
  2246  		peers.Store(peer, peer)
  2247  	}
  2248  	ar.peer = peer.(string)
  2249  	ar.success = msg[24] == 1
  2250  	return ar
  2251  }
  2252  
  2253  // Called when a remove peer proposal has been forwarded
  2254  func (n *raft) handleForwardedRemovePeerProposal(sub *subscription, c *client, _ *Account, _, reply string, msg []byte) {
  2255  	n.debug("Received forwarded remove peer proposal: %q", msg)
  2256  
  2257  	if !n.Leader() {
  2258  		n.debug("Ignoring forwarded peer removal proposal, not leader")
  2259  		return
  2260  	}
  2261  	if len(msg) != idLen {
  2262  		n.warn("Received invalid peer name for remove proposal: %q", msg)
  2263  		return
  2264  	}
  2265  
  2266  	n.RLock()
  2267  	prop, werr := n.prop, n.werr
  2268  	n.RUnlock()
  2269  
  2270  	// Ignore if we have had a write error previous.
  2271  	if werr != nil {
  2272  		return
  2273  	}
  2274  
  2275  	// Need to copy since this is underlying client/route buffer.
  2276  	peer := copyBytes(msg)
  2277  	prop.push(newEntry(EntryRemovePeer, peer))
  2278  }
  2279  
  2280  // Called when a peer has forwarded a proposal.
  2281  func (n *raft) handleForwardedProposal(sub *subscription, c *client, _ *Account, _, reply string, msg []byte) {
  2282  	if !n.Leader() {
  2283  		n.debug("Ignoring forwarded proposal, not leader")
  2284  		return
  2285  	}
  2286  	// Need to copy since this is underlying client/route buffer.
  2287  	msg = copyBytes(msg)
  2288  
  2289  	n.RLock()
  2290  	prop, werr := n.prop, n.werr
  2291  	n.RUnlock()
  2292  
  2293  	// Ignore if we have had a write error previous.
  2294  	if werr != nil {
  2295  		return
  2296  	}
  2297  
  2298  	prop.push(newEntry(EntryNormal, msg))
  2299  }
  2300  
  2301  func (n *raft) runAsLeader() {
  2302  	if n.State() == Closed {
  2303  		return
  2304  	}
  2305  
  2306  	n.RLock()
  2307  	psubj, rpsubj := n.psubj, n.rpsubj
  2308  	n.RUnlock()
  2309  
  2310  	// For forwarded proposals, both normal and remove peer proposals.
  2311  	fsub, err := n.subscribe(psubj, n.handleForwardedProposal)
  2312  	if err != nil {
  2313  		n.warn("Error subscribing to forwarded proposals: %v", err)
  2314  		n.stepdown(noLeader)
  2315  		return
  2316  	}
  2317  	rpsub, err := n.subscribe(rpsubj, n.handleForwardedRemovePeerProposal)
  2318  	if err != nil {
  2319  		n.warn("Error subscribing to forwarded remove peer proposals: %v", err)
  2320  		n.unsubscribe(fsub)
  2321  		n.stepdown(noLeader)
  2322  		return
  2323  	}
  2324  
  2325  	// Cleanup our subscription when we leave.
  2326  	defer func() {
  2327  		n.Lock()
  2328  		n.unsubscribe(fsub)
  2329  		n.unsubscribe(rpsub)
  2330  		n.Unlock()
  2331  	}()
  2332  
  2333  	// To send out our initial peer state.
  2334  	n.sendPeerState()
  2335  
  2336  	hb := time.NewTicker(hbInterval)
  2337  	defer hb.Stop()
  2338  
  2339  	lq := time.NewTicker(lostQuorumCheck)
  2340  	defer lq.Stop()
  2341  
  2342  	for n.State() == Leader {
  2343  		select {
  2344  		case <-n.s.quitCh:
  2345  			n.shutdown(false)
  2346  			return
  2347  		case <-n.quit:
  2348  			return
  2349  		case <-n.resp.ch:
  2350  			ars := n.resp.pop()
  2351  			for _, ar := range ars {
  2352  				n.processAppendEntryResponse(ar)
  2353  			}
  2354  			n.resp.recycle(&ars)
  2355  		case <-n.prop.ch:
  2356  			const maxBatch = 256 * 1024
  2357  			var entries []*Entry
  2358  
  2359  			es := n.prop.pop()
  2360  			sz := 0
  2361  			for i, b := range es {
  2362  				if b.Type == EntryRemovePeer {
  2363  					n.doRemovePeerAsLeader(string(b.Data))
  2364  				}
  2365  				entries = append(entries, b)
  2366  				sz += len(b.Data) + 1
  2367  				if i != len(es)-1 && sz < maxBatch && len(entries) < math.MaxUint16 {
  2368  					continue
  2369  				}
  2370  				n.sendAppendEntry(entries)
  2371  
  2372  				// If this is us sending out a leadership transfer stepdown inline here.
  2373  				if b.Type == EntryLeaderTransfer {
  2374  					n.prop.recycle(&es)
  2375  					n.debug("Stepping down due to leadership transfer")
  2376  					n.stepdown(noLeader)
  2377  					return
  2378  				}
  2379  				// We need to re-create `entries` because there is a reference
  2380  				// to it in the node's pae map.
  2381  				entries = nil
  2382  			}
  2383  			n.prop.recycle(&es)
  2384  
  2385  		case <-hb.C:
  2386  			if n.notActive() {
  2387  				n.sendHeartbeat()
  2388  			}
  2389  		case <-lq.C:
  2390  			if n.lostQuorum() {
  2391  				n.stepdown(noLeader)
  2392  				return
  2393  			}
  2394  		case <-n.votes.ch:
  2395  			// Because of drain() it is possible that we get nil from popOne().
  2396  			vresp, ok := n.votes.popOne()
  2397  			if !ok {
  2398  				continue
  2399  			}
  2400  			if vresp.term > n.Term() {
  2401  				n.stepdown(noLeader)
  2402  				return
  2403  			}
  2404  			n.trackPeer(vresp.peer)
  2405  		case <-n.reqs.ch:
  2406  			// Because of drain() it is possible that we get nil from popOne().
  2407  			if voteReq, ok := n.reqs.popOne(); ok {
  2408  				n.processVoteRequest(voteReq)
  2409  			}
  2410  		case <-n.entry.ch:
  2411  			n.processAppendEntries()
  2412  		}
  2413  	}
  2414  }
  2415  
  2416  // Quorum reports the quorum status. Will be called on former leaders.
  2417  func (n *raft) Quorum() bool {
  2418  	n.RLock()
  2419  	defer n.RUnlock()
  2420  
  2421  	now, nc := time.Now().UnixNano(), 1
  2422  	for _, peer := range n.peers {
  2423  		if now-peer.ts < int64(lostQuorumInterval) {
  2424  			nc++
  2425  			if nc >= n.qn {
  2426  				return true
  2427  			}
  2428  		}
  2429  	}
  2430  	return false
  2431  }
  2432  
  2433  func (n *raft) proposalAutoAcceptLocked() bool {
  2434  	return n.csz == 1
  2435  }
  2436  func (n *raft) implicitLeaderLocked() bool {
  2437  	return n.csz == 1 || (n.csz == 2 && n.ipeer)
  2438  }
  2439  
  2440  func (n *raft) lostQuorum() bool {
  2441  	n.RLock()
  2442  	defer n.RUnlock()
  2443  	return n.lostQuorumLocked()
  2444  }
  2445  
  2446  func (n *raft) lostQuorumLocked() bool {
  2447  	if n.implicitLeaderLocked() {
  2448  		return false
  2449  	}
  2450  
  2451  	// Make sure we let any scale up actions settle before deciding.
  2452  	if !n.lsut.IsZero() && time.Since(n.lsut) < lostQuorumInterval {
  2453  		return false
  2454  	}
  2455  
  2456  	now, nc := time.Now().UnixNano(), 1
  2457  	for _, peer := range n.peers {
  2458  		if now-peer.ts < int64(lostQuorumInterval) {
  2459  			nc++
  2460  			if nc >= n.qn {
  2461  				return false
  2462  			}
  2463  		}
  2464  	}
  2465  	return true
  2466  }
  2467  
  2468  // Check for being not active in terms of sending entries.
  2469  // Used in determining if we need to send a heartbeat.
  2470  func (n *raft) notActive() bool {
  2471  	n.RLock()
  2472  	defer n.RUnlock()
  2473  	return time.Since(n.active) > hbInterval
  2474  }
  2475  
  2476  // Return our current term.
  2477  func (n *raft) Term() uint64 {
  2478  	n.RLock()
  2479  	defer n.RUnlock()
  2480  	return n.term
  2481  }
  2482  
  2483  // Lock should be held.
  2484  func (n *raft) loadFirstEntry() (ae *appendEntry, err error) {
  2485  	var state StreamState
  2486  	n.wal.FastState(&state)
  2487  	return n.loadEntry(state.FirstSeq)
  2488  }
  2489  
  2490  func (n *raft) runCatchup(ar *appendEntryResponse, indexUpdatesQ *ipQueue[uint64]) {
  2491  	n.RLock()
  2492  	s, reply := n.s, n.areply
  2493  	peer, subj, last := ar.peer, ar.reply, n.pindex
  2494  	n.RUnlock()
  2495  
  2496  	defer s.grWG.Done()
  2497  	defer arPool.Put(ar)
  2498  
  2499  	defer func() {
  2500  		n.Lock()
  2501  		delete(n.progress, peer)
  2502  		if len(n.progress) == 0 {
  2503  			n.progress = nil
  2504  		}
  2505  		// Check if this is a new peer and if so go ahead and propose adding them.
  2506  		_, exists := n.peers[peer]
  2507  		n.Unlock()
  2508  		if !exists {
  2509  			n.debug("Catchup done for %q, will add into peers", peer)
  2510  			n.ProposeAddPeer(peer)
  2511  		}
  2512  		indexUpdatesQ.unregister()
  2513  	}()
  2514  
  2515  	n.debug("Running catchup for %q", peer)
  2516  
  2517  	const maxOutstanding = 2 * 1024 * 1024 // 2MB for now.
  2518  	next, total, om := uint64(0), 0, make(map[uint64]int)
  2519  
  2520  	sendNext := func() bool {
  2521  		for total <= maxOutstanding {
  2522  			next++
  2523  			if next > last {
  2524  				return true
  2525  			}
  2526  			ae, err := n.loadEntry(next)
  2527  			if err != nil {
  2528  				if err != ErrStoreEOF {
  2529  					n.warn("Got an error loading %d index: %v", next, err)
  2530  				}
  2531  				return true
  2532  			}
  2533  			// Update our tracking total.
  2534  			om[next] = len(ae.buf)
  2535  			total += len(ae.buf)
  2536  			n.sendRPC(subj, reply, ae.buf)
  2537  		}
  2538  		return false
  2539  	}
  2540  
  2541  	const activityInterval = 2 * time.Second
  2542  	timeout := time.NewTimer(activityInterval)
  2543  	defer timeout.Stop()
  2544  
  2545  	stepCheck := time.NewTicker(100 * time.Millisecond)
  2546  	defer stepCheck.Stop()
  2547  
  2548  	// Run as long as we are leader and still not caught up.
  2549  	for n.Leader() {
  2550  		select {
  2551  		case <-n.s.quitCh:
  2552  			n.shutdown(false)
  2553  			return
  2554  		case <-n.quit:
  2555  			return
  2556  		case <-stepCheck.C:
  2557  			if !n.Leader() {
  2558  				n.debug("Catching up canceled, no longer leader")
  2559  				return
  2560  			}
  2561  		case <-timeout.C:
  2562  			n.debug("Catching up for %q stalled", peer)
  2563  			return
  2564  		case <-indexUpdatesQ.ch:
  2565  			if index, ok := indexUpdatesQ.popOne(); ok {
  2566  				// Update our activity timer.
  2567  				timeout.Reset(activityInterval)
  2568  				// Update outstanding total.
  2569  				total -= om[index]
  2570  				delete(om, index)
  2571  				if next == 0 {
  2572  					next = index
  2573  				}
  2574  				// Check if we are done.
  2575  				if index > last || sendNext() {
  2576  					n.debug("Finished catching up")
  2577  					return
  2578  				}
  2579  			}
  2580  		}
  2581  	}
  2582  }
  2583  
  2584  // Lock should be held.
  2585  func (n *raft) sendSnapshotToFollower(subject string) (uint64, error) {
  2586  	snap, err := n.loadLastSnapshot()
  2587  	if err != nil {
  2588  		// We need to stepdown here when this happens.
  2589  		n.stepdownLocked(noLeader)
  2590  		// We need to reset our state here as well.
  2591  		n.resetWAL()
  2592  		return 0, err
  2593  	}
  2594  	// Go ahead and send the snapshot and peerstate here as first append entry to the catchup follower.
  2595  	ae := n.buildAppendEntry([]*Entry{{EntrySnapshot, snap.data}, {EntryPeerState, snap.peerstate}})
  2596  	ae.pterm, ae.pindex = snap.lastTerm, snap.lastIndex
  2597  	var state StreamState
  2598  	n.wal.FastState(&state)
  2599  
  2600  	fpIndex := state.FirstSeq - 1
  2601  	if snap.lastIndex < fpIndex && state.FirstSeq != 0 {
  2602  		snap.lastIndex = fpIndex
  2603  		ae.pindex = fpIndex
  2604  	}
  2605  
  2606  	encoding, err := ae.encode(nil)
  2607  	if err != nil {
  2608  		return 0, err
  2609  	}
  2610  	n.sendRPC(subject, n.areply, encoding)
  2611  	return snap.lastIndex, nil
  2612  }
  2613  
  2614  func (n *raft) catchupFollower(ar *appendEntryResponse) {
  2615  	n.debug("Being asked to catch up follower: %q", ar.peer)
  2616  	n.Lock()
  2617  	if n.progress == nil {
  2618  		n.progress = make(map[string]*ipQueue[uint64])
  2619  	} else if q, ok := n.progress[ar.peer]; ok {
  2620  		n.debug("Will cancel existing entry for catching up %q", ar.peer)
  2621  		delete(n.progress, ar.peer)
  2622  		q.push(n.pindex)
  2623  	}
  2624  
  2625  	// Check to make sure we have this entry.
  2626  	start := ar.index + 1
  2627  	var state StreamState
  2628  	n.wal.FastState(&state)
  2629  
  2630  	if start < state.FirstSeq || (state.Msgs == 0 && start <= state.LastSeq) {
  2631  		n.debug("Need to send snapshot to follower")
  2632  		if lastIndex, err := n.sendSnapshotToFollower(ar.reply); err != nil {
  2633  			n.error("Error sending snapshot to follower [%s]: %v", ar.peer, err)
  2634  			n.Unlock()
  2635  			arPool.Put(ar)
  2636  			return
  2637  		} else {
  2638  			start = lastIndex + 1
  2639  			// If no other entries, we can just return here.
  2640  			if state.Msgs == 0 || start > state.LastSeq {
  2641  				n.debug("Finished catching up")
  2642  				n.Unlock()
  2643  				arPool.Put(ar)
  2644  				return
  2645  			}
  2646  			n.debug("Snapshot sent, reset first catchup entry to %d", lastIndex)
  2647  		}
  2648  	}
  2649  
  2650  	ae, err := n.loadEntry(start)
  2651  	if err != nil {
  2652  		n.warn("Request from follower for entry at index [%d] errored for state %+v - %v", start, state, err)
  2653  		if err == ErrStoreEOF {
  2654  			// If we are here we are seeing a request for an item beyond our state, meaning we should stepdown.
  2655  			n.stepdownLocked(noLeader)
  2656  			n.Unlock()
  2657  			arPool.Put(ar)
  2658  			return
  2659  		}
  2660  		ae, err = n.loadFirstEntry()
  2661  	}
  2662  	if err != nil || ae == nil {
  2663  		n.warn("Could not find a starting entry for catchup request: %v", err)
  2664  		// If we are here we are seeing a request for an item we do not have, meaning we should stepdown.
  2665  		// This is possible on a reset of our WAL but the other side has a snapshot already.
  2666  		// If we do not stepdown this can cycle.
  2667  		n.stepdownLocked(noLeader)
  2668  		n.Unlock()
  2669  		arPool.Put(ar)
  2670  		return
  2671  	}
  2672  	if ae.pindex != ar.index || ae.pterm != ar.term {
  2673  		n.debug("Our first entry [%d:%d] does not match request from follower [%d:%d]", ae.pterm, ae.pindex, ar.term, ar.index)
  2674  	}
  2675  	// Create a queue for delivering updates from responses.
  2676  	indexUpdates := newIPQueue[uint64](n.s, fmt.Sprintf("[ACC:%s] RAFT '%s' indexUpdates", n.accName, n.group))
  2677  	indexUpdates.push(ae.pindex)
  2678  	n.progress[ar.peer] = indexUpdates
  2679  	n.Unlock()
  2680  
  2681  	n.s.startGoRoutine(func() { n.runCatchup(ar, indexUpdates) })
  2682  }
  2683  
  2684  func (n *raft) loadEntry(index uint64) (*appendEntry, error) {
  2685  	var smp StoreMsg
  2686  	sm, err := n.wal.LoadMsg(index, &smp)
  2687  	if err != nil {
  2688  		return nil, err
  2689  	}
  2690  	return n.decodeAppendEntry(sm.msg, nil, _EMPTY_)
  2691  }
  2692  
  2693  // applyCommit will update our commit index and apply the entry to the apply queue.
  2694  // lock should be held.
  2695  func (n *raft) applyCommit(index uint64) error {
  2696  	if n.State() == Closed {
  2697  		return errNodeClosed
  2698  	}
  2699  	if index <= n.commit {
  2700  		n.debug("Ignoring apply commit for %d, already processed", index)
  2701  		return nil
  2702  	}
  2703  	original := n.commit
  2704  	n.commit = index
  2705  
  2706  	if n.State() == Leader {
  2707  		delete(n.acks, index)
  2708  	}
  2709  
  2710  	var fpae bool
  2711  
  2712  	ae := n.pae[index]
  2713  	if ae == nil {
  2714  		var state StreamState
  2715  		n.wal.FastState(&state)
  2716  		if index < state.FirstSeq {
  2717  			return nil
  2718  		}
  2719  		var err error
  2720  		if ae, err = n.loadEntry(index); err != nil {
  2721  			if err != ErrStoreClosed && err != ErrStoreEOF {
  2722  				n.warn("Got an error loading %d index: %v - will reset", index, err)
  2723  				if n.State() == Leader {
  2724  					n.stepdownLocked(n.selectNextLeader())
  2725  				}
  2726  				// Reset and cancel any catchup.
  2727  				n.resetWAL()
  2728  				n.cancelCatchup()
  2729  			} else {
  2730  				n.commit = original
  2731  			}
  2732  			return errEntryLoadFailed
  2733  		}
  2734  	} else {
  2735  		fpae = true
  2736  	}
  2737  
  2738  	ae.buf = nil
  2739  
  2740  	var committed []*Entry
  2741  	for _, e := range ae.entries {
  2742  		switch e.Type {
  2743  		case EntryNormal:
  2744  			committed = append(committed, e)
  2745  		case EntryOldSnapshot:
  2746  			// For old snapshots in our WAL.
  2747  			committed = append(committed, newEntry(EntrySnapshot, e.Data))
  2748  		case EntrySnapshot:
  2749  			committed = append(committed, e)
  2750  		case EntryPeerState:
  2751  			if n.State() != Leader {
  2752  				if ps, err := decodePeerState(e.Data); err == nil {
  2753  					n.processPeerState(ps)
  2754  				}
  2755  			}
  2756  		case EntryAddPeer:
  2757  			newPeer := string(e.Data)
  2758  			n.debug("Added peer %q", newPeer)
  2759  
  2760  			// Store our peer in our global peer map for all peers.
  2761  			peers.LoadOrStore(newPeer, newPeer)
  2762  
  2763  			// If we were on the removed list reverse that here.
  2764  			if n.removed != nil {
  2765  				delete(n.removed, newPeer)
  2766  			}
  2767  
  2768  			if lp, ok := n.peers[newPeer]; !ok {
  2769  				// We are not tracking this one automatically so we need to bump cluster size.
  2770  				n.peers[newPeer] = &lps{time.Now().UnixNano(), 0, true}
  2771  			} else {
  2772  				// Mark as added.
  2773  				lp.kp = true
  2774  			}
  2775  			// Adjust cluster size and quorum if needed.
  2776  			n.adjustClusterSizeAndQuorum()
  2777  			// Write out our new state.
  2778  			n.writePeerState(&peerState{n.peerNames(), n.csz, n.extSt})
  2779  			// We pass these up as well.
  2780  			committed = append(committed, e)
  2781  
  2782  		case EntryRemovePeer:
  2783  			peer := string(e.Data)
  2784  			n.debug("Removing peer %q", peer)
  2785  
  2786  			// Make sure we have our removed map.
  2787  			if n.removed == nil {
  2788  				n.removed = make(map[string]struct{})
  2789  			}
  2790  			n.removed[peer] = struct{}{}
  2791  
  2792  			if _, ok := n.peers[peer]; ok {
  2793  				delete(n.peers, peer)
  2794  				// We should decrease our cluster size since we are tracking this peer.
  2795  				n.adjustClusterSizeAndQuorum()
  2796  				// Write out our new state.
  2797  				n.writePeerState(&peerState{n.peerNames(), n.csz, n.extSt})
  2798  			}
  2799  
  2800  			// If this is us and we are the leader we should attempt to stepdown.
  2801  			if peer == n.id && n.State() == Leader {
  2802  				n.stepdown(n.selectNextLeader())
  2803  			}
  2804  
  2805  			// Remove from string intern map.
  2806  			peers.Delete(peer)
  2807  
  2808  			// We pass these up as well.
  2809  			committed = append(committed, e)
  2810  		}
  2811  	}
  2812  	if fpae {
  2813  		delete(n.pae, index)
  2814  	}
  2815  	// Pass to the upper layers if we have normal entries. It is
  2816  	// entirely possible that 'committed' might be an empty slice here,
  2817  	// which will happen if we've processed updates inline (like peer
  2818  	// states). In which case the upper layer will just call down with
  2819  	// Applied() with no further action.
  2820  	n.apply.push(newCommittedEntry(index, committed))
  2821  	// Place back in the pool.
  2822  	ae.returnToPool()
  2823  	return nil
  2824  }
  2825  
  2826  // Used to track a success response and apply entries.
  2827  func (n *raft) trackResponse(ar *appendEntryResponse) {
  2828  	if n.State() == Closed {
  2829  		return
  2830  	}
  2831  
  2832  	n.Lock()
  2833  
  2834  	// Update peer's last index.
  2835  	if ps := n.peers[ar.peer]; ps != nil && ar.index > ps.li {
  2836  		ps.li = ar.index
  2837  	}
  2838  
  2839  	// If we are tracking this peer as a catchup follower, update that here.
  2840  	if indexUpdateQ := n.progress[ar.peer]; indexUpdateQ != nil {
  2841  		indexUpdateQ.push(ar.index)
  2842  	}
  2843  
  2844  	// Ignore items already committed.
  2845  	if ar.index <= n.commit {
  2846  		n.Unlock()
  2847  		return
  2848  	}
  2849  
  2850  	// See if we have items to apply.
  2851  	var sendHB bool
  2852  
  2853  	if results := n.acks[ar.index]; results != nil {
  2854  		results[ar.peer] = struct{}{}
  2855  		if nr := len(results); nr >= n.qn {
  2856  			// We have a quorum.
  2857  			for index := n.commit + 1; index <= ar.index; index++ {
  2858  				if err := n.applyCommit(index); err != nil && err != errNodeClosed {
  2859  					n.error("Got an error applying commit for %d: %v", index, err)
  2860  					break
  2861  				}
  2862  			}
  2863  			sendHB = n.prop.len() == 0
  2864  		}
  2865  	}
  2866  	n.Unlock()
  2867  
  2868  	if sendHB {
  2869  		n.sendHeartbeat()
  2870  	}
  2871  }
  2872  
  2873  // Used to adjust cluster size and peer count based on added official peers.
  2874  // lock should be held.
  2875  func (n *raft) adjustClusterSizeAndQuorum() {
  2876  	pcsz, ncsz := n.csz, 0
  2877  	for _, peer := range n.peers {
  2878  		if peer.kp {
  2879  			ncsz++
  2880  		}
  2881  	}
  2882  	n.csz = ncsz
  2883  	n.qn = n.csz/2 + 1
  2884  
  2885  	if ncsz > pcsz {
  2886  		n.debug("Expanding our clustersize: %d -> %d", pcsz, ncsz)
  2887  		n.lsut = time.Now()
  2888  	} else if ncsz < pcsz {
  2889  		n.debug("Decreasing our clustersize: %d -> %d", pcsz, ncsz)
  2890  		if n.State() == Leader {
  2891  			go n.sendHeartbeat()
  2892  		}
  2893  	}
  2894  }
  2895  
  2896  // Track interactions with this peer.
  2897  func (n *raft) trackPeer(peer string) error {
  2898  	n.Lock()
  2899  	var needPeerAdd, isRemoved bool
  2900  	if n.removed != nil {
  2901  		_, isRemoved = n.removed[peer]
  2902  	}
  2903  	if n.State() == Leader {
  2904  		if lp, ok := n.peers[peer]; !ok || !lp.kp {
  2905  			// Check if this peer had been removed previously.
  2906  			needPeerAdd = !isRemoved
  2907  		}
  2908  	}
  2909  	if ps := n.peers[peer]; ps != nil {
  2910  		ps.ts = time.Now().UnixNano()
  2911  	} else if !isRemoved {
  2912  		n.peers[peer] = &lps{time.Now().UnixNano(), 0, false}
  2913  	}
  2914  	n.Unlock()
  2915  
  2916  	if needPeerAdd {
  2917  		n.ProposeAddPeer(peer)
  2918  	}
  2919  	return nil
  2920  }
  2921  
  2922  func (n *raft) runAsCandidate() {
  2923  	n.Lock()
  2924  	// Drain old responses.
  2925  	n.votes.drain()
  2926  	n.Unlock()
  2927  
  2928  	// Send out our request for votes.
  2929  	n.requestVote()
  2930  
  2931  	// We vote for ourselves.
  2932  	votes := map[string]struct{}{
  2933  		n.ID(): {},
  2934  	}
  2935  
  2936  	for n.State() == Candidate {
  2937  		elect := n.electTimer()
  2938  		select {
  2939  		case <-n.entry.ch:
  2940  			n.processAppendEntries()
  2941  		case <-n.resp.ch:
  2942  			// Ignore
  2943  			n.resp.popOne()
  2944  		case <-n.s.quitCh:
  2945  			n.shutdown(false)
  2946  			return
  2947  		case <-n.quit:
  2948  			return
  2949  		case <-elect.C:
  2950  			n.switchToCandidate()
  2951  			return
  2952  		case <-n.votes.ch:
  2953  			// Because of drain() it is possible that we get nil from popOne().
  2954  			vresp, ok := n.votes.popOne()
  2955  			if !ok {
  2956  				continue
  2957  			}
  2958  			n.RLock()
  2959  			nterm := n.term
  2960  			n.RUnlock()
  2961  
  2962  			if vresp.granted && nterm == vresp.term {
  2963  				// only track peers that would be our followers
  2964  				n.trackPeer(vresp.peer)
  2965  				votes[vresp.peer] = struct{}{}
  2966  				if n.wonElection(len(votes)) {
  2967  					// Become LEADER if we have won and gotten a quorum with everyone we should hear from.
  2968  					n.switchToLeader()
  2969  					return
  2970  				}
  2971  			} else if vresp.term > nterm {
  2972  				// if we observe a bigger term, we should start over again or risk forming a quorum fully knowing
  2973  				// someone with a better term exists. This is even the right thing to do if won == true.
  2974  				n.Lock()
  2975  				n.debug("Stepping down from candidate, detected higher term: %d vs %d", vresp.term, n.term)
  2976  				n.term = vresp.term
  2977  				n.vote = noVote
  2978  				n.writeTermVote()
  2979  				n.lxfer = false
  2980  				n.stepdownLocked(noLeader)
  2981  				n.Unlock()
  2982  			}
  2983  		case <-n.reqs.ch:
  2984  			// Because of drain() it is possible that we get nil from popOne().
  2985  			if voteReq, ok := n.reqs.popOne(); ok {
  2986  				n.processVoteRequest(voteReq)
  2987  			}
  2988  		}
  2989  	}
  2990  }
  2991  
  2992  // handleAppendEntry handles an append entry from the wire. This function
  2993  // is an internal callback from the "asubj" append entry subscription.
  2994  func (n *raft) handleAppendEntry(sub *subscription, c *client, _ *Account, subject, reply string, msg []byte) {
  2995  	msg = copyBytes(msg)
  2996  	if ae, err := n.decodeAppendEntry(msg, sub, reply); err == nil {
  2997  		// Push to the new entry channel. From here one of the worker
  2998  		// goroutines (runAsLeader, runAsFollower, runAsCandidate) will
  2999  		// pick it up.
  3000  		n.entry.push(ae)
  3001  	} else {
  3002  		n.warn("AppendEntry failed to be placed on internal channel: corrupt entry")
  3003  	}
  3004  }
  3005  
  3006  // cancelCatchup will stop an in-flight catchup by unsubscribing from the
  3007  // catchup subscription.
  3008  // Lock should be held.
  3009  func (n *raft) cancelCatchup() {
  3010  	n.debug("Canceling catchup subscription since we are now up to date")
  3011  
  3012  	if n.catchup != nil && n.catchup.sub != nil {
  3013  		n.unsubscribe(n.catchup.sub)
  3014  	}
  3015  	n.catchup = nil
  3016  }
  3017  
  3018  // catchupStalled will try to determine if we are stalled. This is called
  3019  // on a new entry from our leader.
  3020  // Lock should be held.
  3021  func (n *raft) catchupStalled() bool {
  3022  	if n.catchup == nil {
  3023  		return false
  3024  	}
  3025  	if n.catchup.pindex == n.pindex {
  3026  		return time.Since(n.catchup.active) > 2*time.Second
  3027  	}
  3028  	n.catchup.pindex = n.pindex
  3029  	n.catchup.active = time.Now()
  3030  	return false
  3031  }
  3032  
  3033  // createCatchup will create the state needed to track a catchup as it
  3034  // runs. It then creates a unique inbox for this catchup and subscribes
  3035  // to it. The remote side will stream entries to that subject.
  3036  // Lock should be held.
  3037  func (n *raft) createCatchup(ae *appendEntry) string {
  3038  	// Cleanup any old ones.
  3039  	if n.catchup != nil && n.catchup.sub != nil {
  3040  		n.unsubscribe(n.catchup.sub)
  3041  	}
  3042  	// Snapshot term and index.
  3043  	n.catchup = &catchupState{
  3044  		cterm:  ae.pterm,
  3045  		cindex: ae.pindex,
  3046  		pterm:  n.pterm,
  3047  		pindex: n.pindex,
  3048  		active: time.Now(),
  3049  	}
  3050  	inbox := n.newCatchupInbox()
  3051  	sub, _ := n.subscribe(inbox, n.handleAppendEntry)
  3052  	n.catchup.sub = sub
  3053  
  3054  	return inbox
  3055  }
  3056  
  3057  // Truncate our WAL and reset.
  3058  // Lock should be held.
  3059  func (n *raft) truncateWAL(term, index uint64) {
  3060  	n.debug("Truncating and repairing WAL to Term %d Index %d", term, index)
  3061  
  3062  	if term == 0 && index == 0 {
  3063  		n.warn("Resetting WAL state")
  3064  	}
  3065  
  3066  	defer func() {
  3067  		// Check to see if we invalidated any snapshots that might have held state
  3068  		// from the entries we are truncating.
  3069  		if snap, _ := n.loadLastSnapshot(); snap != nil && snap.lastIndex >= index {
  3070  			os.Remove(n.snapfile)
  3071  			n.snapfile = _EMPTY_
  3072  		}
  3073  		// Make sure to reset commit and applied if above
  3074  		if n.commit > n.pindex {
  3075  			n.commit = n.pindex
  3076  		}
  3077  		if n.applied > n.commit {
  3078  			n.applied = n.commit
  3079  		}
  3080  	}()
  3081  
  3082  	if err := n.wal.Truncate(index); err != nil {
  3083  		// If we get an invalid sequence, reset our wal all together.
  3084  		if err == ErrInvalidSequence {
  3085  			n.debug("Resetting WAL")
  3086  			n.wal.Truncate(0)
  3087  			index, n.term, n.pterm, n.pindex = 0, 0, 0, 0
  3088  		} else {
  3089  			n.warn("Error truncating WAL: %v", err)
  3090  			n.setWriteErrLocked(err)
  3091  		}
  3092  		return
  3093  	}
  3094  
  3095  	// Set after we know we have truncated properly.
  3096  	n.term, n.pterm, n.pindex = term, term, index
  3097  }
  3098  
  3099  // Reset our WAL. This is equivalent to truncating all data from the log.
  3100  // Lock should be held.
  3101  func (n *raft) resetWAL() {
  3102  	n.truncateWAL(0, 0)
  3103  }
  3104  
  3105  // Lock should be held
  3106  func (n *raft) updateLeader(newLeader string) {
  3107  	n.leader = newLeader
  3108  	if !n.pleader && newLeader != noLeader {
  3109  		n.pleader = true
  3110  	}
  3111  }
  3112  
  3113  // processAppendEntry will process an appendEntry. This is called either
  3114  // during recovery or from processAppendEntries when there are new entries
  3115  // to be committed.
  3116  func (n *raft) processAppendEntry(ae *appendEntry, sub *subscription) {
  3117  	n.Lock()
  3118  	// Don't reset here if we have been asked to assume leader position.
  3119  	if !n.lxfer {
  3120  		n.resetElectionTimeout()
  3121  	}
  3122  
  3123  	// Just return if closed or we had previous write error.
  3124  	if n.State() == Closed || n.werr != nil {
  3125  		n.Unlock()
  3126  		return
  3127  	}
  3128  
  3129  	// Scratch buffer for responses.
  3130  	var scratch [appendEntryResponseLen]byte
  3131  	arbuf := scratch[:]
  3132  
  3133  	// Are we receiving from another leader.
  3134  	if n.State() == Leader {
  3135  		// If we are the same we should step down to break the tie.
  3136  		if ae.term >= n.term {
  3137  			n.term = ae.term
  3138  			n.vote = noVote
  3139  			n.writeTermVote()
  3140  			n.debug("Received append entry from another leader, stepping down to %q", ae.leader)
  3141  			n.stepdownLocked(ae.leader)
  3142  		} else {
  3143  			// Let them know we are the leader.
  3144  			ar := newAppendEntryResponse(n.term, n.pindex, n.id, false)
  3145  			n.debug("AppendEntry ignoring old term from another leader")
  3146  			n.sendRPC(ae.reply, _EMPTY_, ar.encode(arbuf))
  3147  			arPool.Put(ar)
  3148  		}
  3149  		// Always return here from processing.
  3150  		n.Unlock()
  3151  		return
  3152  	}
  3153  
  3154  	// If we received an append entry as a candidate then it would appear that
  3155  	// another node has taken on the leader role already, so we should convert
  3156  	// to a follower of that node instead.
  3157  	if n.State() == Candidate {
  3158  		n.debug("Received append entry in candidate state from %q, converting to follower", ae.leader)
  3159  		if n.term < ae.term {
  3160  			n.term = ae.term
  3161  			n.vote = noVote
  3162  			n.writeTermVote()
  3163  		}
  3164  		n.stepdownLocked(ae.leader)
  3165  	}
  3166  
  3167  	// Catching up state.
  3168  	catchingUp := n.catchup != nil
  3169  	// Is this a new entry? New entries will be delivered on the append entry
  3170  	// sub, rather than a catch-up sub.
  3171  	isNew := sub != nil && sub == n.aesub
  3172  
  3173  	// Track leader directly
  3174  	if isNew && ae.leader != noLeader {
  3175  		if ps := n.peers[ae.leader]; ps != nil {
  3176  			ps.ts = time.Now().UnixNano()
  3177  		} else {
  3178  			n.peers[ae.leader] = &lps{time.Now().UnixNano(), 0, true}
  3179  		}
  3180  	}
  3181  
  3182  	// If we are catching up ignore old catchup subs.
  3183  	// This could happen when we stall or cancel a catchup.
  3184  	if !isNew && catchingUp && sub != n.catchup.sub {
  3185  		n.Unlock()
  3186  		n.debug("AppendEntry ignoring old entry from previous catchup")
  3187  		return
  3188  	}
  3189  
  3190  	// Check state if we are catching up.
  3191  	if catchingUp {
  3192  		if cs := n.catchup; cs != nil && n.pterm >= cs.cterm && n.pindex >= cs.cindex {
  3193  			// If we are here we are good, so if we have a catchup pending we can cancel.
  3194  			n.cancelCatchup()
  3195  			// Reset our notion of catching up.
  3196  			catchingUp = false
  3197  		} else if isNew {
  3198  			var ar *appendEntryResponse
  3199  			var inbox string
  3200  			// Check to see if we are stalled. If so recreate our catchup state and resend response.
  3201  			if n.catchupStalled() {
  3202  				n.debug("Catchup may be stalled, will request again")
  3203  				inbox = n.createCatchup(ae)
  3204  				ar = newAppendEntryResponse(n.pterm, n.pindex, n.id, false)
  3205  			}
  3206  			n.Unlock()
  3207  			if ar != nil {
  3208  				n.sendRPC(ae.reply, inbox, ar.encode(arbuf))
  3209  				arPool.Put(ar)
  3210  			}
  3211  			// Ignore new while catching up or replaying.
  3212  			return
  3213  		}
  3214  	}
  3215  
  3216  	// If this term is greater than ours.
  3217  	if ae.term > n.term {
  3218  		n.pterm = ae.pterm
  3219  		n.term = ae.term
  3220  		n.vote = noVote
  3221  		if isNew {
  3222  			n.writeTermVote()
  3223  		}
  3224  		if n.State() != Follower {
  3225  			n.debug("Term higher than ours and we are not a follower: %v, stepping down to %q", n.State(), ae.leader)
  3226  			n.stepdownLocked(ae.leader)
  3227  		}
  3228  	}
  3229  
  3230  	if isNew && n.leader != ae.leader && n.State() == Follower {
  3231  		n.debug("AppendEntry updating leader to %q", ae.leader)
  3232  		n.updateLeader(ae.leader)
  3233  		n.writeTermVote()
  3234  		n.resetElectionTimeout()
  3235  		n.updateLeadChange(false)
  3236  	}
  3237  
  3238  	if (isNew && ae.pterm != n.pterm) || ae.pindex != n.pindex {
  3239  		// Check if this is a lower or equal index than what we were expecting.
  3240  		if ae.pindex <= n.pindex {
  3241  			n.debug("AppendEntry detected pindex less than ours: %d:%d vs %d:%d", ae.pterm, ae.pindex, n.pterm, n.pindex)
  3242  			var ar *appendEntryResponse
  3243  
  3244  			var success bool
  3245  			if eae, _ := n.loadEntry(ae.pindex); eae == nil {
  3246  				// If terms are equal, and we are not catching up, we have simply already processed this message.
  3247  				// So we will ACK back to the leader. This can happen on server restarts based on timings of snapshots.
  3248  				if ae.pterm == n.pterm && !catchingUp {
  3249  					success = true
  3250  				} else {
  3251  					n.resetWAL()
  3252  				}
  3253  			} else {
  3254  				// If terms mismatched, or we got an error loading, delete that entry and all others past it.
  3255  				// Make sure to cancel any catchups in progress.
  3256  				// Truncate will reset our pterm and pindex. Only do so if we have an entry.
  3257  				n.truncateWAL(ae.pterm, ae.pindex)
  3258  			}
  3259  			// Cancel regardless.
  3260  			n.cancelCatchup()
  3261  
  3262  			// Create response.
  3263  			ar = newAppendEntryResponse(ae.pterm, ae.pindex, n.id, success)
  3264  			n.Unlock()
  3265  			n.sendRPC(ae.reply, _EMPTY_, ar.encode(arbuf))
  3266  			arPool.Put(ar)
  3267  			return
  3268  		}
  3269  
  3270  		// Check if we are catching up. If we are here we know the leader did not have all of the entries
  3271  		// so make sure this is a snapshot entry. If it is not start the catchup process again since it
  3272  		// means we may have missed additional messages.
  3273  		if catchingUp {
  3274  			// Check if only our terms do not match here.
  3275  			if ae.pindex == n.pindex {
  3276  				// Make sure pterms match and we take on the leader's.
  3277  				// This prevents constant spinning.
  3278  				n.truncateWAL(ae.pterm, ae.pindex)
  3279  				n.cancelCatchup()
  3280  				n.Unlock()
  3281  				return
  3282  			}
  3283  			// This means we already entered into a catchup state but what the leader sent us did not match what we expected.
  3284  			// Snapshots and peerstate will always be together when a leader is catching us up in this fashion.
  3285  			if len(ae.entries) != 2 || ae.entries[0].Type != EntrySnapshot || ae.entries[1].Type != EntryPeerState {
  3286  				n.warn("Expected first catchup entry to be a snapshot and peerstate, will retry")
  3287  				n.cancelCatchup()
  3288  				n.Unlock()
  3289  				return
  3290  			}
  3291  
  3292  			if ps, err := decodePeerState(ae.entries[1].Data); err == nil {
  3293  				n.processPeerState(ps)
  3294  				// Also need to copy from client's buffer.
  3295  				ae.entries[0].Data = copyBytes(ae.entries[0].Data)
  3296  			} else {
  3297  				n.warn("Could not parse snapshot peerstate correctly")
  3298  				n.cancelCatchup()
  3299  				n.Unlock()
  3300  				return
  3301  			}
  3302  
  3303  			n.pindex = ae.pindex
  3304  			n.pterm = ae.pterm
  3305  			n.commit = ae.pindex
  3306  
  3307  			if _, err := n.wal.Compact(n.pindex + 1); err != nil {
  3308  				n.setWriteErrLocked(err)
  3309  				n.Unlock()
  3310  				return
  3311  			}
  3312  
  3313  			// Now send snapshot to upper levels. Only send the snapshot, not the peerstate entry.
  3314  			n.apply.push(newCommittedEntry(n.commit, ae.entries[:1]))
  3315  			n.Unlock()
  3316  			return
  3317  
  3318  		} else {
  3319  			n.debug("AppendEntry did not match %d %d with %d %d", ae.pterm, ae.pindex, n.pterm, n.pindex)
  3320  			// Reset our term.
  3321  			n.term = n.pterm
  3322  			if ae.pindex > n.pindex {
  3323  				// Setup our state for catching up.
  3324  				inbox := n.createCatchup(ae)
  3325  				ar := newAppendEntryResponse(n.pterm, n.pindex, n.id, false)
  3326  				n.Unlock()
  3327  				n.sendRPC(ae.reply, inbox, ar.encode(arbuf))
  3328  				arPool.Put(ar)
  3329  				return
  3330  			}
  3331  		}
  3332  	}
  3333  
  3334  	// Save to our WAL if we have entries.
  3335  	if ae.shouldStore() {
  3336  		// Only store if an original which will have sub != nil
  3337  		if sub != nil {
  3338  			if err := n.storeToWAL(ae); err != nil {
  3339  				if err != ErrStoreClosed {
  3340  					n.warn("Error storing entry to WAL: %v", err)
  3341  				}
  3342  				n.Unlock()
  3343  				return
  3344  			}
  3345  			// Save in memory for faster processing during applyCommit.
  3346  			// Only save so many however to avoid memory bloat.
  3347  			if l := len(n.pae); l <= paeDropThreshold {
  3348  				n.pae[n.pindex], l = ae, l+1
  3349  				if l > paeWarnThreshold && l%paeWarnModulo == 0 {
  3350  					n.warn("%d append entries pending", len(n.pae))
  3351  				}
  3352  			} else {
  3353  				n.debug("Not saving to append entries pending")
  3354  			}
  3355  		} else {
  3356  			// This is a replay on startup so just take the appendEntry version.
  3357  			n.pterm = ae.term
  3358  			n.pindex = ae.pindex + 1
  3359  		}
  3360  	}
  3361  
  3362  	// Check to see if we have any related entries to process here.
  3363  	for _, e := range ae.entries {
  3364  		switch e.Type {
  3365  		case EntryLeaderTransfer:
  3366  			// Only process these if they are new, so no replays or catchups.
  3367  			if isNew {
  3368  				maybeLeader := string(e.Data)
  3369  				// This is us. We need to check if we can become the leader.
  3370  				if maybeLeader == n.id {
  3371  					// If not an observer and not paused we are good to go.
  3372  					if !n.observer && !n.paused {
  3373  						n.lxfer = true
  3374  						n.xferCampaign()
  3375  					} else if n.paused && !n.pobserver {
  3376  						// Here we can become a leader but need to wait for resume of the apply queue.
  3377  						n.lxfer = true
  3378  					}
  3379  				} else if n.vote != noVote {
  3380  					// Since we are here we are not the chosen one but we should clear any vote preference.
  3381  					n.vote = noVote
  3382  					n.writeTermVote()
  3383  				}
  3384  			}
  3385  		case EntryAddPeer:
  3386  			if newPeer := string(e.Data); len(newPeer) == idLen {
  3387  				// Track directly, but wait for commit to be official
  3388  				if ps := n.peers[newPeer]; ps != nil {
  3389  					ps.ts = time.Now().UnixNano()
  3390  				} else {
  3391  					n.peers[newPeer] = &lps{time.Now().UnixNano(), 0, false}
  3392  				}
  3393  				// Store our peer in our global peer map for all peers.
  3394  				peers.LoadOrStore(newPeer, newPeer)
  3395  			}
  3396  		}
  3397  	}
  3398  
  3399  	// Apply anything we need here.
  3400  	if ae.commit > n.commit {
  3401  		if n.paused {
  3402  			n.hcommit = ae.commit
  3403  			n.debug("Paused, not applying %d", ae.commit)
  3404  		} else {
  3405  			for index := n.commit + 1; index <= ae.commit; index++ {
  3406  				if err := n.applyCommit(index); err != nil {
  3407  					break
  3408  				}
  3409  			}
  3410  		}
  3411  	}
  3412  
  3413  	var ar *appendEntryResponse
  3414  	if sub != nil {
  3415  		ar = newAppendEntryResponse(n.pterm, n.pindex, n.id, true)
  3416  	}
  3417  	n.Unlock()
  3418  
  3419  	// Success. Send our response.
  3420  	if ar != nil {
  3421  		n.sendRPC(ae.reply, _EMPTY_, ar.encode(arbuf))
  3422  		arPool.Put(ar)
  3423  	}
  3424  }
  3425  
  3426  // processPeerState is called when a peer state entry is received
  3427  // over the wire or when we're updating known peers.
  3428  // Lock should be held.
  3429  func (n *raft) processPeerState(ps *peerState) {
  3430  	// Update our version of peers to that of the leader. Calculate
  3431  	// the number of nodes needed to establish a quorum.
  3432  	n.csz = ps.clusterSize
  3433  	n.qn = n.csz/2 + 1
  3434  
  3435  	old := n.peers
  3436  	n.peers = make(map[string]*lps)
  3437  	for _, peer := range ps.knownPeers {
  3438  		if lp := old[peer]; lp != nil {
  3439  			lp.kp = true
  3440  			n.peers[peer] = lp
  3441  		} else {
  3442  			n.peers[peer] = &lps{0, 0, true}
  3443  		}
  3444  	}
  3445  	n.debug("Update peers from leader to %+v", n.peers)
  3446  	n.writePeerState(ps)
  3447  }
  3448  
  3449  // processAppendEntryResponse is called when we receive an append entry
  3450  // response from another node. They will send a confirmation to tell us
  3451  // whether they successfully committed the entry or not.
  3452  func (n *raft) processAppendEntryResponse(ar *appendEntryResponse) {
  3453  	n.trackPeer(ar.peer)
  3454  
  3455  	if ar.success {
  3456  		// The remote node successfully committed the append entry.
  3457  		n.trackResponse(ar)
  3458  		arPool.Put(ar)
  3459  	} else if ar.term > n.term {
  3460  		// The remote node didn't commit the append entry, it looks like
  3461  		// they are on a newer term than we are. Step down.
  3462  		n.Lock()
  3463  		n.term = ar.term
  3464  		n.vote = noVote
  3465  		n.writeTermVote()
  3466  		n.warn("Detected another leader with higher term, will stepdown and reset")
  3467  		n.stepdownLocked(noLeader)
  3468  		n.resetWAL()
  3469  		n.Unlock()
  3470  		arPool.Put(ar)
  3471  	} else if ar.reply != _EMPTY_ {
  3472  		// The remote node didn't commit the append entry and they are
  3473  		// still on the same term, so let's try to catch them up.
  3474  		n.catchupFollower(ar)
  3475  	}
  3476  }
  3477  
  3478  // handleAppendEntryResponse processes responses to append entries.
  3479  func (n *raft) handleAppendEntryResponse(sub *subscription, c *client, _ *Account, subject, reply string, msg []byte) {
  3480  	ar := n.decodeAppendEntryResponse(msg)
  3481  	ar.reply = reply
  3482  	n.resp.push(ar)
  3483  }
  3484  
  3485  func (n *raft) buildAppendEntry(entries []*Entry) *appendEntry {
  3486  	return newAppendEntry(n.id, n.term, n.commit, n.pterm, n.pindex, entries)
  3487  }
  3488  
  3489  // Determine if we should store an entry. This stops us from storing
  3490  // heartbeat messages.
  3491  func (ae *appendEntry) shouldStore() bool {
  3492  	return ae != nil && len(ae.entries) > 0
  3493  }
  3494  
  3495  // Store our append entry to our WAL.
  3496  // lock should be held.
  3497  func (n *raft) storeToWAL(ae *appendEntry) error {
  3498  	if ae == nil {
  3499  		return fmt.Errorf("raft: Missing append entry for storage")
  3500  	}
  3501  	if n.werr != nil {
  3502  		return n.werr
  3503  	}
  3504  
  3505  	seq, _, err := n.wal.StoreMsg(_EMPTY_, nil, ae.buf)
  3506  	if err != nil {
  3507  		n.setWriteErrLocked(err)
  3508  		return err
  3509  	}
  3510  
  3511  	// Sanity checking for now.
  3512  	if index := ae.pindex + 1; index != seq {
  3513  		n.warn("Wrong index, ae is %+v, index stored was %d, n.pindex is %d, will reset", ae, seq, n.pindex)
  3514  		if n.State() == Leader {
  3515  			n.stepdownLocked(n.selectNextLeader())
  3516  		}
  3517  		// Reset and cancel any catchup.
  3518  		n.resetWAL()
  3519  		n.cancelCatchup()
  3520  		return errEntryStoreFailed
  3521  	}
  3522  
  3523  	n.pterm = ae.term
  3524  	n.pindex = seq
  3525  	return nil
  3526  }
  3527  
  3528  const (
  3529  	paeDropThreshold = 20_000
  3530  	paeWarnThreshold = 10_000
  3531  	paeWarnModulo    = 5_000
  3532  )
  3533  
  3534  func (n *raft) sendAppendEntry(entries []*Entry) {
  3535  	n.Lock()
  3536  	defer n.Unlock()
  3537  	ae := n.buildAppendEntry(entries)
  3538  
  3539  	var err error
  3540  	var scratch [1024]byte
  3541  	ae.buf, err = ae.encode(scratch[:])
  3542  	if err != nil {
  3543  		return
  3544  	}
  3545  
  3546  	// If we have entries store this in our wal.
  3547  	shouldStore := ae.shouldStore()
  3548  	if shouldStore {
  3549  		if err := n.storeToWAL(ae); err != nil {
  3550  			return
  3551  		}
  3552  		// We count ourselves.
  3553  		n.acks[n.pindex] = map[string]struct{}{n.id: {}}
  3554  		n.active = time.Now()
  3555  
  3556  		// Save in memory for faster processing during applyCommit.
  3557  		n.pae[n.pindex] = ae
  3558  		if l := len(n.pae); l > paeWarnThreshold && l%paeWarnModulo == 0 {
  3559  			n.warn("%d append entries pending", len(n.pae))
  3560  		}
  3561  	}
  3562  	if n.proposalAutoAcceptLocked() {
  3563  		n.resp.push(newAppendEntryResponse(n.term, n.pindex, n.id, true))
  3564  	} else {
  3565  		n.sendRPC(n.asubj, n.areply, ae.buf)
  3566  	}
  3567  	if !shouldStore {
  3568  		ae.returnToPool()
  3569  	}
  3570  }
  3571  
  3572  type extensionState uint16
  3573  
  3574  const (
  3575  	extUndetermined = extensionState(iota)
  3576  	extExtended
  3577  	extNotExtended
  3578  )
  3579  
  3580  type peerState struct {
  3581  	knownPeers  []string
  3582  	clusterSize int
  3583  	domainExt   extensionState
  3584  }
  3585  
  3586  func peerStateBufSize(ps *peerState) int {
  3587  	return 4 + 4 + (idLen * len(ps.knownPeers)) + 2
  3588  }
  3589  
  3590  func encodePeerState(ps *peerState) []byte {
  3591  	var le = binary.LittleEndian
  3592  	buf := make([]byte, peerStateBufSize(ps))
  3593  	le.PutUint32(buf[0:], uint32(ps.clusterSize))
  3594  	le.PutUint32(buf[4:], uint32(len(ps.knownPeers)))
  3595  	wi := 8
  3596  	for _, peer := range ps.knownPeers {
  3597  		copy(buf[wi:], peer)
  3598  		wi += idLen
  3599  	}
  3600  	le.PutUint16(buf[wi:], uint16(ps.domainExt))
  3601  	return buf
  3602  }
  3603  
  3604  func decodePeerState(buf []byte) (*peerState, error) {
  3605  	if len(buf) < 8 {
  3606  		return nil, errCorruptPeers
  3607  	}
  3608  	var le = binary.LittleEndian
  3609  	ps := &peerState{clusterSize: int(le.Uint32(buf[0:]))}
  3610  	expectedPeers := int(le.Uint32(buf[4:]))
  3611  	buf = buf[8:]
  3612  	ri := 0
  3613  	for i, n := 0, expectedPeers; i < n && ri < len(buf); i++ {
  3614  		ps.knownPeers = append(ps.knownPeers, string(buf[ri:ri+idLen]))
  3615  		ri += idLen
  3616  	}
  3617  	if len(ps.knownPeers) != expectedPeers {
  3618  		return nil, errCorruptPeers
  3619  	}
  3620  	if len(buf[ri:]) >= 2 {
  3621  		ps.domainExt = extensionState(le.Uint16(buf[ri:]))
  3622  	}
  3623  	return ps, nil
  3624  }
  3625  
  3626  // Lock should be held.
  3627  func (n *raft) peerNames() []string {
  3628  	var peers []string
  3629  	for name, peer := range n.peers {
  3630  		if peer.kp {
  3631  			peers = append(peers, name)
  3632  		}
  3633  	}
  3634  	return peers
  3635  }
  3636  
  3637  func (n *raft) currentPeerState() *peerState {
  3638  	n.RLock()
  3639  	ps := &peerState{n.peerNames(), n.csz, n.extSt}
  3640  	n.RUnlock()
  3641  	return ps
  3642  }
  3643  
  3644  // sendPeerState will send our current peer state to the cluster.
  3645  func (n *raft) sendPeerState() {
  3646  	n.sendAppendEntry([]*Entry{{EntryPeerState, encodePeerState(n.currentPeerState())}})
  3647  }
  3648  
  3649  // Send a heartbeat.
  3650  func (n *raft) sendHeartbeat() {
  3651  	n.sendAppendEntry(nil)
  3652  }
  3653  
  3654  type voteRequest struct {
  3655  	term      uint64
  3656  	lastTerm  uint64
  3657  	lastIndex uint64
  3658  	candidate string
  3659  	// internal only.
  3660  	reply string
  3661  }
  3662  
  3663  const voteRequestLen = 24 + idLen
  3664  
  3665  func (vr *voteRequest) encode() []byte {
  3666  	var buf [voteRequestLen]byte
  3667  	var le = binary.LittleEndian
  3668  	le.PutUint64(buf[0:], vr.term)
  3669  	le.PutUint64(buf[8:], vr.lastTerm)
  3670  	le.PutUint64(buf[16:], vr.lastIndex)
  3671  	copy(buf[24:24+idLen], vr.candidate)
  3672  
  3673  	return buf[:voteRequestLen]
  3674  }
  3675  
  3676  func decodeVoteRequest(msg []byte, reply string) *voteRequest {
  3677  	if len(msg) != voteRequestLen {
  3678  		return nil
  3679  	}
  3680  
  3681  	var le = binary.LittleEndian
  3682  	return &voteRequest{
  3683  		term:      le.Uint64(msg[0:]),
  3684  		lastTerm:  le.Uint64(msg[8:]),
  3685  		lastIndex: le.Uint64(msg[16:]),
  3686  		candidate: string(copyBytes(msg[24 : 24+idLen])),
  3687  		reply:     reply,
  3688  	}
  3689  }
  3690  
  3691  const peerStateFile = "peers.idx"
  3692  const ipeerMarkerFile = "ipeer.mark"
  3693  
  3694  // Lock should be held.
  3695  func (n *raft) writePeerState(ps *peerState) {
  3696  	pse := encodePeerState(ps)
  3697  	if bytes.Equal(n.wps, pse) {
  3698  		return
  3699  	}
  3700  	// Stamp latest and write the peer state file.
  3701  	n.wps = pse
  3702  	if err := writePeerState(n.sd, ps); err != nil && !n.isClosed() {
  3703  		n.setWriteErr(err)
  3704  		n.warn("Error writing peer state file for %q: %v", n.group, err)
  3705  	}
  3706  }
  3707  
  3708  // Writes out our peer state outside of a specific raft context.
  3709  func writePeerState(sd string, ps *peerState) error {
  3710  	psf := filepath.Join(sd, peerStateFile)
  3711  	if _, err := os.Stat(psf); err != nil && !os.IsNotExist(err) {
  3712  		return err
  3713  	}
  3714  
  3715  	<-dios
  3716  	err := os.WriteFile(psf, encodePeerState(ps), defaultFilePerms)
  3717  	dios <- struct{}{}
  3718  
  3719  	return err
  3720  }
  3721  
  3722  func readPeerState(sd string) (ps *peerState, err error) {
  3723  	<-dios
  3724  	buf, err := os.ReadFile(filepath.Join(sd, peerStateFile))
  3725  	dios <- struct{}{}
  3726  
  3727  	if err != nil {
  3728  		return nil, err
  3729  	}
  3730  	return decodePeerState(buf)
  3731  }
  3732  
  3733  const termVoteFile = "tav.idx"
  3734  const termVoteLen = idLen + 8
  3735  
  3736  // Writes out our term & vote outside of a specific raft context.
  3737  func writeTermVote(sd string, wtv []byte) error {
  3738  	psf := filepath.Join(sd, termVoteFile)
  3739  	if _, err := os.Stat(psf); err != nil && !os.IsNotExist(err) {
  3740  		return err
  3741  	}
  3742  
  3743  	<-dios
  3744  	err := os.WriteFile(psf, wtv, defaultFilePerms)
  3745  	dios <- struct{}{}
  3746  
  3747  	return err
  3748  }
  3749  
  3750  // readTermVote will read the largest term and who we voted from to stable storage.
  3751  // Lock should be held.
  3752  func (n *raft) readTermVote() (term uint64, voted string, err error) {
  3753  	<-dios
  3754  	buf, err := os.ReadFile(filepath.Join(n.sd, termVoteFile))
  3755  	dios <- struct{}{}
  3756  
  3757  	if err != nil {
  3758  		return 0, noVote, err
  3759  	}
  3760  	if len(buf) < termVoteLen {
  3761  		return 0, noVote, nil
  3762  	}
  3763  	var le = binary.LittleEndian
  3764  	term = le.Uint64(buf[0:])
  3765  	voted = string(buf[8:])
  3766  	return term, voted, nil
  3767  }
  3768  
  3769  // Lock should be held.
  3770  func (n *raft) setWriteErrLocked(err error) {
  3771  	// Check if we are closed already.
  3772  	if n.State() == Closed {
  3773  		return
  3774  	}
  3775  	// Ignore if already set.
  3776  	if n.werr == err || err == nil {
  3777  		return
  3778  	}
  3779  	// Ignore non-write errors.
  3780  	if err != nil {
  3781  		if err == ErrStoreClosed ||
  3782  			err == ErrStoreEOF ||
  3783  			err == ErrInvalidSequence ||
  3784  			err == ErrStoreMsgNotFound ||
  3785  			err == errNoPending ||
  3786  			err == errPartialCache {
  3787  			return
  3788  		}
  3789  		// If this is a not found report but do not disable.
  3790  		if os.IsNotExist(err) {
  3791  			n.error("Resource not found: %v", err)
  3792  			return
  3793  		}
  3794  		n.error("Critical write error: %v", err)
  3795  	}
  3796  	n.werr = err
  3797  
  3798  	if isOutOfSpaceErr(err) {
  3799  		// For now since this can be happening all under the covers, we will call up and disable JetStream.
  3800  		go n.s.handleOutOfSpace(nil)
  3801  	}
  3802  }
  3803  
  3804  // Helper to check if we are closed when we do not hold a lock already.
  3805  func (n *raft) isClosed() bool {
  3806  	return n.State() == Closed
  3807  }
  3808  
  3809  // Capture our write error if any and hold.
  3810  func (n *raft) setWriteErr(err error) {
  3811  	n.Lock()
  3812  	defer n.Unlock()
  3813  	n.setWriteErrLocked(err)
  3814  }
  3815  
  3816  // writeTermVote will record the largest term and who we voted for to stable storage.
  3817  // Lock should be held.
  3818  func (n *raft) writeTermVote() {
  3819  	var buf [termVoteLen]byte
  3820  	var le = binary.LittleEndian
  3821  	le.PutUint64(buf[0:], n.term)
  3822  	copy(buf[8:], n.vote)
  3823  	b := buf[:8+len(n.vote)]
  3824  
  3825  	// If the term and vote hasn't changed then don't rewrite to disk.
  3826  	if bytes.Equal(n.wtv, b) {
  3827  		return
  3828  	}
  3829  	// Stamp latest and write the term & vote file.
  3830  	n.wtv = b
  3831  	if err := writeTermVote(n.sd, n.wtv); err != nil && !n.isClosed() {
  3832  		n.setWriteErr(err)
  3833  		n.warn("Error writing term and vote file for %q: %v", n.group, err)
  3834  	}
  3835  }
  3836  
  3837  // voteResponse is a response to a vote request.
  3838  type voteResponse struct {
  3839  	term    uint64
  3840  	peer    string
  3841  	granted bool
  3842  }
  3843  
  3844  const voteResponseLen = 8 + 8 + 1
  3845  
  3846  func (vr *voteResponse) encode() []byte {
  3847  	var buf [voteResponseLen]byte
  3848  	var le = binary.LittleEndian
  3849  	le.PutUint64(buf[0:], vr.term)
  3850  	copy(buf[8:], vr.peer)
  3851  	if vr.granted {
  3852  		buf[16] = 1
  3853  	} else {
  3854  		buf[16] = 0
  3855  	}
  3856  	return buf[:voteResponseLen]
  3857  }
  3858  
  3859  func decodeVoteResponse(msg []byte) *voteResponse {
  3860  	if len(msg) != voteResponseLen {
  3861  		return nil
  3862  	}
  3863  	var le = binary.LittleEndian
  3864  	vr := &voteResponse{term: le.Uint64(msg[0:]), peer: string(msg[8:16])}
  3865  	vr.granted = msg[16] == 1
  3866  	return vr
  3867  }
  3868  
  3869  func (n *raft) handleVoteResponse(sub *subscription, c *client, _ *Account, _, reply string, msg []byte) {
  3870  	vr := decodeVoteResponse(msg)
  3871  	n.debug("Received a voteResponse %+v", vr)
  3872  	if vr == nil {
  3873  		n.error("Received malformed vote response for %q", n.group)
  3874  		return
  3875  	}
  3876  
  3877  	if state := n.State(); state != Candidate && state != Leader {
  3878  		n.debug("Ignoring old vote response, we have stepped down")
  3879  		return
  3880  	}
  3881  
  3882  	n.votes.push(vr)
  3883  }
  3884  
  3885  func (n *raft) processVoteRequest(vr *voteRequest) error {
  3886  	// To simplify calling code, we can possibly pass `nil` to this function.
  3887  	// If that is the case, does not consider it an error.
  3888  	if vr == nil {
  3889  		return nil
  3890  	}
  3891  	n.debug("Received a voteRequest %+v", vr)
  3892  
  3893  	if err := n.trackPeer(vr.candidate); err != nil {
  3894  		return err
  3895  	}
  3896  
  3897  	n.Lock()
  3898  	n.resetElectionTimeout()
  3899  
  3900  	vresp := &voteResponse{n.term, n.id, false}
  3901  	defer n.debug("Sending a voteResponse %+v -> %q", vresp, vr.reply)
  3902  
  3903  	// Ignore if we are newer. This is important so that we don't accidentally process
  3904  	// votes from a previous term if they were still in flight somewhere.
  3905  	if vr.term < n.term {
  3906  		n.Unlock()
  3907  		n.sendReply(vr.reply, vresp.encode())
  3908  		return nil
  3909  	}
  3910  
  3911  	// If this is a higher term go ahead and stepdown.
  3912  	if vr.term > n.term {
  3913  		if n.State() != Follower {
  3914  			n.debug("Stepping down from %s, detected higher term: %d vs %d",
  3915  				strings.ToLower(n.State().String()), vr.term, n.term)
  3916  			n.stepdownLocked(noLeader)
  3917  			n.term = vr.term
  3918  		}
  3919  		n.vote = noVote
  3920  		n.writeTermVote()
  3921  	}
  3922  
  3923  	// Only way we get to yes is through here.
  3924  	voteOk := n.vote == noVote || n.vote == vr.candidate
  3925  	if voteOk && (vr.lastTerm > n.pterm || vr.lastTerm == n.pterm && vr.lastIndex >= n.pindex) {
  3926  		vresp.granted = true
  3927  		n.term = vr.term
  3928  		n.vote = vr.candidate
  3929  		n.writeTermVote()
  3930  	} else {
  3931  		if vr.term >= n.term && n.vote == noVote {
  3932  			n.term = vr.term
  3933  			n.resetElect(randCampaignTimeout())
  3934  		}
  3935  	}
  3936  
  3937  	// Term might have changed, make sure response has the most current
  3938  	vresp.term = n.term
  3939  
  3940  	n.Unlock()
  3941  
  3942  	n.sendReply(vr.reply, vresp.encode())
  3943  
  3944  	return nil
  3945  }
  3946  
  3947  func (n *raft) handleVoteRequest(sub *subscription, c *client, _ *Account, subject, reply string, msg []byte) {
  3948  	vr := decodeVoteRequest(msg, reply)
  3949  	if vr == nil {
  3950  		n.error("Received malformed vote request for %q", n.group)
  3951  		return
  3952  	}
  3953  	n.reqs.push(vr)
  3954  }
  3955  
  3956  func (n *raft) requestVote() {
  3957  	n.Lock()
  3958  	if n.State() != Candidate {
  3959  		n.Unlock()
  3960  		return
  3961  	}
  3962  	n.vote = n.id
  3963  	n.writeTermVote()
  3964  	vr := voteRequest{n.term, n.pterm, n.pindex, n.id, _EMPTY_}
  3965  	subj, reply := n.vsubj, n.vreply
  3966  	n.Unlock()
  3967  
  3968  	n.debug("Sending out voteRequest %+v", vr)
  3969  
  3970  	// Now send it out.
  3971  	n.sendRPC(subj, reply, vr.encode())
  3972  }
  3973  
  3974  func (n *raft) sendRPC(subject, reply string, msg []byte) {
  3975  	if n.sq != nil {
  3976  		n.sq.send(subject, reply, nil, msg)
  3977  	}
  3978  }
  3979  
  3980  func (n *raft) sendReply(subject string, msg []byte) {
  3981  	if n.sq != nil {
  3982  		n.sq.send(subject, _EMPTY_, nil, msg)
  3983  	}
  3984  }
  3985  
  3986  func (n *raft) wonElection(votes int) bool {
  3987  	return votes >= n.quorumNeeded()
  3988  }
  3989  
  3990  // Return the quorum size for a given cluster config.
  3991  func (n *raft) quorumNeeded() int {
  3992  	n.RLock()
  3993  	qn := n.qn
  3994  	n.RUnlock()
  3995  	return qn
  3996  }
  3997  
  3998  // Lock should be held.
  3999  func (n *raft) updateLeadChange(isLeader bool) {
  4000  	// We don't care about values that have not been consumed (transitory states),
  4001  	// so we dequeue any state that is pending and push the new one.
  4002  	for {
  4003  		select {
  4004  		case n.leadc <- isLeader:
  4005  			return
  4006  		default:
  4007  			select {
  4008  			case <-n.leadc:
  4009  			default:
  4010  				// May have been consumed by the "reader" go routine, so go back
  4011  				// to the top of the loop and try to send again.
  4012  			}
  4013  		}
  4014  	}
  4015  }
  4016  
  4017  // Lock should be held.
  4018  func (n *raft) switchState(state RaftState) {
  4019  	if n.State() == Closed {
  4020  		return
  4021  	}
  4022  
  4023  	// Reset the election timer.
  4024  	n.resetElectionTimeout()
  4025  
  4026  	if n.State() == Leader && state != Leader {
  4027  		n.updateLeadChange(false)
  4028  		// Drain the response queue.
  4029  		n.resp.drain()
  4030  	} else if state == Leader && n.State() != Leader {
  4031  		if len(n.pae) > 0 {
  4032  			n.pae = make(map[uint64]*appendEntry)
  4033  		}
  4034  		n.updateLeadChange(true)
  4035  	}
  4036  
  4037  	n.state.Store(int32(state))
  4038  	n.writeTermVote()
  4039  }
  4040  
  4041  const (
  4042  	noLeader = _EMPTY_
  4043  	noVote   = _EMPTY_
  4044  )
  4045  
  4046  func (n *raft) switchToFollower(leader string) {
  4047  	n.Lock()
  4048  	defer n.Unlock()
  4049  
  4050  	n.switchToFollowerLocked(leader)
  4051  }
  4052  
  4053  func (n *raft) switchToFollowerLocked(leader string) {
  4054  	if n.State() == Closed {
  4055  		return
  4056  	}
  4057  
  4058  	n.debug("Switching to follower")
  4059  
  4060  	n.lxfer = false
  4061  	n.updateLeader(leader)
  4062  	n.switchState(Follower)
  4063  }
  4064  
  4065  func (n *raft) switchToCandidate() {
  4066  	if n.State() == Closed {
  4067  		return
  4068  	}
  4069  
  4070  	n.Lock()
  4071  	defer n.Unlock()
  4072  
  4073  	// If we are catching up or are in observer mode we can not switch.
  4074  	if n.observer || n.paused {
  4075  		return
  4076  	}
  4077  
  4078  	if n.State() != Candidate {
  4079  		n.debug("Switching to candidate")
  4080  	} else {
  4081  		if n.lostQuorumLocked() && time.Since(n.llqrt) > 20*time.Second {
  4082  			// We signal to the upper layers such that can alert on quorum lost.
  4083  			n.updateLeadChange(false)
  4084  			n.llqrt = time.Now()
  4085  		}
  4086  	}
  4087  	// Increment the term.
  4088  	n.term++
  4089  	// Clear current Leader.
  4090  	n.updateLeader(noLeader)
  4091  	n.switchState(Candidate)
  4092  }
  4093  
  4094  func (n *raft) switchToLeader() {
  4095  	if n.State() == Closed {
  4096  		return
  4097  	}
  4098  
  4099  	n.Lock()
  4100  
  4101  	n.debug("Switching to leader")
  4102  
  4103  	var state StreamState
  4104  	n.wal.FastState(&state)
  4105  
  4106  	// Check if we have items pending as we are taking over.
  4107  	sendHB := state.LastSeq > n.commit
  4108  
  4109  	n.lxfer = false
  4110  	n.updateLeader(n.id)
  4111  	n.switchState(Leader)
  4112  	n.Unlock()
  4113  
  4114  	if sendHB {
  4115  		n.sendHeartbeat()
  4116  	}
  4117  }