github.com/grafana/pyroscope@v1.18.0/pkg/metastore/raftnode/node_read.go (about)

     1  package raftnode
     2  
     3  import (
     4  	"context"
     5  	"errors"
     6  	"fmt"
     7  	"time"
     8  
     9  	"github.com/hashicorp/raft"
    10  	"github.com/prometheus/client_golang/prometheus"
    11  )
    12  
    13  var (
    14  	ErrConsistentRead = errors.New("consistent read failed")
    15  	ErrLagBehind      = errors.New("replica has fallen too far behind")
    16  	ErrAborted        = errors.New("aborted")
    17  )
    18  
    19  // ReadIndex is the lower bound for the state any query must operate against.
    20  // However, it does not guarantee snapshot isolation or an upper bound (which
    21  // is the applied index of the state machine being queried).
    22  //
    23  // Refer to https://web.stanford.edu/~ouster/cgi-bin/papers/OngaroPhD.pdf,
    24  // paragraph 6.4, "Processing read-only queries more efficiently".
    25  type ReadIndex struct {
    26  	// CommitIndex is the index of the last log entry that was committed by
    27  	// the leader and is guaranteed to be present on all followers.
    28  	CommitIndex uint64
    29  	// Term the leader was in when the entry was committed.
    30  	Term uint64
    31  }
    32  
    33  type Leader interface {
    34  	ReadIndex() (ReadIndex, error)
    35  }
    36  
    37  type FSM[Tx any] interface {
    38  	AppliedIndex() uint64
    39  	Read(func(Tx)) error
    40  }
    41  
    42  // StateReader represents the read-only state of the replicated state machine.
    43  // It allows performing read-only transactions on the leader's and follower's
    44  // state machines.
    45  type StateReader[Tx any] struct {
    46  	leader        Leader
    47  	fsm           FSM[Tx]
    48  	checkInterval time.Duration
    49  	maxDistance   uint64
    50  }
    51  
    52  // NewStateReader creates a new interface to query the replicated state.
    53  // If the provided leader implementation is the local node, the interface
    54  // implements the Leader Read pattern. Otherwise, it implements the Follower
    55  // Read pattern.
    56  //
    57  // > This approach is more efficient than committing read-only queries as new
    58  // > entries in the log, since it avoids synchronous disk writes. To improve
    59  // > efficiency further, the leader can amortize the cost of confirming its
    60  // > leadership: it can use a single round of heartbeats for any number of
    61  // > read-only queries that it has accumulated.
    62  // >
    63  // > Followers could also help offload the processing of read-only queries.
    64  // > This would improve the system’s read throughput, and it would also
    65  // > divert load away from the leader, allowing the leader to process more
    66  // > read-write requests. However, these reads would also run the risk of
    67  // > returning stale data without additional precautions. For example, a
    68  // > partitioned follower might not receive any new log entries from the leader
    69  // > for long periods of time, or even if a follower received a heartbeat from
    70  // > a leader, that leader might itself be deposed and not yet know it.
    71  // > To serve reads safely, the follower could issue a request to the leader
    72  // > that just asked for a current readIndex (the leader would execute steps
    73  // > 1–3 above); the follower could then execute steps 4 and 5 on its own state
    74  // > machine for any number of accumulated read-only queries.
    75  //
    76  // The applied index is checked on the configured interval. It the distance
    77  // between the read index and the applied index exceeds the configured
    78  // threshold, the operation fails with ErrLagBehind. Any error returned by
    79  // the reader is wrapped with ErrConsistentRead.
    80  func NewStateReader[Tx any](
    81  	leader Leader,
    82  	fsm FSM[Tx],
    83  	checkInterval time.Duration,
    84  	maxDistance uint64,
    85  ) *StateReader[Tx] {
    86  	return &StateReader[Tx]{
    87  		leader:        leader,
    88  		fsm:           fsm,
    89  		checkInterval: checkInterval,
    90  		maxDistance:   maxDistance,
    91  	}
    92  }
    93  
    94  // ConsistentRead performs a read-only operation on the state machine, whether
    95  // it's a leader or a follower.
    96  //
    97  // The transaction passed to the provided function has read-only access to the
    98  // most up-to-date data, reflecting the updates from all prior write operations
    99  // that were successful. If the function returns an error, it's guaranteed that
   100  // the state has not been accessed. These errors can and should be retried on
   101  // another replica.
   102  //
   103  // Currently, each ConsistentRead requests the new read index from the leader.
   104  // It's possible to "pipeline" such queries to minimize communications by
   105  // obtaining the applied index with WaitLeaderCommitIndexApplied and checking
   106  // the currently applied index every time entering the transaction. Take into
   107  // account that the FSM state might be changed at any time (e.g., restored from
   108  // a snapshot).
   109  //
   110  // It's caller's responsibility to handle errors encountered while using the
   111  // provided transaction, such as I/O errors or logical inconsistencies.
   112  func (r *StateReader[Tx]) ConsistentRead(ctx context.Context, read func(tx Tx, index ReadIndex)) error {
   113  	if err := r.consistentRead(ctx, read); err != nil {
   114  		return fmt.Errorf("%w: %w", ErrConsistentRead, err)
   115  	}
   116  	return nil
   117  }
   118  
   119  func (r *StateReader[Tx]) consistentRead(ctx context.Context, read func(tx Tx, index ReadIndex)) error {
   120  	readIndex, err := r.WaitLeaderCommitIndexApplied(ctx)
   121  	if err != nil {
   122  		return err
   123  	}
   124  	var readErr error
   125  	fn := func(tx Tx) {
   126  		// Now that we've acquired access to the state after catch up with
   127  		// the leader, we can perform the read operation. However, there's a
   128  		// possibility that the FSM has been restored from a snapshot right
   129  		// after the index check and before the transaction begins (blocking
   130  		// state restore). We perform the check again to detect this, and
   131  		// abort the operation if this is the case.
   132  		if r.fsm.AppliedIndex() < readIndex.CommitIndex {
   133  			readErr = ErrAborted
   134  			return
   135  		}
   136  		// NOTE(kolesnikovae): The leader guarantees that the state observed is
   137  		// not older than its committed index but does not guarantee it is the
   138  		// latest possible state at the time of the read.
   139  		read(tx, readIndex)
   140  	}
   141  	if err = r.fsm.Read(fn); err != nil {
   142  		// The FSM might not be able to perform the read operation due to the
   143  		// underlying storage issues. In this case, we return the error before
   144  		// providing the transaction handle to the caller.
   145  		return err
   146  	}
   147  	return readErr
   148  }
   149  
   150  // WaitLeaderCommitIndexApplied blocks until the local
   151  // applied index reaches the leader read index
   152  func (r *StateReader[tx]) WaitLeaderCommitIndexApplied(ctx context.Context) (ReadIndex, error) {
   153  	readIndex, err := r.leader.ReadIndex()
   154  	if err != nil {
   155  		return ReadIndex{}, err
   156  	}
   157  	return readIndex, waitIndexReached(ctx,
   158  		r.fsm.AppliedIndex,
   159  		readIndex.CommitIndex,
   160  		r.checkInterval,
   161  		int(r.maxDistance),
   162  	)
   163  }
   164  
   165  func (n *Node) ReadIndex() (ReadIndex, error) {
   166  	timer := prometheus.NewTimer(n.metrics.read)
   167  	defer timer.ObserveDuration()
   168  	v, err := n.readIndex()
   169  	return v, WithRaftLeaderStatusDetails(err, n.raft)
   170  }
   171  
   172  func (n *Node) AppliedIndex() uint64 { return n.raft.AppliedIndex() }
   173  
   174  func (n *Node) readIndex() (ReadIndex, error) {
   175  	// > If the leader has not yet marked an entry from its current term
   176  	// > committed, it waits until it has done so. The Leader Completeness
   177  	// > Property guarantees that a leader has all committed entries, but
   178  	// > at the start of its term, it may not know which those are. To find
   179  	// > out, it needs to commit an entry from its term. Raft handles this
   180  	// > by having each leader commit a blank no-op entry into the log at
   181  	// > the start of its term. As soon as this no-op entry is committed,
   182  	// > the leader’s commit index will be at least as large as any other
   183  	// > servers’ during its term.
   184  	term := n.raft.CurrentTerm()
   185  	// See the "runLeader" and "dispatchLogs" implementation (hashicorp raft)
   186  	// for details: when the leader is elected, it issues a noop, we only need
   187  	// to ensure that the entry is committed before we access the current
   188  	// commit index. This may incur substantial latency, if replicas are slow,
   189  	// but it's the only way to ensure that the leader has all committed
   190  	// entries. We also keep track of the current term to ensure that the
   191  	// leader has not changed while we were waiting for the noop to be
   192  	// committed and heartbeat messages to be exchanged.
   193  	if err := n.waitLastIndexCommitted(); err != nil {
   194  		return ReadIndex{}, err
   195  	}
   196  	commitIndex := n.raft.CommitIndex()
   197  	// > The leader needs to make sure it has not been superseded by a newer
   198  	// > leader of which it is unaware. It issues a new round of heartbeats
   199  	// > and waits for their acknowledgments from a majority of the cluster.
   200  	// > Once these acknowledgments are received, the leader knows that there
   201  	// > could not have existed a leader for a greater term at the moment it
   202  	// > sent the heartbeats. Thus, the readIndex was, at the time, the
   203  	// > largest commit index ever seen by any server in the cluster.
   204  	if err := n.raft.VerifyLeader().Error(); err != nil {
   205  		// The error includes details about the actual leader the request
   206  		// should be directed to; the client should retry the operation.
   207  		return ReadIndex{}, err
   208  	}
   209  	// The CommitIndex and leader heartbeats must be in the same term.
   210  	// Otherwise, we can't guarantee that this is the leader's commit index
   211  	// (mind the ABA problem), and thus, we can't guarantee completeness.
   212  	if n.raft.CurrentTerm() != term {
   213  		// There's a chance that the leader has changed since we've checked
   214  		// the leader status. The client should retry the operation, to
   215  		// ensure correctness of the read index.
   216  		return ReadIndex{}, raft.ErrLeadershipLost
   217  	}
   218  	// The node was the leader before we saved readIndex, and no elections
   219  	// have occurred while we were confirming leadership.
   220  	return ReadIndex{CommitIndex: commitIndex, Term: term}, nil
   221  }
   222  
   223  func (n *Node) waitLastIndexCommitted() error {
   224  	ctx, cancel := context.WithTimeout(context.Background(), n.config.ApplyTimeout)
   225  	defer cancel()
   226  	return waitIndexReached(ctx,
   227  		n.raft.CommitIndex,
   228  		n.raft.LastIndex(),
   229  		n.config.LogIndexCheckInterval,
   230  		int(n.config.ReadIndexMaxDistance),
   231  	)
   232  }
   233  
   234  // waitIndexReached blocks until a >= b.
   235  // If b - a >= maxDistance, the function return ErrLagBehind.
   236  // reached is guaranteed to be false, if err != nil.
   237  func waitIndexReached(
   238  	ctx context.Context,
   239  	src func() uint64,
   240  	dst uint64,
   241  	interval time.Duration,
   242  	maxDistance int,
   243  ) error {
   244  	if reached, err := compareIndex(src, dst, maxDistance); err != nil || reached {
   245  		return err
   246  	}
   247  	t := time.NewTicker(interval)
   248  	defer t.Stop()
   249  	for {
   250  		select {
   251  		case <-ctx.Done():
   252  			return ctx.Err()
   253  		case <-t.C:
   254  			if reached, err := compareIndex(src, dst, maxDistance); err != nil || reached {
   255  				return err
   256  			}
   257  		}
   258  	}
   259  }
   260  
   261  func compareIndex(src func() uint64, dst uint64, maxDistance int) (bool, error) {
   262  	cur := src()
   263  	if maxDistance > 0 {
   264  		if delta := int(dst) - int(cur); delta > maxDistance {
   265  			return false, ErrLagBehind
   266  		}
   267  	}
   268  	return cur >= dst, nil
   269  }