Skip to content

Commit

Permalink
Merge pull request #18900 from BramGruneir/quiesced
Browse files Browse the repository at this point in the history
server, ui: add quiescent raft state to the debug page
  • Loading branch information
BramGruneir authored Oct 3, 2017
2 parents 5097e16 + 8eef0fc commit c3411a1
Show file tree
Hide file tree
Showing 9 changed files with 344 additions and 257 deletions.
503 changes: 269 additions & 234 deletions pkg/server/serverpb/status.pb.go

Large diffs are not rendered by default.

1 change: 1 addition & 0 deletions pkg/server/serverpb/status.proto
Original file line number Diff line number Diff line change
Expand Up @@ -176,6 +176,7 @@ message RangeInfo {
CommandQueueMetrics cmd_q_local = 11 [(gogoproto.nullable) = false];
CommandQueueMetrics cmd_q_global = 12 [(gogoproto.nullable) = false];
storage.LeaseStatus lease_status = 13 [(gogoproto.nullable) = false];
bool quiescent = 14;
}

message RangesRequest {
Expand Down
23 changes: 12 additions & 11 deletions pkg/server/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -906,21 +906,21 @@ func (s *statusServer) Ranges(
}

convertRaftStatus := func(raftStatus *raft.Status) serverpb.RaftState {
var state serverpb.RaftState
if raftStatus == nil {
state.State = raftStateDormant
return state
return serverpb.RaftState{
State: raftStateDormant,
}
}

state.ReplicaID = raftStatus.ID
state.HardState = raftStatus.HardState
state.Applied = raftStatus.Applied

// Grab Lead and State, which together form the SoftState.
state.Lead = raftStatus.Lead
state.State = raftStatus.RaftState.String()
state := serverpb.RaftState{
ReplicaID: raftStatus.ID,
HardState: raftStatus.HardState,
Applied: raftStatus.Applied,
Lead: raftStatus.Lead,
State: raftStatus.RaftState.String(),
Progress: make(map[uint64]serverpb.RaftState_Progress),
}

state.Progress = make(map[uint64]serverpb.RaftState_Progress)
for id, progress := range raftStatus.Progress {
state.Progress[id] = serverpb.RaftState_Progress{
Match: progress.Match,
Expand Down Expand Up @@ -964,6 +964,7 @@ func (s *statusServer) Ranges(
CmdQLocal: serverpb.CommandQueueMetrics(metrics.CmdQMetricsLocal),
CmdQGlobal: serverpb.CommandQueueMetrics(metrics.CmdQMetricsGlobal),
LeaseStatus: metrics.LeaseStatus,
Quiescent: metrics.Quiescent,
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/server/status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -429,7 +429,7 @@ func TestRangesResponse(t *testing.T) {
for _, ri := range response.Ranges {
// Do some simple validation based on the fact that this is a
// single-node cluster.
if ri.RaftState.State != "StateLeader" && ri.RaftState.State != "StateDormant" {
if ri.RaftState.State != "StateLeader" && ri.RaftState.State != raftStateDormant {
t.Errorf("expected to be Raft leader or dormant, but was '%s'", ri.RaftState.State)
}
expReplica := roachpb.ReplicaDescriptor{
Expand Down
2 changes: 1 addition & 1 deletion pkg/ui/embedded.go

Large diffs are not rendered by default.

7 changes: 7 additions & 0 deletions pkg/ui/src/js/protos.d.ts
Original file line number Diff line number Diff line change
Expand Up @@ -8038,6 +8038,7 @@ export namespace cockroach {
cmd_q_local?: cockroach.server.serverpb.CommandQueueMetrics$Properties;
cmd_q_global?: cockroach.server.serverpb.CommandQueueMetrics$Properties;
lease_status?: cockroach.storage.LeaseStatus$Properties;
quiescent?: boolean;
};

/**
Expand Down Expand Up @@ -8128,6 +8129,12 @@ export namespace cockroach {
*/
public lease_status: (cockroach.storage.LeaseStatus$Properties|null);

/**
* RangeInfo quiescent.
* @type {boolean}
*/
public quiescent: boolean;

/**
* Creates a new RangeInfo instance using the specified properties.
* @param {cockroach.server.serverpb.RangeInfo$Properties=} [properties] Properties to set
Expand Down
20 changes: 20 additions & 0 deletions pkg/ui/src/js/protos.js
Original file line number Diff line number Diff line change
Expand Up @@ -14598,6 +14598,7 @@ export const cockroach = $root.cockroach = (() => {
* @property {cockroach.server.serverpb.CommandQueueMetrics$Properties} [cmd_q_local] RangeInfo cmd_q_local.
* @property {cockroach.server.serverpb.CommandQueueMetrics$Properties} [cmd_q_global] RangeInfo cmd_q_global.
* @property {cockroach.storage.LeaseStatus$Properties} [lease_status] RangeInfo lease_status.
* @property {boolean} [quiescent] RangeInfo quiescent.
*/

/**
Expand Down Expand Up @@ -14686,6 +14687,12 @@ export const cockroach = $root.cockroach = (() => {
*/
RangeInfo.prototype.lease_status = null;

/**
* RangeInfo quiescent.
* @type {boolean}
*/
RangeInfo.prototype.quiescent = false;

/**
* Creates a new RangeInfo instance using the specified properties.
* @param {cockroach.server.serverpb.RangeInfo$Properties=} [properties] Properties to set
Expand Down Expand Up @@ -14729,6 +14736,8 @@ export const cockroach = $root.cockroach = (() => {
$root.cockroach.server.serverpb.CommandQueueMetrics.encode(message.cmd_q_global, writer.uint32(/* id 12, wireType 2 =*/98).fork()).ldelim();
if (message.lease_status != null && message.hasOwnProperty("lease_status"))
$root.cockroach.storage.LeaseStatus.encode(message.lease_status, writer.uint32(/* id 13, wireType 2 =*/106).fork()).ldelim();
if (message.quiescent != null && message.hasOwnProperty("quiescent"))
writer.uint32(/* id 14, wireType 0 =*/112).bool(message.quiescent);
return writer;
};

Expand Down Expand Up @@ -14795,6 +14804,9 @@ export const cockroach = $root.cockroach = (() => {
case 13:
message.lease_status = $root.cockroach.storage.LeaseStatus.decode(reader, reader.uint32());
break;
case 14:
message.quiescent = reader.bool();
break;
default:
reader.skipType(tag & 7);
break;
Expand Down Expand Up @@ -14882,6 +14894,9 @@ export const cockroach = $root.cockroach = (() => {
if (error)
return "lease_status." + error;
}
if (message.quiescent != null && message.hasOwnProperty("quiescent"))
if (typeof message.quiescent !== "boolean")
return "quiescent: boolean expected";
return null;
};

Expand Down Expand Up @@ -14950,6 +14965,8 @@ export const cockroach = $root.cockroach = (() => {
throw TypeError(".cockroach.server.serverpb.RangeInfo.lease_status: object expected");
message.lease_status = $root.cockroach.storage.LeaseStatus.fromObject(object.lease_status);
}
if (object.quiescent != null)
message.quiescent = Boolean(object.quiescent);
return message;
};

Expand Down Expand Up @@ -14986,6 +15003,7 @@ export const cockroach = $root.cockroach = (() => {
object.cmd_q_local = null;
object.cmd_q_global = null;
object.lease_status = null;
object.quiescent = false;
}
if (message.span != null && message.hasOwnProperty("span"))
object.span = $root.cockroach.server.serverpb.PrettySpan.toObject(message.span, options);
Expand Down Expand Up @@ -15014,6 +15032,8 @@ export const cockroach = $root.cockroach = (() => {
object.cmd_q_global = $root.cockroach.server.serverpb.CommandQueueMetrics.toObject(message.cmd_q_global, options);
if (message.lease_status != null && message.hasOwnProperty("lease_status"))
object.lease_status = $root.cockroach.storage.LeaseStatus.toObject(message.lease_status, options);
if (message.quiescent != null && message.hasOwnProperty("quiescent"))
object.quiescent = message.quiescent;
return object;
};

Expand Down
38 changes: 29 additions & 9 deletions pkg/ui/src/views/reports/containers/range/rangeTable.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,10 @@ const rangeTableDisplayList: RangeTableRow[] = [
{ variable: "keyRange", display: "Key Range", compareToLeader: true },
{ variable: "problems", display: "Problems", compareToLeader: true },
{ variable: "raftState", display: "Raft State", compareToLeader: false },
{ variable: "quiescent", display: "Quiescent", compareToLeader: true },
{ variable: "leaseType", display: "Lease Type", compareToLeader: true },
{ variable: "leaseState", display: "Lease State", compareToLeader: true },
{ variable: "leaseHolder", display: "Lease Holder", compareToLeader: true },
{ variable: "leaseType", display: "Lease Type", compareToLeader: true },
{ variable: "leaseEpoch", display: "Lease Epoch", compareToLeader: true },
{ variable: "leaseStart", display: "Lease Start", compareToLeader: true },
{ variable: "leaseExpiration", display: "Lease Expiration", compareToLeader: true },
Expand Down Expand Up @@ -81,6 +82,11 @@ const rangeTableEmptyContentWithWarning: RangeTableCellContent = {
className: ["range-table__cell--warning"],
};

const rangeTableQuiescent: RangeTableCellContent = {
value: ["quiescent"],
className: ["range-table__cell--quiescent"],
};

function convertLeaseState(leaseState: protos.cockroach.storage.LeaseState) {
return protos.cockroach.storage.LeaseState[leaseState].toLowerCase();
}
Expand Down Expand Up @@ -190,6 +196,18 @@ export default class RangeTable extends React.Component<RangeTableProps, {}> {
};
}

// contentIf returns an empty value if the condition is false, and if true,
// executes and returns the content function.
contentIf(
showContent: boolean,
content: () => RangeTableCellContent,
): RangeTableCellContent {
if (!showContent) {
return rangeTableEmptyContent;
}
return content();
}

renderRangeCell(
row: RangeTableRow,
cell: RangeTableCellContent,
Expand Down Expand Up @@ -378,7 +396,8 @@ export default class RangeTable extends React.Component<RangeTableProps, {}> {
"range-table__cell--warning",
);
}
if (raftState.value[0] === "dormant") {
const dormant = raftState.value[0] === "dormant";
if (dormant) {
dormantStoreIDs.add(info.source_store_id);
}
detailsByStoreID.set(info.source_store_id, {
Expand All @@ -391,24 +410,25 @@ export default class RangeTable extends React.Component<RangeTableProps, {}> {
keyRange: this.createContent(`${info.span.start_key} to ${info.span.end_key}`),
problems: this.contentProblems(info.problems, awaitingGC),
raftState: raftState,
quiescent: info.quiescent ? rangeTableQuiescent : rangeTableEmptyContent,
leaseState: leaseState,
leaseHolder: this.createContent(
Print.ReplicaID(rangeID, lease.replica),
leaseHolder ? "range-table__cell--lease-holder" : "range-table__cell--lease-follower",
),
leaseType: this.createContent(epoch ? "Epoch" : "Expiration"),
leaseType: this.createContent(epoch ? "epoch" : "expiration"),
leaseEpoch: epoch ? this.createContent(lease.epoch) : rangeTableEmptyContent,
leaseStart: this.contentTimestamp(lease.start),
leaseExpiration: epoch ? rangeTableEmptyContent : this.contentTimestamp(lease.expiration),
leaseAppliedIndex: this.createContent(FixLong(info.state.state.lease_applied_index)),
raftLeader: this.createContent(
raftLeader: this.contentIf(!dormant, () => this.createContent(
FixLong(info.raft_state.lead),
raftLeader ? "range-table__cell--raftstate-leader" : "range-table__cell--raftstate-follower",
),
vote: this.createContent(vote.greaterThan(0) ? vote : "-"),
term: this.createContent(FixLong(info.raft_state.hard_state.term)),
applied: this.createContent(FixLong(info.raft_state.applied)),
commit: this.createContent(FixLong(info.raft_state.hard_state.commit)),
)),
vote: this.contentIf(!dormant, () => this.createContent(vote.greaterThan(0) ? vote : "-")),
term: this.contentIf(!dormant, () => this.createContent(FixLong(info.raft_state.hard_state.term))),
applied: this.contentIf(!dormant, () => this.createContent(FixLong(info.raft_state.applied))),
commit: this.contentIf(!dormant, () => this.createContent(FixLong(info.raft_state.hard_state.commit))),
lastIndex: this.createContent(FixLong(info.state.last_index)),
logSize: this.createContent(FixLong(info.state.raft_log_size)),
leaseHolderQPS: leaseHolder ? this.createContent(info.stats.queries_per_second.toFixed(4)) : rangeTableEmptyContent,
Expand Down
5 changes: 4 additions & 1 deletion pkg/ui/styl/pages/reports.styl
Original file line number Diff line number Diff line change
Expand Up @@ -245,7 +245,7 @@ $reports-table
color darkorange

&--raftstate-dormant
color gray
color lightgray

&--raftstate-unknown
color red
Expand All @@ -268,6 +268,9 @@ $reports-table
&--warning
color orange

&--quiescent
color violet

.range-entries-list
list-style-type none
margin 0
Expand Down

0 comments on commit c3411a1

Please sign in to comment.