Skip to content

Commit

Permalink
Merge pull request #10903 from tbg/inflights
Browse files Browse the repository at this point in the history
raft: return non-nil Inflights in raft status
  • Loading branch information
tbg authored Jul 18, 2019
2 parents f498392 + 6b03225 commit 62f4fb3
Show file tree
Hide file tree
Showing 4 changed files with 63 additions and 30 deletions.
17 changes: 8 additions & 9 deletions raft/rawnode.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,18 +218,17 @@ func (rn *RawNode) Advance(rd Ready) {
rn.commitReady(rd)
}

// Status returns the current status of the given group.
func (rn *RawNode) Status() *Status {
// Status returns the current status of the given group. This allocates, see
// BasicStatus and WithProgress for allocation-friendlier choices.
func (rn *RawNode) Status() Status {
status := getStatus(rn.raft)
return &status
return status
}

// StatusWithoutProgress returns a Status without populating the Progress field
// (and returns the Status as a value to avoid forcing it onto the heap). This
// is more performant if the Progress is not required. See WithProgress for an
// allocation-free way to introspect the Progress.
func (rn *RawNode) StatusWithoutProgress() Status {
return getStatusWithoutProgress(rn.raft)
// BasicStatus returns a BasicStatus. Notably this does not contain the
// Progress map; see WithProgress for an allocation-free way to inspect it.
func (rn *RawNode) BasicStatus() BasicStatus {
return getBasicStatus(rn.raft)
}

// ProgressType indicates the type of replica a Progress corresponds to.
Expand Down
40 changes: 29 additions & 11 deletions raft/rawnode_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"reflect"
"testing"

"go.etcd.io/etcd/raft/quorum"
"go.etcd.io/etcd/raft/raftpb"
"go.etcd.io/etcd/raft/tracker"
)
Expand All @@ -43,7 +44,7 @@ func (a *rawNodeAdapter) TransferLeadership(ctx context.Context, lead, transfere
func (a *rawNodeAdapter) Stop() {}

// RawNode returns a *Status.
func (a *rawNodeAdapter) Status() Status { return *a.RawNode.Status() }
func (a *rawNodeAdapter) Status() Status { return a.RawNode.Status() }

// RawNode takes a Ready. It doesn't really have to do that I think? It can hold on
// to it internally. But maybe that approach is frail.
Expand Down Expand Up @@ -439,14 +440,33 @@ func TestRawNodeRestartFromSnapshot(t *testing.T) {
// no dependency check between Ready() and Advance()

func TestRawNodeStatus(t *testing.T) {
storage := NewMemoryStorage()
rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage), []Peer{{ID: 1}})
s := NewMemoryStorage()
rn, err := NewRawNode(newTestConfig(1, []uint64{1}, 10, 1, s), nil)
if err != nil {
t.Fatal(err)
}
status := rawNode.Status()
if status == nil {
t.Errorf("expected status struct, got nil")
if status := rn.Status(); status.Progress != nil {
t.Fatalf("expected no Progress because not leader: %+v", status.Progress)
}
if err := rn.Campaign(); err != nil {
t.Fatal(err)
}
status := rn.Status()
if status.Lead != 1 {
t.Fatal("not lead")
}
if status.RaftState != StateLeader {
t.Fatal("not leader")
}
if exp, act := *rn.raft.prs.Progress[1], status.Progress[1]; !reflect.DeepEqual(exp, act) {
t.Fatalf("want: %+v\ngot: %+v", exp, act)
}
expCfg := tracker.Config{Voters: quorum.JointConfig{
quorum.MajorityConfig{1: {}},
nil,
}}
if !reflect.DeepEqual(expCfg, status.Config) {
t.Fatalf("want: %+v\ngot: %+v", expCfg, status.Config)
}
}

Expand Down Expand Up @@ -590,7 +610,7 @@ func TestRawNodeBoundedLogGrowthWithPartition(t *testing.T) {
checkUncommitted(0)
}

func BenchmarkStatusProgress(b *testing.B) {
func BenchmarkStatus(b *testing.B) {
setup := func(members int) *RawNode {
peers := make([]uint64, members)
for i := range peers {
Expand All @@ -607,8 +627,6 @@ func BenchmarkStatusProgress(b *testing.B) {

for _, members := range []int{1, 3, 5, 100} {
b.Run(fmt.Sprintf("members=%d", members), func(b *testing.B) {
// NB: call getStatus through rn.Status because that incurs an additional
// allocation.
rn := setup(members)

b.Run("Status", func(b *testing.B) {
Expand All @@ -630,10 +648,10 @@ func BenchmarkStatusProgress(b *testing.B) {
}
})

b.Run("StatusWithoutProgress", func(b *testing.B) {
b.Run("BasicStatus", func(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = rn.StatusWithoutProgress()
_ = rn.BasicStatus()
}
})

Expand Down
28 changes: 18 additions & 10 deletions raft/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,22 @@ import (
"go.etcd.io/etcd/raft/tracker"
)

// Status contains information about this Raft peer and its view of the system.
// The Progress is only populated on the leader.
type Status struct {
BasicStatus
Config tracker.Config
Progress map[uint64]tracker.Progress
}

// BasicStatus contains basic information about the Raft peer. It does not allocate.
type BasicStatus struct {
ID uint64

pb.HardState
SoftState

Applied uint64
Progress map[uint64]tracker.Progress
Applied uint64

LeadTransferee uint64
}
Expand All @@ -37,19 +45,17 @@ func getProgressCopy(r *raft) map[uint64]tracker.Progress {
m := make(map[uint64]tracker.Progress)
r.prs.Visit(func(id uint64, pr *tracker.Progress) {
var p tracker.Progress
p, pr = *pr, nil /* avoid accidental reuse below */

// The inflight buffer is tricky to copy and besides, it isn't exposed
// to the client, so pretend it's nil.
p.Inflights = nil
p = *pr
p.Inflights = pr.Inflights.Clone()
pr = nil

m[id] = p
})
return m
}

func getStatusWithoutProgress(r *raft) Status {
s := Status{
func getBasicStatus(r *raft) BasicStatus {
s := BasicStatus{
ID: r.id,
LeadTransferee: r.leadTransferee,
}
Expand All @@ -61,10 +67,12 @@ func getStatusWithoutProgress(r *raft) Status {

// getStatus gets a copy of the current raft status.
func getStatus(r *raft) Status {
s := getStatusWithoutProgress(r)
var s Status
s.BasicStatus = getBasicStatus(r)
if s.RaftState == StateLeader {
s.Progress = getProgressCopy(r)
}
s.Config = r.prs.Config.Clone()
return s
}

Expand Down
8 changes: 8 additions & 0 deletions raft/tracker/inflights.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,14 @@ func NewInflights(size int) *Inflights {
}
}

// Clone returns an *Inflights that is identical to but shares no memory with
// the receiver.
func (in *Inflights) Clone() *Inflights {
ins := *in
ins.buffer = append([]uint64(nil), in.buffer...)
return &ins
}

// Add notifies the Inflights that a new message with the given index is being
// dispatched. Full() must be called prior to Add() to verify that there is room
// for one more message, and consecutive calls to add Add() must provide a
Expand Down

0 comments on commit 62f4fb3

Please sign in to comment.