From 89ce060358094c6910a8189b96e08cc59583fc90 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 6 Jul 2020 00:03:25 -0400 Subject: [PATCH 1/5] kvserver: rename members of `livenessUpdate` Pulling out a rename refactor out of future commits that then make use of it. Release note: None --- pkg/kv/kvserver/node_liveness.go | 56 ++++++++++++++++---------------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/pkg/kv/kvserver/node_liveness.go b/pkg/kv/kvserver/node_liveness.go index 151ffbe11eab..c47ad7d4a7aa 100644 --- a/pkg/kv/kvserver/node_liveness.go +++ b/pkg/kv/kvserver/node_liveness.go @@ -361,27 +361,27 @@ func (nl *NodeLiveness) setDrainingInternal( }() update := livenessUpdate{ - updated: kvserverpb.Liveness{ + newLiveness: kvserverpb.Liveness{ NodeID: nodeID, Epoch: 1, }, - old: liveness.Liveness, + oldLiveness: liveness.Liveness, ignoreCache: true, oldRaw: liveness.raw, } if liveness.Liveness != (kvserverpb.Liveness{}) { - update.updated = liveness.Liveness + update.newLiveness = liveness.Liveness } - if reporter != nil && drain && !update.updated.Draining { + if reporter != nil && drain && !update.newLiveness.Draining { // Report progress to the Drain RPC. reporter(1, "liveness record") } - update.updated.Draining = drain + update.newLiveness.Draining = drain written, err := nl.updateLiveness(ctx, update, func(actual LivenessRecord) error { nl.maybeUpdate(actual) - if actual.Draining == update.updated.Draining { + if actual.Draining == update.newLiveness.Draining { return errNodeDrainingSet } return errors.New("failed to update liveness record because record has changed") @@ -402,8 +402,8 @@ func (nl *NodeLiveness) setDrainingInternal( // livenessUpdate contains the information for CPutting a new version of a // liveness record. It has both the new and the old version of the proto. type livenessUpdate struct { - updated kvserverpb.Liveness - old kvserverpb.Liveness + newLiveness kvserverpb.Liveness + oldLiveness kvserverpb.Liveness // When ignoreCache is set, we won't assume that our in-memory cached version // of the liveness record is accurate and will use a CPut on the liveness // table with the old value supplied by the client (oldRaw). This is used for @@ -428,23 +428,23 @@ func (nl *NodeLiveness) setDecommissioningInternal( ctx context.Context, nodeID roachpb.NodeID, liveness LivenessRecord, decommission bool, ) (changeCommitted bool, err error) { update := livenessUpdate{ - updated: kvserverpb.Liveness{ + newLiveness: kvserverpb.Liveness{ NodeID: nodeID, Epoch: 1, }, - old: liveness.Liveness, + oldLiveness: liveness.Liveness, ignoreCache: true, oldRaw: liveness.raw, } if liveness.Liveness != (kvserverpb.Liveness{}) { - update.updated = liveness.Liveness + update.newLiveness = liveness.Liveness } - update.updated.Decommissioning = decommission + update.newLiveness.Decommissioning = decommission var conditionFailed bool if _, err := nl.updateLiveness(ctx, update, func(actual LivenessRecord) error { conditionFailed = true - if actual.Decommissioning == update.updated.Decommissioning { + if actual.Decommissioning == update.newLiveness.Decommissioning { return nil } return errChangeDecommissioningFailed @@ -627,30 +627,30 @@ func (nl *NodeLiveness) heartbeatInternal( }() update := livenessUpdate{ - updated: kvserverpb.Liveness{ + newLiveness: kvserverpb.Liveness{ NodeID: nodeID, Epoch: 1, }, - old: liveness, + oldLiveness: liveness, } if liveness != (kvserverpb.Liveness{}) { - update.updated = liveness + update.newLiveness = liveness if incrementEpoch { - update.updated.Epoch++ + update.newLiveness.Epoch++ // Clear draining field. - update.updated.Draining = false + update.newLiveness.Draining = false } } // We need to add the maximum clock offset to the expiration because it's // used when determining liveness for a node. { - update.updated.Expiration = hlc.LegacyTimestamp( + update.newLiveness.Expiration = hlc.LegacyTimestamp( nl.clock.Now().Add((nl.livenessThreshold).Nanoseconds(), 0)) // This guards against the system clock moving backwards. As long // as the cockroach process is running, checks inside hlc.Clock // will ensure that the clock never moves backwards, but these // checks don't work across process restarts. - if update.updated.Expiration.Less(liveness.Expiration) { + if update.newLiveness.Expiration.Less(liveness.Expiration) { return errors.Errorf("proposed liveness update expires earlier than previous record") } } @@ -814,10 +814,10 @@ func (nl *NodeLiveness) IncrementEpoch(ctx context.Context, liveness kvserverpb. } update := livenessUpdate{ - updated: liveness, - old: liveness, + newLiveness: liveness, + oldLiveness: liveness, } - update.updated.Epoch++ + update.newLiveness.Epoch++ written, err := nl.updateLiveness(ctx, update, func(actual LivenessRecord) error { defer nl.maybeUpdate(actual) if actual.Epoch > liveness.Epoch { @@ -918,11 +918,11 @@ func (nl *NodeLiveness) updateLivenessAttempt( log.Fatalf(ctx, "unexpected oldRaw when ignoreCache not specified") } - l, err := nl.GetLiveness(update.updated.NodeID) + l, err := nl.GetLiveness(update.newLiveness.NodeID) if err != nil && !errors.Is(err, ErrNoLivenessRecord) { return LivenessRecord{}, err } - if err == nil && l.Liveness != update.old { + if err == nil && l.Liveness != update.oldLiveness { return LivenessRecord{}, handleCondFailed(l) } oldRaw = l.raw @@ -931,8 +931,8 @@ func (nl *NodeLiveness) updateLivenessAttempt( v := new(roachpb.Value) if err := nl.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { b := txn.NewBatch() - key := keys.NodeLivenessKey(update.updated.NodeID) - if err := v.SetProto(&update.updated); err != nil { + key := keys.NodeLivenessKey(update.newLiveness.NodeID) + if err := v.SetProto(&update.newLiveness); err != nil { log.Fatalf(ctx, "failed to marshall proto: %s", err) } b.CPut(key, v, oldRaw) @@ -975,7 +975,7 @@ func (nl *NodeLiveness) updateLivenessAttempt( if cb != nil { cb(ctx) } - return LivenessRecord{Liveness: update.updated, raw: v.TagAndDataBytes()}, nil + return LivenessRecord{Liveness: update.newLiveness, raw: v.TagAndDataBytes()}, nil } // maybeUpdate replaces the liveness (if it appears newer) and invokes the From 57033b4bccd693e7b30a3359b256e0aa6375a14b Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Sun, 5 Jul 2020 23:50:32 -0400 Subject: [PATCH 2/5] kvserver,cli,roachtest,sql: introduce a decommissioned bit This PR introduces a fully decommissioned bit to CRDB. Previously our Liveness schema only contained a `decommissioning` bool, with consequently no ability to disamiguate between a node currently undergoing decommissioning, and a node that was fully decommissioned. We used some combination of store dead threshold to surface, in our UI, "fully decommissioned" nodes, but it was never quite so. We need this specificity for the Connect RPC. --- We wire up a new `MembershipStatus` enum that's now part of the liveness record. In doing so it elides usage of the `decommissioning` bool used in v20.1. We're careful to maintain an on-the-wire representation of the Liveness record that will be understood by v20.1 nodes, and do so by ensuring the encoding of the enum type is parsed into the semantically equivalent v20.1 representation. Usage of the fully decommissioned bit is gated behind a version flag. A future commit will introduce a mixed-version roachtest testing cross-version compatibility. A future commit will also re-register/unskip an improved version of the `acceptance/decommission` roachtest. We repurpose the `AdminServer.Decommission` RPC to persist `MembershipStatus`es to KV through the lifetime of a node decommissioning/recommissioning. See `cli/node.go` for where that's done. For recommissioning a node, it suffices to simply persist an `ACTIVE` status. When decommissioning a node, since it's a longer running process, we first persist an in-progress `DECOMMISSIONING` status, and once we've moved off all the Replicas in the node, we finalize the decommissioning process by persisting the `DECOMMISSIONED` status. When transitioning between `MembershipStatus`es, we CPut against what's already there, disallowing illegal state transitions. The appropriate error codes are surfaced back to the user. An example would be in attempting to recommission a fully decommissioned node, in which case we'd error out with the following: > ERROR: can only recommission a decommissioning node; n4 found to be > decommissioned Note that this is a behavioral change for `cockroach node recommission`. Previously it was able to recommission any "fully decommissioned" node, regardless of how long ago it's was removed from the cluster. Now recommission serves to only cancel an accidental decommissioning process that wasn't finalized. The `decommissioning` column in `crdb_internal.gossip_liveness` is now powered by this new `MembershipStatus` instead, and we introduce a new `membership` column to it that should be preferred going forward. We also introduce the same column to the output generated by `cockroach node status --decommission`. The `is_decommissioning` column still exists, but is also powered by this `MembershipStatus`. While here, we iron out the events plumbed into `system.eventlog`: it now has a dedicated event for "node decommissioning". --- Release note (general change): `cockroach node recommission` has new semantics. Previously it was able to recommission any decommissioning node, regardless of how long ago it's was decommissioned, or removed from the cluster. Now recommission serves to only cancel an accidental inflight decommissioning process that wasn't finalized. Release note (cli change): We introduce a `membership` column to the output generated by `cockroach node status --decommission`. It should be used in favor of the `is_decommissioning` column going forward. Release note (cli change): The v20.2 cli `cockroach node` family of subcommands will not work with against servers running older version of cockroach, but the v20.1 cli `cockroach node` subcommands will work against v20.2 servers. Release note (cli change): The `is_decommissioning` column found in the output of `cockroach node decommission` is slated for removal in v20.1. Operators should instead use the new `membership` column to determine node membership status. --- docs/generated/settings/settings.html | 2 +- pkg/cli/demo_cluster.go | 55 +- pkg/cli/error.go | 8 + .../interactive_tests/test_demo_node_cmds.tcl | 45 +- .../interactive_tests/test_multiple_nodes.tcl | 11 +- pkg/cli/node.go | 72 +- pkg/cli/sql.go | 4 +- pkg/cli/zip_test.go | 26 +- pkg/clusterversion/cockroach_versions.go | 8 + pkg/clusterversion/versionkey_string.go | 5 +- pkg/cmd/roachtest/acceptance.go | 3 +- pkg/kv/kvserver/client_raft_test.go | 12 +- pkg/kv/kvserver/helpers_test.go | 7 +- pkg/kv/kvserver/kvserverpb/liveness.go | 71 +- pkg/kv/kvserver/kvserverpb/liveness.pb.go | 172 +++-- pkg/kv/kvserver/kvserverpb/liveness.proto | 58 +- pkg/kv/kvserver/node_liveness.go | 210 +++-- pkg/kv/kvserver/node_liveness_test.go | 42 +- pkg/kv/kvserver/node_liveness_unit_test.go | 57 +- pkg/kv/kvserver/replica_gc_queue.go | 14 +- pkg/kv/kvserver/replica_gossip.go | 10 + pkg/kv/kvserver/store_pool.go | 16 +- pkg/kv/kvserver/store_pool_test.go | 60 +- pkg/server/admin.go | 14 +- pkg/server/server.go | 35 +- pkg/server/serverpb/admin.pb.go | 718 +++++++++--------- pkg/server/serverpb/admin.proto | 13 +- pkg/sql/crdb_internal.go | 20 +- pkg/sql/event_log.go | 6 +- .../testdata/logic_test/crdb_internal | 8 +- pkg/ui/src/redux/alerts.spec.ts | 33 +- pkg/ui/src/redux/nodes.ts | 5 +- pkg/ui/src/util/eventTypes.ts | 8 +- pkg/ui/src/util/events.ts | 2 + 34 files changed, 1149 insertions(+), 681 deletions(-) diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 00beb7e1a023..8b82f844cdfa 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -72,6 +72,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen in the /debug page trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versioncustom validation20.1-10set the active cluster version in the format '.' +versioncustom validation20.1-11set the active cluster version in the format '.' diff --git a/pkg/cli/demo_cluster.go b/pkg/cli/demo_cluster.go index 9877493e1cdb..bd4c1f757f4a 100644 --- a/pkg/cli/demo_cluster.go +++ b/pkg/cli/demo_cluster.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cli/cliflags" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" @@ -361,8 +362,8 @@ func (c *transientCluster) DrainAndShutdown(nodeID roachpb.NodeID) error { return nil } -// CallDecommission calls the Decommission RPC on a node. -func (c *transientCluster) CallDecommission(nodeID roachpb.NodeID, decommissioning bool) error { +// Recommission recommissions a given node. +func (c *transientCluster) Recommission(nodeID roachpb.NodeID) error { nodeIndex := int(nodeID - 1) if nodeIndex < 0 || nodeIndex >= len(c.servers) { @@ -370,8 +371,8 @@ func (c *transientCluster) CallDecommission(nodeID roachpb.NodeID, decommissioni } req := &serverpb.DecommissionRequest{ - NodeIDs: []roachpb.NodeID{nodeID}, - Decommissioning: decommissioning, + NodeIDs: []roachpb.NodeID{nodeID}, + TargetMembership: kvserverpb.MembershipStatus_ACTIVE, } ctx, cancel := context.WithCancel(context.Background()) @@ -387,6 +388,52 @@ func (c *transientCluster) CallDecommission(nodeID roachpb.NodeID, decommissioni if err != nil { return errors.Wrap(err, "while trying to mark as decommissioning") } + + return nil +} + +// Decommission decommissions a given node. +func (c *transientCluster) Decommission(nodeID roachpb.NodeID) error { + nodeIndex := int(nodeID - 1) + + if nodeIndex < 0 || nodeIndex >= len(c.servers) { + return errors.Errorf("node %d does not exist", nodeID) + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + adminClient, finish, err := getAdminClient(ctx, *(c.s.Cfg)) + if err != nil { + return err + } + defer finish() + + // This (cumbersome) two step process is due to the allowed state + // transitions for membership status. To mark a node as fully + // decommissioned, it has to be marked as decommissioning first. + { + req := &serverpb.DecommissionRequest{ + NodeIDs: []roachpb.NodeID{nodeID}, + TargetMembership: kvserverpb.MembershipStatus_DECOMMISSIONING, + } + _, err = adminClient.Decommission(ctx, req) + if err != nil { + return errors.Wrap(err, "while trying to mark as decommissioning") + } + } + + { + req := &serverpb.DecommissionRequest{ + NodeIDs: []roachpb.NodeID{nodeID}, + TargetMembership: kvserverpb.MembershipStatus_DECOMMISSIONED, + } + _, err = adminClient.Decommission(ctx, req) + if err != nil { + return errors.Wrap(err, "while trying to mark as decommissioned") + } + } + return nil } diff --git a/pkg/cli/error.go b/pkg/cli/error.go index 66b7eee51f7e..f2f436d65d38 100644 --- a/pkg/cli/error.go +++ b/pkg/cli/error.go @@ -268,6 +268,14 @@ func MaybeDecorateGRPCError( if pgcode.MakeCode(string(wErr.Code)) == pgcode.ProtocolViolation { return connSecurityHint() } + + // Are we running a v20.2 binary against a v20.1 server? + if strings.Contains(wErr.Message, "column \"membership\" does not exist") { + // The v20.2 binary makes use of columns not present in v20.1, + // so this is a disallowed operation. Surface a better error + // code here. + return fmt.Errorf("cannot use a v20.2 cli against servers running v20.1") + } // Otherwise, there was a regular SQL error. Just report // that. return err diff --git a/pkg/cli/interactive_tests/test_demo_node_cmds.tcl b/pkg/cli/interactive_tests/test_demo_node_cmds.tcl index 74fb005a9f0b..fd8987d4e1ab 100644 --- a/pkg/cli/interactive_tests/test_demo_node_cmds.tcl +++ b/pkg/cli/interactive_tests/test_demo_node_cmds.tcl @@ -35,12 +35,12 @@ eexpect "node 2 is already running" send "\\demo shutdown 3\r" eexpect "node 3 has been shutdown" -send "select node_id, draining, decommissioning from crdb_internal.gossip_liveness ORDER BY node_id;\r" -eexpect "1 | false | false" -eexpect "2 | false | false" -eexpect "3 | true | false" -eexpect "4 | false | false" -eexpect "5 | false | false" +send "select node_id, draining, decommissioning, membership from crdb_internal.gossip_liveness ORDER BY node_id;\r" +eexpect "1 | false | false | active" +eexpect "2 | false | false | active" +eexpect "3 | true | false | active" +eexpect "4 | false | false | active" +eexpect "5 | false | false | active" # Cannot shut it down again. send "\\demo shutdown 3\r" @@ -55,33 +55,26 @@ eexpect "movr>" send "\\demo restart 3\r" eexpect "node 3 has been restarted" -send "select node_id, draining, decommissioning from crdb_internal.gossip_liveness ORDER BY node_id;\r" -eexpect "1 | false | false" -eexpect "2 | false | false" -eexpect "3 | false | false" -eexpect "4 | false | false" -eexpect "5 | false | false" +send "select node_id, draining, decommissioning, membership from crdb_internal.gossip_liveness ORDER BY node_id;\r" +eexpect "1 | false | false | active" +eexpect "2 | false | false | active" +eexpect "3 | false | false | active" +eexpect "4 | false | false | active" +eexpect "5 | false | false | active" # Try commissioning commands send "\\demo decommission 4\r" eexpect "node 4 has been decommissioned" -send "select node_id, draining, decommissioning from crdb_internal.gossip_liveness ORDER BY node_id;\r" -eexpect "1 | false | false" -eexpect "2 | false | false" -eexpect "3 | false | false" -eexpect "4 | false | true" -eexpect "5 | false | false" +send "select node_id, draining, decommissioning, membership from crdb_internal.gossip_liveness ORDER BY node_id;\r" +eexpect "1 | false | false | active" +eexpect "2 | false | false | active" +eexpect "3 | false | false | active" +eexpect "4 | false | true | decommissioned" +eexpect "5 | false | false | active" send "\\demo recommission 4\r" -eexpect "node 4 has been recommissioned" - -send "select node_id, draining, decommissioning from crdb_internal.gossip_liveness ORDER BY node_id;\r" -eexpect "1 | false | false" -eexpect "2 | false | false" -eexpect "3 | false | false" -eexpect "4 | false | false" -eexpect "5 | false | false" +eexpect "can only recommission a decommissioning node" interrupt eexpect eof diff --git a/pkg/cli/interactive_tests/test_multiple_nodes.tcl b/pkg/cli/interactive_tests/test_multiple_nodes.tcl index 84a7798feecd..702bdfdc5782 100644 --- a/pkg/cli/interactive_tests/test_multiple_nodes.tcl +++ b/pkg/cli/interactive_tests/test_multiple_nodes.tcl @@ -20,6 +20,10 @@ eexpect eof end_test +start_test "Check that a recommissioning an active node prints out a warning" +spawn $argv node recommission 2 +eexpect "warning: node 2 is not decommissioned" +eexpect eof start_test "Check that a double decommission prints out a warning" spawn $argv node decommission 2 --wait none @@ -30,13 +34,6 @@ eexpect "warning: node 2 is already decommissioning or decommissioned" eexpect eof end_test -start_test "Check that a double recommission prints out a warning" -spawn $argv node recommission 2 -eexpect eof - -spawn $argv node recommission 2 -eexpect "warning: node 2 is not decommissioned" -eexpect eof end_test diff --git a/pkg/cli/node.go b/pkg/cli/node.go index 080b0411675b..171d4e70d03b 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -27,6 +27,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/errors" "github.com/spf13/cobra" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) const ( @@ -64,9 +66,10 @@ func runLsNodes(cmd *cobra.Command, args []string) error { _, rows, err := runQuery( conn, makeQuery(`SELECT node_id FROM crdb_internal.gossip_liveness - WHERE decommissioning = false OR split_part(expiration,',',1)::decimal > now()::decimal`), + WHERE membership = 'active' OR split_part(expiration,',',1)::decimal > now()::decimal`), false, ) + if err != nil { return err } @@ -105,6 +108,7 @@ var statusNodesColumnHeadersForStats = []string{ var statusNodesColumnHeadersForDecommission = []string{ "gossiped_replicas", "is_decommissioning", + "membership", "is_draining", } @@ -143,7 +147,7 @@ func runStatusNodeInner(showDecommissioned bool, args []string) ([]string, [][]s maybeAddActiveNodesFilter := func(query string) string { if !showDecommissioned { - query += " WHERE decommissioning = false OR split_part(expiration,',',1)::decimal > now()::decimal" + query += " WHERE membership = 'active' OR split_part(expiration,',',1)::decimal > now()::decimal" } return query } @@ -184,10 +188,12 @@ SELECT node_id AS id, FROM crdb_internal.kv_store_status GROUP BY node_id` + // TODO(irfansharif): Remove the `is_decommissioning` column in v20.2. const decommissionQuery = ` SELECT node_id AS id, ranges AS gossiped_replicas, - decommissioning AS is_decommissioning, + membership != 'active' as is_decommissioning, + membership AS membership, draining AS is_draining FROM crdb_internal.gossip_liveness LEFT JOIN crdb_internal.gossip_nodes USING (node_id)` @@ -274,6 +280,7 @@ var decommissionNodesColumnHeaders = []string{ "is_live", "replicas", "is_decommissioning", + "membership", "is_draining", } @@ -330,7 +337,6 @@ func runDecommissionNode(cmd *cobra.Command, args []string) error { } c := serverpb.NewAdminClient(conn) - return runDecommissionNodeImpl(ctx, c, nodeCtx.nodeDecommissionWait, nodeIDs) } @@ -409,11 +415,17 @@ func runDecommissionNodeImpl( MaxBackoff: 20 * time.Second, } + // Marking a node as fully decommissioned is driven by a two-step process. + // We start off by marking each node as 'decommissioning'. In doing so, + // replicas are slowly moved off of these nodes. It's only after when we're + // made aware that the replica counts have all hit zero, and that all nodes + // have been successfully marked as 'decommissioning', that we then go and + // mark each node as 'decommissioned'. prevResponse := serverpb.DecommissionStatusResponse{} for r := retry.StartWithCtx(ctx, opts); r.Next(); { req := &serverpb.DecommissionRequest{ - NodeIDs: nodeIDs, - Decommissioning: true, + NodeIDs: nodeIDs, + TargetMembership: kvserverpb.MembershipStatus_DECOMMISSIONING, } resp, err := c.Decommission(ctx, req) if err != nil { @@ -430,18 +442,43 @@ func runDecommissionNodeImpl( } else { fmt.Fprintf(stderr, ".") } + + anyActive := false var replicaCount int64 - allDecommissioning := true for _, status := range resp.Status { + anyActive = anyActive || status.Membership.Active() replicaCount += status.ReplicaCount - allDecommissioning = allDecommissioning && status.Decommissioning } - if replicaCount == 0 && allDecommissioning { + + if !anyActive && replicaCount == 0 { + // We now mark the nodes as fully decommissioned. + req := &serverpb.DecommissionRequest{ + NodeIDs: nodeIDs, + TargetMembership: kvserverpb.MembershipStatus_DECOMMISSIONED, + } + resp, err := c.Decommission(ctx, req) + if err != nil { + fmt.Fprintln(stderr) + return errors.Wrap(err, "while trying to mark as decommissioned") + } + if !reflect.DeepEqual(&prevResponse, resp) { + fmt.Fprintln(stderr) + if err := printDecommissionStatus(*resp); err != nil { + return err + } + prevResponse = *resp + } + fmt.Fprintln(os.Stdout, "\nNo more data reported on target nodes. "+ "Please verify cluster health before removing the nodes.") return nil } + if wait == nodeDecommissionWaitNone { + // The intent behind --wait=none is for it to be used when polling + // manually from an external system. We'll only mark nodes as + // fully decommissioned once the replica count hits zero and they're + // all marked as decommissioning. return nil } if replicaCount < minReplicaCount { @@ -453,7 +490,7 @@ func runDecommissionNodeImpl( } func decommissionResponseAlignment() string { - return "rcrcc" + return "rcrccc" } // decommissionResponseValueToRows converts DecommissionStatusResponse_Status to @@ -468,7 +505,8 @@ func decommissionResponseValueToRows( strconv.FormatInt(int64(node.NodeID), 10), strconv.FormatBool(node.IsLive), strconv.FormatInt(node.ReplicaCount, 10), - strconv.FormatBool(node.Decommissioning), + strconv.FormatBool(!node.Membership.Active()), + node.Membership.String(), strconv.FormatBool(node.Draining), }) } @@ -522,13 +560,19 @@ func runRecommissionNode(cmd *cobra.Command, args []string) error { } c := serverpb.NewAdminClient(conn) - req := &serverpb.DecommissionRequest{ - NodeIDs: nodeIDs, - Decommissioning: false, + NodeIDs: nodeIDs, + TargetMembership: kvserverpb.MembershipStatus_ACTIVE, } resp, err := c.Decommission(ctx, req) if err != nil { + // If it's a specific illegal membership transition error, we try to + // surface a more readable message to the user. See + // ValidateLivenessTransition in kvserverpb/liveness.go for where this + // error is generated. + if s, ok := status.FromError(err); ok && s.Code() == codes.FailedPrecondition { + return errors.Newf("%s", s.Message()) + } return err } return printDecommissionStatus(*resp) diff --git a/pkg/cli/sql.go b/pkg/cli/sql.go index 047d160a4de9..bb29655a8bc8 100644 --- a/pkg/cli/sql.go +++ b/pkg/cli/sql.go @@ -527,13 +527,13 @@ func (c *cliState) handleDemo(cmd []string, nextState, errState cliStateEnum) cl fmt.Printf("node %d has been restarted\n", nodeID) return nextState case "recommission": - if err := demoCtx.transientCluster.CallDecommission(roachpb.NodeID(nodeID), false /* decommissioning */); err != nil { + if err := demoCtx.transientCluster.Recommission(roachpb.NodeID(nodeID)); err != nil { return c.internalServerError(errState, err) } fmt.Printf("node %d has been recommissioned\n", nodeID) return nextState case "decommission": - if err := demoCtx.transientCluster.CallDecommission(roachpb.NodeID(nodeID), true /* decommissioning */); err != nil { + if err := demoCtx.transientCluster.Decommission(roachpb.NodeID(nodeID)); err != nil { return c.internalServerError(errState, err) } fmt.Printf("node %d has been decommissioned\n", nodeID) diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 31e329ac0ab3..3654158c5f23 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -30,10 +30,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -341,25 +339,21 @@ func TestPartialZip(t *testing.T) { }) // Now mark the stopped node as decommissioned, and check that zip - // skips over it automatically. - s := tc.Server(0) - conn, err := s.RPCContext().GRPCDialNode(s.ServingRPCAddr(), s.NodeID(), - rpc.DefaultClass).Connect(context.Background()) - if err != nil { - t.Fatal(err) - } - as := serverpb.NewAdminClient(conn) - req := &serverpb.DecommissionRequest{ - NodeIDs: []roachpb.NodeID{2}, - Decommissioning: true, - } - if _, err := as.Decommission(context.Background(), req); err != nil { - t.Fatal(err) + // skips over it automatically. We specifically use --wait=none because + // we're decommissioning a node in a 3-node cluster, so there's no node to + // up-replicate the under-replicated ranges to. + { + _, err := c.RunWithCapture(fmt.Sprintf("node decommission --wait=none %d", 2)) + if err != nil { + t.Fatal(err) + } } + // We use .Override() here instead of SET CLUSTER SETTING in SQL to // override the 1m15s minimum placed on the cluster setting. There // is no risk to see the override bumped due to a gossip update // because this setting is not otherwise set in the test cluster. + s := tc.Server(0) kvserver.TimeUntilStoreDead.Override(&s.ClusterSettings().SV, kvserver.TestTimeUntilStoreDead) datadriven.RunTest(t, "testdata/zip/partial2", diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 7e3dc648723f..151b4015e67e 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -70,6 +70,7 @@ const ( VersionUserDefinedSchemas VersionNoOriginFKIndexes VersionClientRangeInfosOnBatchResponse + VersionNodeMembershipStatus // Add new versions here (step one of two). ) @@ -531,6 +532,13 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: VersionClientRangeInfosOnBatchResponse, Version: roachpb.Version{Major: 20, Minor: 1, Unstable: 10}, }, + { + // VersionNodeMembershipStatus gates the usage of the MembershipStatus + // enum in the Liveness proto. See comment on proto definition for more + // details. + Key: VersionNodeMembershipStatus, + Version: roachpb.Version{Major: 20, Minor: 1, Unstable: 11}, + }, // Add new versions here (step two of two). diff --git a/pkg/clusterversion/versionkey_string.go b/pkg/clusterversion/versionkey_string.go index 3757a7994354..d4a7beba0e38 100644 --- a/pkg/clusterversion/versionkey_string.go +++ b/pkg/clusterversion/versionkey_string.go @@ -46,11 +46,12 @@ func _() { _ = x[VersionUserDefinedSchemas-35] _ = x[VersionNoOriginFKIndexes-36] _ = x[VersionClientRangeInfosOnBatchResponse-37] + _ = x[VersionNodeMembershipStatus-38] } -const _VersionKey_name = "Version19_1VersionStart19_2VersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemasVersionProtectedTimestampsVersionPrimaryKeyChangesVersionAuthLocalAndTrustRejectMethodsVersionPrimaryKeyColumnsOutOfFamilyZeroVersionRootPasswordVersionNoExplicitForeignKeyIndexIDsVersionHashShardedIndexesVersionCreateRolePrivilegeVersionStatementDiagnosticsSystemTablesVersionSchemaChangeJobVersionSavepointsVersionTimeTZTypeVersionTimePrecisionVersion20_1VersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponse" +const _VersionKey_name = "Version19_1VersionStart19_2VersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemasVersionProtectedTimestampsVersionPrimaryKeyChangesVersionAuthLocalAndTrustRejectMethodsVersionPrimaryKeyColumnsOutOfFamilyZeroVersionRootPasswordVersionNoExplicitForeignKeyIndexIDsVersionHashShardedIndexesVersionCreateRolePrivilegeVersionStatementDiagnosticsSystemTablesVersionSchemaChangeJobVersionSavepointsVersionTimeTZTypeVersionTimePrecisionVersion20_1VersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatus" -var _VersionKey_index = [...]uint16{0, 11, 27, 49, 75, 109, 136, 176, 200, 211, 227, 258, 287, 322, 354, 380, 404, 441, 480, 499, 534, 559, 585, 624, 646, 663, 680, 700, 711, 727, 748, 760, 782, 811, 852, 880, 905, 929, 967} +var _VersionKey_index = [...]uint16{0, 11, 27, 49, 75, 109, 136, 176, 200, 211, 227, 258, 287, 322, 354, 380, 404, 441, 480, 499, 534, 559, 585, 624, 646, 663, 680, 700, 711, 727, 748, 760, 782, 811, 852, 880, 905, 929, 967, 994} func (i VersionKey) String() string { if i < 0 || i >= VersionKey(len(_VersionKey_index)-1) { diff --git a/pkg/cmd/roachtest/acceptance.go b/pkg/cmd/roachtest/acceptance.go index 2114f80ca7e9..da90a0b9cc75 100644 --- a/pkg/cmd/roachtest/acceptance.go +++ b/pkg/cmd/roachtest/acceptance.go @@ -71,7 +71,8 @@ https://github.com/cockroachdb/cockroach/issues/48676`, // running. If that branch corresponds to an older release, then upgrading // to head after 19.2 fails. minVersion: "v19.2.0", - timeout: 30 * time.Minute}, + timeout: 30 * time.Minute, + }, } tags := []string{"default", "quick"} const numNodes = 4 diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 178e3bb3a949..f80400486d5a 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -3060,8 +3060,8 @@ func TestDecommission(t *testing.T) { // Decommission the first node, which holds most of the leases. _, err = admin.Decommission( ctx, &serverpb.DecommissionRequest{ - NodeIDs: []roachpb.NodeID{1}, - Decommissioning: true, + NodeIDs: []roachpb.NodeID{1}, + TargetMembership: kvserverpb.MembershipStatus_DECOMMISSIONING, }, ) require.NoError(t, err) @@ -3097,8 +3097,8 @@ func TestDecommission(t *testing.T) { _, err = admin.Decommission( ctx, &serverpb.DecommissionRequest{ - NodeIDs: []roachpb.NodeID{2}, - Decommissioning: true, + NodeIDs: []roachpb.NodeID{2}, + TargetMembership: kvserverpb.MembershipStatus_DECOMMISSIONING, }, ) require.NoError(t, err) @@ -3125,8 +3125,8 @@ func TestDecommission(t *testing.T) { // can't use atomic replica swaps because the leaseholder can't be removed. _, err = admin.Decommission( ctx, &serverpb.DecommissionRequest{ - NodeIDs: []roachpb.NodeID{3, 4}, - Decommissioning: true, + NodeIDs: []roachpb.NodeID{3, 4}, + TargetMembership: kvserverpb.MembershipStatus_DECOMMISSIONING, }, ) require.NoError(t, err) diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index d4c35b34db7d..a3a61b766b5b 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -514,9 +514,12 @@ func (nl *NodeLiveness) SetDrainingInternal( } func (nl *NodeLiveness) SetDecommissioningInternal( - ctx context.Context, nodeID roachpb.NodeID, liveness LivenessRecord, decommission bool, + ctx context.Context, + nodeID roachpb.NodeID, + liveness LivenessRecord, + targetStatus kvserverpb.MembershipStatus, ) (changeCommitted bool, err error) { - return nl.setDecommissioningInternal(ctx, nodeID, liveness, decommission) + return nl.setMembershipStatusInternal(ctx, nodeID, liveness, targetStatus) } // GetCircuitBreaker returns the circuit breaker controlling diff --git a/pkg/kv/kvserver/kvserverpb/liveness.go b/pkg/kv/kvserver/kvserverpb/liveness.go index f68fe11edbb3..240c75760abd 100644 --- a/pkg/kv/kvserver/kvserverpb/liveness.go +++ b/pkg/kv/kvserver/kvserverpb/liveness.go @@ -15,6 +15,8 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) // IsLive returns whether the node is considered live at the given time. @@ -41,10 +43,73 @@ func (l *Liveness) IsDead(now time.Time, threshold time.Duration) bool { return !now.Before(deadAsOf) } -func (l Liveness) String() string { +func (l *Liveness) String() string { var extra string - if l.Draining || l.Decommissioning { - extra = fmt.Sprintf(" drain:%t decom:%t", l.Draining, l.Decommissioning) + if l.Draining || l.Membership.Decommissioning() || l.Membership.Decommissioned() { + extra = fmt.Sprintf(" drain:%t membership:%s", l.Draining, l.Membership.String()) } return fmt.Sprintf("liveness(nid:%d epo:%d exp:%s%s)", l.NodeID, l.Epoch, l.Expiration, extra) } + +// Decommissioning is a shorthand to check if the membership status is DECOMMISSIONING. +func (c MembershipStatus) Decommissioning() bool { return c == MembershipStatus_DECOMMISSIONING } + +// Decommissioned is a shorthand to check if the membership status is DECOMMISSIONED. +func (c MembershipStatus) Decommissioned() bool { return c == MembershipStatus_DECOMMISSIONED } + +// Active is a shorthand to check if the membership status is ACTIVE. +func (c MembershipStatus) Active() bool { return c == MembershipStatus_ACTIVE } + +func (c MembershipStatus) String() string { + // NB: These strings must not be changed, since the CLI matches on them. + switch c { + case MembershipStatus_ACTIVE: + return "active" + case MembershipStatus_DECOMMISSIONING: + return "decommissioning" + case MembershipStatus_DECOMMISSIONED: + return "decommissioned" + default: + err := "unknown membership status, expected one of [active,decommissioning,decommissioned]" + panic(err) + } +} + +// ValidateLivenessTransition validates transitions of the liveness record, +// returning an error if the proposed transition is invalid. Ignoring no-ops +// (which also includes decommissioning a decommissioned node) the valid state +// transitions for Membership are as follows: +// +// Decommissioning => Active +// Active => Decommissioning +// Decommissioning => Decommissioned +// +// See diagram above the Membership type for more details. +func ValidateLivenessTransition(old, new Liveness) error { + if old.Membership == new.Membership { + // No-op. + return nil + } + + if old.Membership.Decommissioned() && new.Membership.Decommissioning() { + // No-op. + return nil + } + + if new.Membership.Active() && !old.Membership.Decommissioning() { + err := fmt.Sprintf("can only recommission a decommissioning node; n%d found to be %s", + new.NodeID, old.Membership.String()) + return status.Error(codes.FailedPrecondition, err) + } + + // We don't assert on the new membership being "decommissioning" as all + // previous states are valid (again, consider no-ops). + + if new.Membership.Decommissioned() && !old.Membership.Decommissioning() { + err := fmt.Sprintf("can only fully decommission an already decommissioning node; n%d found to be %s", + new.NodeID, old.Membership.String()) + return status.Error(codes.FailedPrecondition, err) + } + + return nil +} diff --git a/pkg/kv/kvserver/kvserverpb/liveness.pb.go b/pkg/kv/kvserver/kvserverpb/liveness.pb.go index 95861257e154..fe22fd42205b 100644 --- a/pkg/kv/kvserver/kvserverpb/liveness.pb.go +++ b/pkg/kv/kvserver/kvserverpb/liveness.pb.go @@ -23,6 +23,67 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +// MembershipStatus enumerates the possible membership states a node could in. +// +// +--------------------+ +--------------------+ +// | | cockroach node decommission | | +// | |--------------------------------------->| | +// | Active | | Decommissioning | +// | |<---------------------------------------| | +// | | cockroach node recommission | | +// +--------------------+ +--------------------+ +// | +// | +// | +// | +// | +// v +// +--------------------+ +// | | +// | | +// | Decommissioned | +// | | +// | | +// +--------------------+ +// +// Note that we've intentionally elided a 'recommissioning' state. To +// recommission a node is to simply cancel an inflight decommissioning process, +// which we do by persisting the appropriate membership status in the liveness +// record. It then makes no sense to bother with persisting an explicit +// in-progress state. +type MembershipStatus int32 + +const ( + // Active represents a node that is an active member of the cluster, and is + // neither decommissioning nor fully decommissioned. + MembershipStatus_ACTIVE MembershipStatus = 0 + // Decommissioning represents a node that we've only started decommissioning, + // but has not yet fully decommissioned. It is safe to recommission it during + // this time in order to cancel the inflight decommissioning process. + MembershipStatus_DECOMMISSIONING MembershipStatus = 1 + // Decommissioned represents a node that has been fully decommissioned. It + // will no longer be allowed to join the cluster. + // + // TODO(irfansharif): We don't disallow the joining as yet (but will come in + // as part of the Connect RPC subsystem). + MembershipStatus_DECOMMISSIONED MembershipStatus = 2 +) + +var MembershipStatus_name = map[int32]string{ + 0: "ACTIVE", + 1: "DECOMMISSIONING", + 2: "DECOMMISSIONED", +} +var MembershipStatus_value = map[string]int32{ + "ACTIVE": 0, + "DECOMMISSIONING": 1, + "DECOMMISSIONED": 2, +} + +func (MembershipStatus) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_liveness_e11b2e8cd41db0d7, []int{0} +} + // NodeLivenessStatus describes the status of a node from the perspective of the // liveness system. See comment on LivenessStatus() for a description of the // states. @@ -77,7 +138,7 @@ func (x NodeLivenessStatus) String() string { return proto.EnumName(NodeLivenessStatus_name, int32(x)) } func (NodeLivenessStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_liveness_509e0e36d44d5856, []int{0} + return fileDescriptor_liveness_e11b2e8cd41db0d7, []int{1} } // Liveness holds information about a node's latest heartbeat and epoch. @@ -105,15 +166,20 @@ type Liveness struct { // TODO(andrei): Change this to a regular Timestamp field in 21.1. Expiration hlc.LegacyTimestamp `protobuf:"bytes,3,opt,name=expiration,proto3" json:"expiration"` Draining bool `protobuf:"varint,4,opt,name=draining,proto3" json:"draining,omitempty"` - // decommissioning is true if the given node is decommissioning or - // fully decommissioned. - Decommissioning bool `protobuf:"varint,5,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` + // MembershipStatus (one of "active", "decommissioning", "decommissioned") + // is the membership status of the given node. + // + // NB: This field was upgraded from a boolean `decommissioning` field that + // didn't explicitly capture the fully decommissioned state. Care was taken in + // the defining MembershipStatus to be on-the-wire compatible with the boolean + // representation. + Membership MembershipStatus `protobuf:"varint,5,opt,name=membership,proto3,enum=cockroach.kv.kvserver.storagepb.MembershipStatus" json:"membership,omitempty"` } func (m *Liveness) Reset() { *m = Liveness{} } func (*Liveness) ProtoMessage() {} func (*Liveness) Descriptor() ([]byte, []int) { - return fileDescriptor_liveness_509e0e36d44d5856, []int{0} + return fileDescriptor_liveness_e11b2e8cd41db0d7, []int{0} } func (m *Liveness) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -140,6 +206,7 @@ var xxx_messageInfo_Liveness proto.InternalMessageInfo func init() { proto.RegisterType((*Liveness)(nil), "cockroach.kv.kvserver.storagepb.Liveness") + proto.RegisterEnum("cockroach.kv.kvserver.storagepb.MembershipStatus", MembershipStatus_name, MembershipStatus_value) proto.RegisterEnum("cockroach.kv.kvserver.storagepb.NodeLivenessStatus", NodeLivenessStatus_name, NodeLivenessStatus_value) } func (m *Liveness) Marshal() (dAtA []byte, err error) { @@ -185,15 +252,10 @@ func (m *Liveness) MarshalTo(dAtA []byte) (int, error) { } i++ } - if m.Decommissioning { + if m.Membership != 0 { dAtA[i] = 0x28 i++ - if m.Decommissioning { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ + i = encodeVarintLiveness(dAtA, i, uint64(m.Membership)) } return i, nil } @@ -220,7 +282,7 @@ func NewPopulatedLiveness(r randyLiveness, easy bool) *Liveness { v1 := hlc.NewPopulatedLegacyTimestamp(r, easy) this.Expiration = *v1 this.Draining = bool(bool(r.Intn(2) == 0)) - this.Decommissioning = bool(bool(r.Intn(2) == 0)) + this.Membership = MembershipStatus([]int32{0, 1, 2}[r.Intn(3)]) if !easy && r.Intn(10) != 0 { } return this @@ -315,8 +377,8 @@ func (m *Liveness) Size() (n int) { if m.Draining { n += 2 } - if m.Decommissioning { - n += 2 + if m.Membership != 0 { + n += 1 + sovLiveness(uint64(m.Membership)) } return n } @@ -453,9 +515,9 @@ func (m *Liveness) Unmarshal(dAtA []byte) error { m.Draining = bool(v != 0) case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Decommissioning", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Membership", wireType) } - var v int + m.Membership = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowLiveness @@ -465,12 +527,11 @@ func (m *Liveness) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + m.Membership |= (MembershipStatus(b) & 0x7F) << shift if b < 0x80 { break } } - m.Decommissioning = bool(v != 0) default: iNdEx = preIndex skippy, err := skipLiveness(dAtA[iNdEx:]) @@ -598,41 +659,44 @@ var ( ) func init() { - proto.RegisterFile("kv/kvserver/kvserverpb/liveness.proto", fileDescriptor_liveness_509e0e36d44d5856) + proto.RegisterFile("kv/kvserver/kvserverpb/liveness.proto", fileDescriptor_liveness_e11b2e8cd41db0d7) } -var fileDescriptor_liveness_509e0e36d44d5856 = []byte{ - // 501 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x92, 0xcd, 0x8a, 0x9b, 0x50, - 0x1c, 0xc5, 0xbd, 0x99, 0x24, 0x13, 0x6e, 0xa0, 0x91, 0x3b, 0x03, 0x0d, 0x16, 0x54, 0xfa, 0x01, - 0xa1, 0x0c, 0x0a, 0x33, 0x5d, 0x75, 0x67, 0x9a, 0x50, 0xa4, 0x19, 0x03, 0x49, 0x66, 0x0a, 0xb3, - 0x09, 0x7e, 0x5c, 0xcc, 0x25, 0xea, 0x15, 0x35, 0xd2, 0xbe, 0x82, 0xab, 0xd2, 0x4d, 0xbb, 0x11, - 0xe6, 0x31, 0xfa, 0x08, 0x59, 0xce, 0x72, 0x56, 0xa1, 0x35, 0x6f, 0xd1, 0x55, 0x51, 0x27, 0x9f, - 0xd0, 0xdd, 0xef, 0x7f, 0x38, 0xff, 0x73, 0x3d, 0xf2, 0x87, 0x6f, 0xe6, 0xb1, 0x3c, 0x8f, 0x43, - 0x1c, 0xc4, 0x38, 0xd8, 0x82, 0x6f, 0xc8, 0x0e, 0x89, 0xb1, 0x87, 0xc3, 0x50, 0xf2, 0x03, 0x1a, - 0x51, 0x24, 0x98, 0xd4, 0x9c, 0x07, 0x54, 0x37, 0x67, 0xd2, 0x3c, 0x96, 0x36, 0x3e, 0x29, 0x8c, - 0x68, 0xa0, 0xdb, 0xd8, 0x37, 0x38, 0x61, 0x11, 0x11, 0x47, 0x9e, 0x39, 0xa6, 0xec, 0x60, 0x5b, - 0x37, 0xbf, 0x4e, 0x23, 0xe2, 0xe2, 0x30, 0xd2, 0x5d, 0xbf, 0x4c, 0xe0, 0xce, 0x6d, 0x6a, 0xd3, - 0x02, 0xe5, 0x9c, 0x4a, 0xf5, 0xe5, 0x8f, 0x0a, 0x6c, 0x0c, 0x9e, 0x9e, 0x42, 0x77, 0xf0, 0xd4, - 0xa3, 0x16, 0x9e, 0x12, 0xab, 0x0d, 0x44, 0xd0, 0xa9, 0x75, 0x95, 0x6c, 0x25, 0xd4, 0x35, 0x6a, - 0x61, 0xb5, 0xf7, 0x77, 0x25, 0x5c, 0xd9, 0x24, 0x9a, 0x2d, 0x0c, 0xc9, 0xa4, 0xae, 0xbc, 0xfd, - 0x1c, 0xcb, 0xd8, 0xb1, 0xec, 0xcf, 0x6d, 0xb9, 0x20, 0xdf, 0x90, 0xca, 0xb5, 0x51, 0x3d, 0x4f, - 0x54, 0x2d, 0x74, 0x0e, 0x6b, 0xd8, 0xa7, 0xe6, 0xac, 0x5d, 0x11, 0x41, 0xe7, 0x64, 0x54, 0x0e, - 0x48, 0x85, 0x10, 0x7f, 0xf1, 0x49, 0xa0, 0x47, 0x84, 0x7a, 0xed, 0x13, 0x11, 0x74, 0x9a, 0x97, - 0xaf, 0xa4, 0x5d, 0xd7, 0xbc, 0x94, 0x34, 0x73, 0x4c, 0x69, 0x50, 0x94, 0x9a, 0x6c, 0x3a, 0x75, - 0xab, 0xcb, 0x95, 0xc0, 0x8c, 0xf6, 0x96, 0x11, 0x07, 0x1b, 0x56, 0xa0, 0x13, 0x8f, 0x78, 0x76, - 0xbb, 0x2a, 0x82, 0x4e, 0x63, 0xb4, 0x9d, 0x51, 0x07, 0xb6, 0x2c, 0x6c, 0x52, 0xd7, 0x25, 0x61, - 0x48, 0x68, 0x61, 0xa9, 0x15, 0x96, 0x63, 0xf9, 0x7d, 0xe3, 0xe7, 0xbd, 0xc0, 0xfc, 0xba, 0x17, - 0xc0, 0xdb, 0xef, 0x15, 0x88, 0xf2, 0x0e, 0x9b, 0xbf, 0x33, 0x8e, 0xf4, 0x68, 0x11, 0xa2, 0xd7, - 0xf0, 0x4c, 0x1b, 0xf6, 0xfa, 0xd3, 0xf1, 0x44, 0x99, 0xdc, 0x8c, 0xa7, 0x37, 0xda, 0x27, 0x6d, - 0xf8, 0x59, 0x63, 0x19, 0xae, 0x99, 0xa4, 0xe2, 0xe9, 0xd3, 0x88, 0x78, 0xc8, 0xee, 0xbb, 0x7a, - 0x7d, 0xa5, 0xc7, 0x02, 0xae, 0x91, 0xa4, 0x62, 0x35, 0x67, 0x74, 0x01, 0x9f, 0x1f, 0xa6, 0x28, - 0xb7, 0x8a, 0x3a, 0x50, 0xba, 0x83, 0x3e, 0x5b, 0xe1, 0x5a, 0x49, 0x2a, 0x36, 0xf7, 0xa4, 0xe3, - 0xb4, 0x81, 0x7a, 0xdb, 0x67, 0x4f, 0xca, 0xb4, 0x9c, 0xd1, 0x3b, 0xf8, 0xe2, 0xf0, 0xb5, 0x0f, - 0xc3, 0xeb, 0x6b, 0x75, 0x3c, 0x56, 0x87, 0x9a, 0xaa, 0x7d, 0x64, 0xab, 0xdc, 0x59, 0x92, 0x8a, - 0xad, 0x23, 0x19, 0x5d, 0x42, 0xee, 0x7f, 0x5b, 0xfd, 0x1e, 0x5b, 0xe3, 0x50, 0x92, 0x8a, 0xcf, - 0x0e, 0xd5, 0xee, 0xc5, 0xf2, 0x0f, 0xcf, 0x2c, 0x33, 0x1e, 0x3c, 0x64, 0x3c, 0x78, 0xcc, 0x78, - 0xf0, 0x3b, 0xe3, 0xc1, 0xb7, 0x35, 0xcf, 0x3c, 0xac, 0x79, 0xe6, 0x71, 0xcd, 0x33, 0x77, 0x70, - 0x77, 0xc3, 0x46, 0xbd, 0xb8, 0xb1, 0xab, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x3a, 0xde, 0x2c, - 0x89, 0xe4, 0x02, 0x00, 0x00, +var fileDescriptor_liveness_e11b2e8cd41db0d7 = []byte{ + // 549 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x93, 0x3b, 0x6b, 0xdb, 0x50, + 0x14, 0xc7, 0x75, 0xfd, 0x8a, 0x39, 0x86, 0x44, 0x5c, 0x07, 0x6a, 0x54, 0x90, 0x44, 0x1f, 0x60, + 0x42, 0x90, 0xa8, 0xd3, 0xa9, 0x9b, 0x1c, 0x89, 0x22, 0x6a, 0xcb, 0xd4, 0x76, 0x52, 0xc8, 0x62, + 0xf4, 0xb8, 0x48, 0xc2, 0x8f, 0x2b, 0x24, 0xd9, 0xb4, 0x5f, 0xc1, 0x53, 0xe9, 0xd4, 0xc5, 0x90, + 0xa1, 0x43, 0x3f, 0x42, 0xc7, 0x8e, 0x1e, 0x33, 0x66, 0x32, 0xad, 0xfd, 0x2d, 0x3a, 0x15, 0xf9, + 0x6d, 0x43, 0xe9, 0xf6, 0x3f, 0x87, 0xf3, 0xff, 0x1f, 0xfd, 0x8e, 0x24, 0x78, 0xd9, 0x1d, 0xc9, + 0xdd, 0x51, 0x44, 0xc2, 0x11, 0x09, 0xb7, 0x22, 0xb0, 0xe4, 0x9e, 0x3f, 0x22, 0x03, 0x12, 0x45, + 0x52, 0x10, 0xd2, 0x98, 0x62, 0xc1, 0xa6, 0x76, 0x37, 0xa4, 0xa6, 0xed, 0x49, 0xdd, 0x91, 0xb4, + 0x99, 0x93, 0xa2, 0x98, 0x86, 0xa6, 0x4b, 0x02, 0x8b, 0x13, 0x86, 0xb1, 0xdf, 0x93, 0xbd, 0x9e, + 0x2d, 0xf7, 0x88, 0x6b, 0xda, 0x9f, 0x3a, 0xb1, 0xdf, 0x27, 0x51, 0x6c, 0xf6, 0x83, 0x55, 0x02, + 0x77, 0xee, 0x52, 0x97, 0x2e, 0xa5, 0x9c, 0xa8, 0x55, 0xf7, 0xd9, 0xcf, 0x14, 0xe4, 0x6b, 0xeb, + 0x55, 0xf8, 0x0e, 0x4e, 0x06, 0xd4, 0x21, 0x1d, 0xdf, 0x29, 0x21, 0x11, 0x95, 0xb3, 0x55, 0x65, + 0x3e, 0x13, 0x72, 0x06, 0x75, 0x88, 0xae, 0xfe, 0x99, 0x09, 0x57, 0xae, 0x1f, 0x7b, 0x43, 0x4b, + 0xb2, 0x69, 0x5f, 0xde, 0x3e, 0x8e, 0x63, 0xed, 0xb4, 0x1c, 0x74, 0x5d, 0x79, 0xa9, 0x02, 0x4b, + 0x5a, 0xd9, 0x9a, 0xb9, 0x24, 0x51, 0x77, 0xf0, 0x39, 0x64, 0x49, 0x40, 0x6d, 0xaf, 0x94, 0x12, + 0x51, 0x39, 0xdd, 0x5c, 0x15, 0x58, 0x07, 0x20, 0x1f, 0x03, 0x3f, 0x34, 0x63, 0x9f, 0x0e, 0x4a, + 0x69, 0x11, 0x95, 0x0b, 0x95, 0xe7, 0xd2, 0x8e, 0x35, 0x81, 0x92, 0xbc, 0x9e, 0x2d, 0xd5, 0x96, + 0x50, 0xed, 0x0d, 0x53, 0x35, 0x33, 0x9d, 0x09, 0x4c, 0x73, 0xcf, 0x8c, 0x39, 0xc8, 0x3b, 0xa1, + 0xe9, 0x0f, 0xfc, 0x81, 0x5b, 0xca, 0x88, 0xa8, 0x9c, 0x6f, 0x6e, 0x6b, 0xfc, 0x1e, 0xa0, 0x4f, + 0xfa, 0x16, 0x09, 0x23, 0xcf, 0x0f, 0x4a, 0x59, 0x11, 0x95, 0x4f, 0x2b, 0xaf, 0xa4, 0xff, 0x9c, + 0x54, 0xaa, 0x6f, 0x2d, 0xad, 0xd8, 0x8c, 0x87, 0x51, 0x73, 0x2f, 0xe4, 0x4d, 0xfe, 0xeb, 0xbd, + 0xc0, 0xfc, 0xb8, 0x17, 0xd0, 0x45, 0x1d, 0xd8, 0xe3, 0x49, 0x0c, 0x90, 0x53, 0xae, 0xdb, 0xfa, + 0xad, 0xc6, 0x32, 0xb8, 0x08, 0x67, 0xaa, 0x76, 0xdd, 0xa8, 0xd7, 0xf5, 0x56, 0x4b, 0x6f, 0x18, + 0xba, 0xf1, 0x96, 0x45, 0x18, 0xc3, 0xe9, 0x7e, 0x53, 0x53, 0xd9, 0x14, 0x97, 0xf9, 0xfe, 0x8d, + 0x67, 0x2e, 0xbe, 0xa4, 0x00, 0x27, 0xb7, 0xdb, 0xbc, 0x95, 0x75, 0xe2, 0x0b, 0x28, 0x1a, 0x0d, + 0x55, 0xeb, 0xb4, 0xda, 0x4a, 0xfb, 0xa6, 0xd5, 0xb9, 0x31, 0xde, 0x19, 0x8d, 0x0f, 0x06, 0xcb, + 0x70, 0x85, 0xf1, 0x44, 0x3c, 0x59, 0x97, 0x98, 0x07, 0x76, 0x7f, 0x4a, 0xd5, 0x14, 0x95, 0x45, + 0x5c, 0x7e, 0x3c, 0x11, 0x33, 0x89, 0xc6, 0x97, 0xf0, 0xe4, 0x30, 0x45, 0xb9, 0x55, 0xf4, 0x9a, + 0x52, 0xad, 0x69, 0x6c, 0x8a, 0x3b, 0x1b, 0x4f, 0xc4, 0xc2, 0x5e, 0xeb, 0x38, 0xad, 0x96, 0xf0, + 0xa4, 0x57, 0x69, 0x89, 0xc6, 0xaf, 0xe1, 0xe9, 0xe1, 0xb6, 0x43, 0xca, 0x0c, 0x57, 0x1c, 0x4f, + 0xc4, 0x63, 0x78, 0x5c, 0x01, 0xee, 0x5f, 0x2e, 0x4d, 0x65, 0xb3, 0x1c, 0x1e, 0x4f, 0xc4, 0xa3, + 0xe3, 0x54, 0x2f, 0xa7, 0xbf, 0x79, 0x66, 0x3a, 0xe7, 0xd1, 0xc3, 0x9c, 0x47, 0x8f, 0x73, 0x1e, + 0xfd, 0x9a, 0xf3, 0xe8, 0xf3, 0x82, 0x67, 0x1e, 0x16, 0x3c, 0xf3, 0xb8, 0xe0, 0x99, 0x3b, 0xd8, + 0xfd, 0x3b, 0x56, 0x6e, 0xf9, 0x6d, 0x5f, 0xfd, 0x0d, 0x00, 0x00, 0xff, 0xff, 0x0a, 0x92, 0xcd, + 0xa4, 0x5c, 0x03, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/kvserverpb/liveness.proto b/pkg/kv/kvserver/kvserverpb/liveness.proto index 03e8bd92cf63..45047d6cea23 100644 --- a/pkg/kv/kvserver/kvserverpb/liveness.proto +++ b/pkg/kv/kvserver/kvserverpb/liveness.proto @@ -45,9 +45,61 @@ message Liveness { util.hlc.LegacyTimestamp expiration = 3 [(gogoproto.nullable) = false]; bool draining = 4; - // decommissioning is true if the given node is decommissioning or - // fully decommissioned. - bool decommissioning = 5; + // MembershipStatus (one of "active", "decommissioning", "decommissioned") + // is the membership status of the given node. + // + // NB: This field was upgraded from a boolean `decommissioning` field that + // didn't explicitly capture the fully decommissioned state. Care was taken in + // the defining MembershipStatus to be on-the-wire compatible with the boolean + // representation. + MembershipStatus membership = 5; +} + +// MembershipStatus enumerates the possible membership states a node could in. +// +// +--------------------+ +--------------------+ +// | | cockroach node decommission | | +// | |--------------------------------------->| | +// | Active | | Decommissioning | +// | |<---------------------------------------| | +// | | cockroach node recommission | | +// +--------------------+ +--------------------+ +// | +// | +// | +// | +// | +// v +// +--------------------+ +// | | +// | | +// | Decommissioned | +// | | +// | | +// +--------------------+ +// +// Note that we've intentionally elided a 'recommissioning' state. To +// recommission a node is to simply cancel an inflight decommissioning process, +// which we do by persisting the appropriate membership status in the liveness +// record. It then makes no sense to bother with persisting an explicit +// in-progress state. +enum MembershipStatus { + option (gogoproto.goproto_enum_stringer) = false; + // Active represents a node that is an active member of the cluster, and is + // neither decommissioning nor fully decommissioned. + ACTIVE = 0; + + // Decommissioning represents a node that we've only started decommissioning, + // but has not yet fully decommissioned. It is safe to recommission it during + // this time in order to cancel the inflight decommissioning process. + DECOMMISSIONING = 1; + + // Decommissioned represents a node that has been fully decommissioned. It + // will no longer be allowed to join the cluster. + // + // TODO(irfansharif): We don't disallow the joining as yet (but will come in + // as part of the Connect RPC subsystem). + DECOMMISSIONED = 2; } // NodeLivenessStatus describes the status of a node from the perspective of the diff --git a/pkg/kv/kvserver/node_liveness.go b/pkg/kv/kvserver/node_liveness.go index c47ad7d4a7aa..1e9cfa9a594e 100644 --- a/pkg/kv/kvserver/node_liveness.go +++ b/pkg/kv/kvserver/node_liveness.go @@ -43,7 +43,10 @@ var ( // about a node for which nothing is known. ErrNoLivenessRecord = errors.New("node not in the liveness table") - errChangeDecommissioningFailed = errors.New("failed to change the decommissioning status") + // errChangeMembershipStatusFailed is returned when we're not able to + // conditionally write the target membership status. It's safe to retry + // when encountering this error. + errChangeMembershipStatusFailed = errors.New("failed to change the membership status") // ErrEpochIncremented is returned when a heartbeat request fails because // the underlying liveness record has had its epoch incremented. @@ -245,11 +248,11 @@ func (nl *NodeLiveness) sem(nodeID roachpb.NodeID) chan struct{} { func (nl *NodeLiveness) SetDraining(ctx context.Context, drain bool, reporter func(int, string)) { ctx = nl.ambientCtx.AnnotateCtx(ctx) for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); { - liveness, err := nl.SelfEx() + oldLivenessRec, err := nl.SelfEx() if err != nil && !errors.Is(err, ErrNoLivenessRecord) { log.Errorf(ctx, "unexpected error getting liveness: %+v", err) } - err = nl.setDrainingInternal(ctx, liveness, drain, reporter) + err = nl.setDrainingInternal(ctx, oldLivenessRec, drain, reporter) if err != nil { if log.V(1) { log.Infof(ctx, "attempting to set liveness draining status to %v: %v", drain, err) @@ -260,12 +263,14 @@ func (nl *NodeLiveness) SetDraining(ctx context.Context, drain bool, reporter fu } } -// SetDecommissioning runs a best-effort attempt of marking the the liveness -// record as decommissioning. It returns whether the function committed a -// transaction that updated the liveness record. -func (nl *NodeLiveness) SetDecommissioning( - ctx context.Context, nodeID roachpb.NodeID, decommission bool, -) (changeCommitted bool, err error) { +// SetMembershipStatus changes the liveness record to reflect the target +// membership status. It does so idempotently, and may retry internally until it +// observes its target state durably persisted. It returns whether it was able +// to change the membership status (as opposed to it returning early when +// finding the target status possibly set by another node). +func (nl *NodeLiveness) SetMembershipStatus( + ctx context.Context, nodeID roachpb.NodeID, targetStatus kvserverpb.MembershipStatus, +) (statusChanged bool, err error) { ctx = nl.ambientCtx.AnnotateCtx(ctx) attempt := func() (bool, error) { @@ -322,31 +327,31 @@ func (nl *NodeLiveness) SetDecommissioning( return false, errors.Wrap(err, "invalid liveness record") } - rec := LivenessRecord{ + oldLivenessRec := LivenessRecord{ Liveness: oldLiveness, raw: kv.Value.TagAndDataBytes(), } - // We may have discovered a Liveness not yet received via Gossip. Offer it - // to make sure that when we actually try to update the liveness, the - // previous view is correct. This, too, is required to de-flake - // TestNodeLivenessDecommissionAbsent. - nl.maybeUpdate(rec) - - return nl.setDecommissioningInternal(ctx, nodeID, rec, decommission) + // We may have discovered a Liveness not yet received via Gossip. + // Offer it to make sure that when we actually try to update the + // liveness, the previous view is correct. This, too, is required to + // de-flake TestNodeLivenessDecommissionAbsent. + nl.maybeUpdate(oldLivenessRec) + return nl.setMembershipStatusInternal(ctx, nodeID, oldLivenessRec, targetStatus) } for { - changeCommitted, err := attempt() - if errors.Is(err, errChangeDecommissioningFailed) { - continue // expected when epoch incremented + statusChanged, err := attempt() + if errors.Is(err, errChangeMembershipStatusFailed) { + // Expected when epoch incremented, it's safe to retry. + continue } - return changeCommitted, err + return statusChanged, err } } func (nl *NodeLiveness) setDrainingInternal( - ctx context.Context, liveness LivenessRecord, drain bool, reporter func(int, string), + ctx context.Context, oldLivenessRec LivenessRecord, drain bool, reporter func(int, string), ) error { nodeID := nl.gossip.NodeID.Get() sem := nl.sem(nodeID) @@ -360,27 +365,33 @@ func (nl *NodeLiveness) setDrainingInternal( <-sem }() - update := livenessUpdate{ - newLiveness: kvserverpb.Liveness{ + // Let's compute what our new liveness record should be. + var newLiveness kvserverpb.Liveness + if oldLivenessRec.Liveness == (kvserverpb.Liveness{}) { + // Liveness record didn't previously exist, so we create one. + newLiveness = kvserverpb.Liveness{ NodeID: nodeID, Epoch: 1, - }, - oldLiveness: liveness.Liveness, - ignoreCache: true, - oldRaw: liveness.raw, - } - if liveness.Liveness != (kvserverpb.Liveness{}) { - update.newLiveness = liveness.Liveness + } + } else { + newLiveness = oldLivenessRec.Liveness } - if reporter != nil && drain && !update.newLiveness.Draining { + if reporter != nil && drain && !newLiveness.Draining { // Report progress to the Drain RPC. reporter(1, "liveness record") } - update.newLiveness.Draining = drain + newLiveness.Draining = drain + update := livenessUpdate{ + oldLiveness: oldLivenessRec.Liveness, + newLiveness: newLiveness, + oldRaw: oldLivenessRec.raw, + ignoreCache: true, + } written, err := nl.updateLiveness(ctx, update, func(actual LivenessRecord) error { nl.maybeUpdate(actual) + if actual.Draining == update.newLiveness.Draining { return errNodeDrainingSet } @@ -395,6 +406,7 @@ func (nl *NodeLiveness) setDrainingInternal( } return err } + nl.maybeUpdate(written) return nil } @@ -424,35 +436,77 @@ type livenessUpdate struct { oldRaw []byte } -func (nl *NodeLiveness) setDecommissioningInternal( - ctx context.Context, nodeID roachpb.NodeID, liveness LivenessRecord, decommission bool, -) (changeCommitted bool, err error) { - update := livenessUpdate{ - newLiveness: kvserverpb.Liveness{ +func (nl *NodeLiveness) setMembershipStatusInternal( + ctx context.Context, + nodeID roachpb.NodeID, + oldLivenessRec LivenessRecord, + targetStatus kvserverpb.MembershipStatus, +) (statusChanged bool, err error) { + // Let's compute what our new liveness record should be. + var newLiveness kvserverpb.Liveness + if oldLivenessRec.Liveness == (kvserverpb.Liveness{}) { + // Liveness record didn't previously exist, so we create one. + // + // TODO(irfansharif): This code feels a bit unwieldy because it's + // possible for a liveness record to not exist previously. It is just + // generally difficult to write it at startup. When a node joins the + // cluster, this completes before it has had a chance to write its + // liveness record. If it gets decommissioned immediately, there won't + // be one yet. The Connect RPC can solve this though, I think? We can + // bootstrap clusters with a liveness record for n1. Any other node at + // some point has to join the cluster for the first time via the Connect + // RPC, which as part of its job can make sure the liveness record + // exists before responding to the new node. + newLiveness = kvserverpb.Liveness{ NodeID: nodeID, Epoch: 1, - }, - oldLiveness: liveness.Liveness, - ignoreCache: true, - oldRaw: liveness.raw, + } + } else { + // We start off with a copy of our existing liveness record. + newLiveness = oldLivenessRec.Liveness } - if liveness.Liveness != (kvserverpb.Liveness{}) { - update.newLiveness = liveness.Liveness + + newLiveness.Membership = targetStatus + if oldLivenessRec.Membership == newLiveness.Membership { + // No-op. Return early. + return false, nil + } else if oldLivenessRec.Membership.Decommissioned() && + newLiveness.Membership.Decommissioning() { + // Marking a decommissioned node for decommissioning is a no-op. We + // return early. + return false, nil } - update.newLiveness.Decommissioning = decommission - var conditionFailed bool + if oldLivenessRec.Liveness != (kvserverpb.Liveness{}) { + err := kvserverpb.ValidateLivenessTransition(oldLivenessRec.Liveness, newLiveness) + if err != nil { + return false, err + } + } + + update := livenessUpdate{ + newLiveness: newLiveness, + oldLiveness: oldLivenessRec.Liveness, + oldRaw: oldLivenessRec.raw, + ignoreCache: true, + } + statusChanged = true if _, err := nl.updateLiveness(ctx, update, func(actual LivenessRecord) error { - conditionFailed = true - if actual.Decommissioning == update.newLiveness.Decommissioning { - return nil + if actual.Membership != update.newLiveness.Membership { + // We're racing with another attempt at updating the liveness + // record, we error out in order to retry. + return errChangeMembershipStatusFailed } - return errChangeDecommissioningFailed + // The found liveness membership status is the same as the target one, + // so we consider our work done. We inform the caller that this attempt + // was a no-op. + statusChanged = false + return nil }); err != nil { return false, err } - committed := !conditionFailed && liveness.Decommissioning != decommission - return committed, nil + + return statusChanged, nil } // GetLivenessThreshold returns the maximum duration between heartbeats @@ -602,7 +656,7 @@ func (nl *NodeLiveness) Heartbeat(ctx context.Context, liveness kvserverpb.Liven } func (nl *NodeLiveness) heartbeatInternal( - ctx context.Context, liveness kvserverpb.Liveness, incrementEpoch bool, + ctx context.Context, oldLiveness kvserverpb.Liveness, incrementEpoch bool, ) error { ctx, sp := tracing.EnsureChildSpan(ctx, nl.ambientCtx.Tracer, "liveness heartbeat") defer sp.Finish() @@ -626,37 +680,44 @@ func (nl *NodeLiveness) heartbeatInternal( <-sem }() - update := livenessUpdate{ - newLiveness: kvserverpb.Liveness{ + // Let's compute what our new liveness record should be. + var newLiveness kvserverpb.Liveness + if oldLiveness == (kvserverpb.Liveness{}) { + // Liveness record didn't previously exist, so we create one. + newLiveness = kvserverpb.Liveness{ NodeID: nodeID, Epoch: 1, - }, - oldLiveness: liveness, - } - if liveness != (kvserverpb.Liveness{}) { - update.newLiveness = liveness + } + } else { + newLiveness = oldLiveness if incrementEpoch { - update.newLiveness.Epoch++ - // Clear draining field. - update.newLiveness.Draining = false + newLiveness.Epoch++ + newLiveness.Draining = false // Clear draining field. } } + // We need to add the maximum clock offset to the expiration because it's // used when determining liveness for a node. { - update.newLiveness.Expiration = hlc.LegacyTimestamp( + newLiveness.Expiration = hlc.LegacyTimestamp( nl.clock.Now().Add((nl.livenessThreshold).Nanoseconds(), 0)) // This guards against the system clock moving backwards. As long // as the cockroach process is running, checks inside hlc.Clock // will ensure that the clock never moves backwards, but these // checks don't work across process restarts. - if update.newLiveness.Expiration.Less(liveness.Expiration) { + if newLiveness.Expiration.Less(oldLiveness.Expiration) { return errors.Errorf("proposed liveness update expires earlier than previous record") } } + + update := livenessUpdate{ + oldLiveness: oldLiveness, + newLiveness: newLiveness, + } written, err := nl.updateLiveness(ctx, update, func(actual LivenessRecord) error { // Update liveness to actual value on mismatch. nl.maybeUpdate(actual) + // If the actual liveness is different than expected, but is // considered live, treat the heartbeat as a success. This can // happen when the periodic heartbeater races with a concurrent @@ -730,7 +791,7 @@ func (nl *NodeLiveness) GetIsLiveMap() IsLiveMap { now := nl.clock.Now().GoTime() for nID, l := range nl.mu.nodes { isLive := l.IsLive(now) - if !isLive && l.Decommissioning { + if !isLive && !l.Membership.Active() { // This is a node that was completely removed. Skip over it. continue } @@ -818,8 +879,10 @@ func (nl *NodeLiveness) IncrementEpoch(ctx context.Context, liveness kvserverpb. oldLiveness: liveness, } update.newLiveness.Epoch++ + written, err := nl.updateLiveness(ctx, update, func(actual LivenessRecord) error { - defer nl.maybeUpdate(actual) + nl.maybeUpdate(actual) + if actual.Epoch > liveness.Epoch { return ErrEpochAlreadyIncremented } else if actual.Epoch < liveness.Epoch { @@ -987,8 +1050,8 @@ func (nl *NodeLiveness) maybeUpdate(new LivenessRecord) { } nl.mu.Lock() - // Note that this works fine even if `old` is empty. old := nl.mu.nodes[new.NodeID] + should := shouldReplaceLiveness(old.Liveness, new.Liveness) var callbacks []IsLiveCallback if should { @@ -1009,12 +1072,14 @@ func (nl *NodeLiveness) maybeUpdate(new LivenessRecord) { } } +// shouldReplaceLiveness checks to see if the new liveness is in fact newer +// than the old liveness. func shouldReplaceLiveness(old, new kvserverpb.Liveness) bool { if (old == kvserverpb.Liveness{}) { return true } - // Compare first Epoch, and no change there, Expiration. + // Compare Epoch, and if no change there, Expiration. if old.Epoch != new.Epoch { return old.Epoch < new.Epoch } @@ -1030,12 +1095,13 @@ func shouldReplaceLiveness(old, new kvserverpb.Liveness) bool { // number. // // See #18219. - return old.Draining != new.Draining || old.Decommissioning != new.Decommissioning + return old.Draining != new.Draining || + old.Membership != new.Membership } // livenessGossipUpdate is the gossip callback used to keep the // in-memory liveness info up to date. -func (nl *NodeLiveness) livenessGossipUpdate(key string, content roachpb.Value) { +func (nl *NodeLiveness) livenessGossipUpdate(_ string, content roachpb.Value) { var liveness kvserverpb.Liveness if err := content.GetProto(&liveness); err != nil { log.Errorf(context.TODO(), "%v", err) @@ -1111,7 +1177,7 @@ func (nl *NodeLiveness) GetNodeCount() int { defer nl.mu.RUnlock() var count int for _, l := range nl.mu.nodes { - if !l.Decommissioning { + if l.Membership.Active() { count++ } } diff --git a/pkg/kv/kvserver/node_liveness_test.go b/pkg/kv/kvserver/node_liveness_test.go index c7994ab69929..dea397e9eee7 100644 --- a/pkg/kv/kvserver/node_liveness_test.go +++ b/pkg/kv/kvserver/node_liveness_test.go @@ -782,8 +782,11 @@ func verifyNodeIsDecommissioning(t *testing.T, mtc *multiTestContext, nodeID roa for _, nl := range mtc.nodeLivenesses { livenesses := nl.GetLivenesses() for _, liveness := range livenesses { - if liveness.Decommissioning != (liveness.NodeID == nodeID) { - return errors.Errorf("unexpected Decommissioning value of %v for node %v", liveness.Decommissioning, liveness.NodeID) + if liveness.NodeID != nodeID { + continue + } + if !liveness.Membership.Decommissioning() { + return errors.Errorf("unexpected Membership value of %v for node %v", liveness.Membership, liveness.NodeID) } } } @@ -859,15 +862,15 @@ func TestNodeLivenessStatusMap(t *testing.T) { log.Infof(ctx, "done shutting down node %d", deadNodeID) decommissioningNodeID := tc.Server(2).NodeID() - log.Infof(ctx, "decommissioning node %d", decommissioningNodeID) - if err := firstServer.Decommission(ctx, true, []roachpb.NodeID{decommissioningNodeID}); err != nil { + log.Infof(ctx, "marking node %d as decommissioning", decommissioningNodeID) + if err := firstServer.Decommission(ctx, kvserverpb.MembershipStatus_DECOMMISSIONING, []roachpb.NodeID{decommissioningNodeID}); err != nil { t.Fatal(err) } - log.Infof(ctx, "done decommissioning node %d", decommissioningNodeID) + log.Infof(ctx, "marked node %d as decommissioning", decommissioningNodeID) removedNodeID := tc.Server(3).NodeID() - log.Infof(ctx, "decommissioning and shutting down node %d", removedNodeID) - if err := firstServer.Decommission(ctx, true, []roachpb.NodeID{removedNodeID}); err != nil { + log.Infof(ctx, "marking node %d as decommissioning and shutting it down", removedNodeID) + if err := firstServer.Decommission(ctx, kvserverpb.MembershipStatus_DECOMMISSIONING, []roachpb.NodeID{removedNodeID}); err != nil { t.Fatal(err) } tc.StopServer(3) @@ -946,13 +949,14 @@ func testNodeLivenessSetDecommissioning(t *testing.T, decommissionNodeIdx int) { // Verify success on failed update of a liveness record that already has the // given decommissioning setting. if _, err := callerNodeLiveness.SetDecommissioningInternal( - ctx, nodeID, kvserver.LivenessRecord{}, false, + ctx, nodeID, kvserver.LivenessRecord{}, kvserverpb.MembershipStatus_ACTIVE, ); err != nil { t.Fatal(err) } // Set a node to decommissioning state. - if _, err := callerNodeLiveness.SetDecommissioning(ctx, nodeID, true); err != nil { + if _, err := callerNodeLiveness.SetMembershipStatus( + ctx, nodeID, kvserverpb.MembershipStatus_DECOMMISSIONING); err != nil { t.Fatal(err) } verifyNodeIsDecommissioning(t, mtc, nodeID) @@ -1000,8 +1004,8 @@ func TestNodeLivenessDecommissionAbsent(t *testing.T) { const goneNodeID = roachpb.NodeID(10000) // When the node simply never existed, expect an error. - if _, err := mtc.nodeLivenesses[0].SetDecommissioning( - ctx, goneNodeID, true, + if _, err := mtc.nodeLivenesses[0].SetMembershipStatus( + ctx, goneNodeID, kvserverpb.MembershipStatus_DECOMMISSIONING, ); !errors.Is(err, kvserver.ErrNoLivenessRecord) { t.Fatal(err) } @@ -1011,36 +1015,42 @@ func TestNodeLivenessDecommissionAbsent(t *testing.T) { NodeID: goneNodeID, Epoch: 1, Expiration: hlc.LegacyTimestamp(mtc.clock().Now()), + Membership: kvserverpb.MembershipStatus_ACTIVE, }, nil); err != nil { t.Fatal(err) } // Decommission from second node. - if committed, err := mtc.nodeLivenesses[1].SetDecommissioning(ctx, goneNodeID, true); err != nil { + if committed, err := mtc.nodeLivenesses[1].SetMembershipStatus( + ctx, goneNodeID, kvserverpb.MembershipStatus_DECOMMISSIONING); err != nil { t.Fatal(err) } else if !committed { t.Fatal("no change committed") } // Re-decommission from first node. - if committed, err := mtc.nodeLivenesses[0].SetDecommissioning(ctx, goneNodeID, true); err != nil { + if committed, err := mtc.nodeLivenesses[0].SetMembershipStatus( + ctx, goneNodeID, kvserverpb.MembershipStatus_DECOMMISSIONING); err != nil { t.Fatal(err) } else if committed { t.Fatal("spurious change committed") } // Recommission from first node. - if committed, err := mtc.nodeLivenesses[0].SetDecommissioning(ctx, goneNodeID, false); err != nil { + if committed, err := mtc.nodeLivenesses[0].SetMembershipStatus( + ctx, goneNodeID, kvserverpb.MembershipStatus_ACTIVE); err != nil { t.Fatal(err) } else if !committed { t.Fatal("no change committed") } // Decommission from second node (a second time). - if committed, err := mtc.nodeLivenesses[1].SetDecommissioning(ctx, goneNodeID, true); err != nil { + if committed, err := mtc.nodeLivenesses[1].SetMembershipStatus( + ctx, goneNodeID, kvserverpb.MembershipStatus_DECOMMISSIONING); err != nil { t.Fatal(err) } else if !committed { t.Fatal("no change committed") } // Recommission from third node. - if committed, err := mtc.nodeLivenesses[2].SetDecommissioning(ctx, goneNodeID, false); err != nil { + if committed, err := mtc.nodeLivenesses[2].SetMembershipStatus( + ctx, goneNodeID, kvserverpb.MembershipStatus_ACTIVE); err != nil { t.Fatal(err) } else if !committed { t.Fatal("no change committed") diff --git a/pkg/kv/kvserver/node_liveness_unit_test.go b/pkg/kv/kvserver/node_liveness_unit_test.go index ec8066db7571..17fc439995dc 100644 --- a/pkg/kv/kvserver/node_liveness_unit_test.go +++ b/pkg/kv/kvserver/node_liveness_unit_test.go @@ -11,6 +11,7 @@ package kvserver import ( + "fmt" "testing" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -23,12 +24,26 @@ func TestShouldReplaceLiveness(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - l := func(epo int64, expiration hlc.Timestamp, draining, decom bool) kvserverpb.Liveness { + toMembershipStatus := func(membership string) kvserverpb.MembershipStatus { + switch membership { + case "active": + return kvserverpb.MembershipStatus_ACTIVE + case "decommissioning": + return kvserverpb.MembershipStatus_DECOMMISSIONING + case "decommissioned": + return kvserverpb.MembershipStatus_DECOMMISSIONED + default: + err := fmt.Sprintf("unexpected membership: %s", membership) + panic(err) + } + } + + l := func(epo int64, expiration hlc.Timestamp, draining bool, membership string) kvserverpb.Liveness { return kvserverpb.Liveness{ - Epoch: epo, - Expiration: hlc.LegacyTimestamp(expiration), - Draining: draining, - Decommissioning: decom, + Epoch: epo, + Expiration: hlc.LegacyTimestamp(expiration), + Draining: draining, + Membership: toMembershipStatus(membership), } } const ( @@ -44,43 +59,49 @@ func TestShouldReplaceLiveness(t *testing.T) { { // Epoch update only. kvserverpb.Liveness{}, - l(1, hlc.Timestamp{}, false, false), + l(1, hlc.Timestamp{}, false, "active"), yes, }, { // No Epoch update, but Expiration update. - l(1, now, false, false), - l(1, now.Add(0, 1), false, false), + l(1, now, false, "active"), + l(1, now.Add(0, 1), false, "active"), yes, }, { // No update. - l(1, now, false, false), - l(1, now, false, false), + l(1, now, false, "active"), + l(1, now, false, "active"), no, }, { // Only Decommissioning changes. - l(1, now, false, false), - l(1, now, false, true), + l(1, now, false, "active"), + l(1, now, false, "decommissioning"), + yes, + }, + { + // Only Decommissioning changes. + l(1, now, false, "decommissioned"), + l(1, now, false, "decommissioning"), yes, }, { // Only Draining changes. - l(1, now, false, false), - l(1, now, true, false), + l(1, now, false, "active"), + l(1, now, true, "active"), yes, }, { // Decommissioning changes, but Epoch moves backwards. - l(10, now, true, true), - l(9, now, true, false), + l(10, now, true, "decommissioning"), + l(9, now, true, "active"), no, }, { // Draining changes, but Expiration moves backwards.. - l(10, now, false, false), - l(10, now.Add(-1, 0), true, false), + l(10, now, false, "active"), + l(10, now.Add(-1, 0), true, "active"), no, }, } { diff --git a/pkg/kv/kvserver/replica_gc_queue.go b/pkg/kv/kvserver/replica_gc_queue.go index 88c5add46808..2ec4e4daad4a 100644 --- a/pkg/kv/kvserver/replica_gc_queue.go +++ b/pkg/kv/kvserver/replica_gc_queue.go @@ -158,13 +158,15 @@ func (rgcq *replicaGCQueue) shouldQueue( (raftStatus.SoftState.RaftState == raft.StateCandidate || raftStatus.SoftState.RaftState == raft.StatePreCandidate) } else { - // If a replica doesn't have an active raft group, we should check whether - // we're decommissioning. If so, we should process the replica because it - // has probably already been removed from its raft group but doesn't know it. - // Without this, node decommissioning can stall on such dormant ranges. - // Make sure NodeLiveness isn't nil because it can be in tests/benchmarks. + // If a replica doesn't have an active raft group, we should check + // whether or not it is active. If not, we should process the replica + // because it has probably already been removed from its raft group but + // doesn't know it. Without this, node decommissioning can stall on such + // dormant ranges. Make sure NodeLiveness isn't nil because it can be in + // tests/benchmarks. if repl.store.cfg.NodeLiveness != nil { - if liveness, err := repl.store.cfg.NodeLiveness.Self(); err == nil && liveness.Decommissioning { + if liveness, err := repl.store.cfg.NodeLiveness.Self(); err == nil && + !liveness.Membership.Active() { return true, replicaGCPriorityDefault } } diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index 77d4f3e636bd..eb57557e289a 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -13,6 +13,7 @@ package kvserver import ( "context" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" @@ -176,6 +177,15 @@ func (r *Replica) MaybeGossipNodeLiveness(ctx context.Context, span roachpb.Span continue } } + if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionNodeMembershipStatus) { + // We can't transmit liveness records with a backwards incompatible + // representation unless we're told by the user that there are no + // pre-v20.1 nodes around. We should never get here. + if kvLiveness.Membership.Decommissioned() { + log.Fatal(ctx, "programming error: illegal membership status: decommissioned") + } + } + if err := r.store.Gossip().AddInfoProto(key, &kvLiveness, 0); err != nil { return errors.Wrapf(err, "failed to gossip node liveness (%+v)", kvLiveness) } diff --git a/pkg/kv/kvserver/store_pool.go b/pkg/kv/kvserver/store_pool.go index 6f7e73758574..ccf3e133edc8 100644 --- a/pkg/kv/kvserver/store_pool.go +++ b/pkg/kv/kvserver/store_pool.go @@ -127,20 +127,28 @@ func MakeStorePoolNodeLivenessFunc(nodeLiveness *NodeLiveness) NodeLivenessFunc // // - Let's say a node write its liveness record at tWrite. It sets the // Expiration field of the record as tExp=tWrite+livenessThreshold. -// The node is considered LIVE (or DECOMISSIONING or UNAVAILABLE if draining). +// The node is considered LIVE (or DECOMMISSIONING or UNAVAILABLE if draining). // - At tExp, the IsLive() method starts returning false. The state becomes -// UNAVAILABLE (or stays DECOMISSIONING or UNAVAILABLE if draining). +// UNAVAILABLE (or stays DECOMMISSIONING or UNAVAILABLE if draining). // - Once threshold passes, the node is considered DEAD (or DECOMMISSIONED). +// +// NB: There's a bit of discrepancy between what "Decommissioned" represents, as +// seen by NodeStatusLiveness, and what "Decommissioned" represents as +// understood by MembershipStatus. Currently it's possible for a live node, that +// was marked as fully decommissioned, to have a NodeLivenessStatus of +// "Decommissioning". This was kept this way for backwards compatibility, and +// ideally we should remove usage of NodeLivenessStatus altogether. See #50707 +// for more details. func LivenessStatus( l kvserverpb.Liveness, now time.Time, deadThreshold time.Duration, ) kvserverpb.NodeLivenessStatus { if l.IsDead(now, deadThreshold) { - if l.Decommissioning { + if !l.Membership.Active() { return kvserverpb.NodeLivenessStatus_DECOMMISSIONED } return kvserverpb.NodeLivenessStatus_DEAD } - if l.Decommissioning { + if !l.Membership.Active() { return kvserverpb.NodeLivenessStatus_DECOMMISSIONING } if l.Draining { diff --git a/pkg/kv/kvserver/store_pool_test.go b/pkg/kv/kvserver/store_pool_test.go index e74209c32678..5bbbe381efac 100644 --- a/pkg/kv/kvserver/store_pool_test.go +++ b/pkg/kv/kvserver/store_pool_test.go @@ -993,8 +993,7 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { Expiration: hlc.LegacyTimestamp{ WallTime: now.Add(5 * time.Minute).UnixNano(), }, - Decommissioning: false, - Draining: false, + Draining: false, }, expected: kvserverpb.NodeLivenessStatus_LIVE, }, @@ -1006,8 +1005,7 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { // Expires just slightly in the future. WallTime: now.UnixNano() + 1, }, - Decommissioning: false, - Draining: false, + Draining: false, }, expected: kvserverpb.NodeLivenessStatus_LIVE, }, @@ -1020,8 +1018,7 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { // Just expired. WallTime: now.UnixNano(), }, - Decommissioning: false, - Draining: false, + Draining: false, }, expected: kvserverpb.NodeLivenessStatus_UNAVAILABLE, }, @@ -1033,8 +1030,7 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { Expiration: hlc.LegacyTimestamp{ WallTime: now.UnixNano(), }, - Decommissioning: false, - Draining: false, + Draining: false, }, expected: kvserverpb.NodeLivenessStatus_UNAVAILABLE, }, @@ -1046,8 +1042,7 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { Expiration: hlc.LegacyTimestamp{ WallTime: now.Add(-threshold).UnixNano() + 1, }, - Decommissioning: false, - Draining: false, + Draining: false, }, expected: kvserverpb.NodeLivenessStatus_UNAVAILABLE, }, @@ -1059,8 +1054,7 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { Expiration: hlc.LegacyTimestamp{ WallTime: now.Add(-threshold).UnixNano(), }, - Decommissioning: false, - Draining: false, + Draining: false, }, expected: kvserverpb.NodeLivenessStatus_DEAD, }, @@ -1072,12 +1066,12 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { Expiration: hlc.LegacyTimestamp{ WallTime: now.Add(time.Second).UnixNano(), }, - Decommissioning: true, - Draining: false, + Membership: kvserverpb.MembershipStatus_DECOMMISSIONING, + Draining: false, }, expected: kvserverpb.NodeLivenessStatus_DECOMMISSIONING, }, - // Decommissioned. + // Decommissioning + expired. { liveness: kvserverpb.Liveness{ NodeID: 1, @@ -1085,8 +1079,37 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { Expiration: hlc.LegacyTimestamp{ WallTime: now.Add(-threshold).UnixNano(), }, - Decommissioning: true, - Draining: false, + Membership: kvserverpb.MembershipStatus_DECOMMISSIONING, + Draining: false, + }, + expected: kvserverpb.NodeLivenessStatus_DECOMMISSIONED, + }, + // Decommissioned + live. + { + liveness: kvserverpb.Liveness{ + NodeID: 1, + Epoch: 1, + Expiration: hlc.LegacyTimestamp{ + WallTime: now.Add(time.Second).UnixNano(), + }, + Membership: kvserverpb.MembershipStatus_DECOMMISSIONED, + Draining: false, + }, + // Despite having marked the node as fully decommissioned, through + // this NodeLivenessStatus API we still surface the node as + // "Decommissioning". See #50707 for more details. + expected: kvserverpb.NodeLivenessStatus_DECOMMISSIONING, + }, + // Decommissioned + expired. + { + liveness: kvserverpb.Liveness{ + NodeID: 1, + Epoch: 1, + Expiration: hlc.LegacyTimestamp{ + WallTime: now.Add(-threshold).UnixNano(), + }, + Membership: kvserverpb.MembershipStatus_DECOMMISSIONED, + Draining: false, }, expected: kvserverpb.NodeLivenessStatus_DECOMMISSIONED, }, @@ -1098,8 +1121,7 @@ func TestNodeLivenessLivenessStatus(t *testing.T) { Expiration: hlc.LegacyTimestamp{ WallTime: now.Add(5 * time.Minute).UnixNano(), }, - Decommissioning: false, - Draining: true, + Draining: true, }, expected: kvserverpb.NodeLivenessStatus_UNAVAILABLE, }, diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 8e877f2052ca..43521e242629 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -1686,10 +1686,10 @@ func (s *adminServer) DecommissionStatus( return nil, errors.Wrapf(err, "unable to get liveness for %d", nodeID) } nodeResp := serverpb.DecommissionStatusResponse_Status{ - NodeID: l.NodeID, - ReplicaCount: replicaCounts[l.NodeID], - Decommissioning: l.Decommissioning, - Draining: l.Draining, + NodeID: l.NodeID, + ReplicaCount: replicaCounts[l.NodeID], + Membership: l.Membership, + Draining: l.Draining, } if l.IsLive(s.server.clock.Now().GoTime()) { nodeResp.IsLive = true @@ -1715,9 +1715,9 @@ func (s *adminServer) Decommission( return nil, status.Errorf(codes.InvalidArgument, "no node ID specified") } - // Mark the target nodes as decommissioning. They'll find out as they - // heartbeat their liveness. - if err := s.server.Decommission(ctx, req.Decommissioning, nodeIDs); err != nil { + // Mark the target nodes with their new membership status. They'll find out + // as they heartbeat their liveness. + if err := s.server.Decommission(ctx, req.TargetMembership, nodeIDs); err != nil { return nil, err } return s.DecommissionStatus(ctx, &serverpb.DecommissionStatusRequest{NodeIDs: nodeIDs}) diff --git a/pkg/server/server.go b/pkg/server/server.go index ae4c51ad09e2..bddac7e95e42 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/container" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptprovider" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptreconcile" @@ -1880,20 +1881,40 @@ func (s *sqlServer) startServeSQL( } // Decommission idempotently sets the decommissioning flag for specified nodes. -func (s *Server) Decommission(ctx context.Context, setTo bool, nodeIDs []roachpb.NodeID) error { +func (s *Server) Decommission( + ctx context.Context, targetStatus kvserverpb.MembershipStatus, nodeIDs []roachpb.NodeID, +) error { + if !s.st.Version.IsActive(ctx, clusterversion.VersionNodeMembershipStatus) { + if targetStatus.Decommissioned() { + // In mixed-version cluster settings, we need to ensure that we're + // on-the-wire compatible with nodes only familiar with the boolean + // representation of membership state. We do the simple thing and + // simply disallow the setting of the fully decommissioned state until + // we're guaranteed to be on v20.2. + targetStatus = kvserverpb.MembershipStatus_DECOMMISSIONING + } + } + eventLogger := sql.MakeEventLogger(s.sqlServer.execCfg) - eventType := sql.EventLogNodeDecommissioned - if !setTo { + var eventType sql.EventLogType + if targetStatus.Decommissioning() { + eventType = sql.EventLogNodeDecommissioning + } else if targetStatus.Decommissioned() { + eventType = sql.EventLogNodeDecommissioned + } else if targetStatus.Active() { eventType = sql.EventLogNodeRecommissioned + } else { + panic("unexpected target membership status") } + for _, nodeID := range nodeIDs { - changeCommitted, err := s.nodeLiveness.SetDecommissioning(ctx, nodeID, setTo) + statusChanged, err := s.nodeLiveness.SetMembershipStatus(ctx, nodeID, targetStatus) if err != nil { - return errors.Wrapf(err, "during liveness update %d -> %t", nodeID, setTo) + return err } - if changeCommitted { + if statusChanged { // If we die right now or if this transaction fails to commit, the - // commissioning event will not be recorded to the event log. While we + // membership event will not be recorded to the event log. While we // could insert the event record in the same transaction as the liveness // update, this would force a 2PC and potentially leave write intents in // the node liveness range. Better to make the event logging best effort diff --git a/pkg/server/serverpb/admin.pb.go b/pkg/server/serverpb/admin.pb.go index f3e3a56486aa..e8aa01a8ba66 100644 --- a/pkg/server/serverpb/admin.pb.go +++ b/pkg/server/serverpb/admin.pb.go @@ -73,7 +73,7 @@ func (x ZoneConfigurationLevel) String() string { return proto.EnumName(ZoneConfigurationLevel_name, int32(x)) } func (ZoneConfigurationLevel) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{0} } // DatabasesRequest requests a list of databases. @@ -84,7 +84,7 @@ func (m *DatabasesRequest) Reset() { *m = DatabasesRequest{} } func (m *DatabasesRequest) String() string { return proto.CompactTextString(m) } func (*DatabasesRequest) ProtoMessage() {} func (*DatabasesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{0} } func (m *DatabasesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -118,7 +118,7 @@ func (m *DatabasesResponse) Reset() { *m = DatabasesResponse{} } func (m *DatabasesResponse) String() string { return proto.CompactTextString(m) } func (*DatabasesResponse) ProtoMessage() {} func (*DatabasesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{1} + return fileDescriptor_admin_05a80b84e7fbd849, []int{1} } func (m *DatabasesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -154,7 +154,7 @@ func (m *DatabaseDetailsRequest) Reset() { *m = DatabaseDetailsRequest{} func (m *DatabaseDetailsRequest) String() string { return proto.CompactTextString(m) } func (*DatabaseDetailsRequest) ProtoMessage() {} func (*DatabaseDetailsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{2} + return fileDescriptor_admin_05a80b84e7fbd849, []int{2} } func (m *DatabaseDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -201,7 +201,7 @@ func (m *DatabaseDetailsResponse) Reset() { *m = DatabaseDetailsResponse func (m *DatabaseDetailsResponse) String() string { return proto.CompactTextString(m) } func (*DatabaseDetailsResponse) ProtoMessage() {} func (*DatabaseDetailsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{3} + return fileDescriptor_admin_05a80b84e7fbd849, []int{3} } func (m *DatabaseDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -237,7 +237,7 @@ func (m *DatabaseDetailsResponse_Grant) Reset() { *m = DatabaseDetailsRe func (m *DatabaseDetailsResponse_Grant) String() string { return proto.CompactTextString(m) } func (*DatabaseDetailsResponse_Grant) ProtoMessage() {} func (*DatabaseDetailsResponse_Grant) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{3, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{3, 0} } func (m *DatabaseDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -274,7 +274,7 @@ func (m *TableDetailsRequest) Reset() { *m = TableDetailsRequest{} } func (m *TableDetailsRequest) String() string { return proto.CompactTextString(m) } func (*TableDetailsRequest) ProtoMessage() {} func (*TableDetailsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{4} + return fileDescriptor_admin_05a80b84e7fbd849, []int{4} } func (m *TableDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -331,7 +331,7 @@ func (m *TableDetailsResponse) Reset() { *m = TableDetailsResponse{} } func (m *TableDetailsResponse) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse) ProtoMessage() {} func (*TableDetailsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{5} + return fileDescriptor_admin_05a80b84e7fbd849, []int{5} } func (m *TableDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -368,7 +368,7 @@ func (m *TableDetailsResponse_Grant) Reset() { *m = TableDetailsResponse func (m *TableDetailsResponse_Grant) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Grant) ProtoMessage() {} func (*TableDetailsResponse_Grant) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{5, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{5, 0} } func (m *TableDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -412,7 +412,7 @@ func (m *TableDetailsResponse_Column) Reset() { *m = TableDetailsRespons func (m *TableDetailsResponse_Column) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Column) ProtoMessage() {} func (*TableDetailsResponse_Column) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{5, 1} + return fileDescriptor_admin_05a80b84e7fbd849, []int{5, 1} } func (m *TableDetailsResponse_Column) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +458,7 @@ func (m *TableDetailsResponse_Index) Reset() { *m = TableDetailsResponse func (m *TableDetailsResponse_Index) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Index) ProtoMessage() {} func (*TableDetailsResponse_Index) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{5, 2} + return fileDescriptor_admin_05a80b84e7fbd849, []int{5, 2} } func (m *TableDetailsResponse_Index) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -496,7 +496,7 @@ func (m *TableStatsRequest) Reset() { *m = TableStatsRequest{} } func (m *TableStatsRequest) String() string { return proto.CompactTextString(m) } func (*TableStatsRequest) ProtoMessage() {} func (*TableStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{6} + return fileDescriptor_admin_05a80b84e7fbd849, []int{6} } func (m *TableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -550,7 +550,7 @@ func (m *TableStatsResponse) Reset() { *m = TableStatsResponse{} } func (m *TableStatsResponse) String() string { return proto.CompactTextString(m) } func (*TableStatsResponse) ProtoMessage() {} func (*TableStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{7} + return fileDescriptor_admin_05a80b84e7fbd849, []int{7} } func (m *TableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -588,7 +588,7 @@ func (m *TableStatsResponse_MissingNode) Reset() { *m = TableStatsRespon func (m *TableStatsResponse_MissingNode) String() string { return proto.CompactTextString(m) } func (*TableStatsResponse_MissingNode) ProtoMessage() {} func (*TableStatsResponse_MissingNode) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{7, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{7, 0} } func (m *TableStatsResponse_MissingNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -622,7 +622,7 @@ func (m *NonTableStatsRequest) Reset() { *m = NonTableStatsRequest{} } func (m *NonTableStatsRequest) String() string { return proto.CompactTextString(m) } func (*NonTableStatsRequest) ProtoMessage() {} func (*NonTableStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{8} + return fileDescriptor_admin_05a80b84e7fbd849, []int{8} } func (m *NonTableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -661,7 +661,7 @@ func (m *NonTableStatsResponse) Reset() { *m = NonTableStatsResponse{} } func (m *NonTableStatsResponse) String() string { return proto.CompactTextString(m) } func (*NonTableStatsResponse) ProtoMessage() {} func (*NonTableStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{9} + return fileDescriptor_admin_05a80b84e7fbd849, []int{9} } func (m *NonTableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -694,7 +694,7 @@ func (m *UsersRequest) Reset() { *m = UsersRequest{} } func (m *UsersRequest) String() string { return proto.CompactTextString(m) } func (*UsersRequest) ProtoMessage() {} func (*UsersRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{10} + return fileDescriptor_admin_05a80b84e7fbd849, []int{10} } func (m *UsersRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -729,7 +729,7 @@ func (m *UsersResponse) Reset() { *m = UsersResponse{} } func (m *UsersResponse) String() string { return proto.CompactTextString(m) } func (*UsersResponse) ProtoMessage() {} func (*UsersResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{11} + return fileDescriptor_admin_05a80b84e7fbd849, []int{11} } func (m *UsersResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -763,7 +763,7 @@ func (m *UsersResponse_User) Reset() { *m = UsersResponse_User{} } func (m *UsersResponse_User) String() string { return proto.CompactTextString(m) } func (*UsersResponse_User) ProtoMessage() {} func (*UsersResponse_User) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{11, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{11, 0} } func (m *UsersResponse_User) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -810,7 +810,7 @@ func (m *EventsRequest) Reset() { *m = EventsRequest{} } func (m *EventsRequest) String() string { return proto.CompactTextString(m) } func (*EventsRequest) ProtoMessage() {} func (*EventsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{12} + return fileDescriptor_admin_05a80b84e7fbd849, []int{12} } func (m *EventsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -845,7 +845,7 @@ func (m *EventsResponse) Reset() { *m = EventsResponse{} } func (m *EventsResponse) String() string { return proto.CompactTextString(m) } func (*EventsResponse) ProtoMessage() {} func (*EventsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{13} + return fileDescriptor_admin_05a80b84e7fbd849, []int{13} } func (m *EventsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -890,7 +890,7 @@ func (m *EventsResponse_Event) Reset() { *m = EventsResponse_Event{} } func (m *EventsResponse_Event) String() string { return proto.CompactTextString(m) } func (*EventsResponse_Event) ProtoMessage() {} func (*EventsResponse_Event) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{13, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{13, 0} } func (m *EventsResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -926,7 +926,7 @@ func (m *SetUIDataRequest) Reset() { *m = SetUIDataRequest{} } func (m *SetUIDataRequest) String() string { return proto.CompactTextString(m) } func (*SetUIDataRequest) ProtoMessage() {} func (*SetUIDataRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{14} + return fileDescriptor_admin_05a80b84e7fbd849, []int{14} } func (m *SetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -959,7 +959,7 @@ func (m *SetUIDataResponse) Reset() { *m = SetUIDataResponse{} } func (m *SetUIDataResponse) String() string { return proto.CompactTextString(m) } func (*SetUIDataResponse) ProtoMessage() {} func (*SetUIDataResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{15} + return fileDescriptor_admin_05a80b84e7fbd849, []int{15} } func (m *SetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -994,7 +994,7 @@ func (m *GetUIDataRequest) Reset() { *m = GetUIDataRequest{} } func (m *GetUIDataRequest) String() string { return proto.CompactTextString(m) } func (*GetUIDataRequest) ProtoMessage() {} func (*GetUIDataRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{16} + return fileDescriptor_admin_05a80b84e7fbd849, []int{16} } func (m *GetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1031,7 +1031,7 @@ func (m *GetUIDataResponse) Reset() { *m = GetUIDataResponse{} } func (m *GetUIDataResponse) String() string { return proto.CompactTextString(m) } func (*GetUIDataResponse) ProtoMessage() {} func (*GetUIDataResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{17} + return fileDescriptor_admin_05a80b84e7fbd849, []int{17} } func (m *GetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1067,7 +1067,7 @@ func (m *GetUIDataResponse_Value) Reset() { *m = GetUIDataResponse_Value func (m *GetUIDataResponse_Value) String() string { return proto.CompactTextString(m) } func (*GetUIDataResponse_Value) ProtoMessage() {} func (*GetUIDataResponse_Value) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{17, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{17, 0} } func (m *GetUIDataResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1100,7 +1100,7 @@ func (m *ClusterRequest) Reset() { *m = ClusterRequest{} } func (m *ClusterRequest) String() string { return proto.CompactTextString(m) } func (*ClusterRequest) ProtoMessage() {} func (*ClusterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{18} + return fileDescriptor_admin_05a80b84e7fbd849, []int{18} } func (m *ClusterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1139,7 +1139,7 @@ func (m *ClusterResponse) Reset() { *m = ClusterResponse{} } func (m *ClusterResponse) String() string { return proto.CompactTextString(m) } func (*ClusterResponse) ProtoMessage() {} func (*ClusterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{19} + return fileDescriptor_admin_05a80b84e7fbd849, []int{19} } func (m *ClusterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1198,7 +1198,7 @@ func (m *DrainRequest) Reset() { *m = DrainRequest{} } func (m *DrainRequest) String() string { return proto.CompactTextString(m) } func (*DrainRequest) ProtoMessage() {} func (*DrainRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{20} + return fileDescriptor_admin_05a80b84e7fbd849, []int{20} } func (m *DrainRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1292,7 +1292,7 @@ func (m *DrainResponse) Reset() { *m = DrainResponse{} } func (m *DrainResponse) String() string { return proto.CompactTextString(m) } func (*DrainResponse) ProtoMessage() {} func (*DrainResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{21} + return fileDescriptor_admin_05a80b84e7fbd849, []int{21} } func (m *DrainResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1327,7 +1327,7 @@ func (m *DecommissionStatusRequest) Reset() { *m = DecommissionStatusReq func (m *DecommissionStatusRequest) String() string { return proto.CompactTextString(m) } func (*DecommissionStatusRequest) ProtoMessage() {} func (*DecommissionStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{22} + return fileDescriptor_admin_05a80b84e7fbd849, []int{22} } func (m *DecommissionStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1352,20 +1352,20 @@ func (m *DecommissionStatusRequest) XXX_DiscardUnknown() { var xxx_messageInfo_DecommissionStatusRequest proto.InternalMessageInfo -// DecommissionRequest requests the server to set the Decommissioning flag on -// all nodes specified by 'node_id' to the value of 'decommissioning'. +// DecommissionRequest requests the server to set the membership status on +// all nodes specified by NodeIDs to the value of TargetMembership. // -// If no 'node_id' is given, it targets the recipient node. +// If no NodeIDs are given, it targets the recipient node. type DecommissionRequest struct { - NodeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,rep,packed,name=node_ids,json=nodeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_ids,omitempty"` - Decommissioning bool `protobuf:"varint,2,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` + NodeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,rep,packed,name=node_ids,json=nodeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_ids,omitempty"` + TargetMembership kvserverpb.MembershipStatus `protobuf:"varint,2,opt,name=target_membership,json=targetMembership,proto3,enum=cockroach.kv.kvserver.storagepb.MembershipStatus" json:"target_membership,omitempty"` } func (m *DecommissionRequest) Reset() { *m = DecommissionRequest{} } func (m *DecommissionRequest) String() string { return proto.CompactTextString(m) } func (*DecommissionRequest) ProtoMessage() {} func (*DecommissionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{23} + return fileDescriptor_admin_05a80b84e7fbd849, []int{23} } func (m *DecommissionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1400,7 +1400,7 @@ func (m *DecommissionStatusResponse) Reset() { *m = DecommissionStatusRe func (m *DecommissionStatusResponse) String() string { return proto.CompactTextString(m) } func (*DecommissionStatusResponse) ProtoMessage() {} func (*DecommissionStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{24} + return fileDescriptor_admin_05a80b84e7fbd849, []int{24} } func (m *DecommissionStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1429,16 +1429,17 @@ type DecommissionStatusResponse_Status struct { NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` IsLive bool `protobuf:"varint,2,opt,name=is_live,json=isLive,proto3" json:"is_live,omitempty"` // The number of replicas on the node, computed by scanning meta2 ranges. - ReplicaCount int64 `protobuf:"varint,3,opt,name=replica_count,json=replicaCount,proto3" json:"replica_count,omitempty"` - Decommissioning bool `protobuf:"varint,4,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` - Draining bool `protobuf:"varint,5,opt,name=draining,proto3" json:"draining,omitempty"` + ReplicaCount int64 `protobuf:"varint,3,opt,name=replica_count,json=replicaCount,proto3" json:"replica_count,omitempty"` + // The membership status of the given node. + Membership kvserverpb.MembershipStatus `protobuf:"varint,4,opt,name=membership,proto3,enum=cockroach.kv.kvserver.storagepb.MembershipStatus" json:"membership,omitempty"` + Draining bool `protobuf:"varint,5,opt,name=draining,proto3" json:"draining,omitempty"` } func (m *DecommissionStatusResponse_Status) Reset() { *m = DecommissionStatusResponse_Status{} } func (m *DecommissionStatusResponse_Status) String() string { return proto.CompactTextString(m) } func (*DecommissionStatusResponse_Status) ProtoMessage() {} func (*DecommissionStatusResponse_Status) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{24, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{24, 0} } func (m *DecommissionStatusResponse_Status) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1480,7 +1481,7 @@ func (m *SettingsRequest) Reset() { *m = SettingsRequest{} } func (m *SettingsRequest) String() string { return proto.CompactTextString(m) } func (*SettingsRequest) ProtoMessage() {} func (*SettingsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{25} + return fileDescriptor_admin_05a80b84e7fbd849, []int{25} } func (m *SettingsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1514,7 +1515,7 @@ func (m *SettingsResponse) Reset() { *m = SettingsResponse{} } func (m *SettingsResponse) String() string { return proto.CompactTextString(m) } func (*SettingsResponse) ProtoMessage() {} func (*SettingsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{26} + return fileDescriptor_admin_05a80b84e7fbd849, []int{26} } func (m *SettingsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1550,7 +1551,7 @@ func (m *SettingsResponse_Value) Reset() { *m = SettingsResponse_Value{} func (m *SettingsResponse_Value) String() string { return proto.CompactTextString(m) } func (*SettingsResponse_Value) ProtoMessage() {} func (*SettingsResponse_Value) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{26, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{26, 0} } func (m *SettingsResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1603,7 +1604,7 @@ func (m *HealthRequest) Reset() { *m = HealthRequest{} } func (m *HealthRequest) String() string { return proto.CompactTextString(m) } func (*HealthRequest) ProtoMessage() {} func (*HealthRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{27} + return fileDescriptor_admin_05a80b84e7fbd849, []int{27} } func (m *HealthRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1637,7 +1638,7 @@ func (m *HealthResponse) Reset() { *m = HealthResponse{} } func (m *HealthResponse) String() string { return proto.CompactTextString(m) } func (*HealthResponse) ProtoMessage() {} func (*HealthResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{28} + return fileDescriptor_admin_05a80b84e7fbd849, []int{28} } func (m *HealthResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1670,7 +1671,7 @@ func (m *LivenessRequest) Reset() { *m = LivenessRequest{} } func (m *LivenessRequest) String() string { return proto.CompactTextString(m) } func (*LivenessRequest) ProtoMessage() {} func (*LivenessRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{29} + return fileDescriptor_admin_05a80b84e7fbd849, []int{29} } func (m *LivenessRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1705,7 +1706,7 @@ func (m *LivenessResponse) Reset() { *m = LivenessResponse{} } func (m *LivenessResponse) String() string { return proto.CompactTextString(m) } func (*LivenessResponse) ProtoMessage() {} func (*LivenessResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{30} + return fileDescriptor_admin_05a80b84e7fbd849, []int{30} } func (m *LivenessResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1741,7 +1742,7 @@ func (m *JobsRequest) Reset() { *m = JobsRequest{} } func (m *JobsRequest) String() string { return proto.CompactTextString(m) } func (*JobsRequest) ProtoMessage() {} func (*JobsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{31} + return fileDescriptor_admin_05a80b84e7fbd849, []int{31} } func (m *JobsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1775,7 +1776,7 @@ func (m *JobsResponse) Reset() { *m = JobsResponse{} } func (m *JobsResponse) String() string { return proto.CompactTextString(m) } func (*JobsResponse) ProtoMessage() {} func (*JobsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{32} + return fileDescriptor_admin_05a80b84e7fbd849, []int{32} } func (m *JobsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1828,7 +1829,7 @@ func (m *JobsResponse_Job) Reset() { *m = JobsResponse_Job{} } func (m *JobsResponse_Job) String() string { return proto.CompactTextString(m) } func (*JobsResponse_Job) ProtoMessage() {} func (*JobsResponse_Job) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{32, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{32, 0} } func (m *JobsResponse_Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1861,7 +1862,7 @@ func (m *LocationsRequest) Reset() { *m = LocationsRequest{} } func (m *LocationsRequest) String() string { return proto.CompactTextString(m) } func (*LocationsRequest) ProtoMessage() {} func (*LocationsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{33} + return fileDescriptor_admin_05a80b84e7fbd849, []int{33} } func (m *LocationsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1895,7 +1896,7 @@ func (m *LocationsResponse) Reset() { *m = LocationsResponse{} } func (m *LocationsResponse) String() string { return proto.CompactTextString(m) } func (*LocationsResponse) ProtoMessage() {} func (*LocationsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{34} + return fileDescriptor_admin_05a80b84e7fbd849, []int{34} } func (m *LocationsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1931,7 +1932,7 @@ func (m *LocationsResponse_Location) Reset() { *m = LocationsResponse_Lo func (m *LocationsResponse_Location) String() string { return proto.CompactTextString(m) } func (*LocationsResponse_Location) ProtoMessage() {} func (*LocationsResponse_Location) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{34, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{34, 0} } func (m *LocationsResponse_Location) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1973,7 +1974,7 @@ func (m *RangeLogRequest) Reset() { *m = RangeLogRequest{} } func (m *RangeLogRequest) String() string { return proto.CompactTextString(m) } func (*RangeLogRequest) ProtoMessage() {} func (*RangeLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{35} + return fileDescriptor_admin_05a80b84e7fbd849, []int{35} } func (m *RangeLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2007,7 +2008,7 @@ func (m *RangeLogResponse) Reset() { *m = RangeLogResponse{} } func (m *RangeLogResponse) String() string { return proto.CompactTextString(m) } func (*RangeLogResponse) ProtoMessage() {} func (*RangeLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{36} + return fileDescriptor_admin_05a80b84e7fbd849, []int{36} } func (m *RangeLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2047,7 +2048,7 @@ func (m *RangeLogResponse_PrettyInfo) Reset() { *m = RangeLogResponse_Pr func (m *RangeLogResponse_PrettyInfo) String() string { return proto.CompactTextString(m) } func (*RangeLogResponse_PrettyInfo) ProtoMessage() {} func (*RangeLogResponse_PrettyInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{36, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{36, 0} } func (m *RangeLogResponse_PrettyInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2081,7 +2082,7 @@ func (m *RangeLogResponse_Event) Reset() { *m = RangeLogResponse_Event{} func (m *RangeLogResponse_Event) String() string { return proto.CompactTextString(m) } func (*RangeLogResponse_Event) ProtoMessage() {} func (*RangeLogResponse_Event) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{36, 1} + return fileDescriptor_admin_05a80b84e7fbd849, []int{36, 1} } func (m *RangeLogResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2116,7 +2117,7 @@ func (m *QueryPlanRequest) Reset() { *m = QueryPlanRequest{} } func (m *QueryPlanRequest) String() string { return proto.CompactTextString(m) } func (*QueryPlanRequest) ProtoMessage() {} func (*QueryPlanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{37} + return fileDescriptor_admin_05a80b84e7fbd849, []int{37} } func (m *QueryPlanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2151,7 +2152,7 @@ func (m *QueryPlanResponse) Reset() { *m = QueryPlanResponse{} } func (m *QueryPlanResponse) String() string { return proto.CompactTextString(m) } func (*QueryPlanResponse) ProtoMessage() {} func (*QueryPlanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{38} + return fileDescriptor_admin_05a80b84e7fbd849, []int{38} } func (m *QueryPlanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2183,7 +2184,7 @@ func (m *DataDistributionRequest) Reset() { *m = DataDistributionRequest func (m *DataDistributionRequest) String() string { return proto.CompactTextString(m) } func (*DataDistributionRequest) ProtoMessage() {} func (*DataDistributionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{39} + return fileDescriptor_admin_05a80b84e7fbd849, []int{39} } func (m *DataDistributionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2219,7 +2220,7 @@ func (m *DataDistributionResponse) Reset() { *m = DataDistributionRespon func (m *DataDistributionResponse) String() string { return proto.CompactTextString(m) } func (*DataDistributionResponse) ProtoMessage() {} func (*DataDistributionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{40} + return fileDescriptor_admin_05a80b84e7fbd849, []int{40} } func (m *DataDistributionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2257,7 +2258,7 @@ func (m *DataDistributionResponse_ZoneConfig) Reset() { *m = DataDistrib func (m *DataDistributionResponse_ZoneConfig) String() string { return proto.CompactTextString(m) } func (*DataDistributionResponse_ZoneConfig) ProtoMessage() {} func (*DataDistributionResponse_ZoneConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{40, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{40, 0} } func (m *DataDistributionResponse_ZoneConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2292,7 +2293,7 @@ func (m *DataDistributionResponse_TableInfo) Reset() { *m = DataDistribu func (m *DataDistributionResponse_TableInfo) String() string { return proto.CompactTextString(m) } func (*DataDistributionResponse_TableInfo) ProtoMessage() {} func (*DataDistributionResponse_TableInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{40, 1} + return fileDescriptor_admin_05a80b84e7fbd849, []int{40, 1} } func (m *DataDistributionResponse_TableInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2326,7 +2327,7 @@ func (m *DataDistributionResponse_DatabaseInfo) Reset() { *m = DataDistr func (m *DataDistributionResponse_DatabaseInfo) String() string { return proto.CompactTextString(m) } func (*DataDistributionResponse_DatabaseInfo) ProtoMessage() {} func (*DataDistributionResponse_DatabaseInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{40, 2} + return fileDescriptor_admin_05a80b84e7fbd849, []int{40, 2} } func (m *DataDistributionResponse_DatabaseInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2359,7 +2360,7 @@ func (m *MetricMetadataRequest) Reset() { *m = MetricMetadataRequest{} } func (m *MetricMetadataRequest) String() string { return proto.CompactTextString(m) } func (*MetricMetadataRequest) ProtoMessage() {} func (*MetricMetadataRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{41} + return fileDescriptor_admin_05a80b84e7fbd849, []int{41} } func (m *MetricMetadataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2393,7 +2394,7 @@ func (m *MetricMetadataResponse) Reset() { *m = MetricMetadataResponse{} func (m *MetricMetadataResponse) String() string { return proto.CompactTextString(m) } func (*MetricMetadataResponse) ProtoMessage() {} func (*MetricMetadataResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{42} + return fileDescriptor_admin_05a80b84e7fbd849, []int{42} } func (m *MetricMetadataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2436,7 +2437,7 @@ func (m *EnqueueRangeRequest) Reset() { *m = EnqueueRangeRequest{} } func (m *EnqueueRangeRequest) String() string { return proto.CompactTextString(m) } func (*EnqueueRangeRequest) ProtoMessage() {} func (*EnqueueRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{43} + return fileDescriptor_admin_05a80b84e7fbd849, []int{43} } func (m *EnqueueRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2469,7 +2470,7 @@ func (m *EnqueueRangeResponse) Reset() { *m = EnqueueRangeResponse{} } func (m *EnqueueRangeResponse) String() string { return proto.CompactTextString(m) } func (*EnqueueRangeResponse) ProtoMessage() {} func (*EnqueueRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{44} + return fileDescriptor_admin_05a80b84e7fbd849, []int{44} } func (m *EnqueueRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2506,7 +2507,7 @@ func (m *EnqueueRangeResponse_Details) Reset() { *m = EnqueueRangeRespon func (m *EnqueueRangeResponse_Details) String() string { return proto.CompactTextString(m) } func (*EnqueueRangeResponse_Details) ProtoMessage() {} func (*EnqueueRangeResponse_Details) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{44, 0} + return fileDescriptor_admin_05a80b84e7fbd849, []int{44, 0} } func (m *EnqueueRangeResponse_Details) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2539,7 +2540,7 @@ func (m *ChartCatalogRequest) Reset() { *m = ChartCatalogRequest{} } func (m *ChartCatalogRequest) String() string { return proto.CompactTextString(m) } func (*ChartCatalogRequest) ProtoMessage() {} func (*ChartCatalogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{45} + return fileDescriptor_admin_05a80b84e7fbd849, []int{45} } func (m *ChartCatalogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2573,7 +2574,7 @@ func (m *ChartCatalogResponse) Reset() { *m = ChartCatalogResponse{} } func (m *ChartCatalogResponse) String() string { return proto.CompactTextString(m) } func (*ChartCatalogResponse) ProtoMessage() {} func (*ChartCatalogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_4db1a727dcd14ea8, []int{46} + return fileDescriptor_admin_05a80b84e7fbd849, []int{46} } func (m *ChartCatalogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4933,15 +4934,10 @@ func (m *DecommissionRequest) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintAdmin(dAtA, i, uint64(j15)) i += copy(dAtA[i:], dAtA16[:j15]) } - if m.Decommissioning { + if m.TargetMembership != 0 { dAtA[i] = 0x10 i++ - if m.Decommissioning { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ + i = encodeVarintAdmin(dAtA, i, uint64(m.TargetMembership)) } return i, nil } @@ -5011,15 +5007,10 @@ func (m *DecommissionStatusResponse_Status) MarshalTo(dAtA []byte) (int, error) i++ i = encodeVarintAdmin(dAtA, i, uint64(m.ReplicaCount)) } - if m.Decommissioning { + if m.Membership != 0 { dAtA[i] = 0x20 i++ - if m.Decommissioning { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ + i = encodeVarintAdmin(dAtA, i, uint64(m.Membership)) } if m.Draining { dAtA[i] = 0x28 @@ -6859,8 +6850,8 @@ func (m *DecommissionRequest) Size() (n int) { } n += 1 + sovAdmin(uint64(l)) + l } - if m.Decommissioning { - n += 2 + if m.TargetMembership != 0 { + n += 1 + sovAdmin(uint64(m.TargetMembership)) } return n } @@ -6895,8 +6886,8 @@ func (m *DecommissionStatusResponse_Status) Size() (n int) { if m.ReplicaCount != 0 { n += 1 + sovAdmin(uint64(m.ReplicaCount)) } - if m.Decommissioning { - n += 2 + if m.Membership != 0 { + n += 1 + sovAdmin(uint64(m.Membership)) } if m.Draining { n += 2 @@ -11421,9 +11412,9 @@ func (m *DecommissionRequest) Unmarshal(dAtA []byte) error { } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Decommissioning", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TargetMembership", wireType) } - var v int + m.TargetMembership = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowAdmin @@ -11433,12 +11424,11 @@ func (m *DecommissionRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + m.TargetMembership |= (kvserverpb.MembershipStatus(b) & 0x7F) << shift if b < 0x80 { break } } - m.Decommissioning = bool(v != 0) default: iNdEx = preIndex skippy, err := skipAdmin(dAtA[iNdEx:]) @@ -11630,9 +11620,9 @@ func (m *DecommissionStatusResponse_Status) Unmarshal(dAtA []byte) error { } case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Decommissioning", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Membership", wireType) } - var v int + m.Membership = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowAdmin @@ -11642,12 +11632,11 @@ func (m *DecommissionStatusResponse_Status) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + m.Membership |= (kvserverpb.MembershipStatus(b) & 0x7F) << shift if b < 0x80 { break } } - m.Decommissioning = bool(v != 0) case 5: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Draining", wireType) @@ -15796,268 +15785,269 @@ var ( ErrIntOverflowAdmin = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_4db1a727dcd14ea8) } - -var fileDescriptor_admin_4db1a727dcd14ea8 = []byte{ - // 4152 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7a, 0x4d, 0x70, 0x1b, 0xc9, - 0x75, 0xb0, 0x06, 0x20, 0xfe, 0x1e, 0x01, 0x12, 0x6c, 0x51, 0x14, 0x08, 0xc9, 0x04, 0x77, 0x64, - 0x79, 0xb9, 0xda, 0x35, 0xb0, 0xa2, 0xb4, 0x5a, 0x7f, 0x5a, 0xad, 0xfd, 0x11, 0x04, 0x4b, 0x0b, - 0xad, 0xa4, 0x5d, 0x0d, 0x25, 0x6d, 0x59, 0xce, 0x7a, 0x32, 0xc0, 0x34, 0xc1, 0x31, 0x07, 0x33, - 0xc3, 0x99, 0x01, 0xb5, 0xdc, 0x8d, 0x5d, 0xb1, 0x9d, 0xa4, 0x92, 0x4b, 0x6a, 0xcb, 0xce, 0x6d, - 0xab, 0x52, 0x95, 0x1c, 0x92, 0x43, 0x2e, 0xd9, 0x1c, 0x73, 0xc8, 0x79, 0x2b, 0x87, 0xd8, 0xa9, - 0x5c, 0x9c, 0x0b, 0x9d, 0x70, 0x7d, 0x48, 0x39, 0xf7, 0x1c, 0x72, 0x49, 0xaa, 0xff, 0x66, 0x1a, - 0x20, 0x34, 0x02, 0x28, 0x3b, 0x07, 0x12, 0xd3, 0xef, 0x75, 0xbf, 0x7e, 0xfd, 0xfa, 0xf5, 0xfb, - 0xeb, 0x86, 0x0b, 0x01, 0xf6, 0x0f, 0xb0, 0xdf, 0x60, 0x3f, 0x5e, 0xa7, 0x61, 0x98, 0x7d, 0xcb, - 0xa9, 0x7b, 0xbe, 0x1b, 0xba, 0x68, 0xb9, 0xeb, 0x76, 0xf7, 0x7c, 0xd7, 0xe8, 0xee, 0xd6, 0x19, - 0xbe, 0x2e, 0xba, 0x55, 0x2b, 0x5d, 0xd7, 0xd9, 0xb1, 0x7a, 0x8d, 0x8f, 0x5d, 0x07, 0x7b, 0x1d, - 0xfa, 0xc3, 0x06, 0x55, 0x97, 0xbe, 0xe7, 0x76, 0x82, 0x06, 0xf9, 0xe7, 0x75, 0xe8, 0x0f, 0x87, - 0x5f, 0x1c, 0x9d, 0x29, 0x08, 0x8d, 0x70, 0x20, 0xb0, 0x17, 0x82, 0xd0, 0xf5, 0x8d, 0x1e, 0x6e, - 0x60, 0xa7, 0x67, 0x51, 0x92, 0xfd, 0x83, 0x6e, 0x97, 0x23, 0x2f, 0xef, 0x1d, 0x34, 0xf6, 0x0e, - 0xf8, 0x78, 0xf1, 0xe1, 0x75, 0x1a, 0xb6, 0x75, 0x80, 0x1d, 0x1c, 0x08, 0x1a, 0xab, 0xcf, 0xea, - 0xe6, 0xf6, 0x78, 0x8f, 0x95, 0x30, 0x68, 0x74, 0x8d, 0xd0, 0xb0, 0xdd, 0x5e, 0xa3, 0xbb, 0x6b, - 0xf8, 0xa1, 0xce, 0x5b, 0x1c, 0x5f, 0x19, 0x84, 0x96, 0xdd, 0xe8, 0xe3, 0xd0, 0xb7, 0xba, 0xfc, - 0x87, 0x63, 0x16, 0x7b, 0x6e, 0xcf, 0xa5, 0x9f, 0x0d, 0xf2, 0x25, 0xd6, 0xd4, 0x73, 0xdd, 0x9e, - 0x8d, 0x1b, 0x86, 0x67, 0x35, 0x0c, 0xc7, 0x71, 0x43, 0x23, 0xb4, 0x5c, 0x47, 0xf0, 0x53, 0xe3, - 0x58, 0xda, 0xea, 0x0c, 0x76, 0x1a, 0xa1, 0xd5, 0xc7, 0x41, 0x68, 0xf4, 0x3d, 0xd6, 0x41, 0x45, - 0x50, 0x6e, 0x19, 0xa1, 0xd1, 0x31, 0x02, 0x1c, 0x68, 0x78, 0x7f, 0x80, 0x83, 0x50, 0xbd, 0x0a, - 0x0b, 0x12, 0x2c, 0xf0, 0x5c, 0x27, 0xc0, 0xe8, 0x22, 0x14, 0x4c, 0x01, 0xac, 0x28, 0xab, 0xe9, - 0xb5, 0x82, 0x16, 0x03, 0xd4, 0xeb, 0xb0, 0x24, 0x86, 0xb4, 0x70, 0x68, 0x58, 0xb6, 0x20, 0x86, - 0xaa, 0x90, 0x17, 0xdd, 0x2a, 0xca, 0xaa, 0xb2, 0x56, 0xd0, 0xa2, 0xb6, 0xfa, 0x79, 0x1a, 0xce, - 0x9f, 0x18, 0xc6, 0xe7, 0x7b, 0x0c, 0xd9, 0x9e, 0x6f, 0x38, 0x21, 0x9b, 0x6c, 0x76, 0xfd, 0x1b, - 0xf5, 0x67, 0x6a, 0x42, 0xfd, 0x19, 0x34, 0xea, 0xb7, 0x09, 0x81, 0xe6, 0xcc, 0x17, 0x47, 0xb5, - 0x33, 0x1a, 0xa7, 0x86, 0x6a, 0x30, 0x1b, 0x1a, 0x1d, 0x1b, 0xeb, 0x8e, 0xd1, 0xc7, 0x41, 0x25, - 0x45, 0x57, 0x02, 0x14, 0x74, 0x9f, 0x40, 0xd0, 0x1b, 0x50, 0x32, 0x71, 0xd0, 0xf5, 0x2d, 0x2f, - 0x74, 0x7d, 0xdd, 0x32, 0x2b, 0xe9, 0x55, 0x65, 0x2d, 0xdd, 0x2c, 0x1f, 0x1f, 0xd5, 0x8a, 0xad, - 0x08, 0xd1, 0x6e, 0x69, 0xc5, 0xb8, 0x5b, 0xdb, 0x44, 0x77, 0x60, 0x96, 0x68, 0xa0, 0xce, 0x94, - 0xb2, 0x32, 0xb3, 0xaa, 0xac, 0xcd, 0xae, 0x5f, 0x92, 0x98, 0x66, 0x88, 0x3a, 0xd3, 0xd6, 0xfa, - 0x13, 0xd7, 0xc1, 0x9b, 0x14, 0xc2, 0xf9, 0x83, 0x8f, 0x23, 0x08, 0xfa, 0x10, 0x16, 0x24, 0x5a, - 0xba, 0x8d, 0x0f, 0xb0, 0x5d, 0xc9, 0xac, 0x2a, 0x6b, 0x73, 0xeb, 0x57, 0x13, 0xc4, 0x10, 0xd3, - 0x1c, 0xf8, 0x54, 0x0b, 0xee, 0x92, 0x81, 0xda, 0x7c, 0x4c, 0x99, 0x02, 0xaa, 0x6f, 0x41, 0x86, - 0x4a, 0x06, 0x21, 0x98, 0x19, 0x04, 0xd8, 0xe7, 0xfb, 0x42, 0xbf, 0xd1, 0x0a, 0x80, 0xe7, 0x5b, - 0x07, 0x96, 0x8d, 0x7b, 0xb1, 0x78, 0x62, 0x88, 0x7a, 0x1b, 0xce, 0x3e, 0x24, 0xc2, 0x9a, 0x7c, - 0x9b, 0xd1, 0x22, 0x64, 0xa8, 0x7c, 0x2b, 0x29, 0x8a, 0x60, 0x0d, 0xf5, 0x0f, 0xf3, 0xb0, 0x38, - 0x4c, 0x89, 0xef, 0xfc, 0xf6, 0xc8, 0xce, 0xbf, 0x91, 0xb0, 0xe4, 0x71, 0x04, 0xc6, 0x6e, 0xfb, - 0x63, 0xc8, 0x75, 0x5d, 0x7b, 0xd0, 0x77, 0xd8, 0x9a, 0x66, 0xd7, 0x6f, 0x4c, 0x4b, 0x75, 0x93, - 0x0e, 0xe7, 0x64, 0x05, 0x31, 0xf4, 0x08, 0x72, 0x96, 0x63, 0xe2, 0x8f, 0x70, 0x50, 0x49, 0x9f, - 0x8e, 0xdb, 0x36, 0x19, 0x2e, 0xc8, 0x72, 0x5a, 0x44, 0x4b, 0x7d, 0xc3, 0xe9, 0x11, 0x15, 0x18, - 0x38, 0x21, 0xd5, 0xa6, 0xb4, 0x06, 0x14, 0xb4, 0x49, 0x20, 0xe8, 0x3a, 0x2c, 0x75, 0x7d, 0x6c, - 0x84, 0x58, 0x67, 0xda, 0x4c, 0x2c, 0x19, 0xee, 0x63, 0x27, 0xa4, 0x7a, 0x52, 0xd0, 0x16, 0x19, - 0x96, 0xce, 0xb8, 0x2d, 0x70, 0xa3, 0x4a, 0x9a, 0xfd, 0x8d, 0x2b, 0x69, 0xee, 0x37, 0xa5, 0xa4, - 0x27, 0x8f, 0x61, 0x7e, 0x92, 0x63, 0xf8, 0x42, 0xba, 0x5d, 0xfd, 0x07, 0x05, 0xb2, 0x6c, 0x9b, - 0xc9, 0x70, 0x62, 0x20, 0xc4, 0x70, 0xf2, 0x4d, 0x60, 0xe1, 0xa1, 0x27, 0xd4, 0x98, 0x7e, 0x13, - 0xbd, 0x77, 0x06, 0xb6, 0x4d, 0xd5, 0x9b, 0x18, 0x8a, 0xbc, 0x16, 0xb5, 0xd1, 0x25, 0xb2, 0x84, - 0x1d, 0x63, 0x60, 0x87, 0xfa, 0x81, 0x61, 0x0f, 0x30, 0xdd, 0xc6, 0x02, 0x61, 0x98, 0x02, 0x1f, - 0x13, 0x18, 0xba, 0x06, 0xe7, 0x7a, 0xd8, 0xc1, 0x4c, 0x16, 0x3a, 0xfe, 0xc8, 0xf3, 0x71, 0x10, - 0x58, 0xae, 0x23, 0xf6, 0x31, 0x46, 0x6e, 0x45, 0x38, 0xb4, 0x04, 0xd9, 0x5d, 0xcb, 0x34, 0xb1, - 0x43, 0xb7, 0x30, 0xaf, 0xf1, 0x56, 0xf5, 0x73, 0x05, 0x32, 0x54, 0x9f, 0xc6, 0xf2, 0xbf, 0x04, - 0xd9, 0x81, 0x63, 0xed, 0x0f, 0xd8, 0x0a, 0xf2, 0x1a, 0x6f, 0xa1, 0x32, 0xa4, 0x03, 0xbc, 0xcf, - 0xec, 0x9c, 0x46, 0x3e, 0x49, 0x4f, 0xa6, 0xe0, 0x9c, 0x65, 0xde, 0xa2, 0x4e, 0xc0, 0xf2, 0x71, - 0x37, 0x8c, 0x19, 0x8c, 0x01, 0xa8, 0x02, 0x39, 0xe2, 0x42, 0x2d, 0xa7, 0xc7, 0xd9, 0x12, 0x4d, - 0x22, 0x25, 0xab, 0xef, 0xd9, 0x56, 0xd7, 0x0a, 0xa9, 0x8a, 0xe4, 0xb5, 0xa8, 0xad, 0x6e, 0xc1, - 0x42, 0xa4, 0xa5, 0x2f, 0x60, 0x4e, 0x3e, 0x4f, 0x03, 0x92, 0xe9, 0x70, 0x63, 0x32, 0x72, 0x90, - 0x94, 0x13, 0x07, 0xe9, 0x12, 0x94, 0x7c, 0x4c, 0x58, 0x31, 0x78, 0x97, 0x14, 0xed, 0x52, 0xe4, - 0x40, 0xd6, 0xe9, 0x2b, 0x00, 0x8e, 0x6b, 0x0a, 0x22, 0x4c, 0x50, 0x05, 0x02, 0x61, 0xe8, 0x0d, - 0xc8, 0x90, 0xf3, 0x17, 0x70, 0xab, 0x7f, 0x59, 0x56, 0x7f, 0x16, 0x53, 0xd4, 0x45, 0x4c, 0x51, - 0xbf, 0xf7, 0x78, 0x73, 0x93, 0xb2, 0xc8, 0x8f, 0x14, 0x1b, 0x89, 0x4c, 0x28, 0xf5, 0xad, 0x20, - 0xb0, 0x9c, 0x9e, 0x4e, 0xe8, 0x06, 0x95, 0x0c, 0xb5, 0x26, 0xff, 0xef, 0x79, 0xd6, 0x64, 0x68, - 0xb5, 0xf5, 0x7b, 0x8c, 0xc4, 0x7d, 0xd7, 0xc4, 0x9c, 0x7c, 0xb1, 0x1f, 0x83, 0x02, 0x62, 0x35, - 0x0c, 0xcf, 0xf3, 0xdd, 0x8f, 0xac, 0x3e, 0x31, 0x1d, 0xa6, 0x15, 0xec, 0xe9, 0x9d, 0xc3, 0x10, - 0x07, 0x74, 0xc3, 0x66, 0xb4, 0x45, 0x09, 0xdb, 0xb2, 0x82, 0xbd, 0x26, 0xc1, 0x55, 0x3f, 0x80, - 0x59, 0x89, 0x30, 0xba, 0x04, 0x39, 0x2a, 0x0c, 0xcb, 0x64, 0x5b, 0xd3, 0x84, 0xe3, 0xa3, 0x5a, - 0x96, 0xa0, 0xda, 0x2d, 0x2d, 0x4b, 0x50, 0x6d, 0x93, 0x88, 0x15, 0xfb, 0xbe, 0xeb, 0xeb, 0x7d, - 0x1c, 0x04, 0x46, 0x4f, 0x6c, 0x56, 0x91, 0x02, 0xef, 0x31, 0x98, 0xba, 0x04, 0x8b, 0xf7, 0x5d, - 0xe7, 0xc4, 0xee, 0xab, 0x3f, 0x53, 0xe0, 0xdc, 0x08, 0x82, 0x6f, 0xe7, 0xb7, 0x61, 0x81, 0x44, - 0x30, 0x7a, 0x80, 0x7d, 0x0b, 0x07, 0x3a, 0x93, 0xba, 0x42, 0xa5, 0xfe, 0xf5, 0xa9, 0x44, 0xa5, - 0xcd, 0x13, 0x3a, 0xdb, 0x94, 0x0c, 0x45, 0xa0, 0xef, 0x00, 0xb2, 0x9c, 0x10, 0xfb, 0x8e, 0x61, - 0xeb, 0x83, 0x00, 0x73, 0xda, 0xa9, 0xd3, 0xd0, 0x2e, 0x0b, 0x42, 0x8f, 0x02, 0x86, 0x51, 0xe7, - 0xa0, 0xf8, 0x28, 0xc0, 0x7e, 0xb4, 0xc2, 0x1f, 0x40, 0x89, 0xb7, 0xf9, 0xc2, 0xda, 0x90, 0x21, - 0x26, 0x4a, 0xf8, 0xbc, 0xa4, 0x09, 0x87, 0x06, 0xd2, 0x96, 0x50, 0x25, 0x4a, 0xa1, 0xaa, 0xc2, - 0x0c, 0x01, 0x92, 0x33, 0x44, 0x00, 0x92, 0x19, 0x88, 0xda, 0xea, 0x8f, 0x15, 0x28, 0x6d, 0x1d, - 0x60, 0x27, 0x3e, 0x71, 0xc2, 0xb8, 0x29, 0x92, 0x71, 0xbb, 0x00, 0x85, 0xd0, 0xf0, 0x7b, 0x38, - 0x24, 0x7b, 0xcd, 0xce, 0x45, 0x9e, 0x01, 0xda, 0x26, 0x39, 0x86, 0xb6, 0xd5, 0xb7, 0xd8, 0x71, - 0xc8, 0x68, 0xac, 0x81, 0x5e, 0x85, 0x85, 0x81, 0xe3, 0x63, 0xd3, 0xe8, 0x86, 0xd8, 0xd4, 0x31, - 0x9d, 0x82, 0x1e, 0x8b, 0xbc, 0x56, 0x8e, 0x11, 0x6c, 0x6a, 0xf5, 0xd7, 0x29, 0x98, 0x13, 0x5c, - 0x70, 0x39, 0xdc, 0x83, 0x2c, 0x1f, 0xc4, 0x04, 0xd1, 0x48, 0x10, 0xc4, 0xf0, 0x50, 0xd6, 0x14, - 0x6e, 0x9f, 0x11, 0xa9, 0xfe, 0x71, 0x0a, 0x32, 0x14, 0x8e, 0x9a, 0x50, 0x88, 0x62, 0x5f, 0xae, - 0x31, 0xd5, 0x3a, 0x8b, 0x8e, 0xeb, 0x22, 0x3a, 0xae, 0x3f, 0x14, 0x3d, 0x9a, 0x79, 0x42, 0xe6, - 0xd3, 0x5f, 0xd6, 0x14, 0x2d, 0x1e, 0x46, 0xcc, 0x00, 0xa5, 0xab, 0x4b, 0x6e, 0xa0, 0x40, 0x21, - 0x0f, 0x89, 0xb8, 0x5e, 0x91, 0xc5, 0xc5, 0xa2, 0xc6, 0xe2, 0xf1, 0x51, 0x2d, 0xff, 0x90, 0x89, - 0xac, 0x25, 0x09, 0x6f, 0x1d, 0x88, 0x81, 0x71, 0xfd, 0x90, 0x1c, 0x78, 0xcb, 0x64, 0x0e, 0xbe, - 0x39, 0x7f, 0x7c, 0x54, 0x9b, 0xd5, 0x04, 0xbc, 0xdd, 0xd2, 0x66, 0xa3, 0x4e, 0x6d, 0x93, 0xec, - 0x90, 0xe5, 0xec, 0xb8, 0xdc, 0xee, 0xd2, 0x6f, 0x32, 0x25, 0x33, 0xe2, 0x84, 0x08, 0x39, 0xc3, - 0x45, 0x36, 0xe5, 0x23, 0x0a, 0x24, 0x53, 0x32, 0x74, 0xdb, 0x54, 0xff, 0x46, 0x81, 0xf2, 0x36, - 0x0e, 0x1f, 0xb5, 0x49, 0xb4, 0x2c, 0x76, 0xfd, 0xdb, 0x00, 0x7b, 0xf8, 0x90, 0xb9, 0x27, 0x21, - 0xf2, 0x9b, 0x09, 0x22, 0x1f, 0x25, 0x50, 0x7f, 0x17, 0x1f, 0x52, 0x3f, 0x16, 0x6c, 0x39, 0xa1, - 0x7f, 0xa8, 0x15, 0xf6, 0x44, 0xbb, 0x7a, 0x0b, 0xe6, 0x86, 0x91, 0xc4, 0xcf, 0xec, 0xe1, 0x43, - 0xae, 0x61, 0xe4, 0x93, 0xe8, 0x10, 0xf3, 0x8c, 0x44, 0x96, 0x45, 0x8d, 0x35, 0x6e, 0xa6, 0xbe, - 0xa1, 0xa8, 0x67, 0x61, 0x41, 0x9a, 0x8b, 0xed, 0xb0, 0xfa, 0x35, 0x28, 0xdf, 0x1e, 0x5d, 0x01, - 0x82, 0x99, 0x3d, 0x7c, 0x28, 0x52, 0x12, 0xfa, 0xad, 0xfe, 0x2c, 0x05, 0x0b, 0xb7, 0x47, 0x47, - 0xa3, 0xdf, 0x1d, 0xb3, 0xd6, 0xb7, 0x12, 0xd6, 0x7a, 0x82, 0xc2, 0xc8, 0x62, 0xb9, 0xaa, 0x49, - 0x4b, 0xde, 0x81, 0x0c, 0x73, 0xea, 0xd1, 0xba, 0x14, 0x69, 0x5d, 0xe8, 0x36, 0x14, 0x6d, 0x23, - 0x08, 0xf5, 0x81, 0x67, 0x1a, 0x21, 0x36, 0xb9, 0x6d, 0x99, 0x4c, 0x0b, 0x67, 0xc9, 0xc8, 0x47, - 0x6c, 0x60, 0xd5, 0x9b, 0x40, 0xb4, 0xef, 0xc8, 0xa2, 0x9d, 0x5d, 0x5f, 0x9f, 0x6a, 0xa1, 0x94, - 0xb4, 0xbc, 0x1d, 0x65, 0x98, 0xdb, 0xb4, 0x07, 0x41, 0x88, 0x7d, 0x61, 0xc1, 0x3e, 0x53, 0x60, - 0x3e, 0x02, 0x71, 0x09, 0xbf, 0x06, 0xd0, 0x65, 0xa0, 0xd8, 0x39, 0x94, 0x8e, 0x8f, 0x6a, 0x05, - 0xde, 0xb1, 0xdd, 0xd2, 0x0a, 0xbc, 0x43, 0xdb, 0x24, 0xa6, 0x22, 0x3e, 0x03, 0xd8, 0x21, 0x66, - 0xd4, 0xe4, 0x91, 0x49, 0x39, 0x42, 0x6c, 0x31, 0x38, 0xfa, 0x3a, 0x20, 0x4c, 0x8c, 0xaa, 0xe7, - 0x5b, 0x01, 0x8e, 0x7a, 0xb3, 0x88, 0x6b, 0x21, 0xc6, 0xf0, 0xee, 0xea, 0x9f, 0x28, 0x50, 0x6c, - 0xf9, 0x86, 0xe5, 0x08, 0x35, 0xb9, 0x05, 0x55, 0x13, 0x7b, 0x3e, 0xee, 0x12, 0x01, 0xea, 0x9e, - 0xef, 0x76, 0xb0, 0x6e, 0x39, 0xa6, 0xd5, 0x35, 0x42, 0xd7, 0xa7, 0xca, 0x90, 0xd1, 0x2a, 0x71, - 0x8f, 0xf7, 0x49, 0x87, 0xb6, 0xc0, 0x13, 0x53, 0x1a, 0xec, 0x0e, 0x42, 0xd3, 0x7d, 0xea, 0x88, - 0x28, 0x4f, 0xb4, 0xd1, 0x32, 0xe4, 0x4d, 0x57, 0x37, 0xc9, 0x64, 0xdc, 0xd0, 0xe5, 0x4c, 0x97, - 0xce, 0x7d, 0x67, 0x26, 0x9f, 0x2a, 0xa7, 0xd5, 0x5f, 0x29, 0x50, 0xe2, 0xbc, 0x70, 0x39, 0xdd, - 0x80, 0xf3, 0x12, 0x33, 0x74, 0xa8, 0xce, 0x4a, 0x11, 0x9c, 0x93, 0x73, 0x31, 0x9a, 0x8e, 0xdc, - 0xa6, 0x48, 0x12, 0xcc, 0x58, 0x01, 0xeb, 0x4f, 0x82, 0x2c, 0x26, 0x2b, 0xb0, 0x82, 0x16, 0x87, - 0xa0, 0x9b, 0xb0, 0xcc, 0xa8, 0xf9, 0xb8, 0xcf, 0x40, 0xd2, 0x22, 0xd3, 0xd4, 0xc5, 0x9f, 0x37, - 0x19, 0x2b, 0x1c, 0x1f, 0xaf, 0xf1, 0x9b, 0x70, 0x61, 0x74, 0xac, 0x88, 0xac, 0x49, 0xb4, 0xc7, - 0x02, 0xc1, 0xe5, 0xe1, 0xd1, 0xad, 0xb8, 0x83, 0xfa, 0x31, 0x2c, 0xb7, 0x70, 0xd7, 0xed, 0xd3, - 0x80, 0xc3, 0xe5, 0x2c, 0x0b, 0xf1, 0x7f, 0x08, 0x79, 0x1e, 0x33, 0xf0, 0x25, 0x36, 0x9b, 0xc7, - 0x47, 0xb5, 0x1c, 0x0b, 0x1a, 0x82, 0xff, 0x3e, 0xaa, 0x5d, 0xeb, 0x59, 0xe1, 0xee, 0xa0, 0x53, - 0xef, 0xba, 0xfd, 0x46, 0xa4, 0xa9, 0x66, 0x27, 0xfe, 0x6e, 0x78, 0x7b, 0xbd, 0x06, 0xfd, 0xf2, - 0x3a, 0x75, 0x1e, 0x6c, 0xe4, 0x58, 0xb0, 0x11, 0xa8, 0x7f, 0xae, 0xc0, 0x59, 0x79, 0xf2, 0xff, - 0x9b, 0x69, 0xd1, 0x1a, 0xcc, 0x9b, 0xd2, 0xac, 0xf1, 0x9e, 0x8c, 0x82, 0xd5, 0x2f, 0x53, 0x50, - 0x1d, 0x27, 0x1d, 0xae, 0x10, 0x4f, 0x20, 0xcb, 0xf7, 0x9f, 0x25, 0xa7, 0xb7, 0x92, 0x8a, 0x1d, - 0xcf, 0x24, 0x53, 0x67, 0x4d, 0xe1, 0x02, 0x19, 0xc5, 0xea, 0x7f, 0x28, 0x90, 0xe5, 0xfa, 0xf3, - 0x64, 0x38, 0x72, 0xcb, 0x34, 0x37, 0xe2, 0xc8, 0xed, 0xb4, 0xc2, 0x10, 0x01, 0xdf, 0x79, 0xc8, - 0x59, 0x81, 0x6e, 0x5b, 0x07, 0x51, 0x76, 0x61, 0x05, 0x77, 0xad, 0x03, 0x7c, 0x32, 0xc0, 0x4e, - 0x8f, 0x09, 0xb0, 0xc7, 0x48, 0x72, 0x66, 0xac, 0x24, 0x69, 0x66, 0x20, 0x0e, 0x40, 0x86, 0x1d, - 0x45, 0xd1, 0x56, 0x35, 0x98, 0xdf, 0xc6, 0x21, 0x31, 0x1b, 0x41, 0x82, 0x7b, 0x18, 0x89, 0x51, - 0xb8, 0x3f, 0x48, 0x8d, 0xc6, 0x28, 0xcc, 0xbc, 0xaa, 0xff, 0x9c, 0xa2, 0x6e, 0x93, 0x13, 0xe5, - 0xfb, 0xa5, 0x4f, 0xef, 0x36, 0x87, 0x08, 0x3c, 0xd7, 0x93, 0xec, 0x8d, 0xf5, 0x24, 0x05, 0xe1, - 0x49, 0xc6, 0x65, 0xa2, 0xab, 0x30, 0x2b, 0x9f, 0xd7, 0x34, 0x45, 0xc9, 0x20, 0x92, 0xd5, 0x79, - 0x83, 0x8e, 0x6d, 0x75, 0xb9, 0x6c, 0x79, 0xab, 0xea, 0x4e, 0xe0, 0x4e, 0x6e, 0x0f, 0xbb, 0x93, - 0xab, 0xd3, 0x2c, 0xf6, 0x84, 0x37, 0xb9, 0x0c, 0xa5, 0x77, 0xb0, 0x61, 0x87, 0xbb, 0x62, 0x97, - 0x16, 0x21, 0xe3, 0x63, 0xc3, 0x64, 0x33, 0xe6, 0x35, 0xd6, 0x20, 0x4e, 0x47, 0x74, 0xe3, 0x01, - 0xc0, 0x02, 0xcc, 0xdf, 0xe5, 0x05, 0x57, 0xe1, 0x87, 0x7e, 0x9d, 0x82, 0x72, 0x0c, 0xe3, 0xfb, - 0xf3, 0x1e, 0x80, 0x28, 0xcc, 0x46, 0xfb, 0xf3, 0x8a, 0xc4, 0xf2, 0xde, 0x41, 0x5d, 0x14, 0x67, - 0x45, 0x86, 0xe6, 0x75, 0xea, 0x82, 0x8c, 0x28, 0x76, 0xc4, 0x24, 0xd0, 0x4f, 0x14, 0xc8, 0xb3, - 0xf3, 0x84, 0xc5, 0x19, 0x4d, 0x4a, 0xcd, 0x46, 0x19, 0xe2, 0x27, 0x53, 0x6c, 0xf7, 0x5b, 0x84, - 0xfe, 0x8f, 0x7e, 0x79, 0xba, 0x03, 0x17, 0xf1, 0x51, 0xf5, 0xa0, 0x34, 0x44, 0x57, 0xde, 0xb6, - 0x0c, 0xdb, 0xb6, 0xb6, 0xbc, 0x6d, 0x73, 0xeb, 0xd7, 0x9e, 0x2b, 0x03, 0x32, 0x89, 0xe0, 0x9e, - 0x5b, 0x17, 0x69, 0xe3, 0x3c, 0x98, 0xbd, 0xe3, 0x76, 0x02, 0x69, 0xdb, 0x58, 0x0a, 0xa0, 0xc8, - 0x29, 0xc0, 0x92, 0x64, 0xcc, 0x68, 0xf1, 0x80, 0xb5, 0xd0, 0x35, 0xae, 0xb4, 0x69, 0xca, 0x4a, - 0x4d, 0x16, 0xdf, 0xbe, 0x5d, 0xa7, 0x65, 0x7a, 0x56, 0xb2, 0xaf, 0x93, 0x68, 0x9a, 0x69, 0xb5, - 0xfa, 0x4f, 0x59, 0x28, 0xb2, 0x29, 0xf9, 0xd6, 0x6e, 0xc1, 0x0c, 0xe9, 0xc5, 0x37, 0xf5, 0xd5, - 0x84, 0x4d, 0x90, 0x87, 0x91, 0x06, 0xdf, 0x56, 0x3a, 0xbc, 0xfa, 0x5f, 0x19, 0x48, 0xdf, 0x71, - 0x3b, 0x68, 0x09, 0x52, 0xdc, 0x1a, 0xa6, 0x9b, 0xd9, 0xe3, 0xa3, 0x5a, 0xaa, 0xdd, 0xd2, 0x52, - 0x96, 0x79, 0xca, 0x13, 0x26, 0xa7, 0x5c, 0x33, 0xc3, 0x29, 0x17, 0x72, 0x61, 0x6e, 0xa8, 0xa0, - 0xc5, 0x52, 0xfc, 0x52, 0xf3, 0x9d, 0xe3, 0xa3, 0x5a, 0x49, 0xae, 0x68, 0x4d, 0xee, 0x97, 0x82, - 0x7d, 0x9b, 0xfc, 0x75, 0x8c, 0x00, 0xd7, 0xdb, 0x2d, 0xad, 0x24, 0x57, 0xc2, 0x02, 0x69, 0x1f, - 0xb2, 0x43, 0xfb, 0x70, 0x13, 0x72, 0xac, 0x38, 0x68, 0xd2, 0x5a, 0x4c, 0x72, 0x04, 0x3a, 0x43, - 0xa3, 0x4f, 0x31, 0x80, 0x8c, 0x0d, 0x42, 0xc3, 0x27, 0x63, 0xf3, 0x93, 0x8e, 0xe5, 0x03, 0xd0, - 0x2d, 0xc8, 0xef, 0x58, 0x8e, 0x15, 0xec, 0x62, 0xb3, 0x52, 0x98, 0x70, 0x70, 0x34, 0x82, 0x8c, - 0xee, 0xbb, 0xa6, 0xb5, 0x63, 0x61, 0xb3, 0x02, 0x93, 0x8e, 0x16, 0x23, 0x48, 0xf8, 0xb8, 0xe3, - 0x1b, 0xb4, 0x4c, 0xa5, 0x77, 0xdd, 0xbe, 0x67, 0x63, 0xb2, 0x84, 0xd9, 0x55, 0x65, 0x2d, 0xa5, - 0x2d, 0x08, 0xcc, 0xa6, 0x40, 0x10, 0xc5, 0xa6, 0x85, 0x8a, 0x4a, 0x91, 0x59, 0x5d, 0xda, 0x40, - 0x0f, 0xe0, 0xec, 0xae, 0xd5, 0xdb, 0x7d, 0x6a, 0x90, 0x00, 0x37, 0x4e, 0x26, 0x4b, 0x13, 0x72, - 0x83, 0xa2, 0xc1, 0x11, 0x86, 0xb8, 0xa2, 0x98, 0xa4, 0x89, 0xbb, 0x56, 0xdf, 0xb0, 0x2b, 0x73, - 0x74, 0xd2, 0x72, 0x84, 0x68, 0x31, 0x38, 0xba, 0x0c, 0x73, 0xfe, 0xc0, 0xa1, 0x91, 0x19, 0xdf, - 0xd8, 0x79, 0xda, 0xb3, 0xc4, 0xa1, 0xdc, 0xcb, 0x5f, 0x84, 0x42, 0x5c, 0x0d, 0x2e, 0xb3, 0x24, - 0x35, 0x02, 0xa8, 0x08, 0xca, 0x77, 0xdd, 0x2e, 0xbb, 0x24, 0x12, 0x36, 0xf4, 0x7f, 0x14, 0x58, - 0x90, 0x80, 0x51, 0xad, 0xa5, 0x60, 0x0b, 0xe0, 0x04, 0xa5, 0xf8, 0x13, 0x04, 0x22, 0x88, 0x70, - 0x6f, 0x11, 0xb5, 0xea, 0x9f, 0x2a, 0x90, 0x17, 0x58, 0xf4, 0x12, 0x14, 0x09, 0xc6, 0xb6, 0xc2, - 0x43, 0x3d, 0xf6, 0x3a, 0xb3, 0x02, 0xf6, 0x2e, 0x3e, 0x24, 0x2b, 0x8f, 0xba, 0xc4, 0xf6, 0xac, - 0xa0, 0x95, 0x04, 0x94, 0x39, 0xcb, 0x2a, 0xe4, 0x6d, 0x23, 0xb4, 0xc2, 0x81, 0xc9, 0xac, 0x8c, - 0xa2, 0x45, 0x6d, 0x22, 0x15, 0xdb, 0x75, 0x7a, 0x0c, 0x39, 0x43, 0x91, 0x31, 0x40, 0x6d, 0xc2, - 0xbc, 0x66, 0x38, 0x3d, 0x7c, 0xd7, 0xed, 0x09, 0xe3, 0xb6, 0x0c, 0x79, 0x56, 0x39, 0x14, 0xf6, - 0x41, 0xcb, 0xd1, 0xb6, 0x5c, 0xfa, 0x48, 0x49, 0x76, 0x4f, 0xfd, 0xcf, 0x34, 0x94, 0x63, 0x22, - 0x91, 0x27, 0x12, 0xf5, 0x0c, 0xe6, 0x35, 0x92, 0x1c, 0xe7, 0xe8, 0xe0, 0xb1, 0x15, 0x8d, 0x7f, - 0x54, 0x00, 0xde, 0xf7, 0x71, 0x18, 0x1e, 0xb6, 0x9d, 0x1d, 0x97, 0x08, 0x8f, 0xa7, 0x93, 0x34, - 0x5a, 0x17, 0xc2, 0xe3, 0x30, 0x62, 0x56, 0xc8, 0x42, 0x1c, 0xfc, 0x94, 0xa1, 0x99, 0xd8, 0x72, - 0x0e, 0x7e, 0x4a, 0x51, 0x97, 0xa0, 0x64, 0x98, 0x26, 0x36, 0x75, 0x1e, 0x8c, 0x71, 0x9b, 0x56, - 0xa4, 0x40, 0x8d, 0xc1, 0xd0, 0xcb, 0x30, 0xef, 0xe3, 0xbe, 0x7b, 0x20, 0x75, 0x63, 0xb6, 0x6d, - 0x8e, 0x83, 0x45, 0xc7, 0x25, 0xc8, 0xfa, 0xd8, 0x08, 0xa2, 0xd2, 0x30, 0x6f, 0xa1, 0x0a, 0xe4, - 0x4c, 0x76, 0xe9, 0xc1, 0x2d, 0x91, 0x68, 0x56, 0xff, 0x4e, 0x11, 0xe5, 0x99, 0x3b, 0x90, 0xa1, - 0x0b, 0xe4, 0xa5, 0x99, 0xfa, 0x73, 0x1d, 0x95, 0x10, 0x96, 0x2c, 0x23, 0x46, 0x02, 0x7d, 0x08, - 0xb3, 0x1e, 0x95, 0x90, 0x4e, 0xeb, 0x25, 0x2c, 0x62, 0xb9, 0x31, 0x8d, 0xe0, 0x63, 0x01, 0x8b, - 0x58, 0xc0, 0x8b, 0x20, 0x77, 0x66, 0xf2, 0x4a, 0x39, 0xa5, 0xae, 0x41, 0xf9, 0xc1, 0x00, 0xfb, - 0x87, 0xef, 0xdb, 0x86, 0x23, 0xf9, 0xc3, 0x7d, 0x02, 0x13, 0xc1, 0x1a, 0x6d, 0xa8, 0x1e, 0x2c, - 0x48, 0x3d, 0xb9, 0x5e, 0x7c, 0x07, 0x2e, 0x98, 0x56, 0x10, 0x06, 0xfb, 0xb6, 0xee, 0xed, 0x1e, - 0x06, 0x56, 0xd7, 0xb0, 0x75, 0xda, 0x5d, 0xf7, 0x6c, 0xc3, 0xe1, 0xb9, 0xf3, 0xc5, 0xe3, 0xa3, - 0x5a, 0xa5, 0x65, 0x05, 0xe1, 0xf6, 0x83, 0xbb, 0xef, 0xf3, 0x5e, 0x31, 0xa9, 0x0a, 0x27, 0x70, - 0x02, 0xa3, 0x2e, 0xb3, 0x6b, 0x55, 0x32, 0xd2, 0xb7, 0x3a, 0x83, 0x30, 0xce, 0x88, 0xd4, 0xbf, - 0x05, 0xa8, 0x9c, 0xc4, 0x71, 0xa6, 0x3c, 0x28, 0x89, 0x2a, 0x3b, 0x13, 0x1d, 0x3b, 0xf5, 0x5b, - 0xcf, 0xb9, 0x7a, 0x1d, 0x47, 0x2b, 0xba, 0x93, 0x25, 0x22, 0x93, 0x83, 0xdc, 0xa2, 0x29, 0x21, - 0x50, 0x1f, 0x8a, 0xd2, 0x25, 0x92, 0xb8, 0x43, 0x6b, 0x9d, 0x66, 0xc2, 0xf8, 0x62, 0x69, 0x28, - 0xa8, 0x9e, 0x8d, 0x2f, 0x96, 0x82, 0xea, 0x67, 0x0a, 0x40, 0xdc, 0x8f, 0x28, 0x2c, 0xab, 0xc8, - 0xf1, 0x0d, 0xe3, 0x2d, 0xb4, 0x01, 0x59, 0x7e, 0x43, 0x96, 0x9a, 0xf6, 0x86, 0x8c, 0x0f, 0xa4, - 0xa5, 0x10, 0x76, 0x31, 0x16, 0xec, 0xdb, 0xec, 0xbc, 0xf0, 0x52, 0x08, 0x85, 0x6e, 0x3f, 0xb8, - 0xab, 0x15, 0x58, 0x87, 0xed, 0x7d, 0xfb, 0xce, 0x4c, 0x3e, 0x5d, 0x9e, 0xa9, 0xfe, 0x51, 0x1a, - 0x0a, 0xb4, 0x9a, 0x4c, 0x45, 0xf3, 0x85, 0x02, 0x95, 0xa1, 0xc4, 0x49, 0xef, 0x1c, 0xea, 0x71, - 0xfa, 0x46, 0xe4, 0xf4, 0xc1, 0x69, 0xe4, 0x14, 0xcd, 0x50, 0xd7, 0xa4, 0x0c, 0xac, 0x79, 0x48, - 0xc3, 0x4b, 0x93, 0x89, 0xee, 0xcd, 0xd3, 0x06, 0xa7, 0x8b, 0xfe, 0x18, 0x9a, 0xe8, 0xab, 0x30, - 0x27, 0x5f, 0x15, 0x46, 0xc5, 0xe4, 0x62, 0xbc, 0x37, 0x6d, 0x13, 0x7d, 0x0b, 0xc0, 0xf4, 0x5d, - 0xcf, 0xc3, 0xa6, 0x6e, 0xb0, 0x2c, 0x71, 0x12, 0xaf, 0x5a, 0xe0, 0x63, 0x36, 0xc2, 0xea, 0x6d, - 0x58, 0x7e, 0xe6, 0x92, 0xc6, 0xc4, 0xc6, 0x43, 0xc5, 0xc7, 0xb4, 0x14, 0xe6, 0x56, 0x7f, 0x98, - 0x82, 0xa2, 0xac, 0xbf, 0x28, 0x04, 0xf6, 0x42, 0x40, 0x3e, 0x15, 0xef, 0xbd, 0xe8, 0xa9, 0x88, - 0x77, 0x62, 0x28, 0x09, 0x0c, 0x05, 0xb4, 0xfa, 0x09, 0xcc, 0x0d, 0x77, 0x19, 0x93, 0x97, 0x6d, - 0x0f, 0xe7, 0x65, 0x6f, 0xbf, 0x90, 0x46, 0x0c, 0xc9, 0x40, 0x89, 0x5f, 0x81, 0x24, 0x31, 0xf0, - 0x78, 0x98, 0x81, 0xff, 0xff, 0xa2, 0x52, 0x91, 0x79, 0xf8, 0x01, 0x94, 0x47, 0x4f, 0xf5, 0x18, - 0x0e, 0x1e, 0x0e, 0x73, 0xf0, 0xcd, 0x17, 0x33, 0x1e, 0xd2, 0xfc, 0xbc, 0x7e, 0x77, 0x1e, 0xce, - 0xdd, 0xa3, 0xef, 0x70, 0xee, 0xe1, 0xd0, 0x30, 0xe3, 0xd2, 0xb3, 0xfa, 0xaf, 0x0a, 0x2c, 0x8d, - 0x62, 0xb8, 0x25, 0x35, 0x20, 0xdf, 0xe7, 0x30, 0xae, 0x2e, 0xdf, 0x4a, 0x60, 0x6b, 0x3c, 0x91, - 0xba, 0x00, 0xc8, 0xea, 0x11, 0x91, 0xad, 0xfe, 0x0e, 0x94, 0x86, 0x3a, 0x8c, 0x91, 0xcc, 0x1b, - 0xc3, 0x92, 0x91, 0x53, 0xae, 0x41, 0x68, 0xd9, 0x75, 0xfe, 0xbc, 0x28, 0x9a, 0x58, 0xca, 0xf4, - 0x7e, 0x9a, 0x82, 0xb3, 0x5b, 0xce, 0xfe, 0x00, 0x0f, 0x30, 0xf5, 0x8e, 0xc2, 0xc5, 0xfd, 0x36, - 0x4b, 0x48, 0xcc, 0x7d, 0x46, 0xc1, 0x1d, 0x6b, 0xa0, 0xef, 0x4a, 0x71, 0x18, 0xbd, 0x6a, 0x6a, - 0x6e, 0x1e, 0x1f, 0xd5, 0x72, 0x94, 0x2b, 0x3a, 0xe7, 0xf5, 0xa9, 0xe6, 0xe4, 0xe3, 0xe2, 0x60, - 0xee, 0x0a, 0x2c, 0x04, 0x7b, 0x96, 0xa7, 0x07, 0xbb, 0xee, 0xc0, 0x36, 0x75, 0xc6, 0x01, 0x2f, - 0x3e, 0x11, 0xc4, 0x36, 0x85, 0x3f, 0x20, 0x60, 0xf5, 0xaf, 0x52, 0xb0, 0x38, 0x2c, 0x15, 0xbe, - 0xdf, 0x0f, 0xe2, 0x10, 0x87, 0x6d, 0xf7, 0x9b, 0x49, 0xf7, 0x56, 0x63, 0x28, 0xd4, 0xc5, 0xb3, - 0x90, 0x28, 0x36, 0xfa, 0x7b, 0x05, 0x72, 0x1c, 0xf8, 0x5b, 0x95, 0xfa, 0xdb, 0x23, 0x11, 0xea, - 0xe5, 0xa4, 0xbb, 0x4e, 0xdf, 0xe8, 0x62, 0x1a, 0x71, 0x89, 0x78, 0x34, 0x4e, 0x95, 0xd2, 0x52, - 0xaa, 0xa4, 0x9e, 0x83, 0xb3, 0x9b, 0xbb, 0x86, 0x1f, 0x6e, 0xb2, 0xb7, 0x6d, 0xe2, 0xc4, 0x3c, - 0x81, 0xc5, 0x61, 0x30, 0x17, 0x5f, 0x13, 0x72, 0xfc, 0x15, 0x1c, 0x17, 0x9f, 0x2a, 0x31, 0x11, - 0x06, 0x75, 0xf1, 0x44, 0x8e, 0x0e, 0xde, 0x66, 0xcf, 0x0d, 0xa2, 0x97, 0x38, 0x0c, 0x77, 0xe5, - 0x5d, 0x58, 0x1a, 0xff, 0xb6, 0x04, 0xcd, 0x42, 0xee, 0xd1, 0xfd, 0x77, 0xef, 0xbf, 0xf7, 0xc1, - 0xfd, 0xf2, 0x19, 0xd2, 0xd8, 0xbc, 0xfb, 0x68, 0xfb, 0xe1, 0x96, 0x56, 0x56, 0x50, 0x11, 0xf2, - 0xad, 0x8d, 0x87, 0x1b, 0xcd, 0x8d, 0xed, 0xad, 0x72, 0x0a, 0x15, 0x20, 0xf3, 0x70, 0xa3, 0x79, - 0x77, 0xab, 0x9c, 0x5e, 0xff, 0x49, 0x15, 0x32, 0x1b, 0x66, 0xdf, 0x72, 0x50, 0x08, 0x19, 0x7a, - 0xe1, 0x8a, 0x5e, 0x7e, 0xfe, 0x95, 0x2c, 0x5d, 0x64, 0x75, 0x6d, 0xd2, 0xbb, 0x5b, 0xb5, 0xf2, - 0xa3, 0x7f, 0xf9, 0xd5, 0x4f, 0x53, 0x08, 0x95, 0x1b, 0x3a, 0x7d, 0xf3, 0xd8, 0x38, 0xb8, 0xda, - 0xa0, 0x77, 0xb8, 0xe8, 0x0f, 0x14, 0x28, 0x44, 0x6f, 0xf0, 0xd0, 0xab, 0x13, 0xbc, 0x7d, 0x8b, - 0xa6, 0x7f, 0x6d, 0xb2, 0xce, 0x9c, 0x85, 0x8b, 0x94, 0x85, 0x25, 0xb4, 0x28, 0xb1, 0x10, 0x3d, - 0xeb, 0x43, 0x7f, 0xa1, 0xc0, 0xfc, 0xc8, 0xe3, 0x3a, 0x74, 0x75, 0x9a, 0x87, 0x78, 0x8c, 0xa5, - 0xf5, 0xe9, 0xdf, 0xee, 0xa9, 0x2f, 0x53, 0xc6, 0x5e, 0x42, 0xb5, 0x71, 0x8c, 0x35, 0x3e, 0x11, - 0x9f, 0xdf, 0x47, 0x7f, 0xad, 0x40, 0x51, 0x7e, 0x58, 0x85, 0xea, 0x13, 0xbf, 0xc0, 0x62, 0xdc, - 0x35, 0xa6, 0x7c, 0xb1, 0xa5, 0xde, 0xa0, 0xac, 0xbd, 0x8e, 0xea, 0xcf, 0x61, 0xad, 0x41, 0x5d, - 0x79, 0xd0, 0xf8, 0x84, 0xfe, 0x52, 0x4e, 0x21, 0x7e, 0x2d, 0x80, 0x5e, 0x9b, 0xf0, 0x51, 0x01, - 0xe3, 0x72, 0xba, 0x27, 0x08, 0xea, 0x2d, 0xca, 0xe3, 0x0d, 0x74, 0x7d, 0x3a, 0x1e, 0x1b, 0xec, - 0x35, 0xca, 0x9f, 0x29, 0x50, 0x1a, 0x7a, 0x80, 0x81, 0x92, 0x84, 0x34, 0xee, 0x0d, 0x47, 0xf5, - 0xf5, 0xc9, 0x07, 0x70, 0x96, 0x57, 0x29, 0xcb, 0x55, 0x54, 0x91, 0x58, 0x76, 0x5c, 0x87, 0x31, - 0x48, 0x99, 0xf8, 0x08, 0xb2, 0xec, 0xce, 0x1f, 0xad, 0x4d, 0xf0, 0x2c, 0x80, 0xf1, 0xf1, 0xca, - 0xc4, 0x0f, 0x08, 0xd4, 0x65, 0xca, 0xc0, 0x59, 0xb4, 0x20, 0x31, 0xc0, 0xad, 0x1c, 0x39, 0x8f, - 0xd1, 0x7d, 0x74, 0xe2, 0x79, 0x1c, 0xbd, 0x21, 0x4f, 0x3c, 0x8f, 0x27, 0xaf, 0xb8, 0xf9, 0x79, - 0x54, 0x65, 0x1e, 0x06, 0x16, 0xd9, 0xae, 0x9b, 0xca, 0x15, 0xf4, 0x43, 0x05, 0x0a, 0xb7, 0x27, - 0x62, 0xe3, 0xf6, 0x34, 0x6c, 0x9c, 0xb8, 0x00, 0x1e, 0x2b, 0x0a, 0xc6, 0x06, 0xfa, 0x3d, 0xc8, - 0xf1, 0xeb, 0x5c, 0x94, 0x24, 0xdb, 0xe1, 0xeb, 0xe2, 0xea, 0x95, 0x49, 0xba, 0xf2, 0xc9, 0xab, - 0x74, 0xf2, 0x45, 0x84, 0xa4, 0xc9, 0xf9, 0xb5, 0x31, 0xfa, 0x7d, 0x05, 0xf2, 0xe2, 0x82, 0x01, - 0x5d, 0x99, 0xe8, 0x16, 0x82, 0x31, 0xf0, 0xea, 0x14, 0x37, 0x16, 0xea, 0x05, 0xca, 0xc1, 0x39, - 0x74, 0x56, 0xe2, 0x20, 0x10, 0xb3, 0xfe, 0x58, 0x81, 0x2c, 0xbb, 0x97, 0x48, 0x54, 0xc3, 0xa1, - 0x1b, 0x8e, 0x44, 0x35, 0x1c, 0xb9, 0xe4, 0xf8, 0x2a, 0x9d, 0x7c, 0x65, 0x48, 0xf6, 0xbb, 0xb4, - 0xcb, 0x93, 0x02, 0xca, 0xf1, 0x4f, 0x2a, 0x08, 0x51, 0xa8, 0x4f, 0x14, 0xc4, 0xc8, 0x85, 0x49, - 0xa2, 0x20, 0x46, 0xef, 0x2d, 0xc6, 0x0a, 0x42, 0x5c, 0x8b, 0x20, 0x0f, 0x66, 0xee, 0xb8, 0x9d, - 0x00, 0x7d, 0xed, 0xb9, 0x45, 0x78, 0x36, 0xf3, 0xcb, 0x13, 0x16, 0xeb, 0xd5, 0xf3, 0x74, 0xd6, - 0x05, 0x34, 0x2f, 0xcd, 0xfa, 0x3d, 0x32, 0x13, 0x39, 0x86, 0x51, 0x9d, 0x31, 0x51, 0xff, 0x47, - 0x6b, 0x9c, 0x89, 0xfa, 0x7f, 0xa2, 0x74, 0x39, 0xd6, 0x2d, 0x46, 0xe5, 0x4b, 0xca, 0x46, 0x54, - 0x6d, 0x49, 0x64, 0x63, 0xb4, 0x44, 0x94, 0xc8, 0xc6, 0x89, 0x2a, 0xd1, 0x58, 0x36, 0x68, 0x95, - 0xc8, 0x23, 0x13, 0x7f, 0x17, 0x32, 0xf4, 0xe6, 0x3f, 0x31, 0x34, 0x91, 0x5f, 0x41, 0x24, 0x86, - 0x26, 0x43, 0x4f, 0x14, 0xd4, 0x33, 0xaf, 0x2b, 0xe8, 0x29, 0x14, 0xe5, 0xcb, 0xe6, 0x44, 0xc7, - 0x3a, 0xe6, 0xf6, 0xbd, 0xfa, 0xc6, 0xa9, 0x6e, 0xb1, 0xd5, 0x33, 0xe4, 0x84, 0xa1, 0x93, 0x1d, - 0xd0, 0xf5, 0x29, 0xe9, 0xbd, 0x20, 0x17, 0x7f, 0xa9, 0x40, 0x5e, 0x54, 0x06, 0x13, 0x4f, 0xd8, - 0x48, 0xe5, 0x38, 0xf1, 0x84, 0x8d, 0x96, 0x1a, 0xd5, 0xb7, 0xe9, 0x16, 0xbf, 0x39, 0x74, 0xc2, - 0x68, 0x6a, 0x62, 0xbb, 0xbd, 0x27, 0xab, 0x68, 0x65, 0x0c, 0xb8, 0xf1, 0x89, 0x48, 0x88, 0xbe, - 0x4f, 0x22, 0xb4, 0xf2, 0x68, 0x56, 0x8b, 0xd6, 0xa7, 0x4a, 0x81, 0x19, 0xd3, 0xd7, 0x4e, 0x91, - 0x36, 0x47, 0xa6, 0xea, 0xe2, 0x48, 0x94, 0xa1, 0x9b, 0x32, 0x3b, 0x9f, 0x29, 0xb0, 0xb0, 0x61, - 0xdb, 0xc3, 0x59, 0x2e, 0x7a, 0x7d, 0x8a, 0x84, 0x98, 0xb1, 0x78, 0x75, 0xea, 0x14, 0x5a, 0x7d, - 0x89, 0x32, 0x78, 0x01, 0x2d, 0x4b, 0x0c, 0xb2, 0xac, 0x57, 0xe4, 0xd1, 0xe8, 0x53, 0x05, 0x8a, - 0x72, 0x52, 0x92, 0xa8, 0xe6, 0x63, 0x92, 0x9a, 0xc4, 0xf8, 0x71, 0x5c, 0xb6, 0xa3, 0xd6, 0x28, - 0x53, 0xcb, 0xe8, 0xbc, 0xec, 0xdf, 0x48, 0x47, 0x9e, 0xca, 0x90, 0xf0, 0xab, 0x28, 0x27, 0x89, - 0x89, 0x2c, 0x8d, 0xc9, 0xd2, 0x13, 0x59, 0x1a, 0x97, 0x7d, 0xaa, 0x97, 0x28, 0x4b, 0x5f, 0x51, - 0xe5, 0xd8, 0x0b, 0xb3, 0x8e, 0x3a, 0x55, 0xb6, 0x9b, 0xca, 0x95, 0xe6, 0x95, 0x2f, 0xfe, 0x7d, - 0xe5, 0xcc, 0x17, 0xc7, 0x2b, 0xca, 0xcf, 0x8f, 0x57, 0x94, 0x5f, 0x1c, 0xaf, 0x28, 0xff, 0x76, - 0xbc, 0xa2, 0x7c, 0xfa, 0xe5, 0xca, 0x99, 0x9f, 0x7f, 0xb9, 0x72, 0xe6, 0x17, 0x5f, 0xae, 0x9c, - 0x79, 0x92, 0x17, 0xb3, 0x74, 0xb2, 0xb4, 0x60, 0x77, 0xed, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, - 0xfc, 0x17, 0x6a, 0x6a, 0xdd, 0x35, 0x00, 0x00, +func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_05a80b84e7fbd849) } + +var fileDescriptor_admin_05a80b84e7fbd849 = []byte{ + // 4172 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x4d, 0x74, 0x1b, 0xc7, + 0x79, 0x5a, 0x80, 0xf8, 0xfb, 0x08, 0x90, 0xe0, 0x88, 0xa2, 0x40, 0x48, 0x21, 0xe8, 0x55, 0x1c, + 0xcb, 0xb2, 0x03, 0x58, 0x94, 0x2c, 0xa7, 0xb2, 0x9c, 0x94, 0x20, 0xf8, 0x64, 0xc8, 0x92, 0x6c, + 0x2d, 0x25, 0xf9, 0x45, 0xa9, 0xbd, 0x5d, 0x60, 0x87, 0xe0, 0x86, 0x8b, 0xdd, 0xe5, 0xee, 0x82, + 0x32, 0xed, 0x26, 0xaf, 0x49, 0xda, 0xbe, 0xf6, 0xbd, 0xbe, 0x3e, 0x37, 0xe9, 0xcd, 0xa7, 0xf6, + 0xd0, 0x1e, 0x7a, 0xa9, 0x7b, 0xec, 0xa1, 0x67, 0xbf, 0x1e, 0x9a, 0xf4, 0xb5, 0x87, 0xf4, 0xc2, + 0xb4, 0x74, 0x4e, 0xe9, 0xbd, 0x87, 0x5e, 0xda, 0x37, 0x7f, 0xbb, 0x03, 0x10, 0x5a, 0x02, 0x54, + 0xd2, 0x03, 0x89, 0x9d, 0xef, 0x9b, 0xf9, 0xe6, 0x9b, 0x6f, 0xbe, 0xf9, 0xfe, 0x66, 0xe0, 0x42, + 0x80, 0xfd, 0x7d, 0xec, 0x37, 0xd8, 0x8f, 0xd7, 0x69, 0x18, 0x66, 0xdf, 0x72, 0xea, 0x9e, 0xef, + 0x86, 0x2e, 0x5a, 0xee, 0xba, 0xdd, 0x5d, 0xdf, 0x35, 0xba, 0x3b, 0x75, 0x86, 0xaf, 0x8b, 0x6e, + 0xd5, 0x4a, 0xd7, 0x75, 0xb6, 0xad, 0x5e, 0xe3, 0x63, 0xd7, 0xc1, 0x5e, 0x87, 0xfe, 0xb0, 0x41, + 0xd5, 0xa5, 0xef, 0xba, 0x9d, 0xa0, 0x41, 0xfe, 0x79, 0x1d, 0xfa, 0xc3, 0xe1, 0x17, 0x47, 0x67, + 0x0a, 0x42, 0x23, 0x1c, 0x08, 0xec, 0x85, 0x20, 0x74, 0x7d, 0xa3, 0x87, 0x1b, 0xd8, 0xe9, 0x59, + 0x94, 0x64, 0x7f, 0xbf, 0xdb, 0xe5, 0xc8, 0x17, 0x77, 0xf7, 0x1b, 0xbb, 0xfb, 0x7c, 0xbc, 0xf8, + 0xf0, 0x3a, 0x0d, 0xdb, 0xda, 0xc7, 0x0e, 0x0e, 0x04, 0x8d, 0xd5, 0x67, 0x75, 0x73, 0x7b, 0xbc, + 0xc7, 0x4a, 0x18, 0x34, 0xba, 0x46, 0x68, 0xd8, 0x6e, 0xaf, 0xd1, 0xdd, 0x31, 0xfc, 0x50, 0xe7, + 0x2d, 0x8e, 0xaf, 0x0c, 0x42, 0xcb, 0x6e, 0xf4, 0x71, 0xe8, 0x5b, 0x5d, 0xfe, 0xc3, 0x31, 0x8b, + 0x3d, 0xb7, 0xe7, 0xd2, 0xcf, 0x06, 0xf9, 0x12, 0x6b, 0xea, 0xb9, 0x6e, 0xcf, 0xc6, 0x0d, 0xc3, + 0xb3, 0x1a, 0x86, 0xe3, 0xb8, 0xa1, 0x11, 0x5a, 0xae, 0x23, 0xf8, 0xa9, 0x71, 0x2c, 0x6d, 0x75, + 0x06, 0xdb, 0x8d, 0xd0, 0xea, 0xe3, 0x20, 0x34, 0xfa, 0x1e, 0xeb, 0xa0, 0x22, 0x28, 0xb7, 0x8c, + 0xd0, 0xe8, 0x18, 0x01, 0x0e, 0x34, 0xbc, 0x37, 0xc0, 0x41, 0xa8, 0x5e, 0x85, 0x05, 0x09, 0x16, + 0x78, 0xae, 0x13, 0x60, 0x74, 0x11, 0x0a, 0xa6, 0x00, 0x56, 0x94, 0xd5, 0xf4, 0xe5, 0x82, 0x16, + 0x03, 0xd4, 0xeb, 0xb0, 0x24, 0x86, 0xb4, 0x70, 0x68, 0x58, 0xb6, 0x20, 0x86, 0xaa, 0x90, 0x17, + 0xdd, 0x2a, 0xca, 0xaa, 0x72, 0xb9, 0xa0, 0x45, 0x6d, 0xf5, 0xf3, 0x34, 0x9c, 0x3f, 0x36, 0x8c, + 0xcf, 0xf7, 0x18, 0xb2, 0x3d, 0xdf, 0x70, 0x42, 0x36, 0xd9, 0xec, 0xda, 0x37, 0xea, 0xcf, 0xd4, + 0x84, 0xfa, 0x33, 0x68, 0xd4, 0x6f, 0x13, 0x02, 0xcd, 0x99, 0x2f, 0x0e, 0x6b, 0x67, 0x34, 0x4e, + 0x0d, 0xd5, 0x60, 0x36, 0x34, 0x3a, 0x36, 0xd6, 0x1d, 0xa3, 0x8f, 0x83, 0x4a, 0x8a, 0xae, 0x04, + 0x28, 0xe8, 0x3e, 0x81, 0xa0, 0xd7, 0xa1, 0x64, 0xe2, 0xa0, 0xeb, 0x5b, 0x5e, 0xe8, 0xfa, 0xba, + 0x65, 0x56, 0xd2, 0xab, 0xca, 0xe5, 0x74, 0xb3, 0x7c, 0x74, 0x58, 0x2b, 0xb6, 0x22, 0x44, 0xbb, + 0xa5, 0x15, 0xe3, 0x6e, 0x6d, 0x13, 0xdd, 0x81, 0x59, 0xa2, 0x81, 0x3a, 0x53, 0xca, 0xca, 0xcc, + 0xaa, 0x72, 0x79, 0x76, 0xed, 0x92, 0xc4, 0x34, 0x43, 0xd4, 0x99, 0xb6, 0xd6, 0x9f, 0xb8, 0x0e, + 0xde, 0xa0, 0x10, 0xce, 0x1f, 0x7c, 0x1c, 0x41, 0xd0, 0x07, 0xb0, 0x20, 0xd1, 0xd2, 0x6d, 0xbc, + 0x8f, 0xed, 0x4a, 0x66, 0x55, 0xb9, 0x3c, 0xb7, 0x76, 0x35, 0x41, 0x0c, 0x31, 0xcd, 0x81, 0x4f, + 0xb5, 0xe0, 0x2e, 0x19, 0xa8, 0xcd, 0xc7, 0x94, 0x29, 0xa0, 0xfa, 0x26, 0x64, 0xa8, 0x64, 0x10, + 0x82, 0x99, 0x41, 0x80, 0x7d, 0xbe, 0x2f, 0xf4, 0x1b, 0xad, 0x00, 0x78, 0xbe, 0xb5, 0x6f, 0xd9, + 0xb8, 0x17, 0x8b, 0x27, 0x86, 0xa8, 0xb7, 0xe1, 0xec, 0x43, 0x22, 0xac, 0xc9, 0xb7, 0x19, 0x2d, + 0x42, 0x86, 0xca, 0xb7, 0x92, 0xa2, 0x08, 0xd6, 0x50, 0xff, 0x30, 0x0f, 0x8b, 0xc3, 0x94, 0xf8, + 0xce, 0x6f, 0x8d, 0xec, 0xfc, 0xeb, 0x09, 0x4b, 0x1e, 0x47, 0x60, 0xec, 0xb6, 0x3f, 0x86, 0x5c, + 0xd7, 0xb5, 0x07, 0x7d, 0x87, 0xad, 0x69, 0x76, 0xed, 0xc6, 0xb4, 0x54, 0x37, 0xe8, 0x70, 0x4e, + 0x56, 0x10, 0x43, 0x8f, 0x20, 0x67, 0x39, 0x26, 0xfe, 0x08, 0x07, 0x95, 0xf4, 0xe9, 0xb8, 0x6d, + 0x93, 0xe1, 0x82, 0x2c, 0xa7, 0x45, 0xb4, 0xd4, 0x37, 0x9c, 0x1e, 0x51, 0x81, 0x81, 0x13, 0x52, + 0x6d, 0x4a, 0x6b, 0x40, 0x41, 0x1b, 0x04, 0x82, 0xae, 0xc3, 0x52, 0xd7, 0xc7, 0x46, 0x88, 0x75, + 0xa6, 0xcd, 0xc4, 0x92, 0xe1, 0x3e, 0x76, 0x42, 0xaa, 0x27, 0x05, 0x6d, 0x91, 0x61, 0xe9, 0x8c, + 0x5b, 0x02, 0x37, 0xaa, 0xa4, 0xd9, 0x5f, 0xbb, 0x92, 0xe6, 0x7e, 0x5d, 0x4a, 0x7a, 0xfc, 0x18, + 0xe6, 0x27, 0x39, 0x86, 0xcf, 0xa5, 0xdb, 0xd5, 0x7f, 0x54, 0x20, 0xcb, 0xb6, 0x99, 0x0c, 0x27, + 0x06, 0x42, 0x0c, 0x27, 0xdf, 0x04, 0x16, 0x1e, 0x78, 0x42, 0x8d, 0xe9, 0x37, 0xd1, 0x7b, 0x67, + 0x60, 0xdb, 0x54, 0xbd, 0x89, 0xa1, 0xc8, 0x6b, 0x51, 0x1b, 0x5d, 0x22, 0x4b, 0xd8, 0x36, 0x06, + 0x76, 0xa8, 0xef, 0x1b, 0xf6, 0x00, 0xd3, 0x6d, 0x2c, 0x10, 0x86, 0x29, 0xf0, 0x31, 0x81, 0xa1, + 0x6b, 0x70, 0xae, 0x87, 0x1d, 0xcc, 0x64, 0xa1, 0xe3, 0x8f, 0x3c, 0x1f, 0x07, 0x81, 0xe5, 0x3a, + 0x62, 0x1f, 0x63, 0xe4, 0x66, 0x84, 0x43, 0x4b, 0x90, 0xdd, 0xb1, 0x4c, 0x13, 0x3b, 0x74, 0x0b, + 0xf3, 0x1a, 0x6f, 0x55, 0x3f, 0x57, 0x20, 0x43, 0xf5, 0x69, 0x2c, 0xff, 0x4b, 0x90, 0x1d, 0x38, + 0xd6, 0xde, 0x80, 0xad, 0x20, 0xaf, 0xf1, 0x16, 0x2a, 0x43, 0x3a, 0xc0, 0x7b, 0xcc, 0xce, 0x69, + 0xe4, 0x93, 0xf4, 0x64, 0x0a, 0xce, 0x59, 0xe6, 0x2d, 0xea, 0x04, 0x2c, 0x1f, 0x77, 0xc3, 0x98, + 0xc1, 0x18, 0x80, 0x2a, 0x90, 0x23, 0x2e, 0xd4, 0x72, 0x7a, 0x9c, 0x2d, 0xd1, 0x24, 0x52, 0xb2, + 0xfa, 0x9e, 0x6d, 0x75, 0xad, 0x90, 0xaa, 0x48, 0x5e, 0x8b, 0xda, 0xea, 0x26, 0x2c, 0x44, 0x5a, + 0xfa, 0x1c, 0xe6, 0xe4, 0xf3, 0x34, 0x20, 0x99, 0x0e, 0x37, 0x26, 0x23, 0x07, 0x49, 0x39, 0x76, + 0x90, 0x2e, 0x41, 0xc9, 0xc7, 0x84, 0x15, 0x83, 0x77, 0x49, 0xd1, 0x2e, 0x45, 0x0e, 0x64, 0x9d, + 0xbe, 0x02, 0xe0, 0xb8, 0xa6, 0x20, 0xc2, 0x04, 0x55, 0x20, 0x10, 0x86, 0x5e, 0x87, 0x0c, 0x39, + 0x7f, 0x01, 0xb7, 0xfa, 0x2f, 0xca, 0xea, 0xcf, 0x62, 0x8a, 0xba, 0x88, 0x29, 0xea, 0xf7, 0x1e, + 0x6f, 0x6c, 0x50, 0x16, 0xf9, 0x91, 0x62, 0x23, 0x91, 0x09, 0xa5, 0xbe, 0x15, 0x04, 0x96, 0xd3, + 0xd3, 0x09, 0xdd, 0xa0, 0x92, 0xa1, 0xd6, 0xe4, 0xb7, 0x4e, 0xb2, 0x26, 0x43, 0xab, 0xad, 0xdf, + 0x63, 0x24, 0xee, 0xbb, 0x26, 0xe6, 0xe4, 0x8b, 0xfd, 0x18, 0x14, 0x10, 0xab, 0x61, 0x78, 0x9e, + 0xef, 0x7e, 0x64, 0xf5, 0x89, 0xe9, 0x30, 0xad, 0x60, 0x57, 0xef, 0x1c, 0x84, 0x38, 0xa0, 0x1b, + 0x36, 0xa3, 0x2d, 0x4a, 0xd8, 0x96, 0x15, 0xec, 0x36, 0x09, 0xae, 0xfa, 0x3e, 0xcc, 0x4a, 0x84, + 0xd1, 0x25, 0xc8, 0x51, 0x61, 0x58, 0x26, 0xdb, 0x9a, 0x26, 0x1c, 0x1d, 0xd6, 0xb2, 0x04, 0xd5, + 0x6e, 0x69, 0x59, 0x82, 0x6a, 0x9b, 0x44, 0xac, 0xd8, 0xf7, 0x5d, 0x5f, 0xef, 0xe3, 0x20, 0x30, + 0x7a, 0x62, 0xb3, 0x8a, 0x14, 0x78, 0x8f, 0xc1, 0xd4, 0x25, 0x58, 0xbc, 0xef, 0x3a, 0xc7, 0x76, + 0x5f, 0xfd, 0xa9, 0x02, 0xe7, 0x46, 0x10, 0x7c, 0x3b, 0xbf, 0x0d, 0x0b, 0x24, 0x82, 0xd1, 0x03, + 0xec, 0x5b, 0x38, 0xd0, 0x99, 0xd4, 0x15, 0x2a, 0xf5, 0xaf, 0x4f, 0x25, 0x2a, 0x6d, 0x9e, 0xd0, + 0xd9, 0xa2, 0x64, 0x28, 0x02, 0x7d, 0x07, 0x90, 0xe5, 0x84, 0xd8, 0x77, 0x0c, 0x5b, 0x1f, 0x04, + 0x98, 0xd3, 0x4e, 0x9d, 0x86, 0x76, 0x59, 0x10, 0x7a, 0x14, 0x30, 0x8c, 0x3a, 0x07, 0xc5, 0x47, + 0x01, 0xf6, 0xa3, 0x15, 0x7e, 0x1f, 0x4a, 0xbc, 0xcd, 0x17, 0xd6, 0x86, 0x0c, 0x31, 0x51, 0xc2, + 0xe7, 0x25, 0x4d, 0x38, 0x34, 0x90, 0xb6, 0x84, 0x2a, 0x51, 0x0a, 0x55, 0x15, 0x66, 0x08, 0x90, + 0x9c, 0x21, 0x02, 0x90, 0xcc, 0x40, 0xd4, 0x56, 0x7f, 0xa4, 0x40, 0x69, 0x73, 0x1f, 0x3b, 0xf1, + 0x89, 0x13, 0xc6, 0x4d, 0x91, 0x8c, 0xdb, 0x05, 0x28, 0x84, 0x86, 0xdf, 0xc3, 0x21, 0xd9, 0x6b, + 0x76, 0x2e, 0xf2, 0x0c, 0xd0, 0x36, 0xc9, 0x31, 0xb4, 0xad, 0xbe, 0xc5, 0x8e, 0x43, 0x46, 0x63, + 0x0d, 0xf4, 0x0a, 0x2c, 0x0c, 0x1c, 0x1f, 0x9b, 0x46, 0x37, 0xc4, 0xa6, 0x8e, 0xe9, 0x14, 0xf4, + 0x58, 0xe4, 0xb5, 0x72, 0x8c, 0x60, 0x53, 0xab, 0xbf, 0x4a, 0xc1, 0x9c, 0xe0, 0x82, 0xcb, 0xe1, + 0x1e, 0x64, 0xf9, 0x20, 0x26, 0x88, 0x46, 0x82, 0x20, 0x86, 0x87, 0xb2, 0xa6, 0x70, 0xfb, 0x8c, + 0x48, 0xf5, 0x8f, 0x53, 0x90, 0xa1, 0x70, 0xd4, 0x84, 0x42, 0x14, 0xfb, 0x72, 0x8d, 0xa9, 0xd6, + 0x59, 0x74, 0x5c, 0x17, 0xd1, 0x71, 0xfd, 0xa1, 0xe8, 0xd1, 0xcc, 0x13, 0x32, 0x9f, 0xfe, 0xa2, + 0xa6, 0x68, 0xf1, 0x30, 0x62, 0x06, 0x28, 0x5d, 0x5d, 0x72, 0x03, 0x05, 0x0a, 0x79, 0x48, 0xc4, + 0xf5, 0xb2, 0x2c, 0x2e, 0x16, 0x35, 0x16, 0x8f, 0x0e, 0x6b, 0xf9, 0x87, 0x4c, 0x64, 0x2d, 0x49, + 0x78, 0x6b, 0x40, 0x0c, 0x8c, 0xeb, 0x87, 0xe4, 0xc0, 0x5b, 0x26, 0x73, 0xf0, 0xcd, 0xf9, 0xa3, + 0xc3, 0xda, 0xac, 0x26, 0xe0, 0xed, 0x96, 0x36, 0x1b, 0x75, 0x6a, 0x9b, 0x64, 0x87, 0x2c, 0x67, + 0xdb, 0xe5, 0x76, 0x97, 0x7e, 0x93, 0x29, 0x99, 0x11, 0x27, 0x44, 0xc8, 0x19, 0x2e, 0xb2, 0x29, + 0x1f, 0x51, 0x20, 0x99, 0x92, 0xa1, 0xdb, 0xa6, 0xfa, 0xb7, 0x0a, 0x94, 0xb7, 0x70, 0xf8, 0xa8, + 0x4d, 0xa2, 0x65, 0xb1, 0xeb, 0xdf, 0x06, 0xd8, 0xc5, 0x07, 0xcc, 0x3d, 0x09, 0x91, 0xdf, 0x4c, + 0x10, 0xf9, 0x28, 0x81, 0xfa, 0x3b, 0xf8, 0x80, 0xfa, 0xb1, 0x60, 0xd3, 0x09, 0xfd, 0x03, 0xad, + 0xb0, 0x2b, 0xda, 0xd5, 0x5b, 0x30, 0x37, 0x8c, 0x24, 0x7e, 0x66, 0x17, 0x1f, 0x70, 0x0d, 0x23, + 0x9f, 0x44, 0x87, 0x98, 0x67, 0x24, 0xb2, 0x2c, 0x6a, 0xac, 0x71, 0x33, 0xf5, 0x0d, 0x45, 0x3d, + 0x0b, 0x0b, 0xd2, 0x5c, 0x6c, 0x87, 0xd5, 0xaf, 0x41, 0xf9, 0xf6, 0xe8, 0x0a, 0x10, 0xcc, 0xec, + 0xe2, 0x03, 0x91, 0x92, 0xd0, 0x6f, 0xf5, 0xa7, 0x29, 0x58, 0xb8, 0x3d, 0x3a, 0x1a, 0xfd, 0xee, + 0x98, 0xb5, 0xbe, 0x99, 0xb0, 0xd6, 0x63, 0x14, 0x46, 0x16, 0xcb, 0x55, 0x4d, 0x5a, 0xf2, 0x36, + 0x64, 0x98, 0x53, 0x8f, 0xd6, 0xa5, 0x48, 0xeb, 0x42, 0xb7, 0xa1, 0x68, 0x1b, 0x41, 0xa8, 0x0f, + 0x3c, 0xd3, 0x08, 0xb1, 0xc9, 0x6d, 0xcb, 0x64, 0x5a, 0x38, 0x4b, 0x46, 0x3e, 0x62, 0x03, 0xab, + 0xde, 0x04, 0xa2, 0x7d, 0x5b, 0x16, 0xed, 0xec, 0xda, 0xda, 0x54, 0x0b, 0xa5, 0xa4, 0xe5, 0xed, + 0x28, 0xc3, 0xdc, 0x86, 0x3d, 0x08, 0x42, 0xec, 0x0b, 0x0b, 0xf6, 0x99, 0x02, 0xf3, 0x11, 0x88, + 0x4b, 0xf8, 0x55, 0x80, 0x2e, 0x03, 0xc5, 0xce, 0xa1, 0x74, 0x74, 0x58, 0x2b, 0xf0, 0x8e, 0xed, + 0x96, 0x56, 0xe0, 0x1d, 0xda, 0x26, 0x31, 0x15, 0xf1, 0x19, 0xc0, 0x0e, 0x31, 0xa3, 0x26, 0x8f, + 0x4c, 0xca, 0x11, 0x62, 0x93, 0xc1, 0xd1, 0xd7, 0x01, 0x61, 0x62, 0x54, 0x3d, 0xdf, 0x0a, 0x70, + 0xd4, 0x9b, 0x45, 0x5c, 0x0b, 0x31, 0x86, 0x77, 0x57, 0xff, 0x44, 0x81, 0x62, 0xcb, 0x37, 0x2c, + 0x47, 0xa8, 0xc9, 0x2d, 0xa8, 0x9a, 0xd8, 0xf3, 0x71, 0x97, 0x08, 0x50, 0xf7, 0x7c, 0xb7, 0x83, + 0x75, 0xcb, 0x31, 0xad, 0xae, 0x11, 0xba, 0x3e, 0x55, 0x86, 0x8c, 0x56, 0x89, 0x7b, 0xbc, 0x47, + 0x3a, 0xb4, 0x05, 0x9e, 0x98, 0xd2, 0x60, 0x67, 0x10, 0x9a, 0xee, 0x53, 0x47, 0x44, 0x79, 0xa2, + 0x8d, 0x96, 0x21, 0x6f, 0xba, 0xba, 0x49, 0x26, 0xe3, 0x86, 0x2e, 0x67, 0xba, 0x74, 0xee, 0x3b, + 0x33, 0xf9, 0x54, 0x39, 0xad, 0xfe, 0x52, 0x81, 0x12, 0xe7, 0x85, 0xcb, 0xe9, 0x06, 0x9c, 0x97, + 0x98, 0xa1, 0x43, 0x75, 0x56, 0x8a, 0xe0, 0x9c, 0x9c, 0x8b, 0xd1, 0x74, 0xe4, 0x16, 0x45, 0x92, + 0x60, 0xc6, 0x0a, 0x58, 0x7f, 0x12, 0x64, 0x31, 0x59, 0x81, 0x15, 0xb4, 0x38, 0x04, 0xdd, 0x84, + 0x65, 0x46, 0xcd, 0xc7, 0x7d, 0x06, 0x92, 0x16, 0x99, 0xa6, 0x2e, 0xfe, 0xbc, 0xc9, 0x58, 0xe1, + 0xf8, 0x78, 0x8d, 0xdf, 0x84, 0x0b, 0xa3, 0x63, 0x45, 0x64, 0x4d, 0xa2, 0x3d, 0x16, 0x08, 0x2e, + 0x0f, 0x8f, 0x6e, 0xc5, 0x1d, 0xd4, 0x8f, 0x61, 0xb9, 0x85, 0xbb, 0x6e, 0x9f, 0x06, 0x1c, 0x2e, + 0x67, 0x59, 0x88, 0xff, 0x03, 0xc8, 0xf3, 0x98, 0x81, 0x2f, 0xb1, 0xd9, 0x3c, 0x3a, 0xac, 0xe5, + 0x58, 0xd0, 0x10, 0xfc, 0xcf, 0x61, 0xed, 0x5a, 0xcf, 0x0a, 0x77, 0x06, 0x9d, 0x7a, 0xd7, 0xed, + 0x37, 0x22, 0x4d, 0x35, 0x3b, 0xf1, 0x77, 0xc3, 0xdb, 0xed, 0x35, 0xe8, 0x97, 0xd7, 0xa9, 0xf3, + 0x60, 0x23, 0xc7, 0x82, 0x8d, 0x40, 0xfd, 0x37, 0x05, 0xce, 0xca, 0x93, 0xff, 0xff, 0x4c, 0x8b, + 0x3e, 0x84, 0x05, 0x6e, 0xf0, 0xfb, 0xb8, 0xdf, 0xc1, 0x7e, 0xb0, 0x63, 0x79, 0x74, 0x57, 0x86, + 0x53, 0xa0, 0xdd, 0xfd, 0xba, 0x28, 0x05, 0x89, 0x78, 0x90, 0x04, 0x82, 0xd1, 0x10, 0x2e, 0xaa, + 0x32, 0xa3, 0x15, 0xc3, 0xd5, 0x3f, 0x4d, 0x43, 0x75, 0x9c, 0x4c, 0xb9, 0x1a, 0x3d, 0x81, 0x2c, + 0xd7, 0x1a, 0x96, 0xd2, 0xde, 0x4a, 0x2a, 0x91, 0x3c, 0x93, 0x4c, 0x9d, 0x35, 0x85, 0xe3, 0x64, + 0x14, 0xab, 0x7f, 0x9e, 0x82, 0x2c, 0xd7, 0xba, 0x27, 0xc3, 0xf1, 0x5e, 0xa6, 0xb9, 0x1e, 0xc7, + 0x7b, 0xa7, 0x15, 0xa1, 0x08, 0x13, 0xcf, 0x43, 0xce, 0x0a, 0x74, 0xdb, 0xda, 0x8f, 0x72, 0x12, + 0x2b, 0xb8, 0x6b, 0xed, 0xe3, 0xe3, 0x61, 0x79, 0x7a, 0x4c, 0x58, 0xfe, 0x00, 0x40, 0x12, 0xfc, + 0xcc, 0x69, 0x05, 0x2f, 0x11, 0xa1, 0x89, 0x87, 0x38, 0x5f, 0x19, 0x76, 0xd2, 0x45, 0x5b, 0xd5, + 0x60, 0x7e, 0x0b, 0x87, 0xc4, 0x2a, 0x05, 0x09, 0xde, 0x67, 0x24, 0x04, 0xe2, 0xee, 0x26, 0x35, + 0x1a, 0x02, 0x31, 0xeb, 0xad, 0xfe, 0x4b, 0x8a, 0x7a, 0x65, 0x4e, 0x94, 0x6f, 0xac, 0x3e, 0xbd, + 0x57, 0x1e, 0x22, 0x70, 0xa2, 0xa3, 0xda, 0x1d, 0xeb, 0xa8, 0x0a, 0xc2, 0x51, 0x8d, 0x4b, 0x74, + 0x57, 0x61, 0x56, 0x36, 0x07, 0x69, 0x8a, 0x92, 0x41, 0x24, 0x69, 0xf4, 0x06, 0x1d, 0xdb, 0xea, + 0x72, 0x33, 0xc8, 0x5b, 0x55, 0x77, 0x02, 0x6f, 0x75, 0x7b, 0xd8, 0x5b, 0x5d, 0x9d, 0x66, 0xb1, + 0xc7, 0x9c, 0xd5, 0x8b, 0x50, 0x7a, 0x1b, 0x1b, 0x76, 0xb8, 0x23, 0x76, 0x69, 0x11, 0x32, 0x3e, + 0x36, 0x4c, 0x36, 0x63, 0x5e, 0x63, 0x0d, 0xe2, 0xd3, 0x44, 0x37, 0x1e, 0x5f, 0x2c, 0xc0, 0xfc, + 0x5d, 0x5e, 0xcf, 0x15, 0x6e, 0xee, 0x57, 0x29, 0x28, 0xc7, 0x30, 0xbe, 0x3f, 0xef, 0x02, 0x88, + 0xba, 0x6f, 0xb4, 0x3f, 0x2f, 0x9f, 0xa8, 0x77, 0x82, 0x8c, 0xa8, 0xa5, 0xc4, 0x24, 0xd0, 0x8f, + 0x15, 0xc8, 0xb3, 0x83, 0x87, 0xc5, 0x61, 0x4e, 0xca, 0xfc, 0x46, 0x19, 0xe2, 0x47, 0x58, 0x6c, + 0xf7, 0x9b, 0x84, 0xfe, 0x0f, 0x7f, 0x71, 0xba, 0x93, 0x19, 0xf1, 0x51, 0xf5, 0xa0, 0x34, 0x44, + 0x57, 0xde, 0xb6, 0x0c, 0xdb, 0xb6, 0xb6, 0xbc, 0x6d, 0x73, 0x6b, 0xd7, 0x4e, 0x94, 0x01, 0x99, + 0x44, 0x70, 0xcf, 0x4f, 0x9f, 0xb4, 0x71, 0x1e, 0xcc, 0xde, 0x71, 0x3b, 0x81, 0xb4, 0x6d, 0x2c, + 0xc3, 0x50, 0xe4, 0x0c, 0x63, 0x49, 0xb2, 0x7a, 0xb4, 0x36, 0xc1, 0x5a, 0xe8, 0x1a, 0x57, 0xda, + 0x34, 0x65, 0xa5, 0x26, 0x8b, 0x6f, 0xcf, 0xae, 0xd3, 0x5b, 0x00, 0x76, 0x23, 0x50, 0x27, 0xc1, + 0x3a, 0xd3, 0x6a, 0xf5, 0x9f, 0xb3, 0x50, 0x64, 0x53, 0xf2, 0xad, 0xdd, 0x84, 0x19, 0xd2, 0x8b, + 0x6f, 0xea, 0x2b, 0x09, 0x9b, 0x20, 0x0f, 0x23, 0x0d, 0xbe, 0xad, 0x74, 0x78, 0xf5, 0xbf, 0x33, + 0x90, 0xbe, 0xe3, 0x76, 0xd0, 0x12, 0xa4, 0xb8, 0xd9, 0x4c, 0x37, 0xb3, 0x47, 0x87, 0xb5, 0x54, + 0xbb, 0xa5, 0xa5, 0x2c, 0xf3, 0x94, 0x27, 0x4c, 0xce, 0xe8, 0x66, 0x86, 0x33, 0x3a, 0xe4, 0xc2, + 0xdc, 0x50, 0xbd, 0x8c, 0x55, 0x10, 0x4a, 0xcd, 0xb7, 0x8f, 0x0e, 0x6b, 0x25, 0xb9, 0x60, 0x36, + 0xb9, 0xdb, 0x0b, 0xf6, 0x6c, 0xf2, 0xd7, 0x31, 0x02, 0x5c, 0x6f, 0xb7, 0xb4, 0x92, 0x5c, 0x68, + 0x0b, 0xa4, 0x7d, 0xc8, 0x0e, 0xed, 0xc3, 0x4d, 0xc8, 0xb1, 0xda, 0xa3, 0x49, 0x4b, 0x3d, 0xc9, + 0x01, 0xee, 0x0c, 0x0d, 0x6e, 0xc5, 0x00, 0x32, 0x36, 0x08, 0x0d, 0x9f, 0x8c, 0xcd, 0x4f, 0x3a, + 0x96, 0x0f, 0x40, 0xb7, 0x20, 0xbf, 0x6d, 0x39, 0x56, 0xb0, 0x83, 0xcd, 0x4a, 0x61, 0xc2, 0xc1, + 0xd1, 0x08, 0x32, 0xba, 0xef, 0x9a, 0xd6, 0xb6, 0x85, 0xcd, 0x0a, 0x4c, 0x3a, 0x5a, 0x8c, 0x20, + 0xd1, 0xe9, 0xb6, 0x6f, 0xd0, 0x2a, 0x98, 0xde, 0x75, 0xfb, 0x9e, 0x8d, 0xc9, 0x12, 0x66, 0x57, + 0x95, 0xcb, 0x29, 0x6d, 0x41, 0x60, 0x36, 0x04, 0x82, 0x28, 0x36, 0xad, 0x83, 0x54, 0x8a, 0xcc, + 0xea, 0xd2, 0x06, 0x7a, 0x00, 0x67, 0x77, 0xac, 0xde, 0xce, 0x53, 0x83, 0xc4, 0xcf, 0x71, 0xae, + 0x5a, 0x9a, 0x90, 0x1b, 0x14, 0x0d, 0x8e, 0x30, 0xc4, 0x15, 0xc5, 0x24, 0x4d, 0xdc, 0xb5, 0xfa, + 0x86, 0x5d, 0x99, 0xa3, 0x93, 0x96, 0x23, 0x44, 0x8b, 0xc1, 0xd1, 0x8b, 0x30, 0xe7, 0x0f, 0x1c, + 0x1a, 0xf8, 0xf1, 0x8d, 0x9d, 0xa7, 0x3d, 0x4b, 0x1c, 0xca, 0xc3, 0x81, 0x8b, 0x50, 0x88, 0x8b, + 0xcd, 0x65, 0x96, 0x03, 0x47, 0x00, 0x15, 0x41, 0xf9, 0xae, 0xdb, 0x65, 0x77, 0x50, 0xc2, 0x86, + 0xfe, 0xaf, 0x02, 0x0b, 0x12, 0x30, 0x2a, 0xe5, 0x14, 0x6c, 0x01, 0x9c, 0xa0, 0xd2, 0x7f, 0x8c, + 0x40, 0x04, 0x11, 0xee, 0x2d, 0xa2, 0x56, 0xfd, 0x33, 0x05, 0xf2, 0x02, 0x8b, 0x5e, 0x80, 0x22, + 0xc1, 0xd8, 0x56, 0x78, 0xa0, 0xc7, 0x5e, 0x67, 0x56, 0xc0, 0xde, 0xc1, 0x07, 0x64, 0xe5, 0x51, + 0x97, 0xd8, 0x9e, 0x15, 0xb4, 0x92, 0x80, 0x32, 0x67, 0x59, 0x85, 0xbc, 0x6d, 0x84, 0x56, 0x38, + 0x30, 0x99, 0x95, 0x51, 0xb4, 0xa8, 0x4d, 0xa4, 0x62, 0xbb, 0x4e, 0x8f, 0x21, 0x67, 0x28, 0x32, + 0x06, 0xa8, 0x4d, 0x98, 0xd7, 0x0c, 0xa7, 0x87, 0xef, 0xba, 0x3d, 0x61, 0xdc, 0x96, 0x21, 0xcf, + 0x0a, 0x93, 0xc2, 0x3e, 0x68, 0x39, 0xda, 0x96, 0x2b, 0x2b, 0x29, 0xc9, 0xee, 0xa9, 0xff, 0x95, + 0x86, 0x72, 0x4c, 0x24, 0xf2, 0x44, 0xa2, 0x5c, 0xc2, 0xbc, 0x46, 0x92, 0xe3, 0x1c, 0x1d, 0x3c, + 0xb6, 0x60, 0xf2, 0x4f, 0x0a, 0xc0, 0x7b, 0x3e, 0x0e, 0xc3, 0x83, 0xb6, 0xb3, 0xed, 0x12, 0xe1, + 0xf1, 0x6c, 0x95, 0x26, 0x03, 0x42, 0x78, 0x1c, 0x46, 0xcc, 0x0a, 0x59, 0x88, 0x83, 0x9f, 0x32, + 0x34, 0x13, 0x5b, 0xce, 0xc1, 0x4f, 0x29, 0xea, 0x12, 0x94, 0x0c, 0xd3, 0xc4, 0xa6, 0xce, 0xa3, + 0x36, 0x6e, 0xd3, 0x8a, 0x14, 0xa8, 0x31, 0x18, 0x7a, 0x09, 0xe6, 0x7d, 0xdc, 0x77, 0xf7, 0xa5, + 0x6e, 0xcc, 0xb6, 0xcd, 0x71, 0xb0, 0xe8, 0xb8, 0x04, 0x59, 0x1f, 0x1b, 0x41, 0x54, 0x79, 0xe6, + 0x2d, 0x54, 0x81, 0x9c, 0xc9, 0xee, 0x54, 0xb8, 0x25, 0x12, 0xcd, 0xea, 0xdf, 0x2b, 0xa2, 0xfa, + 0x73, 0x07, 0x32, 0x74, 0x81, 0xbc, 0xf2, 0x53, 0x3f, 0xd1, 0x51, 0x09, 0x61, 0xc9, 0x32, 0x62, + 0x24, 0xd0, 0x07, 0x30, 0xeb, 0x51, 0x09, 0xe9, 0xb4, 0x1c, 0xc3, 0x22, 0x96, 0x1b, 0xd3, 0x08, + 0x3e, 0x16, 0xb0, 0x88, 0x05, 0xbc, 0x08, 0x72, 0x67, 0x26, 0xaf, 0x94, 0x53, 0xea, 0x65, 0x28, + 0x3f, 0x18, 0x60, 0xff, 0xe0, 0x3d, 0xdb, 0x70, 0x24, 0x7f, 0xb8, 0x47, 0x60, 0x22, 0x58, 0xa3, + 0x0d, 0xd5, 0x83, 0x05, 0xa9, 0x27, 0xd7, 0x8b, 0xef, 0xc0, 0x05, 0xd3, 0x0a, 0xc2, 0x60, 0xcf, + 0xd6, 0xbd, 0x9d, 0x83, 0xc0, 0xea, 0x1a, 0xb6, 0x4e, 0xbb, 0xeb, 0x9e, 0x6d, 0x38, 0x3c, 0x35, + 0xbf, 0x78, 0x74, 0x58, 0xab, 0xb4, 0xac, 0x20, 0xdc, 0x7a, 0x70, 0xf7, 0x3d, 0xde, 0x2b, 0x26, + 0x55, 0xe1, 0x04, 0x8e, 0x61, 0xd4, 0x65, 0x76, 0x6b, 0x4b, 0x46, 0xfa, 0x56, 0x67, 0x10, 0xc6, + 0x09, 0x97, 0xfa, 0x77, 0x00, 0x95, 0xe3, 0x38, 0xce, 0x94, 0x07, 0x25, 0x51, 0xc4, 0x67, 0xa2, + 0x63, 0xa7, 0x7e, 0xf3, 0x84, 0x9b, 0xdd, 0x71, 0xb4, 0xa2, 0x2b, 0x5f, 0x22, 0x32, 0x39, 0xc8, + 0x2d, 0x9a, 0x12, 0x02, 0xf5, 0xa1, 0x28, 0xdd, 0x51, 0x89, 0x2b, 0xba, 0xd6, 0x69, 0x26, 0x8c, + 0xef, 0xad, 0x86, 0x82, 0xea, 0xd9, 0xf8, 0xde, 0x2a, 0xa8, 0x7e, 0xa6, 0x00, 0xc4, 0xfd, 0x88, + 0xc2, 0xb2, 0x94, 0x8e, 0x6f, 0x18, 0x6f, 0xa1, 0x75, 0xc8, 0xf2, 0x0b, 0xb8, 0xd4, 0xb4, 0x17, + 0x70, 0x7c, 0x20, 0xad, 0xb4, 0xb0, 0x7b, 0xb7, 0x60, 0xcf, 0x66, 0xe7, 0x85, 0x57, 0x5a, 0x28, + 0x74, 0xeb, 0xc1, 0x5d, 0xad, 0xc0, 0x3a, 0x6c, 0xed, 0xd9, 0x77, 0x66, 0xf2, 0xe9, 0xf2, 0x4c, + 0xf5, 0x8f, 0xd2, 0x50, 0xa0, 0xc5, 0x6a, 0x2a, 0x9a, 0x2f, 0x14, 0xa8, 0x0c, 0x65, 0x58, 0x7a, + 0xe7, 0x40, 0x8f, 0xf3, 0x3c, 0x22, 0xa7, 0xf7, 0x4f, 0x23, 0xa7, 0x68, 0x86, 0xba, 0x26, 0xa5, + 0x6a, 0xcd, 0x03, 0x1a, 0x5e, 0x9a, 0x4c, 0x74, 0x6f, 0x9c, 0x36, 0x38, 0x5d, 0xf4, 0xc7, 0xd0, + 0x44, 0x5f, 0x85, 0x39, 0xf9, 0x26, 0x32, 0xaa, 0x55, 0x17, 0xe3, 0xbd, 0x69, 0x9b, 0xe8, 0x5b, + 0x00, 0xa6, 0xef, 0x7a, 0x1e, 0x36, 0x75, 0x83, 0xa5, 0x93, 0x93, 0x78, 0xd5, 0x02, 0x1f, 0xb3, + 0x1e, 0x56, 0x6f, 0xc3, 0xf2, 0x33, 0x97, 0x34, 0x26, 0x36, 0x1e, 0xaa, 0x6d, 0xa6, 0xa5, 0x30, + 0xb7, 0xfa, 0x83, 0x14, 0x14, 0x65, 0xfd, 0x45, 0x21, 0xb0, 0x07, 0x08, 0xf2, 0xa9, 0x78, 0xf7, + 0x79, 0x4f, 0x45, 0xbc, 0x13, 0x43, 0x49, 0x60, 0x28, 0xa0, 0xd5, 0x4f, 0x60, 0x6e, 0xb8, 0xcb, + 0x98, 0xbc, 0x6c, 0x6b, 0x38, 0x2f, 0x7b, 0xeb, 0xb9, 0x34, 0x62, 0x48, 0x06, 0x4a, 0xfc, 0xc8, + 0x24, 0x89, 0x81, 0xc7, 0xc3, 0x0c, 0xfc, 0xf6, 0xf3, 0x4a, 0x45, 0xe6, 0xe1, 0xfb, 0x50, 0x1e, + 0x3d, 0xd5, 0x63, 0x38, 0x78, 0x38, 0xcc, 0xc1, 0x37, 0x9f, 0xcf, 0x78, 0x48, 0xf3, 0xf3, 0xf2, + 0xe0, 0x79, 0x38, 0x77, 0x8f, 0x3e, 0xf3, 0xb9, 0x87, 0x43, 0xc3, 0x8c, 0x2b, 0xdb, 0xea, 0xbf, + 0x2b, 0xb0, 0x34, 0x8a, 0xe1, 0x96, 0xd4, 0x80, 0x7c, 0x9f, 0xc3, 0xb8, 0xba, 0x7c, 0x2b, 0x81, + 0xad, 0xf1, 0x44, 0xea, 0x02, 0x20, 0xab, 0x47, 0x44, 0xb6, 0xfa, 0x3b, 0x50, 0x1a, 0xea, 0x30, + 0x46, 0x32, 0xaf, 0x0f, 0x4b, 0x46, 0x4e, 0xb9, 0x06, 0xa1, 0x65, 0xd7, 0xf9, 0xeb, 0xa5, 0x68, + 0x62, 0x29, 0xd3, 0xfb, 0x49, 0x0a, 0xce, 0x6e, 0x3a, 0x7b, 0x03, 0x3c, 0xc0, 0xd4, 0x3b, 0x0a, + 0x17, 0xf7, 0x9b, 0xac, 0x35, 0x31, 0xf7, 0x19, 0x05, 0x77, 0xac, 0x81, 0x3e, 0x94, 0xe2, 0x30, + 0x7a, 0x93, 0xd5, 0xdc, 0x38, 0x3a, 0xac, 0xe5, 0x28, 0x57, 0x74, 0xce, 0xeb, 0x53, 0xcd, 0xc9, + 0xc7, 0xc5, 0xc1, 0xdc, 0x15, 0x58, 0x08, 0x76, 0x2d, 0x4f, 0x0f, 0x76, 0xdc, 0x81, 0x6d, 0xea, + 0x8c, 0x03, 0x56, 0x20, 0x99, 0x27, 0x88, 0x2d, 0x0a, 0x7f, 0x40, 0xc0, 0xea, 0x5f, 0xa7, 0x60, + 0x71, 0x58, 0x2a, 0x7c, 0xbf, 0x1f, 0xc4, 0x21, 0x0e, 0xdb, 0xee, 0x37, 0x92, 0xae, 0xc5, 0xc6, + 0x50, 0xa8, 0x8b, 0x57, 0x27, 0x51, 0x6c, 0xf4, 0x0f, 0x0a, 0xe4, 0x38, 0xf0, 0x37, 0x2a, 0xf5, + 0xb7, 0x46, 0x22, 0xd4, 0x17, 0x93, 0xae, 0x52, 0x7d, 0xa3, 0x8b, 0x69, 0xc4, 0x25, 0xe2, 0xd1, + 0x38, 0x55, 0x4a, 0x4b, 0xa9, 0x92, 0x7a, 0x0e, 0xce, 0x6e, 0xec, 0x18, 0x7e, 0xb8, 0xc1, 0x9e, + 0xce, 0x89, 0x13, 0xf3, 0x04, 0x16, 0x87, 0xc1, 0x5c, 0x7c, 0x4d, 0xc8, 0xf1, 0x47, 0x76, 0x5c, + 0x7c, 0xaa, 0xc4, 0x44, 0x18, 0xd4, 0xc5, 0x0b, 0x3c, 0x3a, 0x78, 0x8b, 0xbd, 0x66, 0x88, 0x1e, + 0xfa, 0x30, 0xdc, 0x95, 0x77, 0x60, 0x69, 0xfc, 0xd3, 0x15, 0x34, 0x0b, 0xb9, 0x47, 0xf7, 0xdf, + 0xb9, 0xff, 0xee, 0xfb, 0xf7, 0xcb, 0x67, 0x48, 0x63, 0xe3, 0xee, 0xa3, 0xad, 0x87, 0x9b, 0x5a, + 0x59, 0x41, 0x45, 0xc8, 0xb7, 0xd6, 0x1f, 0xae, 0x37, 0xd7, 0xb7, 0x36, 0xcb, 0x29, 0x54, 0x80, + 0xcc, 0xc3, 0xf5, 0xe6, 0xdd, 0xcd, 0x72, 0x7a, 0xed, 0xc7, 0x55, 0xc8, 0xac, 0x9b, 0x7d, 0xcb, + 0x41, 0x21, 0x64, 0xe8, 0x7d, 0x2e, 0x7a, 0xe9, 0xe4, 0x1b, 0x5f, 0xba, 0xc8, 0xea, 0xe5, 0x49, + 0xaf, 0x86, 0xd5, 0xca, 0x0f, 0xff, 0xf5, 0x97, 0x3f, 0x49, 0x21, 0x54, 0x6e, 0xe8, 0xf4, 0x49, + 0x65, 0x63, 0xff, 0x6a, 0x83, 0x5e, 0x11, 0xa3, 0x3f, 0x50, 0xa0, 0x10, 0x3d, 0xf1, 0x43, 0xaf, + 0x4c, 0xf0, 0xb4, 0x2e, 0x9a, 0xfe, 0xd5, 0xc9, 0x3a, 0x73, 0x16, 0x2e, 0x52, 0x16, 0x96, 0xd0, + 0xa2, 0xc4, 0x42, 0xf4, 0x6a, 0x10, 0xfd, 0xa5, 0x02, 0xf3, 0x23, 0x6f, 0xf7, 0xd0, 0xd5, 0x69, + 0xde, 0xf9, 0x31, 0x96, 0xd6, 0xa6, 0x7f, 0x1a, 0xa8, 0xbe, 0x44, 0x19, 0x7b, 0x01, 0xd5, 0xc6, + 0x31, 0xd6, 0xf8, 0x44, 0x7c, 0x7e, 0x0f, 0xfd, 0x8d, 0x02, 0x45, 0xf9, 0xdd, 0x16, 0xaa, 0x4f, + 0xfc, 0xc0, 0x8b, 0x71, 0xd7, 0x98, 0xf2, 0x41, 0x98, 0x7a, 0x83, 0xb2, 0xf6, 0x1a, 0xaa, 0x9f, + 0xc0, 0x5a, 0x83, 0xba, 0xf2, 0xa0, 0xf1, 0x09, 0xfd, 0xa5, 0x9c, 0x42, 0xfc, 0x18, 0x01, 0xbd, + 0x3a, 0xe1, 0x9b, 0x05, 0xc6, 0xe5, 0x74, 0x2f, 0x1c, 0xd4, 0x5b, 0x94, 0xc7, 0x1b, 0xe8, 0xfa, + 0x74, 0x3c, 0x36, 0xd8, 0x63, 0x97, 0xbf, 0x50, 0xa0, 0x34, 0xf4, 0xbe, 0x03, 0x25, 0x09, 0x69, + 0xdc, 0x13, 0x91, 0xea, 0x6b, 0x93, 0x0f, 0xe0, 0x2c, 0xaf, 0x52, 0x96, 0xab, 0xa8, 0x22, 0xb1, + 0xec, 0xb8, 0x0e, 0x63, 0x90, 0x32, 0xf1, 0x11, 0x64, 0xd9, 0x93, 0x02, 0x74, 0x79, 0x82, 0x57, + 0x07, 0x8c, 0x8f, 0x97, 0x27, 0x7e, 0x9f, 0xa0, 0x2e, 0x53, 0x06, 0xce, 0xa2, 0x05, 0x89, 0x01, + 0x6e, 0xe5, 0xc8, 0x79, 0x8c, 0xae, 0xbb, 0x13, 0xcf, 0xe3, 0xe8, 0x05, 0x7c, 0xe2, 0x79, 0x3c, + 0x7e, 0x83, 0xce, 0xcf, 0xa3, 0x2a, 0xf3, 0x30, 0xb0, 0xc8, 0x76, 0xdd, 0x54, 0xae, 0xa0, 0x1f, + 0x28, 0x50, 0xb8, 0x3d, 0x11, 0x1b, 0xb7, 0xa7, 0x61, 0xe3, 0xd8, 0xfd, 0xf2, 0x58, 0x51, 0x30, + 0x36, 0xd0, 0xef, 0x41, 0x8e, 0xdf, 0x16, 0xa3, 0x24, 0xd9, 0x0e, 0xdf, 0x46, 0x57, 0xaf, 0x4c, + 0xd2, 0x95, 0x4f, 0x5e, 0xa5, 0x93, 0x2f, 0x22, 0x24, 0x4d, 0xce, 0x6f, 0xa5, 0xd1, 0xef, 0x2b, + 0x90, 0x17, 0x17, 0x0c, 0xe8, 0xca, 0x44, 0xb7, 0x10, 0x8c, 0x81, 0x57, 0xa6, 0xb8, 0xb1, 0x50, + 0x2f, 0x50, 0x0e, 0xce, 0xa1, 0xb3, 0x12, 0x07, 0x81, 0x98, 0xf5, 0x47, 0x0a, 0x64, 0xd9, 0xbd, + 0x44, 0xa2, 0x1a, 0x0e, 0xdd, 0x70, 0x24, 0xaa, 0xe1, 0xc8, 0x25, 0xc7, 0x57, 0xe9, 0xe4, 0x2b, + 0x43, 0xb2, 0xdf, 0xa1, 0x5d, 0x9e, 0x14, 0x50, 0x8e, 0x7f, 0x52, 0x41, 0x88, 0x42, 0x7d, 0xa2, + 0x20, 0x46, 0x2e, 0x4c, 0x12, 0x05, 0x31, 0x7a, 0x6f, 0x31, 0x56, 0x10, 0xe2, 0x5a, 0x04, 0x79, + 0x30, 0x73, 0xc7, 0xed, 0x04, 0xe8, 0x6b, 0x27, 0x16, 0xe1, 0xd9, 0xcc, 0x2f, 0x4d, 0x58, 0xac, + 0x57, 0xcf, 0xd3, 0x59, 0x17, 0xd0, 0xbc, 0x34, 0xeb, 0x77, 0xc9, 0x4c, 0xe4, 0x18, 0x46, 0x75, + 0xc6, 0x44, 0xfd, 0x1f, 0xad, 0x71, 0x26, 0xea, 0xff, 0xb1, 0xd2, 0xe5, 0x58, 0xb7, 0x18, 0x95, + 0x2f, 0x29, 0x1b, 0x51, 0xb5, 0x25, 0x91, 0x8d, 0xd1, 0x12, 0x51, 0x22, 0x1b, 0xc7, 0xaa, 0x44, + 0x63, 0xd9, 0xa0, 0x55, 0x22, 0x8f, 0x4c, 0xfc, 0x21, 0x64, 0xe8, 0xc3, 0x82, 0xc4, 0xd0, 0x44, + 0x7e, 0x64, 0x91, 0x18, 0x9a, 0x0c, 0xbd, 0x80, 0x50, 0xcf, 0xbc, 0xa6, 0xa0, 0xa7, 0x50, 0x94, + 0x6f, 0xa5, 0x13, 0x1d, 0xeb, 0x98, 0xcb, 0xfd, 0xea, 0xeb, 0xa7, 0xba, 0xee, 0x56, 0xcf, 0x90, + 0x13, 0x86, 0x8e, 0x77, 0x40, 0xd7, 0xa7, 0xa4, 0xf7, 0x9c, 0x5c, 0xfc, 0x95, 0x02, 0x79, 0x51, + 0x19, 0x4c, 0x3c, 0x61, 0x23, 0x95, 0xe3, 0xc4, 0x13, 0x36, 0x5a, 0x6a, 0x54, 0xdf, 0xa2, 0x5b, + 0xfc, 0xc6, 0xd0, 0x09, 0xa3, 0xa9, 0x89, 0xed, 0xf6, 0x9e, 0xac, 0xa2, 0x95, 0x31, 0xe0, 0xc6, + 0x27, 0x22, 0x21, 0xfa, 0x1e, 0x89, 0xd0, 0xca, 0xa3, 0x59, 0x2d, 0x5a, 0x9b, 0x2a, 0x05, 0x66, + 0x4c, 0x5f, 0x3b, 0x45, 0xda, 0x1c, 0x99, 0xaa, 0x8b, 0x23, 0x51, 0x86, 0x6e, 0xca, 0xec, 0x7c, + 0xa6, 0xc0, 0xc2, 0xba, 0x6d, 0x0f, 0x67, 0xb9, 0xe8, 0xb5, 0x29, 0x12, 0x62, 0xc6, 0xe2, 0xd5, + 0xa9, 0x53, 0x68, 0xf5, 0x05, 0xca, 0xe0, 0x05, 0xb4, 0x2c, 0x31, 0xc8, 0xb2, 0x5e, 0x91, 0x47, + 0xa3, 0x4f, 0x15, 0x28, 0xca, 0x49, 0x49, 0xa2, 0x9a, 0x8f, 0x49, 0x6a, 0x12, 0xe3, 0xc7, 0x71, + 0xd9, 0x8e, 0x5a, 0xa3, 0x4c, 0x2d, 0xa3, 0xf3, 0xb2, 0x7f, 0x23, 0x1d, 0x79, 0x2a, 0x43, 0xc2, + 0xaf, 0xa2, 0x9c, 0x24, 0x26, 0xb2, 0x34, 0x26, 0x4b, 0x4f, 0x64, 0x69, 0x5c, 0xf6, 0xa9, 0x5e, + 0xa2, 0x2c, 0x7d, 0x45, 0x95, 0x63, 0x2f, 0xcc, 0x3a, 0xea, 0x54, 0xd9, 0x6e, 0x2a, 0x57, 0x9a, + 0x57, 0xbe, 0xf8, 0xcf, 0x95, 0x33, 0x5f, 0x1c, 0xad, 0x28, 0x3f, 0x3b, 0x5a, 0x51, 0x7e, 0x7e, + 0xb4, 0xa2, 0xfc, 0xc7, 0xd1, 0x8a, 0xf2, 0xe9, 0x97, 0x2b, 0x67, 0x7e, 0xf6, 0xe5, 0xca, 0x99, + 0x9f, 0x7f, 0xb9, 0x72, 0xe6, 0x49, 0x5e, 0xcc, 0xd2, 0xc9, 0xd2, 0x82, 0xdd, 0xb5, 0xff, 0x0b, + 0x00, 0x00, 0xff, 0xff, 0x6e, 0xf0, 0x63, 0x09, 0x3c, 0x36, 0x00, 0x00, } diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index 2d4491386334..06102b5ec25c 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -439,14 +439,14 @@ message DecommissionStatusRequest { (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; } -// DecommissionRequest requests the server to set the Decommissioning flag on -// all nodes specified by 'node_id' to the value of 'decommissioning'. +// DecommissionRequest requests the server to set the membership status on +// all nodes specified by NodeIDs to the value of TargetMembership. // -// If no 'node_id' is given, it targets the recipient node. +// If no NodeIDs are given, it targets the recipient node. message DecommissionRequest { repeated int32 node_ids = 1 [(gogoproto.customname) = "NodeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; - bool decommissioning = 2; + cockroach.kv.kvserver.storagepb.MembershipStatus target_membership = 2; } // DecommissionStatusResponse lists decommissioning statuses for a number of NodeIDs. @@ -457,7 +457,8 @@ message DecommissionStatusResponse { bool is_live = 2; // The number of replicas on the node, computed by scanning meta2 ranges. int64 replica_count = 3; - bool decommissioning = 4; + // The membership status of the given node. + cockroach.kv.kvserver.storagepb.MembershipStatus membership = 4; bool draining = 5; } // Status of all affected nodes. @@ -669,7 +670,7 @@ message DataDistributionResponse { map zone_configs = 3 [(gogoproto.nullable) = false]; } - // MetricMetadataRequest requests metadata for all metrics. +// MetricMetadataRequest requests metadata for all metrics. message MetricMetadataRequest { } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 1c686202e363..922b5443b4a5 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2735,16 +2735,21 @@ CREATE TABLE crdb_internal.gossip_nodes ( // crdbInternalGossipLivenessTable exposes local information about the nodes' // liveness. The data exposed in this table can be stale/incomplete because // gossip doesn't provide guarantees around freshness or consistency. +// +// TODO(irfansharif): Remove this decommissioning field in v21.1. It's retained +// for compatibility with v20.1 binaries where the `cockroach node` cli +// processes make use of it. var crdbInternalGossipLivenessTable = virtualSchemaTable{ comment: "locally known gossiped node liveness (RAM; local node only)", schema: ` CREATE TABLE crdb_internal.gossip_liveness ( - node_id INT NOT NULL, - epoch INT NOT NULL, - expiration STRING NOT NULL, - draining BOOL NOT NULL, - decommissioning BOOL NOT NULL, - updated_at TIMESTAMP + node_id INT NOT NULL, + epoch INT NOT NULL, + expiration STRING NOT NULL, + draining BOOL NOT NULL, + decommissioning BOOL NOT NULL, + membership STRING NOT NULL, + updated_at TIMESTAMP ) `, populate: func(ctx context.Context, p *planner, _ *sqlbase.ImmutableDatabaseDescriptor, addRow func(...tree.Datum) error) error { @@ -2804,7 +2809,8 @@ CREATE TABLE crdb_internal.gossip_liveness ( tree.NewDInt(tree.DInt(l.Epoch)), tree.NewDString(l.Expiration.String()), tree.MakeDBool(tree.DBool(l.Draining)), - tree.MakeDBool(tree.DBool(l.Decommissioning)), + tree.MakeDBool(tree.DBool(!l.Membership.Active())), + tree.NewDString(l.Membership.String()), updatedTSDatum, ); err != nil { return err diff --git a/pkg/sql/event_log.go b/pkg/sql/event_log.go index 7e8a73beca93..5b0bf980ea84 100644 --- a/pkg/sql/event_log.go +++ b/pkg/sql/event_log.go @@ -81,10 +81,14 @@ const ( // EventLogNodeRestart is recorded when an existing node rejoins the cluster // after being offline. EventLogNodeRestart EventLogType = "node_restart" + // EventLogNodeDecommissioned is recorded when a node is marked as // decommissioning. + EventLogNodeDecommissioning EventLogType = "node_decommissioning" + // EventLogNodeDecommissioned is recorded when a node is marked as + // decommissioned. EventLogNodeDecommissioned EventLogType = "node_decommissioned" - // EventLogNodeRecommissioned is recorded when a decommissioned node is + // EventLogNodeRecommissioned is recorded when a decommissioning node is // recommissioned. EventLogNodeRecommissioned EventLogType = "node_recommissioned" diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 84503673614c..7037b526c5b8 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -321,11 +321,11 @@ SELECT node_id, network, regexp_replace(address, '\d+$', '') as address, a node_id network address attrs locality server_version 1 tcp 127.0.0.1: [] region=test,dc=dc1 -query IITBB colnames -SELECT node_id, epoch, regexp_replace(expiration, '^\d+\.\d+,\d+$', '') as expiration, draining, decommissioning FROM crdb_internal.gossip_liveness WHERE node_id = 1 +query IITBBT colnames +SELECT node_id, epoch, regexp_replace(expiration, '^\d+\.\d+,\d+$', '') as expiration, draining, decommissioning, membership FROM crdb_internal.gossip_liveness WHERE node_id = 1 ---- -node_id epoch expiration draining decommissioning -1 1 false false +node_id epoch expiration draining decommissioning membership +1 1 false false active query ITTTTTT colnames SELECT node_id, network, regexp_replace(address, '\d+$', '') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-\d+)?$', '') as server_version, regexp_replace(go_version, '^go.+$', '') as go_version diff --git a/pkg/ui/src/redux/alerts.spec.ts b/pkg/ui/src/redux/alerts.spec.ts index 325945b59289..912b24a66119 100644 --- a/pkg/ui/src/redux/alerts.spec.ts +++ b/pkg/ui/src/redux/alerts.spec.ts @@ -15,6 +15,7 @@ import sinon from "sinon"; import { createHashHistory } from "history"; import * as protos from "src/js/protos"; +import { cockroach } from "src/js/protos"; import { API_PREFIX } from "src/util/api"; import fetchMock from "src/util/fetch-mock"; @@ -39,6 +40,7 @@ import { import { livenessReducerObj, versionReducerObj, nodesReducerObj, clusterReducerObj, healthReducerObj, } from "./apiReducers"; +import MembershipStatus = cockroach.kv.kvserver.storagepb.MembershipStatus; const sandbox = sinon.createSandbox(); @@ -80,9 +82,12 @@ describe("alerts", function() { assert.deepEqual(versions, ["0.1", "0.2"]); }); - it("ignores decommissioned nodes", function () { + it("ignores decommissioning/decommissioned nodes", function () { dispatch(nodesReducerObj.receiveData([ { + desc: { + node_id: 1, + }, build_info: { tag: "0.1", }, @@ -95,14 +100,32 @@ describe("alerts", function() { tag: "0.2", }, }, + { + desc: { + node_id: 3, + }, + build_info: { + tag: "0.3", + }, + }, ])); dispatch(livenessReducerObj.receiveData( new protos.cockroach.server.serverpb.LivenessResponse({ - livenesses: [{ - node_id: 2, - decommissioning: true, - }], + livenesses: [ + { + node_id: 1, + membership: MembershipStatus.ACTIVE, + }, + { + node_id: 2, + membership: MembershipStatus.DECOMMISSIONING, + }, + { + node_id: 3, + membership: MembershipStatus.DECOMMISSIONED, + }, + ], }), )); diff --git a/pkg/ui/src/redux/nodes.ts b/pkg/ui/src/redux/nodes.ts index 98b704261ff7..c42bea888b15 100644 --- a/pkg/ui/src/redux/nodes.ts +++ b/pkg/ui/src/redux/nodes.ts @@ -24,6 +24,8 @@ import { nullOfReturnType } from "src/util/types"; * keyword. */ export import LivenessStatus = protos.cockroach.kv.kvserver.storagepb.NodeLivenessStatus; +import { cockroach } from "src/js/protos"; +import MembershipStatus = cockroach.kv.kvserver.storagepb.MembershipStatus; /** * livenessNomenclature resolves a mismatch between the terms used for liveness @@ -357,7 +359,8 @@ export const versionsSelector = createSelector( // Exclude this node if it's known to be decommissioning. .filter((status) => !status.desc || !livenessStatusByNodeID[status.desc.node_id] || - !livenessStatusByNodeID[status.desc.node_id].decommissioning) + !livenessStatusByNodeID[status.desc.node_id].membership || + !(livenessStatusByNodeID[status.desc.node_id].membership !== MembershipStatus.ACTIVE)) // Collect the surviving nodes' build tags. .map((status) => status.build_info.tag) .uniq() diff --git a/pkg/ui/src/util/eventTypes.ts b/pkg/ui/src/util/eventTypes.ts index 0f185e11a70e..501304b2f548 100644 --- a/pkg/ui/src/util/eventTypes.ts +++ b/pkg/ui/src/util/eventTypes.ts @@ -55,9 +55,11 @@ export const FINISH_SCHEMA_CHANGE_ROLLBACK = "finish_schema_change_rollback"; export const NODE_JOIN = "node_join"; // Recorded when an existing node rejoins the cluster after being offline. export const NODE_RESTART = "node_restart"; -// Recorded when a node is marked as decommissioning. +// Recorded when a node is marked for decommissioning. +export const NODE_DECOMMISSIONING = "node_decommissioning"; +// Recorded when a node is marked as decommissioned. export const NODE_DECOMMISSIONED = "node_decommissioned"; -// Recorded when a decommissioned node is recommissioned. +// Recorded when a decommissioning node is recommissioned. export const NODE_RECOMMISSIONED = "node_recommissioned"; // Recorded when a cluster setting is changed. export const SET_CLUSTER_SETTING = "set_cluster_setting"; @@ -69,7 +71,7 @@ export const REMOVE_ZONE_CONFIG = "remove_zone_config"; export const CREATE_STATISTICS = "create_statistics"; // Node Event Types -export const nodeEvents = [NODE_JOIN, NODE_RESTART, NODE_DECOMMISSIONED, NODE_RECOMMISSIONED]; +export const nodeEvents = [NODE_JOIN, NODE_RESTART, NODE_DECOMMISSIONING, NODE_DECOMMISSIONED, NODE_RECOMMISSIONED]; export const databaseEvents = [CREATE_DATABASE, DROP_DATABASE]; export const tableEvents = [ CREATE_TABLE, DROP_TABLE, TRUNCATE_TABLE, ALTER_TABLE, CREATE_INDEX, diff --git a/pkg/ui/src/util/events.ts b/pkg/ui/src/util/events.ts index a3f372227628..02135f2183c8 100644 --- a/pkg/ui/src/util/events.ts +++ b/pkg/ui/src/util/events.ts @@ -60,6 +60,8 @@ export function getEventDescription(e: Event$Properties): string { return `Schema Change Rollback Completed: Rollback of schema change with ID ${info.MutationID} was completed.`; case eventTypes.NODE_JOIN: return `Node Joined: Node ${targetId} joined the cluster`; + case eventTypes.NODE_DECOMMISSIONING: + return `Node Decommissioning: Node ${targetId} was marked as decommissioning`; case eventTypes.NODE_DECOMMISSIONED: return `Node Decommissioned: Node ${targetId} was decommissioned`; case eventTypes.NODE_RECOMMISSIONED: From 68264941a2984f09f0be97c89fad829a637ec62c Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Sun, 5 Jul 2020 23:58:16 -0400 Subject: [PATCH 3/5] roachtest: add `decommission/mixed-versions` Add a roachtest stressing randomized `cockroach node {decommission,recommission}` usage in multi-version clusters. Release note: None --- .../roachtest/mixed_version_decommission.go | 288 ++++++++++++++++++ pkg/cmd/roachtest/registry.go | 1 + 2 files changed, 289 insertions(+) create mode 100644 pkg/cmd/roachtest/mixed_version_decommission.go diff --git a/pkg/cmd/roachtest/mixed_version_decommission.go b/pkg/cmd/roachtest/mixed_version_decommission.go new file mode 100644 index 000000000000..65f317fc01c7 --- /dev/null +++ b/pkg/cmd/roachtest/mixed_version_decommission.go @@ -0,0 +1,288 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package main + +import ( + "context" + "math/rand" + "strconv" + + "github.com/cockroachdb/cockroach/pkg/util/version" +) + +// runDecommissionMixedVersions runs through randomized +// decommission/recommission processes in mixed-version clusters. +func runDecommissionMixedVersions( + ctx context.Context, t *test, c *cluster, buildVersion version.Version, +) { + predecessorVersion, err := PredecessorVersion(buildVersion) + if err != nil { + t.Fatal(err) + } + + var nodeIDs []int + for i := 1; i <= c.spec.NodeCount; i++ { + nodeIDs = append(nodeIDs, i) + } + getRandNode := func() int { + return nodeIDs[rand.Intn(len(nodeIDs))] + } + + // The v20.2 CLI can only be run against servers running v20.2. For this + // reason, we grab a handle on a specific server slated for an upgrade. + pinnedUpgrade := getRandNode() + t.l.Printf("pinned n%d for upgrade", pinnedUpgrade) + + // An empty string means that the cockroach binary specified by flag + // `cockroach` will be used. + const mainVersion = "" + allNodes := c.All() + u := newVersionUpgradeTest(c, + // We upload both binaries to each node, to be able to vary the binary + // used when issuing `cockroach node` subcommands. + uploadVersion(allNodes, predecessorVersion), + uploadVersion(allNodes, mainVersion), + + startVersion(allNodes, predecessorVersion), + waitForUpgradeStep(allNodes), + preventAutoUpgradeStep(nodeIDs[0]), + + // We upgrade a subset of the cluster to v20.2 (these may end up + // resolving to the same node). + binaryUpgradeStep(c.Node(pinnedUpgrade), mainVersion), + binaryUpgradeStep(c.Node(getRandNode()), mainVersion), + checkAllMembership(pinnedUpgrade, "active"), + + // 1. Partially decommission a random node from another random node. We + // use the v20.1 CLI to do so. + partialDecommissionStep(getRandNode(), getRandNode(), predecessorVersion), + checkOneDecommissioning(getRandNode()), + checkOneMembership(pinnedUpgrade, "decommissioning"), + + // 2. Recommission all nodes, including the partially decommissioned + // one, from a random node. Use the v20.1 CLI to do so. + recommissionAllStep(getRandNode(), predecessorVersion), + checkNoDecommissioning(getRandNode()), + checkAllMembership(pinnedUpgrade, "active"), + // + // 3. Attempt to fully decommission a from a random node, again using + // the v20.1 CLI. + fullyDecommissionStep(getRandNode(), getRandNode(), predecessorVersion), + checkOneDecommissioning(getRandNode()), + checkOneMembership(pinnedUpgrade, "decommissioning"), + + // Roll back, which should to be fine because the cluster upgrade was + // not finalized. + binaryUpgradeStep(allNodes, predecessorVersion), + checkOneDecommissioning(getRandNode()), + + // Repeat similar recommission/decommission cycles as above. We can no + // longer assert against the `membership` column as none of the servers + // are running v20.2. + recommissionAllStep(getRandNode(), predecessorVersion), + checkNoDecommissioning(getRandNode()), + + partialDecommissionStep(getRandNode(), getRandNode(), predecessorVersion), + checkOneDecommissioning(getRandNode()), + + // Roll all nodes forward, and finalize upgrade. + binaryUpgradeStep(allNodes, mainVersion), + allowAutoUpgradeStep(1), + waitForUpgradeStep(allNodes), + + checkOneMembership(getRandNode(), "decommissioning"), + + // Use the v20.2 CLI here on forth. Lets start with recommissioning all + // the nodes in the cluster. + recommissionAllStep(getRandNode(), mainVersion), + checkNoDecommissioning(getRandNode()), + checkAllMembership(getRandNode(), "active"), + + // We partially decommission a random node. + partialDecommissionStep(getRandNode(), getRandNode(), mainVersion), + checkOneDecommissioning(getRandNode()), + checkOneMembership(getRandNode(), "decommissioning"), + + // We check that recommissioning is still functional. + recommissionAllStep(getRandNode(), mainVersion), + checkNoDecommissioning(getRandNode()), + checkAllMembership(getRandNode(), "active"), + + // We fully decommission a random node. We need to use the v20.2 CLI to + // do so. + fullyDecommissionStep(getRandNode(), getRandNode(), mainVersion), + checkOneDecommissioning(getRandNode()), + checkOneMembership(getRandNode(), "decommissioned"), + ) + + u.run(ctx, t) +} + +// cockroachBinaryPath is a shorthand to retrieve the path for a cockroach +// binary of a given version. +func cockroachBinaryPath(version string) string { + path := "./cockroach" + if version != "" { + path += "-" + version + } + return path +} + +// partialDecommissionStep runs `cockroach node decommission --wait=none` from a +// given node, targeting another. It uses the specified binary version to run +// the command. +func partialDecommissionStep(target, from int, binaryVersion string) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + c := u.c + c.Run(ctx, c.Node(from), cockroachBinaryPath(binaryVersion), "node", "decommission", + "--wait=none", "--insecure", strconv.Itoa(target)) + } +} + +// recommissionAllStep runs `cockroach node recommission` from a given node, +// targeting all nodes in the cluster. It uses the specified binary version to +// run the command. +func recommissionAllStep(from int, binaryVersion string) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + c := u.c + c.Run(ctx, c.Node(from), cockroachBinaryPath(binaryVersion), "node", "recommission", + "--insecure", c.All().nodeIDsString()) + } +} + +// fullyDecommissionStep is like partialDecommissionStep, except it uses +// `--wait=all`. +func fullyDecommissionStep(target, from int, binaryVersion string) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + c := u.c + c.Run(ctx, c.Node(from), cockroachBinaryPath(binaryVersion), "node", "decommission", + "--wait=all", "--insecure", strconv.Itoa(target)) + } +} + +// checkOneDecommissioning checks against the `decommissioning` column in +// crdb_internal.gossip_liveness, asserting that only one node is marked as +// decommissioning. This check can be run against both v20.1 and v20.2 servers. +func checkOneDecommissioning(from int) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + db := u.conn(ctx, t, from) + var count int + if err := db.QueryRow( + `select count(*) from crdb_internal.gossip_liveness where decommissioning = true;`).Scan(&count); err != nil { + t.Fatal(err) + } + + if count != 1 { + t.Fatalf("expected to find 1 node with decommissioning=true, found %d", count) + } + + var nodeID int + if err := db.QueryRow( + `select node_id from crdb_internal.gossip_liveness where decommissioning = true;`).Scan(&nodeID); err != nil { + t.Fatal(err) + } + t.l.Printf("n%d decommissioning=true", nodeID) + } +} + +// checkNoDecommissioning checks against the `decommissioning` column in +// crdb_internal.gossip_liveness, asserting that only no nodes are marked as +// decommissioning. This check can be run against both v20.1 and v20.2 servers. +func checkNoDecommissioning(from int) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + db := u.conn(ctx, t, from) + var count int + if err := db.QueryRow( + `select count(*) from crdb_internal.gossip_liveness where decommissioning = true;`).Scan(&count); err != nil { + t.Fatal(err) + } + + if count != 0 { + t.Fatalf("expected to find 0 nodes with decommissioning=false, found %d", count) + } + } +} + +// checkOneMembership checks against the `membership` column in +// crdb_internal.gossip_liveness, asserting that only one node is marked with +// the specified membership status. This check can be only be run against +// servers running v20.2 and beyond. +func checkOneMembership(from int, membership string) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + db := u.conn(ctx, t, from) + var count int + if err := db.QueryRow( + `select count(*) from crdb_internal.gossip_liveness where membership = $1;`, membership).Scan(&count); err != nil { + t.Fatal(err) + } + + if count != 1 { + t.Fatalf("expected to find 1 node with membership=%s, found %d", membership, count) + } + + var nodeID int + if err := db.QueryRow( + `select node_id from crdb_internal.gossip_liveness where decommissioning = true;`).Scan(&nodeID); err != nil { + t.Fatal(err) + } + t.l.Printf("n%d membership=%s", nodeID, membership) + } +} + +// checkAllMembership checks against the `membership` column in +// crdb_internal.gossip_liveness, asserting that all nodes are marked with +// the specified membership status. This check can be only be run against +// servers running v20.2 and beyond. +func checkAllMembership(from int, membership string) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + db := u.conn(ctx, t, from) + var count int + if err := db.QueryRow( + `select count(*) from crdb_internal.gossip_liveness where membership != $1;`, membership).Scan(&count); err != nil { + t.Fatal(err) + } + + if count != 0 { + t.Fatalf("expected to find 0 nodes with membership!=%s, found %d", membership, count) + } + } +} + +// uploadVersion uploads the specified cockroach binary version on the specified +// nodes. +func uploadVersion(nodes nodeListOption, version string) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + // Put the binary. + u.uploadVersion(ctx, t, nodes, version) + } +} + +// startVersion starts the specified cockroach binary version on the specified +// nodes. +func startVersion(nodes nodeListOption, version string) versionStep { + return func(ctx context.Context, t *test, u *versionUpgradeTest) { + args := startArgs("--binary=" + cockroachBinaryPath(version)) + u.c.Start(ctx, t, nodes, args, startArgsDontEncrypt, roachprodArgOption{"--sequential=false"}) + } +} + +func registerDecommissionMixedVersion(r *testRegistry) { + r.Add(testSpec{ + Name: "decommission/mixed-versions", + Owner: OwnerKV, + MinVersion: "v20.1.0", + Cluster: makeClusterSpec(4), + Run: func(ctx context.Context, t *test, c *cluster) { + runDecommissionMixedVersions(ctx, t, c, r.buildVersion) + }, + }) + +} diff --git a/pkg/cmd/roachtest/registry.go b/pkg/cmd/roachtest/registry.go index f52aed066a05..32e76c02cdb1 100644 --- a/pkg/cmd/roachtest/registry.go +++ b/pkg/cmd/roachtest/registry.go @@ -27,6 +27,7 @@ func registerTests(r *testRegistry) { registerClockMonotonicTests(r) registerCopy(r) registerDecommission(r) + registerDecommissionMixedVersion(r) registerDiskFull(r) registerDiskStalledDetection(r) registerDjango(r) From 58875533d668b9297df72d8fef3e0a345c93abe8 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Sun, 5 Jul 2020 23:57:36 -0400 Subject: [PATCH 4/5] roachtest: improve decommissioning roachtests Re-write the previously skipped `acceptance/decommission` to account for new node {d,r}ecommissioning semantics. The minimum version to run this test against is v20.2. ```go // runDecommissionRecommission tests a bunch of node // decommissioning/recommissioning procedures, all the while checking for // replica movement and appropriate membership status detection behavior. We go // through partial decommissioning of random nodes, ensuring we're able to undo // those operations. We then fully decommission nodes, verifying it's an // irreversible operation. ``` Release note: None --- pkg/cmd/roachtest/acceptance.go | 7 - pkg/cmd/roachtest/cluster.go | 23 + pkg/cmd/roachtest/decommission.go | 1043 ++++++++++++++++++----------- 3 files changed, 666 insertions(+), 407 deletions(-) diff --git a/pkg/cmd/roachtest/acceptance.go b/pkg/cmd/roachtest/acceptance.go index da90a0b9cc75..1ad304c3f29b 100644 --- a/pkg/cmd/roachtest/acceptance.go +++ b/pkg/cmd/roachtest/acceptance.go @@ -35,13 +35,6 @@ func registerAcceptance(r *testRegistry) { {name: "build-info", fn: runBuildInfo}, {name: "build-analyze", fn: runBuildAnalyze}, {name: "cli/node-status", fn: runCLINodeStatus}, - { - name: "decommission", fn: runDecommissionAcceptance, - skip: `https://github.com/cockroachdb/cockroach/issues/50919 -https://github.com/cockroachdb/cockroach/issues/50861 -https://github.com/cockroachdb/cockroach/issues/50640 -https://github.com/cockroachdb/cockroach/issues/48676`, - }, {name: "cluster-init", fn: runClusterInit}, {name: "event-log", fn: runEventLog}, {name: "gossip/peerings", fn: runGossipPeerings}, diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index c9dcfe6cb34a..a19244393dd1 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -734,6 +734,16 @@ func (n nodeListOption) randNode() nodeListOption { return nodeListOption{n[rand.Intn(len(n))]} } +// nodeIDsString returns a space separated list of all node IDs comprising this +// list. +func (n nodeListOption) nodeIDsString() string { + result := "" + for _, i := range n { + result += fmt.Sprintf("%s ", strconv.Itoa(i)) + } + return result +} + func (n nodeListOption) String() string { if len(n) == 0 { return "" @@ -1406,6 +1416,19 @@ func (c *cluster) Range(begin, end int) nodeListOption { return r } +// All returns a node list containing only the node i. +func (c *cluster) Nodes(ns ...int) nodeListOption { + r := make(nodeListOption, 0, len(ns)) + for _, n := range ns { + if n < 1 || n > c.spec.NodeCount { + c.t.Fatalf("invalid node range: %d (1-%d)", n, c.spec.NodeCount) + } + + r = append(r, n) + } + return r +} + // All returns a node list containing only the node i. func (c *cluster) Node(i int) nodeListOption { return c.Range(i, i) diff --git a/pkg/cmd/roachtest/decommission.go b/pkg/cmd/roachtest/decommission.go index a8513e1d2328..76b85c871251 100644 --- a/pkg/cmd/roachtest/decommission.go +++ b/pkg/cmd/roachtest/decommission.go @@ -14,6 +14,7 @@ import ( "context" "encoding/csv" "fmt" + "math/rand" "reflect" "regexp" "strconv" @@ -29,6 +30,39 @@ import ( "golang.org/x/sync/errgroup" ) +func registerDecommission(r *testRegistry) { + { + numNodes := 4 + duration := time.Hour + + r.Add(testSpec{ + Name: fmt.Sprintf("decommission/nodes=%d/duration=%s", numNodes, duration), + Owner: OwnerKV, + Cluster: makeClusterSpec(4), + Run: func(ctx context.Context, t *test, c *cluster) { + if local { + duration = 3 * time.Minute + t.l.Printf("running with duration=%s in local mode\n", duration) + } + runDecommission(t, c, numNodes, duration) + }, + }) + } + { + numNodes := 6 + r.Add(testSpec{ + Name: "decommission-recommission", + Owner: OwnerKV, + MinVersion: "v20.2.0", + Timeout: 10 * time.Minute, + Cluster: makeClusterSpec(numNodes), + Run: func(ctx context.Context, t *test, c *cluster) { + runDecommissionRecommission(ctx, t, c) + }, + }) + } +} + // TODO(tschottdorf): verify that the logs don't contain the messages // that would spam the log before #23605. I wonder if we should really // start grepping the logs. An alternative is to introduce a metric @@ -227,465 +261,489 @@ func runDecommission(t *test, c *cluster, nodes int, duration time.Duration) { } } -func registerDecommission(r *testRegistry) { - const numNodes = 4 - duration := time.Hour - - r.Add(testSpec{ - Name: fmt.Sprintf("decommission/nodes=%d/duration=%s", numNodes, duration), - Owner: OwnerKV, - Cluster: makeClusterSpec(numNodes), - Run: func(ctx context.Context, t *test, c *cluster) { - if local { - duration = 3 * time.Minute - t.l.Printf("running with duration=%s in local mode\n", duration) - } - runDecommission(t, c, numNodes, duration) - }, - }) -} - -func execCLI( - ctx context.Context, t *test, c *cluster, runNode int, extraArgs ...string, -) (string, error) { - args := []string{"./cockroach"} - args = append(args, extraArgs...) - args = append(args, "--insecure") - args = append(args, fmt.Sprintf("--port={pgport:%d}", runNode)) - buf, err := c.RunWithBuffer(ctx, t.l, c.Node(runNode), args...) - t.l.Printf("%s\n", buf) - return string(buf), err -} - -func runDecommissionAcceptance(ctx context.Context, t *test, c *cluster) { +// runDecommissionRecommission tests a bunch of node +// decommissioning/recommissioning procedures, all the while checking for +// replica movement and appropriate membership status detection behavior. We go +// through partial decommissioning of random nodes, ensuring we're able to undo +// those operations. We then fully decommission nodes, verifying it's an +// irreversible operation. +func runDecommissionRecommission(ctx context.Context, t *test, c *cluster) { args := startArgs("--env=COCKROACH_SCAN_MAX_IDLE_TIME=5ms") c.Put(ctx, cockroach, "./cockroach") c.Start(ctx, t, args) - decommission := func( - ctx context.Context, - runNode int, - targetNodes nodeListOption, - verbs ...string, - ) (string, error) { - args := []string{"node"} - args = append(args, verbs...) - for _, target := range targetNodes { - args = append(args, strconv.Itoa(target)) - } - return execCLI(ctx, t, c, runNode, args...) + // We use a few helpers for random node ID access. + var nodeIDs []int + for i := 1; i <= c.spec.NodeCount; i++ { + nodeIDs = append(nodeIDs, i) } - - getCsvNumCols := func(csvStr string) (cols int, err error) { - reader := csv.NewReader(strings.NewReader(csvStr)) - records, err := reader.Read() - if err != nil { - return 0, errors.Errorf("error reading csv input: \n %v\n errors:%s", csvStr, err) - } - return len(records), nil + getRandNode := func() int { + return nodeIDs[rand.Intn(len(nodeIDs))] } - - matchCSV := func(csvStr string, matchColRow [][]string) (err error) { - defer func() { - if err != nil { - err = errors.Errorf("csv input:\n%v\nexpected:\n%s\nerrors:%s", - csvStr, pretty.Sprint(matchColRow), err) - } - }() - - reader := csv.NewReader(strings.NewReader(csvStr)) - reader.FieldsPerRecord = -1 - records, err := reader.ReadAll() - if err != nil { - return err - } - - lr, lm := len(records), len(matchColRow) - if lr < lm { - return errors.Errorf("csv has %d rows, but expected at least %d", lr, lm) - } - - // Compare only the last len(matchColRow) records. That is, if we want to - // match 4 rows and we have 100 records, we only really compare - // records[96:], that is, the last four rows. - records = records[lr-lm:] - - for i := range records { - if lr, lm := len(records[i]), len(matchColRow[i]); lr != lm { - return errors.Errorf("row #%d: csv has %d columns, but expected %d", i+1, lr, lm) - } - for j := range records[i] { - pat, str := matchColRow[i][j], records[i][j] - re := regexp.MustCompile(pat) - if !re.MatchString(str) { - err = errors.Errorf("%v\nrow #%d, col #%d: found %q which does not match %q", - err, i+1, j+1, str, pat) + getRandNodeOtherThan := func(ids ...int) int { + for { + cur := nodeIDs[rand.Intn(len(nodeIDs))] + inBlockList := false + for _, id := range ids { + if cur == id { + inBlockList = true } } - } - return err - } - - decommissionHeader := []string{ - "id", "is_live", "replicas", "is_decommissioning", "is_draining", - } - decommissionFooter := []string{ - "No more data reported on target nodes. " + - "Please verify cluster health before removing the nodes.", - } - - // Different output here to be backwards compatible with earlier - // versions of cockroach (versions pre commit 888813c, which - // extends the node status command to include locality information). - statusHeaderWithLocality := []string{ - "id", "address", "sql_address", "build", "started_at", "updated_at", "locality", "is_available", "is_live", - } - statusHeaderNoLocality := []string{ - "id", "address", "sql_address", "build", "started_at", "updated_at", "is_available", "is_live", - } - statusHeaderNoLocalityNoSQLAddress := []string{ - "id", "address", "build", "started_at", "updated_at", "is_available", "is_live", - } - getStatusCsvOutput := func(ids []string, numCols int) [][]string { - var res [][]string - switch numCols { - case len(statusHeaderNoLocality): - res = append(res, statusHeaderNoLocality) - case len(statusHeaderWithLocality): - res = append(res, statusHeaderWithLocality) - case len(statusHeaderNoLocalityNoSQLAddress): - res = append(res, statusHeaderNoLocalityNoSQLAddress) - default: - t.Fatalf( - "Expected status output numCols to be either %d, %d or %d, found %d", - len(statusHeaderNoLocalityNoSQLAddress), - len(statusHeaderNoLocality), - len(statusHeaderWithLocality), - numCols, - ) - } - for _, id := range ids { - build := []string{id} - for i := 0; i < numCols-1; i++ { - build = append(build, `.*`) + if inBlockList { + continue } - res = append(res, build) + return cur } - return res } - t.l.Printf("decommissioning first node from the second, polling the status manually\n") + h := &decommTestHelper{c: c, t: t} retryOpts := retry.Options{ InitialBackoff: time.Second, MaxBackoff: 5 * time.Second, - Multiplier: 1, + Multiplier: 2, } - if err := retry.WithMaxAttempts(ctx, retryOpts, 50, func() error { - o, err := decommission(ctx, 2, c.Node(1), - "decommission", "--wait=none", "--format=csv") + + // Partially decommission then recommission a random node, from another + // random node. Run a couple of status checks while doing so. + { + targetNode, runNode := getRandNode(), getRandNode() + t.l.Printf("partially decommissioning n%d from n%d\n", targetNode, runNode) + o, err := h.decommission(ctx, c.Node(targetNode), runNode, + "--wait=none", "--format=csv") if err != nil { t.Fatalf("decommission failed: %v", err) } exp := [][]string{ decommissionHeader, - {"1", "true", "0", "true", "false"}, - decommissionFooter, + {strconv.Itoa(targetNode), "true", `\d+`, "true", "decommissioning", "false"}, + } + if err := h.matchCSV(o, exp); err != nil { + t.Fatal(err) } - return matchCSV(o, exp) - }); err != nil { - t.Fatal(err) - } + // Check that `node status` reflects an ongoing decommissioning status + // for the second node. + { + runNode = getRandNode() + t.l.Printf("checking that `node status` (from n%d) shows n%d as decommissioning\n", + runNode, targetNode) + o, err := execCLI(ctx, t, c, runNode, "node", "status", "--format=csv", "--decommission") + if err != nil { + t.Fatalf("node-status failed: %v", err) + } - // Check that even though the node is decommissioned, we still see it (since - // it remains live) in `node ls`. - { - o, err := execCLI(ctx, t, c, 2, "node", "ls", "--format=csv") - if err != nil { - t.Fatalf("node-ls failed: %v", err) + numCols := h.getCsvNumCols(o) + exp := h.expectCell(targetNode-1, /* node IDs are 1-indexed */ + statusHeaderMembershipColumnIdx, `decommissioning`, c.spec.NodeCount, numCols) + if err := h.matchCSV(o, exp); err != nil { + t.Fatal(err) + } } - exp := [][]string{ - {"id"}, - {"1"}, - {"2"}, - {"3"}, - {"4"}, + + // Recommission the target node, cancel the in-flight decommissioning + // process. + { + runNode = getRandNode() + t.l.Printf("recommissioning n%d (from n%d)\n", targetNode, runNode) + if _, err := h.recommission(ctx, c.Node(targetNode), runNode); err != nil { + t.Fatalf("recommission failed: %v", err) + } } - if err := matchCSV(o, exp); err != nil { - t.Fatal(err) + + // Check that `node status` now reflects a 'active' status for the + // target node. + { + runNode = getRandNode() + t.l.Printf("checking that `node status` (from n%d) shows n%d as active\n", + targetNode, runNode) + o, err := execCLI(ctx, t, c, runNode, "node", "status", "--format=csv", "--decommission") + if err != nil { + t.Fatalf("node-status failed: %v", err) + } + + numCols := h.getCsvNumCols(o) + exp := h.expectCell(targetNode-1, /* node IDs are 1-indexed */ + statusHeaderMembershipColumnIdx, `active`, c.spec.NodeCount, numCols) + if err := h.matchCSV(o, exp); err != nil { + t.Fatal(err) + } } } - // Ditto `node status`. + + // Check to see that operators aren't able to decommission into + // availability. We'll undo the attempted decommissioning event by + // recommissioning the targeted nodes. { - o, err := execCLI(ctx, t, c, 2, "node", "status", "--format=csv") - if err != nil { - t.Fatalf("node-status failed: %v", err) - } - numCols, err := getCsvNumCols(o) - if err != nil { - t.Fatal(err) + // Attempt to decommission all the nodes. + { + runNode := getRandNode() + t.l.Printf("attempting to decommission all nodes from n%d\n", runNode) + o, err := h.decommission(ctx, c.All(), runNode, + "--wait=none", "--format=csv") + if err != nil { + t.Fatalf("decommission failed: %v", err) + } + + exp := [][]string{decommissionHeader} + for i := 1; i <= c.spec.NodeCount; i++ { + rowRegex := []string{strconv.Itoa(i), "true", `\d+`, "true", "decommissioning", "false"} + exp = append(exp, rowRegex) + } + if err := h.matchCSV(o, exp); err != nil { + t.Fatalf("decommission failed: %v", err) + } } - exp := getStatusCsvOutput([]string{`1`, `2`, `3`, `4`}, numCols) - if err := matchCSV(o, exp); err != nil { - t.Fatal(err) + + // Check that `node status` reflects an ongoing decommissioning status for + // all nodes. + { + runNode := getRandNode() + t.l.Printf("checking that `node status` (from n%d) shows all nodes as decommissioning\n", + runNode) + o, err := execCLI(ctx, t, c, runNode, "node", "status", "--format=csv", "--decommission") + if err != nil { + t.Fatalf("node-status failed: %v", err) + } + + numCols := h.getCsvNumCols(o) + var colRegex []string + for i := 1; i <= c.spec.NodeCount; i++ { + colRegex = append(colRegex, `decommissioning`) + } + exp := h.expectColumn(statusHeaderMembershipColumnIdx, colRegex, c.spec.NodeCount, numCols) + if err := h.matchCSV(o, exp); err != nil { + t.Fatal(err) + } } - } - t.l.Printf("recommissioning first node (from third node)\n") - if _, err := decommission(ctx, 3, c.Node(1), "recommission"); err != nil { - t.Fatalf("recommission failed: %v", err) - } + // Check that we can still do stuff, creating a database should be good + // enough. + { + runNode := getRandNode() + t.l.Printf("checking that we're able to create a database (from n%d)\n", runNode) + db := c.Conn(ctx, runNode) + defer db.Close() - // Verify the --self flag works. - t.l.Printf("re-decommissioning first node, from itself\n") - selfFlagSupported := true - if cmdOutput, err := decommission(ctx, 1, nil, "decommission", "--self"); err != nil { - // Until 20.2, --self is not supported. - // TODO(knz): Remove this alternative when roachtest does not - // test any version lower than 20.2. - if strings.Contains(cmdOutput, ": unknown flag: --self") { - t.l.Printf("--self not supported; skipping recommission with --self") - selfFlagSupported = false - } else { - t.Fatalf("decommission failed: %v", err) + if _, err := db.Exec(`create database still_working;`); err != nil { + t.Fatal(err) + } + } + + // Cancel in-flight decommissioning process of all nodes. + { + runNode := getRandNode() + t.l.Printf("recommissioning all nodes (from n%d)\n", runNode) + if _, err := h.recommission(ctx, c.All(), runNode); err != nil { + t.Fatalf("recommission failed: %v", err) + } } - } - if selfFlagSupported { - t.l.Printf("re-recommissioning first node, from itself\n") - if _, err := decommission(ctx, 1, nil, "recommission", "--self"); err != nil { - t.Fatalf("recommission failed: %v", err) + // Check that `node status` now reflects an 'active' status for all + // nodes. + { + runNode := getRandNode() + t.l.Printf("checking that `node status` (from n%d) shows all nodes as active\n", + runNode) + o, err := execCLI(ctx, t, c, runNode, "node", "status", "--format=csv", "--decommission") + if err != nil { + t.Fatalf("node-status failed: %v", err) + } + + numCols := h.getCsvNumCols(o) + var colRegex []string + for i := 1; i <= c.spec.NodeCount; i++ { + colRegex = append(colRegex, `active`) + } + exp := h.expectColumn(statusHeaderMembershipColumnIdx, colRegex, c.spec.NodeCount, numCols) + if err := h.matchCSV(o, exp); err != nil { + t.Fatal(err) + } } } - t.l.Printf("decommissioning second node from third, using --wait=all\n") + // Fully recommission two random nodes, from a random node, randomly choosing + // between using --wait={all,none}. We pin these two nodes to not re-use + // them for the block after, as they will have been fully decommissioned and + // by definition, non-operational. + decommissionedNodeA := getRandNode() + decommissionedNodeB := getRandNodeOtherThan(decommissionedNodeA) { - o, err := decommission(ctx, 3, c.Node(2), - "decommission", "--wait=all", "--format=csv") - if err != nil { - t.Fatalf("decommission failed: %v", err) + targetNodeA, targetNodeB := decommissionedNodeA, decommissionedNodeB + if targetNodeB < targetNodeA { + targetNodeB, targetNodeA = targetNodeA, targetNodeB } - exp := [][]string{ - decommissionHeader, - {"2", "true", "0", "true", "false"}, - decommissionFooter, + runNode := getRandNode() + waitStrategy := "all" // Blocking decommission. + if i := rand.Intn(2); i == 0 { + waitStrategy = "none" // Polling decommission. } - if err := matchCSV(o, exp); err != nil { - t.Fatal(err) - } - } - t.l.Printf("recommissioning second node from itself\n") - if _, err := decommission(ctx, 2, c.Node(2), "recommission"); err != nil { - t.Fatalf("recommission failed: %v", err) - } + t.l.Printf("fully decommissioning [n%d,n%d] from n%d, using --wait=%s\n", + targetNodeA, targetNodeB, runNode, waitStrategy) - t.l.Printf("decommissioning third node (from itself)\n") - func() { - // This should not take longer than five minutes, and if it does, it's - // likely stuck forever and we want to see the output. - timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Minute) - defer cancel() - o, err := decommission(timeoutCtx, 3, c.Node(3), - "decommission", "--wait=all", "--format=csv") - if err != nil { - t.Fatalf("decommission failed: %v", err) + // When using --wait=none, we poll the decommission status. + maxAttempts := 50 + if waitStrategy == "all" { + // --wait=all is a one shot attempt at decommissioning, that polls + // internally. + maxAttempts = 1 } - exp := [][]string{ - decommissionHeader, - {"3", "true", "0", "true", "false"}, - decommissionFooter, - } - if err := matchCSV(o, exp); err != nil { + // Decommission two nodes. + if err := retry.WithMaxAttempts(ctx, retryOpts, maxAttempts, func() error { + o, err := h.decommission(ctx, c.Nodes(targetNodeA, targetNodeB), runNode, + fmt.Sprintf("--wait=%s", waitStrategy), "--format=csv") + if err != nil { + t.Fatalf("decommission failed: %v", err) + } + + exp := [][]string{ + decommissionHeader, + {strconv.Itoa(targetNodeA), "true", "0", "true", "decommissioned", "false"}, + {strconv.Itoa(targetNodeB), "true", "0", "true", "decommissioned", "false"}, + decommissionFooter, + } + return h.matchCSV(o, exp) + }); err != nil { t.Fatal(err) } - }() - // Now that the third node is down and decommissioned, decommissioning it - // again should be a no-op. We do it from node one but as always it doesn't - // matter. - t.l.Printf("checking that other nodes see node three as successfully decommissioned\n") - { - o, err := decommission(ctx, 2, c.Node(3), - "decommission", "--format=csv") // wait=all is implied - if err != nil { - t.Fatalf("decommission failed: %v", err) + // Check that even though two nodes are decommissioned, we still see + // them (since they remain live) in `node ls`. + { + runNode = getRandNode() + t.l.Printf("checking that `node ls` (from n%d) shows all nodes\n", runNode) + o, err := execCLI(ctx, t, c, runNode, "node", "ls", "--format=csv") + if err != nil { + t.Fatalf("node-ls failed: %v", err) + } + exp := [][]string{{"id"}} + for i := 1; i <= c.spec.NodeCount; i++ { + exp = append(exp, []string{strconv.Itoa(i)}) + } + if err := h.matchCSV(o, exp); err != nil { + t.Fatal(err) + } } - exp := [][]string{ - decommissionHeader, - {"3", "true", "0", "true", "false"}, - decommissionFooter, + // Ditto for `node status`. + { + runNode = getRandNode() + t.l.Printf("checking that `node status` (from n%d) shows all nodes\n", runNode) + o, err := execCLI(ctx, t, c, runNode, "node", "status", "--format=csv") + if err != nil { + t.Fatalf("node-status failed: %v", err) + } + + numCols := h.getCsvNumCols(o) + colRegex := []string{} + for i := 1; i <= c.spec.NodeCount; i++ { + colRegex = append(colRegex, strconv.Itoa(i)) + } + exp := h.expectIDsInStatusOut(colRegex, numCols) + if err := h.matchCSV(o, exp); err != nil { + t.Fatal(err) + } } - if err := matchCSV(o, exp); err != nil { - t.Fatal(err) + + // Attempt to recommission the fully decommissioned nodes (expecting it + // to fail). + { + runNode = getRandNode() + t.l.Printf("expected to fail: recommissioning [n%d,n%d] (from n%d)\n", + targetNodeA, targetNodeB, runNode) + if _, err := h.recommission(ctx, c.Nodes(targetNodeA, targetNodeB), runNode); err == nil { + t.Fatal("expected recommission to fail") + } } - // Bring the node back up. It's still decommissioned, so it won't be of much use. - c.Stop(ctx, c.Node(3)) - c.Start(ctx, t, c.Node(3), args) + // Decommissioning the same nodes again should be a no-op. We do it from + // a random node. + { + runNode = getRandNode() + t.l.Printf("checking that decommissioning [n%d,n%d] (from n%d) is a no-op\n", + targetNodeA, targetNodeB, runNode) + o, err := h.decommission(ctx, c.Nodes(targetNodeA, targetNodeB), runNode, + "--wait=all", "--format=csv") + if err != nil { + t.Fatalf("decommission failed: %v", err) + } - // Recommission. Welcome back! - if _, err = decommission(ctx, 2, c.Node(3), "recommission"); err != nil { - t.Fatalf("recommission failed: %v", err) + exp := [][]string{ + decommissionHeader, + {strconv.Itoa(targetNodeA), "true", "0", "true", "decommissioned", "false"}, + {strconv.Itoa(targetNodeB), "true", "0", "true", "decommissioned", "false"}, + decommissionFooter, + } + if err := h.matchCSV(o, exp); err != nil { + t.Fatal(err) + } + } + + // We restart the nodes and attempt to recommission (should still fail). + { + runNode = getRandNode() + t.l.Printf("expected to fail: restarting [n%d,n%d] and attempting to recommission through n%d\n", + targetNodeA, targetNodeB, runNode) + c.Stop(ctx, c.Nodes(targetNodeA, targetNodeB)) + c.Start(ctx, t, c.Nodes(targetNodeA, targetNodeB), args) + + if _, err := h.recommission(ctx, c.Nodes(targetNodeA, targetNodeB), runNode); err == nil { + t.Fatalf("expected recommission to fail") + } } } - // Kill the first node and verify that we can decommission it while it's down, - // bringing it back up to verify that its replicas still get removed. - t.l.Printf("intentionally killing first node\n") - c.Stop(ctx, c.Node(1)) - t.l.Printf("decommission first node, starting with it down but restarting it for verification\n") + // Decommission a downed node (random selected), randomly choosing between + // bringing the node back to life or leaving it permanently dead. + // + // TODO(irfansharif): We could pull merge this "deadness" check into the + // previous block, when fully decommissioning multiple nodes, to reduce the + // total number of nodes needed in the cluster. { - _, err := decommission(ctx, 2, c.Node(1), "decommission", "--wait=all") - if err != nil { - t.Fatalf("decommission failed: %v", err) + restartDownedNode := false + if i := rand.Intn(2); i == 0 { + restartDownedNode = true + } + + if !restartDownedNode { + // We want to test decommissioning a truly dead node. Make sure we + // don't waste too much time waiting for the node to be recognized + // as dead. Note that we don't want to set this number too low or + // everything will seem dead to the allocator at all times, so + // nothing will ever happen. + func() { + db := c.Conn(ctx, 1) + defer db.Close() + const stmt = "SET CLUSTER SETTING server.time_until_store_dead = '1m15s'" + if _, err := db.ExecContext(ctx, stmt); err != nil { + t.Fatal(err) + } + }() } - c.Start(ctx, t, c.Node(1), args) - // Run a second time to wait until the replicas have all been GC'ed. - // Note that we specify "all" because even though the first node is - // now running, it may not be live by the time the command runs. - o, err := decommission(ctx, 2, c.Node(1), - "decommission", "--wait=all", "--format=csv") - if err != nil { + targetNode := getRandNodeOtherThan(decommissionedNodeA, decommissionedNodeB) + t.l.Printf("intentionally killing n%d to later decommission it when down\n", targetNode) + c.Stop(ctx, c.Node(targetNode)) + + runNode := getRandNodeOtherThan(targetNode) + t.l.Printf("decommissioning n%d (from n%d) in absentia\n", targetNode, runNode) + if _, err := h.decommission(ctx, c.Node(targetNode), runNode, + "--wait=all", "--format=csv"); err != nil { t.Fatalf("decommission failed: %v", err) } - exp := [][]string{ - decommissionHeader, - {"1", "true|false", "0", "true", "false"}, - decommissionFooter, - } - if err := matchCSV(o, exp); err != nil { - t.Fatal(err) - } - } + if restartDownedNode { + t.l.Printf("restarting n%d for verification\n", targetNode) - // Now we want to test decommissioning a truly dead node. Make sure we don't - // waste too much time waiting for the node to be recognized as dead. Note that - // we don't want to set this number too low or everything will seem dead to the - // allocator at all times, so nothing will ever happen. - func() { - db := c.Conn(ctx, 2) - defer db.Close() - const stmt = "SET CLUSTER SETTING server.time_until_store_dead = '1m15s'" - if _, err := db.ExecContext(ctx, stmt); err != nil { - t.Fatal(err) + // Bring targetNode it back up to verify that its replicas still get + // removed. + c.Start(ctx, t, c.Node(targetNode), args) } - }() - t.l.Printf("intentionally killing first node\n") - c.Stop(ctx, c.Node(1)) - // It is being decommissioned in absentia, meaning that its replicas are - // being removed due to deadness. We can't see that reflected in the output - // since the current mechanism gets its replica counts from what the node - // reports about itself, so our assertion here is somewhat weak. - t.l.Printf("decommission first node in absentia using --wait=all\n") - { - o, err := decommission(ctx, 3, c.Node(1), "decommission", "--wait=all", "--format=csv") + // Run decommission a second time to wait until the replicas have + // all been GC'ed. Note that we specify "all" because even though + // the target node is now running, it may not be live by the time + // the command runs. + o, err := h.decommission(ctx, c.Node(targetNode), runNode, + "--wait=all", "--format=csv") if err != nil { t.Fatalf("decommission failed: %v", err) } - // Note we don't check precisely zero replicas (which the node would write - // itself, but it's dead). Also note that the target node may still be - // "live" when it's marked as decommissioned, as its replica count may - // drop to zero faster than liveness times out. exp := [][]string{ decommissionHeader, - {"1", `true|false`, "0", `true`, `false`}, + {strconv.Itoa(targetNode), "true|false", "0", "true", "decommissioned", "false"}, decommissionFooter, } - if err := matchCSV(o, exp); err != nil { + if err := h.matchCSV(o, exp); err != nil { t.Fatal(err) } - } - // Check that (at least after a bit) the node disappears from `node ls` - // because it is decommissioned and not live. - for { - o, err := execCLI(ctx, t, c, 2, "node", "ls", "--format=csv") - if err != nil { - t.Fatalf("node-ls failed: %v", err) - } + if !restartDownedNode { + // Check that (at least after a bit) the node disappears from `node + // ls` because it is decommissioned and not live. + if err := retry.WithMaxAttempts(ctx, retryOpts, 50, func() error { + runNode := getRandNodeOtherThan(targetNode) + o, err := execCLI(ctx, t, c, runNode, "node", "ls", "--format=csv") + if err != nil { + t.Fatalf("node-ls failed: %v", err) + } - exp := [][]string{ - {"id"}, - {"2"}, - {"3"}, - {"4"}, - } + var exp [][]string + for i := 1; i <= c.spec.NodeCount; i++ { + exp = append(exp, []string{fmt.Sprintf("[^%d]", targetNode)}) + } - if err := matchCSV(o, exp); err != nil { - time.Sleep(time.Second) - continue - } - break - } - for { - o, err := execCLI(ctx, t, c, 2, "node", "status", "--format=csv") - if err != nil { - t.Fatalf("node-status failed: %v", err) - } - numCols, err := getCsvNumCols(o) - if err != nil { - t.Fatal(err) - } - exp := getStatusCsvOutput([]string{`2`, `3`, `4`}, numCols) - if err := matchCSV(o, exp); err != nil { - time.Sleep(time.Second) - continue + return h.matchCSV(o, exp) + }); err != nil { + t.Fatal(err) + } + + // Ditto for `node status` + if err := retry.WithMaxAttempts(ctx, retryOpts, 50, func() error { + runNode := getRandNodeOtherThan(targetNode) + o, err := execCLI(ctx, t, c, runNode, "node", "status", "--format=csv") + if err != nil { + t.Fatalf("node-status failed: %v", err) + } + + numCols := h.getCsvNumCols(o) + var expC []string + // We're checking for n-1 rows, where n is the node count. + for i := 1; i < c.spec.NodeCount; i++ { + expC = append(expC, fmt.Sprintf("[^%d].*", targetNode)) + } + exp := h.expectIDsInStatusOut(expC, numCols) + return h.matchCSV(o, exp) + }); err != nil { + t.Fatal(err) + } } - break - } - // Wipe data of node 1 and start it as a new node. - // It will join the cluster with a node id of 5. - // This is done to verify that node status works when a new node is started - // with an address belonging to an old decommissioned node. - { - c.Wipe(ctx, c.Node(1)) - c.Start(ctx, t, c.Node(1), startArgs(fmt.Sprintf("-a=--join %s", - c.InternalAddr(ctx, c.Node(2))[0]))) - } + { + t.l.Printf("wiping n%d and adding it back to the cluster as a new node\n", targetNode) - if err := retry.WithMaxAttempts(ctx, retryOpts, 50, func() error { - o, err := execCLI(ctx, t, c, 2, "node", "status", "--format=csv") - if err != nil { - t.Fatalf("node-status failed: %v", err) + c.Stop(ctx, c.Node(targetNode)) + c.Wipe(ctx, c.Node(targetNode)) + + joinNode := targetNode%c.spec.NodeCount + 1 + joinAddr := c.InternalAddr(ctx, c.Node(joinNode))[0] + c.Start(ctx, t, c.Node(targetNode), startArgs( + fmt.Sprintf("-a=--join %s", joinAddr), + )) } - numCols, err := getCsvNumCols(o) - if err != nil { + + if err := retry.WithMaxAttempts(ctx, retryOpts, 50, func() error { + o, err := execCLI(ctx, t, c, getRandNode(), "node", "status", "--format=csv") + if err != nil { + t.Fatalf("node-status failed: %v", err) + } + numCols := h.getCsvNumCols(o) + var expC []string + for i := 1; i <= c.spec.NodeCount; i++ { + expC = append(expC, fmt.Sprintf("[^%d].*", targetNode)) + } + exp := h.expectIDsInStatusOut(expC, numCols) + return h.matchCSV(o, exp) + }); err != nil { t.Fatal(err) } - exp := getStatusCsvOutput([]string{`2`, `3`, `4`, `5`}, numCols) - return matchCSV(o, exp) - }); err != nil { - t.Fatal(err) } + // We'll verify the set of events, in order, we expect to get posted to + // system.eventlog. if err := retry.ForDuration(time.Minute, func() error { // Verify the event log has recorded exactly one decommissioned or - // recommissioned event for each commissioning operation. - // - // Spurious errors appear to be possible since we might be trying to - // send RPCs to the (relatively recently) down node: - // - // pq: rpc error: code = Unavailable desc = grpc: the connection is - // unavailable - // - // Seen in https://teamcity.cockroachdb.com/viewLog.html?buildId=344802. - db := c.Conn(ctx, 2) + // recommissioned event for each membership operation. + db := c.Conn(ctx, 1) defer db.Close() rows, err := db.Query(` -SELECT "eventType", "targetID" FROM system.eventlog -WHERE "eventType" IN ($1, $2) ORDER BY timestamp`, - "node_decommissioned", "node_recommissioned", + SELECT "eventType" FROM system.eventlog WHERE "eventType" IN ($1, $2, $3) ORDER BY timestamp + `, "node_decommissioned", "node_decommissioning", "node_recommissioned", ) if err != nil { t.l.Printf("retrying: %v\n", err) @@ -699,21 +757,35 @@ WHERE "eventType" IN ($1, $2) ORDER BY timestamp`, } expMatrix := [][]string{ - {"node_decommissioned", "1"}, - {"node_recommissioned", "1"}, - {"node_decommissioned", "1"}, - {"node_recommissioned", "1"}, - {"node_decommissioned", "2"}, - {"node_recommissioned", "2"}, - {"node_decommissioned", "3"}, - {"node_recommissioned", "3"}, - {"node_decommissioned", "1"}, - } - - if !selfFlagSupported { - // If `--self` is not supported then we don't expect to see - // node 1 self decommission and recommission (step 3 and 4). - expMatrix = append(expMatrix[:2], expMatrix[4:]...) + // Partial decommission attempt of a single node. + {"node_decommissioning"}, + {"node_recommissioned"}, + + // Cluster wide decommissioning attempt. + {"node_decommissioning"}, + {"node_decommissioning"}, + {"node_decommissioning"}, + {"node_decommissioning"}, + {"node_decommissioning"}, + {"node_decommissioning"}, + + // Cluster wide recommissioning, to undo previous decommissioning attempt. + {"node_recommissioned"}, + {"node_recommissioned"}, + {"node_recommissioned"}, + {"node_recommissioned"}, + {"node_recommissioned"}, + {"node_recommissioned"}, + + // Full decommission of two nodes. + {"node_decommissioning"}, + {"node_decommissioning"}, + {"node_decommissioned"}, + {"node_decommissioned"}, + + // Full decommission of a single node. + {"node_decommissioning"}, + {"node_decommissioned"}, } if !reflect.DeepEqual(matrix, expMatrix) { @@ -723,33 +795,204 @@ WHERE "eventType" IN ($1, $2) ORDER BY timestamp`, }); err != nil { t.Fatal(err) } +} - // Last, verify that the operator can't shoot themselves in the foot by - // accidentally decommissioning all nodes. - // - // Specify wait=none because the command would block forever (the replicas have - // nowhere to go). - if _, err := decommission(ctx, 2, c.All(), "decommission", "--wait=none"); err != nil { - t.Fatalf("decommission failed: %v", err) +// Header from the output of `cockroach node decommission`. +var decommissionHeader = []string{ + "id", "is_live", "replicas", "is_decommissioning", "membership", "is_draining", +} + +// Footer from the output of `cockroach node decommission`, after successful +// decommission. +var decommissionFooter = []string{ + "No more data reported on target nodes. " + + "Please verify cluster health before removing the nodes.", +} + +// Header from the output of `cockroach node status`. +var statusHeader = []string{ + "id", "address", "sql_address", "build", "started_at", "updated_at", "locality", "is_available", "is_live", +} + +// Header from the output of `cockroach node status --decommission`. +var statusHeaderWithDecommission = []string{ + "id", "address", "sql_address", "build", "started_at", "updated_at", "locality", "is_available", "is_live", + "gossiped_replicas", "is_decommissioning", "membership", "is_draining", +} + +// Index of `membership` column in statusHeaderWithDecommission +const statusHeaderMembershipColumnIdx = 11 + +type decommTestHelper struct { + t *test + c *cluster +} + +// decommission decommissions the given targetNodes, running the process +// through the specified runNode. +func (h *decommTestHelper) decommission( + ctx context.Context, targetNodes nodeListOption, runNode int, verbs ...string, +) (string, error) { + args := []string{"node", "decommission"} + args = append(args, verbs...) + + if len(targetNodes) == 1 && targetNodes[0] == runNode { + args = append(args, "--self") + } else { + for _, target := range targetNodes { + args = append(args, strconv.Itoa(target)) + } } + return execCLI(ctx, h.t, h.c, runNode, args...) +} - // Check that we can still do stuff. Creating a database should be good enough. - db := c.Conn(ctx, 2) - defer db.Close() +// recommission recommissions the given targetNodes, running the process +// through the specified runNode. +func (h *decommTestHelper) recommission( + ctx context.Context, targetNodes nodeListOption, runNode int, verbs ...string, +) (string, error) { + args := []string{"node", "recommission"} + args = append(args, verbs...) - if _, err := db.Exec(`CREATE DATABASE still_working;`); err != nil { - t.Fatal(err) + if len(targetNodes) == 1 && targetNodes[0] == runNode { + args = append(args, "--self") + } else { + for _, target := range targetNodes { + args = append(args, strconv.Itoa(target)) + } } + return execCLI(ctx, h.t, h.c, runNode, args...) +} - // Recommission all nodes. - if _, err := decommission(ctx, 2, c.All(), "recommission"); err != nil { - t.Fatalf("recommission failed: %v", err) +// getCsvNumCols returns the number of columns in the given csv string. +func (h *decommTestHelper) getCsvNumCols(csvStr string) (cols int) { + reader := csv.NewReader(strings.NewReader(csvStr)) + records, err := reader.Read() + if err != nil { + h.t.Fatal(errors.Errorf("error reading csv input: \n %v\n errors:%s", csvStr, err)) } + return len(records) +} - // To verify that all nodes are actually accepting replicas again, decommission - // the first nodes (blocking until it's done). This proves that the other nodes - // absorb the first one's replicas. - if _, err := decommission(ctx, 2, c.Node(1), "decommission"); err != nil { - t.Fatalf("decommission failed: %v", err) +// matchCSV matches a multi-line csv string with the provided regex +// (matchColRow[i][j] will be matched against the i-th line, j-th column). +func (h *decommTestHelper) matchCSV(csvStr string, matchColRow [][]string) (err error) { + defer func() { + if err != nil { + err = errors.Errorf("csv input:\n%v\nexpected:\n%s\nerrors:%s", + csvStr, pretty.Sprint(matchColRow), err) + } + }() + + reader := csv.NewReader(strings.NewReader(csvStr)) + reader.FieldsPerRecord = -1 + records, err := reader.ReadAll() + if err != nil { + return err } + + lr, lm := len(records), len(matchColRow) + if lr < lm { + return errors.Errorf("csv has %d rows, but expected at least %d", lr, lm) + } + + // Compare only the last len(matchColRow) records. That is, if we want to + // match 4 rows and we have 100 records, we only really compare + // records[96:], that is, the last four rows. + records = records[lr-lm:] + + for i := range records { + if lr, lm := len(records[i]), len(matchColRow[i]); lr != lm { + return errors.Errorf("row #%d: csv has %d columns, but expected %d", i+1, lr, lm) + } + for j := range records[i] { + pat, str := matchColRow[i][j], records[i][j] + re := regexp.MustCompile(pat) + if !re.MatchString(str) { + err = errors.Errorf("%v\nrow #%d, col #%d: found %q which does not match %q", + err, i+1, j+1, str, pat) + } + } + } + return err +} + +// expectColumn constructs a matching regex for a given column (identified +// by its column index). +func (h *decommTestHelper) expectColumn( + column int, columnRegex []string, numRows, numCols int, +) [][]string { + var res [][]string + for r := 0; r < numRows; r++ { + build := []string{} + for c := 0; c < numCols; c++ { + if c == column { + build = append(build, columnRegex[r]) + } else { + build = append(build, `.*`) + } + } + res = append(res, build) + } + return res +} + +// expectCell constructs a matching regex for a given cell (identified by +// its row and column indexes). +func (h *decommTestHelper) expectCell( + row, column int, regex string, numRows, numCols int, +) [][]string { + var res [][]string + for r := 0; r < numRows; r++ { + build := []string{} + for c := 0; c < numCols; c++ { + if r == row && c == column { + build = append(build, regex) + } else { + build = append(build, `.*`) + } + } + res = append(res, build) + } + return res +} + +// expectIDsInStatusOut constructs a matching regex for output of `cockroach +// node status`. It matches against the `id` column in the output generated +// with and without the `--decommission` flag. +func (h *decommTestHelper) expectIDsInStatusOut(ids []string, numCols int) [][]string { + var res [][]string + switch numCols { + case len(statusHeader): + res = append(res, statusHeader) + case len(statusHeaderWithDecommission): + res = append(res, statusHeaderWithDecommission) + default: + h.t.Fatalf( + "Expected status output numCols to be one of %d or %d, found %d", + len(statusHeader), + len(statusHeaderWithDecommission), + numCols, + ) + } + for _, id := range ids { + build := []string{id} + for i := 0; i < numCols-1; i++ { + build = append(build, `.*`) + } + res = append(res, build) + } + return res +} + +func execCLI( + ctx context.Context, t *test, c *cluster, runNode int, extraArgs ...string, +) (string, error) { + args := []string{"./cockroach"} + args = append(args, extraArgs...) + args = append(args, "--insecure") + args = append(args, fmt.Sprintf("--port={pgport:%d}", runNode)) + buf, err := c.RunWithBuffer(ctx, t.l, c.Node(runNode), args...) + t.l.Printf("%s\n", buf) + return string(buf), err } From c045ad84c003d1276447e09139fba2173ee3bacc Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Tue, 30 Jun 2020 17:51:13 -0400 Subject: [PATCH 5/5] cli: improve help prompt for --wait={all,none} for node decommissioning Now that we have a fully decommissioned bit, we clarify the mechanics of how that is interfaced with through the `--wait` flag. Release note (cli change): We slightly change the mechanics of how the `--wait` flag, as used by `cockroach node decommission`, behaves. Copying over from the help prompt: ``` Specifies when to return during the decommissioning process. Takes any of the following values: - all waits until all target nodes' replica counts have dropped to zero and marks the nodes as fully decommissioned. This is the default. - none marks the targets as decommissioning, but does not wait for the replica counts to drop to zero before returning. If the replica counts are found to be zero, nodes are marked as fully decommissioned. Use when polling manually from an external system. ``` --- pkg/cli/cliflags/flags.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index 409e43e07802..d5089d5349b4 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -939,14 +939,16 @@ drain all active client connections and migrate away range leases.`, Wait = FlagInfo{ Name: "wait", Description: ` -Specifies when to return after having marked the targets as decommissioning. +Specifies when to return during the decommissioning process. Takes any of the following values:
 
-- all: waits until all target nodes' replica counts have dropped to zero.
-  This is the default.
-- none: marks the targets as decommissioning, but does not wait for the
-  process to complete. Use when polling manually from an external system.
+  - all   waits until all target nodes' replica counts have dropped to zero and
+          marks the nodes as fully decommissioned. This is the default.
+  - none  marks the targets as decommissioning, but does not wait for the
+          replica counts to drop to zero before returning. If the replica counts
+          are found to be zero, nodes are marked as fully decommissioned. Use
+          when polling manually from an external system.
 
`, }