From 58413cb2bc08441c534e0e8d18dd93759e41352d Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 15 Oct 2018 19:08:30 -0400 Subject: [PATCH] storage: replace remote proposal tracking with uncommitted log size protection This change reverts most of the non-testing code from 03b116f and f2f3fd2 and replaces it with use of the MaxUncommittedEntriesSize config. This configuration was added in etcd-io/etcd#10167 and provides protection against unbounded Raft log growth when a Raft group stops being able to commit entries. It makes proposals into Raft safer because proposers don't need to verify before the fact that the proposal isn't a duplicate that might be blowing up the size of the Raft group. By default, the configuration is set to double the Replica's proposal quota. The logic here is that the quotaPool should be responsible for throttling proposals in all cases except for unbounded Raft re-proposals because it queues efficiently instead of dropping proposals on the floor indiscriminately. Release note (bug fix): Fix a bug where Raft proposals could get stuck if forwarded to a leader who could not itself append a new entry to its log. --- pkg/base/config.go | 29 ++++ pkg/storage/client_raft_test.go | 7 +- pkg/storage/replica.go | 111 +------------ pkg/storage/replica_test.go | 145 ---------------- pkg/storage/storagepb/state.pb.go | 156 +++++++----------- pkg/storage/storagepb/state.proto | 1 - pkg/storage/store.go | 17 +- .../reports/containers/range/rangeTable.tsx | 4 +- 8 files changed, 108 insertions(+), 362 deletions(-) diff --git a/pkg/base/config.go b/pkg/base/config.go index ffe1a9108d54..278fafd85fef 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -446,6 +446,23 @@ type RaftConfig struct { // performing log truncations. RaftLogMaxSize int64 + // RaftProposalQuota controls the maximum aggregate size of Raft commands + // that a leader is allowed to propose concurrently. + // + // By default, the quota is set to a fraction of the RaftLogMaxSize. In + // doing so, we ensure all replicas have sufficiently up to date logs so + // that when the log gets truncated, the followers do not need + // non-preemptive snapshots. Changing this deserves care. Too low and + // everything comes to a grinding halt, too high and we're not really + // throttling anything (we'll still generate snapshots). + RaftProposalQuota int64 + + // RaftMaxUncommittedEntriesSize controls how large the uncommitted tail of + // the Raft log can grow. The limit is meant to provide protection against + // unbounded Raft log growth when quorum is lost and entries stop being + // committed but continue to be proposed. + RaftMaxUncommittedEntriesSize uint64 + // RaftMaxSizePerMsg controls how many Raft log entries the leader will send to // followers in a single MsgApp. RaftMaxSizePerMsg uint64 @@ -474,6 +491,18 @@ func (cfg *RaftConfig) SetDefaults() { if cfg.RaftLogMaxSize == 0 { cfg.RaftLogMaxSize = defaultRaftLogMaxSize } + if cfg.RaftProposalQuota == 0 { + // By default, set this to a fraction of RaftLogMaxSize. See comment + // above for the tradeoffs of setting this higher or lower. + cfg.RaftProposalQuota = cfg.RaftLogMaxSize / 4 + } + if cfg.RaftMaxUncommittedEntriesSize == 0 { + // By default, set this to twice the RaftProposalQuota. The logic here + // is that the quotaPool should be responsible for throttling proposals + // in all cases except for unbounded Raft re-proposals because it queues + // efficiently instead of dropping proposals on the floor indiscriminately. + cfg.RaftMaxUncommittedEntriesSize = uint64(2 * cfg.RaftProposalQuota) + } if cfg.RaftMaxSizePerMsg == 0 { cfg.RaftMaxSizePerMsg = uint64(defaultRaftMaxSizePerMsg) } diff --git a/pkg/storage/client_raft_test.go b/pkg/storage/client_raft_test.go index fb0ad9b08c98..fce86f5e7beb 100644 --- a/pkg/storage/client_raft_test.go +++ b/pkg/storage/client_raft_test.go @@ -1155,6 +1155,8 @@ func TestLogGrowthWhenRefreshingPendingCommands(t *testing.T) { sc.RaftTickInterval = 10 * time.Millisecond // Don't timeout raft leader. We don't want leadership moving. sc.RaftElectionTimeoutTicks = 1000000 + // Reduce the max uncommitted entry size. + sc.RaftMaxUncommittedEntriesSize = 64 << 10 // 64 KB // Disable leader transfers during leaseholder changes so that we // can easily create leader-not-leaseholder scenarios. sc.TestingKnobs.DisableLeaderFollowsLeaseholder = true @@ -1233,7 +1235,7 @@ func TestLogGrowthWhenRefreshingPendingCommands(t *testing.T) { // While a majority nodes are down, write some data. putRes := make(chan *roachpb.Error) go func() { - putArgs := putArgs([]byte("b"), make([]byte, 8<<10 /* 8 KB */)) + putArgs := putArgs([]byte("b"), make([]byte, sc.RaftMaxUncommittedEntriesSize/8)) _, err := client.SendWrapped(context.Background(), propNode, putArgs) putRes <- err }() @@ -1254,11 +1256,10 @@ func TestLogGrowthWhenRefreshingPendingCommands(t *testing.T) { } // Check raft log size. - const logSizeLimit = 64 << 10 // 64 KB curlogSize := leaderRepl.GetRaftLogSize() logSize := curlogSize - initLogSize logSizeStr := humanizeutil.IBytes(logSize) - if logSize > logSizeLimit { + if uint64(logSize) > sc.RaftMaxUncommittedEntriesSize { t.Fatalf("raft log size grew to %s", logSizeStr) } t.Logf("raft log size grew to %s", logSizeStr) diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index b8383c9d8385..b2958f576039 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -387,12 +387,7 @@ type Replica struct { // map must only be referenced while Replica.mu is held, except if the // element is removed from the map first. The notable exception is the // contained RaftCommand, which we treat as immutable. - localProposals map[storagebase.CmdIDKey]*ProposalData - // remoteProposals is maintained by Raft leaders and stores in-flight - // commands that were forwarded to the leader during its current term. - // The set allows leaders to detect duplicate forwarded commands and - // avoid re-proposing the same forwarded command multiple times. - remoteProposals map[storagebase.CmdIDKey]struct{} + localProposals map[storagebase.CmdIDKey]*ProposalData internalRaftGroup *raft.RawNode // The ID of the replica within the Raft group. May be 0 if the replica has // been created from a preemptive snapshot (i.e. before being added to the @@ -883,7 +878,6 @@ func (r *Replica) cancelPendingCommandsLocked() { r.cleanupFailedProposalLocked(p) p.finishApplication(pr) } - r.mu.remoteProposals = nil } // cleanupFailedProposalLocked cleans up after a proposal that has failed. It @@ -1118,22 +1112,12 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( log.Fatalf(ctx, "len(r.mu.commandSizes) = %d, expected 0", commandSizesLen) } - // We set the defaultProposalQuota to be less than RaftLogMaxSize, - // in doing so we ensure all replicas have sufficiently up to date - // logs so that when the log gets truncated, the followers do not - // need non-preemptive snapshots. Changing this deserves care. Too - // low and everything comes to a grinding halt, too high and we're - // not really throttling anything (we'll still generate snapshots). - // - // TODO(nvanbenschoten): clean this up in later commits. - proposalQuota := r.store.cfg.RaftLogMaxSize / 4 - // Raft may propose commands itself (specifically the empty // commands when leadership changes), and these commands don't go // through the code paths where we acquire quota from the pool. To // offset this we reset the quota pool whenever leadership changes // hands. - r.mu.proposalQuota = newQuotaPool(proposalQuota) + r.mu.proposalQuota = newQuotaPool(r.store.cfg.RaftProposalQuota) r.mu.lastUpdateTimes = make(map[roachpb.ReplicaID]time.Time) r.mu.commandSizes = make(map[storagebase.CmdIDKey]int) } else if r.mu.proposalQuota != nil { @@ -1913,7 +1897,6 @@ func (r *Replica) State() storagepb.RangeInfo { ri.ReplicaState = *(protoutil.Clone(&r.mu.state)).(*storagepb.ReplicaState) ri.LastIndex = r.mu.lastIndex ri.NumPending = uint64(len(r.mu.localProposals)) - ri.NumRemotePending = uint64(len(r.mu.remoteProposals)) ri.RaftLogSize = r.mu.raftLogSize ri.NumDropped = uint64(r.mu.droppedMessages) if r.mu.proposalQuota != nil { @@ -4042,20 +4025,7 @@ func (r *Replica) stepRaftGroup(req *RaftMessageRequest) error { // we expect the originator to campaign instead. r.unquiesceWithOptionsLocked(false /* campaignOnWake */) r.refreshLastUpdateTimeForReplicaLocked(req.FromReplica.ReplicaID) - - // Check if the message is a proposal that should be dropped. - if r.shouldDropForwardedProposalLocked(req) { - // If we could signal to the sender that its proposal was accepted - // or dropped then we wouldn't need to track anything. - return false /* unquiesceAndWakeLeader */, nil - } - err := raftGroup.Step(req.Message) - if err == nil { - // If we stepped successfully and the request is a proposal, consider - // tracking it so that we can ignore identical proposals in the future. - r.maybeTrackForwardedProposalLocked(raftGroup, req) - } if err == raft.ErrProposalDropped { // A proposal was forwarded to this replica but we couldn't propose it. // Swallow the error since we don't have an effective way of signaling @@ -4068,68 +4038,6 @@ func (r *Replica) stepRaftGroup(req *RaftMessageRequest) error { }) } -func (r *Replica) shouldDropForwardedProposalLocked(req *RaftMessageRequest) bool { - if req.Message.Type != raftpb.MsgProp { - // Not a proposal. - return false - } - - for _, e := range req.Message.Entries { - switch e.Type { - case raftpb.EntryNormal: - cmdID, _ := DecodeRaftCommand(e.Data) - if _, ok := r.mu.remoteProposals[cmdID]; !ok { - // Untracked remote proposal. Don't drop. - return false - } - case raftpb.EntryConfChange: - // Never drop EntryConfChange proposals. - return false - default: - log.Fatalf(context.TODO(), "unexpected Raft entry: %v", e) - } - } - // All entries tracked. - return true -} - -func (r *Replica) maybeTrackForwardedProposalLocked(rg *raft.RawNode, req *RaftMessageRequest) { - if req.Message.Type != raftpb.MsgProp { - // Not a proposal. - return - } - - if rg.Status().RaftState != raft.StateLeader { - // We're not the leader. We can't be sure that the proposal made it into - // the Raft log, so don't track it. - return - } - - // Record that each of the proposal's entries was seen and appended. This - // allows us to catch duplicate forwarded proposals in the future and - // prevent them from being repeatedly appended to a leader's raft log. - for _, e := range req.Message.Entries { - switch e.Type { - case raftpb.EntryNormal: - cmdID, data := DecodeRaftCommand(e.Data) - if len(data) == 0 { - // An empty command is proposed to unquiesce a range and - // wake the leader. Don't keep track of these forwarded - // proposals because they will never be cleaned up. - } else { - if r.mu.remoteProposals == nil { - r.mu.remoteProposals = map[storagebase.CmdIDKey]struct{}{} - } - r.mu.remoteProposals[cmdID] = struct{}{} - } - case raftpb.EntryConfChange: - // Don't track EntryConfChanges. - default: - log.Fatalf(context.TODO(), "unexpected Raft entry: %v", e) - } - } -} - type handleRaftReadyStats struct { processed int } @@ -4394,7 +4302,6 @@ func (r *Replica) handleRaftReadyRaftMuLocked( r.mu.leaderID = leaderID // Clear the remote proposal set. Would have been nil already if not // previously the leader. - r.mu.remoteProposals = nil becameLeader = r.mu.leaderID == r.mu.replicaID } r.mu.Unlock() @@ -4599,22 +4506,13 @@ func (r *Replica) tick(livenessMap IsLiveMap) (bool, error) { if knob := r.store.TestingKnobs().RefreshReasonTicksPeriod; knob > 0 { refreshAtDelta = knob } - if !r.store.TestingKnobs().DisableRefreshReasonTicks && - r.mu.replicaID != r.mu.leaderID && - r.mu.ticks%refreshAtDelta == 0 { + if !r.store.TestingKnobs().DisableRefreshReasonTicks && r.mu.ticks%refreshAtDelta == 0 { // RaftElectionTimeoutTicks is a reasonable approximation of how long we // should wait before deciding that our previous proposal didn't go // through. Note that the combination of the above condition and passing // RaftElectionTimeoutTicks to refreshProposalsLocked means that commands // will be refreshed when they have been pending for 1 to 2 election // cycles. - // - // However, we don't refresh proposals if we are the leader because - // doing so would be useless. The commands tracked by a leader replica - // were either all proposed when the replica was a leader or were - // re-proposed when the replica became a leader. Either way, they are - // guaranteed to be in the leader's Raft log so re-proposing won't do - // anything. r.refreshProposalsLocked(refreshAtDelta, reasonTicks) } return true, nil @@ -5407,9 +5305,6 @@ func (r *Replica) processRaftCommand( delete(r.mu.localProposals, idKey) } - // Delete the entry for a forwarded proposal set. - delete(r.mu.remoteProposals, idKey) - leaseIndex, proposalRetry, forcedErr := r.checkForcedErrLocked(ctx, idKey, raftCmd, proposal, proposedLocally) r.mu.Unlock() diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index a017dc9592b3..e37058b7854d 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -8489,151 +8489,6 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { } } -func TestReplicaShouldDropForwardedProposal(t *testing.T) { - defer leaktest.AfterTest(t)() - - cmdSeen, cmdNotSeen := makeIDKey(), makeIDKey() - data, noData := []byte("data"), []byte("") - - testCases := []struct { - name string - leader bool - msg raftpb.Message - expDrop bool - expRemotePropsAfter int - }{ - { - name: "new proposal", - leader: true, - msg: raftpb.Message{ - Type: raftpb.MsgProp, - Entries: []raftpb.Entry{ - {Type: raftpb.EntryNormal, Data: encodeRaftCommandV1(cmdNotSeen, data)}, - }, - }, - expDrop: false, - expRemotePropsAfter: 2, - }, - { - name: "duplicate proposal", - leader: true, - msg: raftpb.Message{ - Type: raftpb.MsgProp, - Entries: []raftpb.Entry{ - {Type: raftpb.EntryNormal, Data: encodeRaftCommandV1(cmdSeen, data)}, - }, - }, - expDrop: true, - expRemotePropsAfter: 1, - }, - { - name: "partially new proposal", - leader: true, - msg: raftpb.Message{ - Type: raftpb.MsgProp, - Entries: []raftpb.Entry{ - {Type: raftpb.EntryNormal, Data: encodeRaftCommandV1(cmdNotSeen, data)}, - {Type: raftpb.EntryNormal, Data: encodeRaftCommandV1(cmdSeen, data)}, - }, - }, - expDrop: false, - expRemotePropsAfter: 2, - }, - { - name: "empty proposal", - leader: true, - msg: raftpb.Message{ - Type: raftpb.MsgProp, - Entries: []raftpb.Entry{ - {Type: raftpb.EntryNormal, Data: encodeRaftCommandV1(cmdNotSeen, noData)}, - }, - }, - expDrop: false, - expRemotePropsAfter: 1, - }, - { - name: "conf change", - leader: true, - msg: raftpb.Message{ - Type: raftpb.MsgProp, - Entries: []raftpb.Entry{ - {Type: raftpb.EntryConfChange, Data: encodeRaftCommandV1(cmdNotSeen, data)}, - }, - }, - expDrop: false, - expRemotePropsAfter: 1, - }, - { - name: "non proposal", - leader: true, - msg: raftpb.Message{ - Type: raftpb.MsgApp, - }, - expDrop: false, - expRemotePropsAfter: 1, - }, - { - name: "not leader", - leader: false, - msg: raftpb.Message{ - Type: raftpb.MsgProp, - Entries: []raftpb.Entry{ - {Type: raftpb.EntryNormal, Data: encodeRaftCommandV1(cmdNotSeen, data)}, - }, - }, - expDrop: false, - expRemotePropsAfter: 0, - }, - } - for _, c := range testCases { - t.Run(c.name, func(t *testing.T) { - var tc testContext - stopper := stop.NewStopper() - defer stopper.Stop(context.TODO()) - tc.Start(t, stopper) - tc.repl.mu.Lock() - defer tc.repl.mu.Unlock() - - rg := tc.repl.mu.internalRaftGroup - if c.leader { - // Set the remoteProposals map to only contain cmdSeen. - tc.repl.mu.remoteProposals = map[storagebase.CmdIDKey]struct{}{ - cmdSeen: {}, - } - // Make sure the replica is the leader. - if s := rg.Status(); s.RaftState != raft.StateLeader { - t.Errorf("Replica not leader: %v", s) - } - } else { - // Clear the remoteProposals map. - tc.repl.mu.remoteProposals = nil - // Force the replica to step down as the leader by sending it a - // heartbeat at a high term. - if err := rg.Step(raftpb.Message{ - Type: raftpb.MsgHeartbeat, - Term: 999, - }); err != nil { - t.Error(err) - } - if s := rg.Status(); s.RaftState != raft.StateFollower { - t.Errorf("Replica not follower: %v", s) - } - } - - req := &RaftMessageRequest{Message: c.msg} - drop := tc.repl.shouldDropForwardedProposalLocked(req) - if c.expDrop != drop { - t.Errorf("expected drop=%t, found %t", c.expDrop, drop) - } - - tc.repl.maybeTrackForwardedProposalLocked(rg, req) - if l := len(tc.repl.mu.remoteProposals); c.expRemotePropsAfter != l { - t.Errorf("expected %d tracked remote proposals, found %d", c.expRemotePropsAfter, l) - } - }) - } -} - // checkValue asserts that the value for a key is the expected one. // The function will attempt to resolve the intent present on the key, if any. func checkValue(ctx context.Context, tc *testContext, key []byte, expectedVal []byte) error { diff --git a/pkg/storage/storagepb/state.pb.go b/pkg/storage/storagepb/state.pb.go index 9b54b101a2df..dc11483f05f5 100644 --- a/pkg/storage/storagepb/state.pb.go +++ b/pkg/storage/storagepb/state.pb.go @@ -84,10 +84,9 @@ func (*ReplicaState) Descriptor() ([]byte, []int) { return fileDescriptorState, type RangeInfo struct { ReplicaState `protobuf:"bytes,1,opt,name=state,embedded=state" json:"state"` // The highest (and last) index in the Raft log. - LastIndex uint64 `protobuf:"varint,2,opt,name=last_index,json=lastIndex,proto3" json:"last_index,omitempty"` - NumPending uint64 `protobuf:"varint,3,opt,name=num_pending,json=numPending,proto3" json:"num_pending,omitempty"` - NumRemotePending uint64 `protobuf:"varint,9,opt,name=num_remote_pending,json=numRemotePending,proto3" json:"num_remote_pending,omitempty"` - NumDropped uint64 `protobuf:"varint,5,opt,name=num_dropped,json=numDropped,proto3" json:"num_dropped,omitempty"` + LastIndex uint64 `protobuf:"varint,2,opt,name=last_index,json=lastIndex,proto3" json:"last_index,omitempty"` + NumPending uint64 `protobuf:"varint,3,opt,name=num_pending,json=numPending,proto3" json:"num_pending,omitempty"` + NumDropped uint64 `protobuf:"varint,5,opt,name=num_dropped,json=numDropped,proto3" json:"num_dropped,omitempty"` // raft_log_size may be initially inaccurate after a server restart. // See storage.Replica.mu.raftLogSize. RaftLogSize int64 `protobuf:"varint,6,opt,name=raft_log_size,json=raftLogSize,proto3" json:"raft_log_size,omitempty"` @@ -216,9 +215,6 @@ func (this *RangeInfo) Equal(that interface{}) bool { if this.NumPending != that1.NumPending { return false } - if this.NumRemotePending != that1.NumRemotePending { - return false - } if this.NumDropped != that1.NumDropped { return false } @@ -384,11 +380,6 @@ func (m *RangeInfo) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintState(dAtA, i, uint64(m.RangeMaxBytes)) } - if m.NumRemotePending != 0 { - dAtA[i] = 0x48 - i++ - i = encodeVarintState(dAtA, i, uint64(m.NumRemotePending)) - } return i, nil } @@ -620,9 +611,6 @@ func (m *RangeInfo) Size() (n int) { if m.RangeMaxBytes != 0 { n += 1 + sovState(uint64(m.RangeMaxBytes)) } - if m.NumRemotePending != 0 { - n += 1 + sovState(uint64(m.NumRemotePending)) - } return n } @@ -1173,25 +1161,6 @@ func (m *RangeInfo) Unmarshal(dAtA []byte) error { break } } - case 9: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NumRemotePending", wireType) - } - m.NumRemotePending = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.NumRemotePending |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skipState(dAtA[iNdEx:]) @@ -1864,64 +1833,63 @@ var ( func init() { proto.RegisterFile("storage/storagepb/state.proto", fileDescriptorState) } var fileDescriptorState = []byte{ - // 944 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x55, 0x3f, 0x6f, 0xdb, 0x46, - 0x1c, 0x35, 0x45, 0xc9, 0x96, 0x8e, 0x49, 0xac, 0x5e, 0x93, 0x86, 0x50, 0x60, 0x49, 0x50, 0xd1, - 0x42, 0x05, 0x52, 0x0a, 0x70, 0xff, 0xa0, 0x30, 0xba, 0x44, 0x4a, 0x11, 0x38, 0x71, 0x0a, 0xe7, - 0x64, 0x74, 0xe8, 0x42, 0x9c, 0xc8, 0x33, 0x45, 0xf8, 0x78, 0x77, 0x39, 0x1e, 0x03, 0xc9, 0x1f, - 0xa0, 0x73, 0x3f, 0x42, 0xf7, 0x7e, 0x86, 0xee, 0x1e, 0x33, 0x76, 0x12, 0x5a, 0x75, 0xc9, 0xdc, - 0xb5, 0x4b, 0x71, 0x47, 0x52, 0x96, 0x5c, 0x23, 0x0d, 0x3c, 0x64, 0x12, 0xf9, 0x7e, 0xef, 0xf7, - 0xef, 0xdd, 0x3b, 0x0a, 0xec, 0xa5, 0x8a, 0x4b, 0x1c, 0x91, 0x41, 0xf1, 0x2b, 0x26, 0x83, 0x54, - 0x61, 0x45, 0x3c, 0x21, 0xb9, 0xe2, 0xf0, 0x41, 0xc0, 0x83, 0x33, 0xc9, 0x71, 0x30, 0xf5, 0x0a, - 0x82, 0xb7, 0x22, 0xb6, 0x7a, 0x65, 0x2e, 0x61, 0x51, 0xcc, 0xca, 0x1f, 0x31, 0x19, 0x24, 0xaf, - 0x82, 0x20, 0x2f, 0xd0, 0x7a, 0x60, 0x92, 0xc5, 0x64, 0x10, 0x33, 0x45, 0x24, 0xc3, 0xd4, 0x97, - 0xf8, 0x54, 0x15, 0xc1, 0x8f, 0xca, 0x60, 0x42, 0x14, 0x0e, 0xb1, 0xc2, 0x05, 0x0e, 0x4b, 0x7c, - 0x0d, 0x73, 0x33, 0x15, 0xd3, 0xc1, 0x94, 0x06, 0x03, 0x15, 0x27, 0x24, 0x55, 0x38, 0x11, 0x45, - 0xe4, 0x6e, 0xc4, 0x23, 0x6e, 0x1e, 0x07, 0xfa, 0x29, 0x47, 0x7b, 0xbf, 0xd6, 0xc0, 0x2d, 0x44, - 0x04, 0x8d, 0x03, 0x3c, 0xd6, 0x0b, 0xc1, 0x87, 0x00, 0xea, 0xd6, 0x3e, 0x16, 0x82, 0xc6, 0x24, - 0xf4, 0x63, 0x16, 0x92, 0x99, 0x6b, 0x75, 0xad, 0x7e, 0x15, 0x35, 0x75, 0xe4, 0x51, 0x1e, 0x38, - 0xd4, 0x38, 0xf4, 0xc0, 0x87, 0x94, 0xe0, 0x94, 0x5c, 0xa1, 0x57, 0x0c, 0xfd, 0x03, 0x13, 0xda, - 0xe0, 0x7f, 0x0d, 0xaa, 0x21, 0x49, 0x03, 0xd7, 0xee, 0x5a, 0x7d, 0x67, 0xbf, 0xe7, 0x5d, 0xea, - 0x56, 0xec, 0xe2, 0x21, 0xcc, 0x22, 0xf2, 0x98, 0xa4, 0x81, 0x8c, 0x85, 0xe2, 0x12, 0x19, 0x3e, - 0xf4, 0x40, 0xcd, 0x14, 0x73, 0xab, 0x26, 0xd1, 0xbd, 0x26, 0xf1, 0x48, 0xc7, 0x51, 0x4e, 0x83, - 0xdf, 0x83, 0x5d, 0x25, 0x33, 0x16, 0x60, 0x45, 0x42, 0xdf, 0x9c, 0x94, 0x5b, 0x33, 0x99, 0x9f, - 0x5c, 0xdb, 0xf2, 0x54, 0x9d, 0x94, 0x6c, 0xa3, 0x02, 0xba, 0xa3, 0x36, 0xde, 0x21, 0x07, 0xb7, - 0xa2, 0xc0, 0x57, 0x53, 0x49, 0xd2, 0x29, 0xa7, 0xa1, 0xbb, 0x6d, 0x8a, 0xed, 0xad, 0x15, 0xd3, - 0xba, 0x7b, 0x53, 0x1a, 0x78, 0x27, 0xa5, 0xee, 0xc3, 0xcf, 0x97, 0x8b, 0x8e, 0xf3, 0x64, 0x74, - 0x52, 0x66, 0xfd, 0xbd, 0xe8, 0xb4, 0x56, 0x09, 0xe1, 0xe4, 0xa0, 0x27, 0x31, 0x0b, 0x59, 0x46, - 0x29, 0x9e, 0x50, 0xd2, 0x43, 0x4e, 0x14, 0xac, 0xa8, 0x70, 0x08, 0x6a, 0x7a, 0xec, 0xd4, 0xdd, - 0x31, 0x9d, 0x1e, 0x7a, 0xff, 0x75, 0x58, 0xee, 0x23, 0xaf, 0xb4, 0x93, 0xf7, 0xfc, 0x87, 0xd1, - 0x48, 0x4f, 0x9b, 0xa2, 0x3c, 0x15, 0xfe, 0x64, 0x81, 0x7b, 0x6a, 0xc6, 0xfc, 0x54, 0x60, 0xe6, - 0x6f, 0x8c, 0xdf, 0x78, 0x97, 0xf1, 0xbf, 0x5c, 0x2e, 0x3a, 0xf0, 0x64, 0xc6, 0xc6, 0x02, 0xb3, - 0x77, 0xdf, 0x02, 0xaa, 0x22, 0x63, 0x6d, 0x99, 0xaf, 0xc0, 0xfd, 0x2c, 0x8d, 0x59, 0xb4, 0x72, - 0x89, 0x39, 0x11, 0xff, 0x8c, 0xcc, 0x5d, 0xa7, 0x6b, 0xf5, 0xeb, 0xe8, 0xae, 0x09, 0x17, 0x4e, - 0x31, 0x8a, 0x3f, 0x23, 0xf3, 0x83, 0xea, 0x9b, 0x5f, 0x3a, 0xd6, 0xd3, 0x6a, 0xbd, 0xde, 0x6c, - 0x3c, 0xad, 0xd6, 0x41, 0xd3, 0xe9, 0xfd, 0x53, 0x01, 0x0d, 0x63, 0x90, 0x43, 0x76, 0xca, 0xe1, - 0x61, 0xae, 0x11, 0x31, 0xee, 0x74, 0xf6, 0x3f, 0xf3, 0xde, 0x72, 0x0b, 0xbd, 0x75, 0x93, 0x0f, - 0xeb, 0x17, 0x8b, 0xce, 0xd6, 0xeb, 0x45, 0xc7, 0xca, 0xa5, 0x22, 0x70, 0x0f, 0x00, 0x8a, 0x53, - 0xb5, 0x61, 0xdf, 0x86, 0x46, 0x72, 0xdb, 0x76, 0x80, 0xc3, 0xb2, 0xc4, 0x17, 0x84, 0x85, 0x31, - 0x8b, 0x8c, 0x7b, 0xab, 0x08, 0xb0, 0x2c, 0x39, 0xce, 0x91, 0x92, 0x10, 0x4a, 0x2e, 0x04, 0x09, - 0x8d, 0xd7, 0x72, 0xc2, 0xe3, 0x1c, 0x81, 0x3d, 0x70, 0xdb, 0x5c, 0x2b, 0xca, 0x23, 0x3f, 0x8d, - 0xcf, 0x89, 0x71, 0x90, 0x8d, 0x1c, 0x0d, 0x1e, 0xf1, 0x68, 0x1c, 0x9f, 0x13, 0xf8, 0x2d, 0x68, - 0x61, 0x21, 0x24, 0x9f, 0xc5, 0x89, 0x96, 0x47, 0x48, 0x2e, 0x78, 0x8a, 0xa9, 0xff, 0x32, 0xe3, - 0x0a, 0x1b, 0x23, 0xd8, 0xc8, 0x5d, 0x63, 0x1c, 0x17, 0x84, 0x17, 0x3a, 0x0e, 0x3f, 0x05, 0xbb, - 0x52, 0x4b, 0xe3, 0x27, 0x78, 0xe6, 0x4f, 0xe6, 0x8a, 0xa4, 0x6e, 0xdd, 0xa4, 0xdc, 0x36, 0xf0, - 0x73, 0x3c, 0x1b, 0x6a, 0x50, 0x5f, 0x70, 0x3d, 0xaa, 0x24, 0x09, 0xd7, 0x4d, 0x8a, 0x95, 0x1a, - 0xf9, 0x05, 0x67, 0x59, 0x82, 0x4c, 0xa0, 0x58, 0x6c, 0x75, 0x06, 0xd5, 0x66, 0xad, 0xf7, 0xa6, - 0x06, 0xee, 0x8d, 0x78, 0x92, 0x60, 0x16, 0xbe, 0xc8, 0x48, 0x46, 0xd2, 0x31, 0xc3, 0x22, 0x9d, - 0x72, 0x05, 0x1f, 0x81, 0xc6, 0xea, 0x73, 0x53, 0x9c, 0xc6, 0xff, 0x98, 0xab, 0xaa, 0x4f, 0x00, - 0x5d, 0x66, 0xc1, 0x29, 0x00, 0x94, 0x07, 0x98, 0x8e, 0x03, 0x2e, 0x88, 0x5b, 0xe9, 0xda, 0x7d, - 0x67, 0x7f, 0xf8, 0xd6, 0x13, 0xbd, 0x76, 0x14, 0xef, 0x68, 0x55, 0xe4, 0x3b, 0xa6, 0xe4, 0xbc, - 0x68, 0xb4, 0x56, 0x1b, 0x9e, 0x01, 0x27, 0xa2, 0x7c, 0x52, 0xb6, 0xb2, 0x4d, 0xab, 0xd1, 0x0d, - 0x5a, 0x3d, 0xb9, 0xac, 0xb2, 0xde, 0x6b, 0xbd, 0x7a, 0xeb, 0x37, 0x0b, 0xec, 0x14, 0xd9, 0xf0, - 0x0e, 0xa8, 0xc4, 0xa1, 0x91, 0xc7, 0x46, 0x95, 0x38, 0x84, 0x4d, 0x60, 0xeb, 0x2b, 0xa0, 0xdd, - 0xd6, 0x40, 0xfa, 0x11, 0xde, 0x07, 0x3b, 0x84, 0x85, 0xe6, 0x62, 0xd8, 0x06, 0xdd, 0x26, 0x2c, - 0x7c, 0x46, 0xe6, 0xb0, 0x05, 0xea, 0x92, 0xe0, 0x90, 0x33, 0x3a, 0x37, 0x9f, 0xc0, 0x3a, 0x5a, - 0xbd, 0x6f, 0x8a, 0x5f, 0xbb, 0x91, 0xf8, 0x2e, 0xd8, 0x11, 0x92, 0x48, 0xf2, 0x32, 0x75, 0xb7, - 0xbb, 0x76, 0xdf, 0x46, 0xe5, 0x6b, 0x6b, 0x0e, 0x76, 0xaf, 0x28, 0x5a, 0x8e, 0x9d, 0xef, 0x61, - 0xc6, 0x3e, 0x06, 0xb5, 0x57, 0x98, 0x66, 0xc4, 0xac, 0xe2, 0xec, 0x1f, 0xdc, 0x40, 0xcb, 0x02, - 0x45, 0x79, 0xa1, 0x83, 0xca, 0x37, 0x56, 0xeb, 0x1c, 0x34, 0xaf, 0x2a, 0xfc, 0xbe, 0x7a, 0x0f, - 0x3f, 0xbe, 0xf8, 0xb3, 0xbd, 0x75, 0xb1, 0x6c, 0x5b, 0xaf, 0x97, 0x6d, 0xeb, 0xf7, 0x65, 0xdb, - 0xfa, 0x63, 0xd9, 0xb6, 0x7e, 0xfe, 0xab, 0xbd, 0xf5, 0x63, 0x63, 0x55, 0x6d, 0xb2, 0x6d, 0xfe, - 0x42, 0xbf, 0xf8, 0x37, 0x00, 0x00, 0xff, 0xff, 0xf9, 0xc1, 0x7f, 0xb8, 0x1d, 0x08, 0x00, 0x00, + // 925 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x55, 0x4d, 0x6f, 0xdb, 0x46, + 0x10, 0x35, 0x45, 0xc9, 0x96, 0x96, 0x49, 0xac, 0x6e, 0x93, 0x86, 0x50, 0x60, 0x49, 0x50, 0xd1, + 0x42, 0x05, 0x52, 0x0a, 0x70, 0x3f, 0x50, 0x18, 0xbd, 0x44, 0x4a, 0x11, 0x38, 0x71, 0x0a, 0x87, + 0x32, 0x7a, 0xe8, 0x85, 0x58, 0x91, 0x63, 0x8a, 0xf0, 0x72, 0x77, 0x43, 0x2e, 0x03, 0xc9, 0x3f, + 0xa0, 0xe7, 0xfe, 0x84, 0xde, 0xfb, 0x1b, 0x7a, 0xf7, 0xa5, 0x40, 0x8e, 0x3d, 0x09, 0xad, 0x7a, + 0xc9, 0xb9, 0xbf, 0xa0, 0xd8, 0x25, 0x29, 0x4b, 0xae, 0x91, 0x06, 0x3e, 0xe4, 0x24, 0xf2, 0xcd, + 0x7b, 0x33, 0xb3, 0x8f, 0x8f, 0x14, 0xda, 0x4b, 0x25, 0x4f, 0x48, 0x08, 0x83, 0xe2, 0x57, 0x4c, + 0x06, 0xa9, 0x24, 0x12, 0x1c, 0x91, 0x70, 0xc9, 0xf1, 0x03, 0x9f, 0xfb, 0x67, 0x09, 0x27, 0xfe, + 0xd4, 0x29, 0x08, 0xce, 0x8a, 0xd8, 0xea, 0x95, 0x5a, 0x60, 0x61, 0xc4, 0xca, 0x1f, 0x31, 0x19, + 0xc4, 0xaf, 0x7c, 0x3f, 0x6f, 0xd0, 0x7a, 0xa0, 0xc5, 0x62, 0x32, 0x88, 0x98, 0x84, 0x84, 0x11, + 0xea, 0x25, 0xe4, 0x54, 0x16, 0xc5, 0x8f, 0xca, 0x62, 0x0c, 0x92, 0x04, 0x44, 0x92, 0x02, 0xc7, + 0x25, 0xbe, 0x86, 0xd9, 0x99, 0x8c, 0xe8, 0x60, 0x4a, 0xfd, 0x81, 0x8c, 0x62, 0x48, 0x25, 0x89, + 0x45, 0x51, 0xb9, 0x1b, 0xf2, 0x90, 0xeb, 0xcb, 0x81, 0xba, 0xca, 0xd1, 0xde, 0xaf, 0x35, 0x74, + 0xcb, 0x05, 0x41, 0x23, 0x9f, 0x8c, 0xd5, 0x81, 0xf0, 0x43, 0x84, 0xd5, 0x68, 0x8f, 0x08, 0x41, + 0x23, 0x08, 0xbc, 0x88, 0x05, 0x30, 0xb3, 0x8d, 0xae, 0xd1, 0xaf, 0xba, 0x4d, 0x55, 0x79, 0x94, + 0x17, 0x0e, 0x15, 0x8e, 0x1d, 0xf4, 0x21, 0x05, 0x92, 0xc2, 0x15, 0x7a, 0x45, 0xd3, 0x3f, 0xd0, + 0xa5, 0x0d, 0xfe, 0xd7, 0xa8, 0x1a, 0x40, 0xea, 0xdb, 0x66, 0xd7, 0xe8, 0x5b, 0xfb, 0x3d, 0xe7, + 0xd2, 0xb7, 0xe2, 0x2c, 0x8e, 0x4b, 0x58, 0x08, 0x8f, 0x21, 0xf5, 0x93, 0x48, 0x48, 0x9e, 0xb8, + 0x9a, 0x8f, 0x1d, 0x54, 0xd3, 0xcd, 0xec, 0xaa, 0x16, 0xda, 0xd7, 0x08, 0x8f, 0x54, 0xdd, 0xcd, + 0x69, 0xf8, 0x7b, 0xb4, 0x2b, 0x93, 0x8c, 0xf9, 0x44, 0x42, 0xe0, 0xe9, 0x27, 0x65, 0xd7, 0xb4, + 0xf2, 0x93, 0x6b, 0x47, 0x9e, 0xca, 0x93, 0x92, 0xad, 0x5d, 0x70, 0xef, 0xc8, 0x8d, 0x7b, 0xcc, + 0xd1, 0xad, 0xd0, 0xf7, 0xe4, 0x34, 0x81, 0x74, 0xca, 0x69, 0x60, 0x6f, 0xeb, 0x66, 0x7b, 0x6b, + 0xcd, 0x94, 0xef, 0xce, 0x94, 0xfa, 0xce, 0x49, 0xe9, 0xfb, 0xf0, 0xf3, 0xe5, 0xa2, 0x63, 0x3d, + 0x19, 0x9d, 0x94, 0xaa, 0x7f, 0x16, 0x9d, 0xd6, 0x4a, 0x10, 0x4c, 0x0e, 0x7a, 0x09, 0x61, 0x01, + 0xcb, 0x28, 0x25, 0x13, 0x0a, 0x3d, 0xd7, 0x0a, 0xfd, 0x15, 0x15, 0x0f, 0x51, 0x4d, 0xad, 0x9d, + 0xda, 0x3b, 0x7a, 0xd2, 0x43, 0xe7, 0xbf, 0x09, 0xcb, 0x73, 0xe4, 0x94, 0x71, 0x72, 0x9e, 0xff, + 0x30, 0x1a, 0xa9, 0x6d, 0x53, 0x37, 0x97, 0xe2, 0x9f, 0x0c, 0x74, 0x4f, 0xce, 0x98, 0x97, 0x0a, + 0xc2, 0xbc, 0x8d, 0xf5, 0x1b, 0xef, 0xb2, 0xfe, 0x97, 0xcb, 0x45, 0x07, 0x9f, 0xcc, 0xd8, 0x58, + 0x10, 0xf6, 0xee, 0xa7, 0xc0, 0xb2, 0x50, 0xac, 0x1d, 0xe6, 0x2b, 0x74, 0x3f, 0x4b, 0x23, 0x16, + 0xae, 0x52, 0xa2, 0x9f, 0x88, 0x77, 0x06, 0x73, 0xdb, 0xea, 0x1a, 0xfd, 0xba, 0x7b, 0x57, 0x97, + 0x8b, 0xa4, 0x68, 0xc7, 0x9f, 0xc1, 0xfc, 0xa0, 0xfa, 0xe6, 0x97, 0x8e, 0xf1, 0xb4, 0x5a, 0xaf, + 0x37, 0x1b, 0x4f, 0xab, 0x75, 0xd4, 0xb4, 0x7a, 0xbf, 0x57, 0x50, 0x43, 0x07, 0xe4, 0x90, 0x9d, + 0x72, 0x7c, 0x98, 0x7b, 0x04, 0x3a, 0x9d, 0xd6, 0xfe, 0x67, 0xce, 0x5b, 0xde, 0x42, 0x67, 0x3d, + 0xe4, 0xc3, 0xfa, 0xc5, 0xa2, 0xb3, 0xf5, 0x7a, 0xd1, 0x31, 0x72, 0xab, 0x00, 0xef, 0x21, 0x44, + 0x49, 0x2a, 0x37, 0xe2, 0xdb, 0x50, 0x48, 0x1e, 0xdb, 0x0e, 0xb2, 0x58, 0x16, 0x7b, 0x02, 0x58, + 0x10, 0xb1, 0x50, 0xa7, 0xb7, 0xea, 0x22, 0x96, 0xc5, 0xc7, 0x39, 0x52, 0x12, 0x82, 0x84, 0x0b, + 0x01, 0x81, 0xce, 0x5a, 0x4e, 0x78, 0x9c, 0x23, 0xb8, 0x87, 0x6e, 0xeb, 0xd7, 0x8a, 0xf2, 0xd0, + 0x4b, 0xa3, 0x73, 0xd0, 0x09, 0x32, 0x5d, 0x4b, 0x81, 0x47, 0x3c, 0x1c, 0x47, 0xe7, 0x80, 0xbf, + 0x45, 0x2d, 0x22, 0x44, 0xc2, 0x67, 0x51, 0xac, 0xec, 0x11, 0x09, 0x17, 0x3c, 0x25, 0xd4, 0x7b, + 0x99, 0x71, 0x49, 0x74, 0x10, 0x4c, 0xd7, 0x5e, 0x63, 0x1c, 0x17, 0x84, 0x17, 0xaa, 0x8e, 0x3f, + 0x45, 0xbb, 0x89, 0xb2, 0xc6, 0x8b, 0xc9, 0xcc, 0x9b, 0xcc, 0x25, 0xa4, 0x76, 0x5d, 0x4b, 0x6e, + 0x6b, 0xf8, 0x39, 0x99, 0x0d, 0x15, 0xb8, 0x72, 0xb5, 0xda, 0xac, 0xf5, 0xde, 0xd4, 0xd0, 0xbd, + 0x11, 0x8f, 0x63, 0xc2, 0x82, 0x17, 0x19, 0x64, 0x90, 0x8e, 0x19, 0x11, 0xe9, 0x94, 0x4b, 0xfc, + 0x08, 0x35, 0x56, 0x1f, 0x90, 0xc2, 0xdf, 0xff, 0x89, 0x4b, 0x55, 0x79, 0xea, 0x5e, 0xaa, 0xf0, + 0x14, 0x21, 0xca, 0x7d, 0x42, 0xc7, 0x3e, 0x17, 0x60, 0x57, 0xba, 0x66, 0xdf, 0xda, 0x1f, 0xbe, + 0xf5, 0x19, 0x5d, 0xbb, 0x8a, 0x73, 0xb4, 0x6a, 0xf2, 0x1d, 0x93, 0xc9, 0xbc, 0x18, 0xb4, 0xd6, + 0x1b, 0x9f, 0x21, 0x2b, 0xa4, 0x7c, 0x52, 0x8e, 0x32, 0xf5, 0xa8, 0xd1, 0x0d, 0x46, 0x3d, 0xb9, + 0xec, 0xb2, 0x3e, 0x6b, 0xbd, 0x7b, 0xeb, 0x37, 0x03, 0xed, 0x14, 0x6a, 0x7c, 0x07, 0x55, 0xa2, + 0x40, 0xdb, 0x63, 0xba, 0x95, 0x28, 0xc0, 0x4d, 0x64, 0xaa, 0x50, 0xab, 0xfc, 0x34, 0x5c, 0x75, + 0x89, 0xef, 0xa3, 0x1d, 0x60, 0x81, 0x8e, 0xba, 0xa9, 0xd1, 0x6d, 0x60, 0xc1, 0x33, 0x98, 0xe3, + 0x16, 0xaa, 0x27, 0x40, 0x02, 0xce, 0xe8, 0x5c, 0x7f, 0xd4, 0xea, 0xee, 0xea, 0x7e, 0xd3, 0xfc, + 0xda, 0x8d, 0xcc, 0xb7, 0xd1, 0x8e, 0x48, 0x20, 0x81, 0x97, 0xa9, 0xbd, 0xdd, 0x35, 0xfb, 0xa6, + 0x5b, 0xde, 0xb6, 0xe6, 0x68, 0xf7, 0x8a, 0xa3, 0xe5, 0xda, 0xf9, 0x39, 0xf4, 0xda, 0xc7, 0xa8, + 0xf6, 0x8a, 0xd0, 0x0c, 0xf4, 0x51, 0xac, 0xfd, 0x83, 0x1b, 0x78, 0x59, 0xa0, 0x6e, 0xde, 0xe8, + 0xa0, 0xf2, 0x8d, 0xd1, 0x3a, 0x47, 0xcd, 0xab, 0x0e, 0xbf, 0xaf, 0xd9, 0xc3, 0x8f, 0x2f, 0xfe, + 0x6a, 0x6f, 0x5d, 0x2c, 0xdb, 0xc6, 0xeb, 0x65, 0xdb, 0xf8, 0x63, 0xd9, 0x36, 0xfe, 0x5c, 0xb6, + 0x8d, 0x9f, 0xff, 0x6e, 0x6f, 0xfd, 0xd8, 0x58, 0x75, 0x9b, 0x6c, 0xeb, 0x3f, 0xc5, 0x2f, 0xfe, + 0x0d, 0x00, 0x00, 0xff, 0xff, 0xe7, 0x76, 0xbe, 0x31, 0xef, 0x07, 0x00, 0x00, } diff --git a/pkg/storage/storagepb/state.proto b/pkg/storage/storagepb/state.proto index 3d74ecac6332..e15a9a382123 100644 --- a/pkg/storage/storagepb/state.proto +++ b/pkg/storage/storagepb/state.proto @@ -90,7 +90,6 @@ message RangeInfo { // The highest (and last) index in the Raft log. uint64 last_index = 2; uint64 num_pending = 3; - uint64 num_remote_pending = 9; reserved 4; // previously last verification timestamp for verify queue. uint64 num_dropped = 5; // raft_log_size may be initially inaccurate after a server restart. diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 2a7108ebe3e0..b15472a0be18 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -167,14 +167,15 @@ func newRaftConfig( strg raft.Storage, id uint64, appliedIndex uint64, storeCfg StoreConfig, logger raft.Logger, ) *raft.Config { return &raft.Config{ - ID: id, - Applied: appliedIndex, - ElectionTick: storeCfg.RaftElectionTimeoutTicks, - HeartbeatTick: storeCfg.RaftHeartbeatIntervalTicks, - MaxSizePerMsg: storeCfg.RaftMaxSizePerMsg, - MaxInflightMsgs: storeCfg.RaftMaxInflightMsgs, - Storage: strg, - Logger: logger, + ID: id, + Applied: appliedIndex, + ElectionTick: storeCfg.RaftElectionTimeoutTicks, + HeartbeatTick: storeCfg.RaftHeartbeatIntervalTicks, + MaxUncommittedEntriesSize: storeCfg.RaftMaxUncommittedEntriesSize, + MaxSizePerMsg: storeCfg.RaftMaxSizePerMsg, + MaxInflightMsgs: storeCfg.RaftMaxInflightMsgs, + Storage: strg, + Logger: logger, PreVote: true, } diff --git a/pkg/ui/src/views/reports/containers/range/rangeTable.tsx b/pkg/ui/src/views/reports/containers/range/rangeTable.tsx index 5b5bfbac007e..8931a7316365 100644 --- a/pkg/ui/src/views/reports/containers/range/rangeTable.tsx +++ b/pkg/ui/src/views/reports/containers/range/rangeTable.tsx @@ -74,8 +74,7 @@ const rangeTableDisplayList: RangeTableRow[] = [ { variable: "leaseHolderQPS", display: "Lease Holder QPS", compareToLeader: false }, { variable: "keysWrittenPS", display: "Average Keys Written Per Second", compareToLeader: false }, { variable: "approxProposalQuota", display: "Approx Proposal Quota", compareToLeader: false }, - { variable: "pendingCommands", display: "Pending Local Commands", compareToLeader: false }, - { variable: "remoteCommands", display: "Pending Remote Commands", compareToLeader: false }, + { variable: "pendingCommands", display: "Pending Commands", compareToLeader: false }, { variable: "droppedCommands", display: "Dropped Commands", compareToLeader: false }, { variable: "truncatedIndex", display: "Truncated Index", compareToLeader: true }, { variable: "truncatedTerm", display: "Truncated Term", compareToLeader: true }, @@ -500,7 +499,6 @@ export default class RangeTable extends React.Component { keysWrittenPS: this.createContent(info.stats.writes_per_second.toFixed(4)), approxProposalQuota: raftLeader ? this.createContent(FixLong(info.state.approximate_proposal_quota)) : rangeTableEmptyContent, pendingCommands: this.createContent(FixLong(info.state.num_pending)), - remoteCommands: raftLeader ? this.createContent(FixLong(info.state.num_remote_pending)) : rangeTableEmptyContent, droppedCommands: this.createContent( FixLong(info.state.num_dropped), FixLong(info.state.num_dropped).greaterThan(0) ? "range-table__cell--warning" : "",