diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 74bddfd106f8..5510fcfa2536 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -35,6 +35,7 @@ server.rangelog.ttlduration720h0m0sif nonzero, range log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours. server.remote_debugging.modestringlocalset to enable remote debugging, localhost-only or disable (any, local, off) server.shutdown.drain_waitduration0sthe amount of time a server waits in an unready state before proceeding with the rest of the shutdown process +server.shutdown.lease_transfer_waitduration5sthe amount of time a server waits to transfer range leases before proceeding with the rest of the shutdown process server.shutdown.query_waitduration10sthe server will wait for at least this amount of time for active queries to finish server.time_until_store_deadduration5m0sthe time after which if there is no new gossiped information about a store, it is considered dead server.user_login.timeoutduration10stimeout after which client authentication times out if some system range is unavailable (0 = no timeout) diff --git a/pkg/cli/cli_test.go b/pkg/cli/cli_test.go index 3c92a27b5862..eb25f13dbae6 100644 --- a/pkg/cli/cli_test.go +++ b/pkg/cli/cli_test.go @@ -1387,11 +1387,11 @@ Available Commands: start-single-node start a single-node cluster init initialize a cluster cert create ca, node, and client certs - quit drain and shutdown node + quit drain and shut down a node sql open a sql shell auth-session log in and out of HTTP sessions - node list, inspect or remove nodes + node list, inspect, drain or remove nodes dump dump sql tables nodelocal upload and delete nodelocal files diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index ed5445746377..44b2c9b9bc11 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -903,6 +903,13 @@ long and not particularly human-readable.`, Description: `Deprecated: use 'node decommission' instead.`, } + DrainWait = FlagInfo{ + Name: "drain-wait", + Description: ` +When non-zero, wait for the specified amount of time for the node to +drain all active client connections and migrate away range leases.`, + } + Wait = FlagInfo{ Name: "wait", Description: ` diff --git a/pkg/cli/context.go b/pkg/cli/context.go index 27e1ca24957b..4bba341d4643 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -131,6 +131,7 @@ func initCLIDefaults() { startCtx.inBackground = false quitCtx.serverDecommission = false + quitCtx.drainWait = 10 * time.Minute nodeCtx.nodeDecommissionWait = nodeDecommissionWaitAll nodeCtx.statusShowRanges = false @@ -334,10 +335,16 @@ var startCtx struct { logDir log.DirName } -// quitCtx captures the command-line parameters of the `quit` command. +// quitCtx captures the command-line parameters of the `quit` and +// `node drain` commands. // Defaults set by InitCLIDefaults() above. var quitCtx struct { + // serverDecommission indicates the server should be decommissioned + // before it is drained. serverDecommission bool + // drainWait is the amount of time to wait for the server + // to drain. Set to 0 to disable a timeout (let the server decide). + drainWait time.Duration } // nodeCtx captures the command-line parameters of the `node` command. diff --git a/pkg/cli/demo_cluster.go b/pkg/cli/demo_cluster.go index 164956d54db0..2e4e073aa5bc 100644 --- a/pkg/cli/demo_cluster.go +++ b/pkg/cli/demo_cluster.go @@ -353,12 +353,7 @@ func (c *transientCluster) DrainNode(nodeID roachpb.NodeID) error { } defer finish() - onModes := make([]int32, len(server.GracefulDrainModes)) - for i, m := range server.GracefulDrainModes { - onModes[i] = int32(m) - } - - if err := doShutdown(ctx, adminClient, onModes); err != nil { + if err := drainAndShutdown(ctx, adminClient); err != nil { return err } c.servers[nodeIndex] = nil diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index bb2ee9a49be2..16efbb067652 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -568,6 +568,13 @@ func init() { _ = f.MarkDeprecated(cliflags.Decommission.Name, `use 'cockroach node decommission' then 'cockroach quit' instead`) } + // Quit and node drain. + for _, cmd := range []*cobra.Command{quitCmd, drainNodeCmd} { + f := cmd.Flags() + DurationFlag(f, &quitCtx.drainWait, cliflags.DrainWait, quitCtx.drainWait) + } + + // SQL and demo commands. for _, cmd := range append([]*cobra.Command{sqlShellCmd, demoCmd}, demoCmd.Commands()...) { f := cmd.Flags() VarFlag(f, &sqlCtx.setStmts, cliflags.Set) diff --git a/pkg/cli/interactive_tests/test_missing_log_output.tcl b/pkg/cli/interactive_tests/test_missing_log_output.tcl index 402884d703c2..e1ad0ebe495e 100644 --- a/pkg/cli/interactive_tests/test_missing_log_output.tcl +++ b/pkg/cli/interactive_tests/test_missing_log_output.tcl @@ -99,7 +99,7 @@ start_test "Test that quit does not show INFO by default with --logtostderr" # that the default logging level is WARNING, so that no INFO messages # are printed between the marker and the (first line) error message # from quit. Quit will error out because the server is already stopped. -send "echo marker; $argv quit --logtostderr 2>&1 | grep -vE '^\[WEF\]\[0-9\]+'\r" +send "echo marker; $argv quit --logtostderr 2>&1 | grep -vE '^\[WEF\]\[0-9\]+|^node is draining'\r" eexpect "marker\r\nok" eexpect ":/# " end_test diff --git a/pkg/cli/node.go b/pkg/cli/node.go index c76331f9a1b8..51fcbf7d5b46 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -493,18 +493,55 @@ func runRecommissionNode(cmd *cobra.Command, args []string) error { return printDecommissionStatus(*resp) } +var drainNodeCmd = &cobra.Command{ + Use: "drain", + Short: "drain a node without shutting it down", + Long: ` +Prepare a server for shutting down. This stops accepting client +connections, stops extant connections, and finally pushes range +leases onto other nodes, subject to various timeout parameters +configurable via cluster settings.`, + Args: cobra.NoArgs, + RunE: MaybeDecorateGRPCError(runDrain), +} + +// runNodeDrain calls the Drain RPC without the flag to stop the +// server process. +func runDrain(cmd *cobra.Command, args []string) (err error) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // At the end, we'll report "ok" if there was no error. + defer func() { + if err == nil { + fmt.Println("ok") + } + }() + + // Establish a RPC connection. + c, finish, err := getAdminClient(ctx, serverCfg) + if err != nil { + return err + } + defer finish() + + _, _, err = doDrain(ctx, c) + return err +} + // Sub-commands for node command. var nodeCmds = []*cobra.Command{ lsNodesCmd, statusNodeCmd, decommissionNodeCmd, recommissionNodeCmd, + drainNodeCmd, } var nodeCmd = &cobra.Command{ Use: "node [command]", - Short: "list, inspect or remove nodes", - Long: "List, inspect or remove nodes.", + Short: "list, inspect, drain or remove nodes", + Long: "List, inspect, drain or remove nodes.", RunE: usageAndErr, } diff --git a/pkg/cli/quit.go b/pkg/cli/quit.go new file mode 100644 index 000000000000..7a4ceed12f33 --- /dev/null +++ b/pkg/cli/quit.go @@ -0,0 +1,293 @@ +// 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 cli + +import ( + "context" + "fmt" + "io" + "strings" + "time" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/util/contextutil" + "github.com/cockroachdb/cockroach/pkg/util/grpcutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/spf13/cobra" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +// quitCmd command shuts down the node server. +var quitCmd = &cobra.Command{ + Use: "quit", + Short: "drain and shut down a node\n", + Long: ` +Shut down the server. The first stage is drain, where the server +stops accepting client connections, then stops extant +connections, and finally pushes range leases onto other nodes, +subject to various timeout parameters configurable via +cluster settings. After the first stage completes, +the server process is shut down. + +See also 'cockroach node drain' to drain a server +without stopping the server process. +`, + Args: cobra.NoArgs, + RunE: MaybeDecorateGRPCError(runQuit), +} + +// runQuit accesses the quit shutdown path. +func runQuit(cmd *cobra.Command, args []string) (err error) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // At the end, we'll report "ok" if there was no error. + defer func() { + if err == nil { + fmt.Println("ok") + } + }() + + // Establish a RPC connection. + c, finish, err := getAdminClient(ctx, serverCfg) + if err != nil { + return err + } + defer finish() + + // If --decommission was passed, perform the decommission as first + // step. (Note that this flag is deprecated. It will be removed.) + if quitCtx.serverDecommission { + var myself []roachpb.NodeID // will remain empty, which means target yourself + if err := runDecommissionNodeImpl(ctx, c, nodeDecommissionWaitAll, myself); err != nil { + log.Warningf(ctx, "%v", err) + if server.IsWaitingForInit(err) { + err = errors.New("node cannot be decommissioned before it has been initialized") + } + return err + } + } + + return drainAndShutdown(ctx, c) +} + +// drainAndShutdown attempts to drain the server and then shut it +// down. When given an empty onModes slice, it's a hard shutdown. +func drainAndShutdown(ctx context.Context, c serverpb.AdminClient) (err error) { + hardError, remainingWork, err := doDrain(ctx, c) + if hardError { + return err + } + + if remainingWork { + log.Warningf(ctx, "graceful shutdown may not have completed successfully; check the node's logs for details.") + } + + if err != nil { + log.Warningf(ctx, "drain did not complete successfully; hard shutdown may cause disruption") + } + // We have already performed the drain above. We use a nil array + // of drain modes to indicate no further drain needs to be attempted + // and go straight to shutdown. We try two times just in case there + // is a transient error. + hardErr, err := doShutdown(ctx, c) + if err != nil && !hardErr { + log.Warningf(ctx, "hard shutdown attempt failed, retrying: %v", err) + _, err = doShutdown(ctx, c) + } + return errors.Wrap(err, "hard shutdown failed") +} + +// doDrain calls a graceful drain. +// +// If the function returns hardError true, then the caller should not +// proceed with an alternate strategy (it's likely the server has gone +// away). +func doDrain( + ctx context.Context, c serverpb.AdminClient, +) (hardError, remainingWork bool, err error) { + // The next step is to drain. The timeout is configurable + // via --drain-wait. + if quitCtx.drainWait == 0 { + return doDrainNoTimeout(ctx, c) + } + + err = contextutil.RunWithTimeout(ctx, "drain", quitCtx.drainWait, func(ctx context.Context) (err error) { + hardError, remainingWork, err = doDrainNoTimeout(ctx, c) + return err + }) + if _, ok := err.(*contextutil.TimeoutError); ok { + log.Infof(ctx, "drain timed out: %v", err) + err = errors.New("drain timeout") + } + return +} + +func doDrainNoTimeout( + ctx context.Context, c serverpb.AdminClient, +) (hardError, remainingWork bool, err error) { + defer func() { + if server.IsWaitingForInit(err) { + log.Infof(ctx, "%v", err) + err = errors.New("node cannot be drained before it has been initialized") + } + }() + + remainingWork = true + for { + // Tell the user we're starting to drain. This enables the user to + // mentally prepare for something to take some time, as opposed to + // wondering why nothing is happening. + fmt.Fprintf(stderr, "node is draining... ") // notice no final newline. + + // Send a drain request with the drain bit set and the shutdown bit + // unset. + stream, err := c.Drain(ctx, &serverpb.DrainRequest{ + DeprecatedProbeIndicator: server.DeprecatedDrainParameter, + DoDrain: true, + }) + if err != nil { + fmt.Fprintf(stderr, "\n") // finish the line started above. + return true, remainingWork, errors.Wrap(err, "error sending drain request") + } + for { + resp, err := stream.Recv() + if err == io.EOF { + // Done. + break + } + if err != nil { + // Unexpected error. + fmt.Fprintf(stderr, "\n") // finish the line started above. + log.Infof(ctx, "graceful shutdown failed: %v", err) + return false, remainingWork, err + } + + if resp.IsDraining { + // We want to assert that the node is quitting, and tell the + // story about how much work was performed in logs for + // debugging. + finalString := "" + if resp.DrainRemainingIndicator == 0 { + finalString = " (complete)" + } + // We use stderr so that 'cockroach quit''s stdout remains a + // simple 'ok' in case of success (for compatibility with + // scripts). + fmt.Fprintf(stderr, "remaining: %d%s\n", + resp.DrainRemainingIndicator, finalString) + remainingWork = resp.DrainRemainingIndicator > 0 + } else { + // Either the server has decided it wanted to stop quitting; or + // we're running a pre-20.1 node which doesn't populate IsDraining. + // In either case, we need to stop sending drain requests. + remainingWork = false + fmt.Fprintf(stderr, "done\n") + } + + if resp.DrainRemainingDescription != "" { + // Only show this information in the log; we'd use this for debugging. + // (This can be revealed e.g. via --logtostderr.) + log.Infof(ctx, "drain details: %s\n", resp.DrainRemainingDescription) + } + + // Iterate until end of stream, which indicates the drain is + // complete. + } + if !remainingWork { + break + } + // Avoid a busy wait with high CPU/network usage if the server + // replies with an incomplete drain too quickly. + time.Sleep(200 * time.Millisecond) + } + return false, remainingWork, nil +} + +// doShutdown attempts to trigger a server shutdown *without* +// draining. Use doDrain() prior to perform a drain, or +// drainAndShutdown() to combine both. +func doShutdown(ctx context.Context, c serverpb.AdminClient) (hardError bool, err error) { + defer func() { + if server.IsWaitingForInit(err) { + log.Infof(ctx, "encountered error: %v", err) + err = errors.New("node cannot be shut down before it has been initialized") + err = errors.WithHint(err, "You can still stop the process using a service manager or a signal.") + hardError = true + } + if grpcutil.IsClosedConnection(err) || grpcConfusedErrConnClosedByPeer(err) { + // This most likely means that we shut down successfully. Note + // that sometimes the connection can be shut down even before a + // DrainResponse gets sent back to us, so we don't require a + // response on the stream (see #14184). + err = nil + } + }() + + // We use a shorter timeout because a shutdown request has nothing + // else to do than shut down the node immediately. + err = contextutil.RunWithTimeout(ctx, "hard shutdown", 10*time.Second, func(ctx context.Context) error { + // Send a drain request with the drain bit unset (no drain). + // and the shutdown bit set. + stream, err := c.Drain(ctx, &serverpb.DrainRequest{Shutdown: true}) + if err != nil { + return errors.Wrap(err, "error sending shutdown request") + } + for { + _, err := stream.Recv() + if err == io.EOF { + return nil + } + if err != nil { + return err + } + } + }) + if _, ok := err.(*contextutil.TimeoutError); !ok { + hardError = true + } + return hardError, err +} + +// getAdminClient returns an AdminClient and a closure that must be invoked +// to free associated resources. +func getAdminClient(ctx context.Context, cfg server.Config) (serverpb.AdminClient, func(), error) { + conn, _, finish, err := getClientGRPCConn(ctx, cfg) + if err != nil { + return nil, nil, errors.Wrap(err, "Failed to connect to the node") + } + return serverpb.NewAdminClient(conn), finish, nil +} + +// grpcConfusedErrConnClosedByPeer returns true if the given error +// has been likely produced by a gRPC handshake that was confused +// by the remote end closing the connection. +// This situation occurs semi-frequently (10-15% of cases) in +// go 1.13, and may have been eliminated in 1.14. +func grpcConfusedErrConnClosedByPeer(err error) bool { + err = errors.Cause(err) + s, ok := status.FromError(err) + if !ok { + return false + } + switch { + case s.Code() == codes.Internal && strings.Contains(err.Error(), "compressed flag set with identity or empty encoding"): + return true + case s.Code() == codes.Unimplemented && strings.Contains(err.Error(), "Decompressor is not installed"): + return true + default: + return false + } +} diff --git a/pkg/cli/start.go b/pkg/cli/start.go index cbe301ac7679..7a677ce4087b 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -15,7 +15,6 @@ import ( "context" "flag" "fmt" - "io" "io/ioutil" "net" "net/url" @@ -31,11 +30,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/cli/cliflags" - "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/server/status" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -933,9 +930,27 @@ If problems persist, please see ` + base.DocsURL("cluster-setup-troubleshooting. ac := log.AmbientContext{} ac.AddLogTag("server drain process", nil) drainCtx := ac.AnnotateCtx(context.Background()) - if _, err := s.Drain(drainCtx, server.GracefulDrainModes); err != nil { - log.Warning(drainCtx, err) + + // Perform a graceful drain. We keep retrying forever, in + // case there are many range leases or some unavailability + // preventing progress. If the operator wants to expedite + // the shutdown, they will need to make it ungraceful + // via a 2nd signal. + for { + remaining, _, err := s.Drain(drainCtx) + if err != nil { + log.Errorf(drainCtx, "graceful drain failed: %v", err) + break + } + if remaining == 0 { + // No more work to do. + break + } + // Avoid a busy wait with high CPU usage if the server replies + // with an incomplete drain too quickly. + time.Sleep(200 * time.Millisecond) } + stopper.Stop(drainCtx) }() @@ -1311,156 +1326,3 @@ func getClientGRPCConn( } return conn, clock, closer, nil } - -// getAdminClient returns an AdminClient and a closure that must be invoked -// to free associated resources. -func getAdminClient(ctx context.Context, cfg server.Config) (serverpb.AdminClient, func(), error) { - conn, _, finish, err := getClientGRPCConn(ctx, cfg) - if err != nil { - return nil, nil, errors.Wrap(err, "Failed to connect to the node") - } - return serverpb.NewAdminClient(conn), finish, nil -} - -// quitCmd command shuts down the node server. -var quitCmd = &cobra.Command{ - Use: "quit", - Short: "drain and shutdown node\n", - Long: ` -Shutdown the server. The first stage is drain, where any new requests -will be ignored by the server. When all extant requests have been -completed, the server exits. -`, - Args: cobra.NoArgs, - RunE: MaybeDecorateGRPCError(runQuit), -} - -// checkNodeRunning performs a no-op RPC and returns an error if it failed to -// connect to the server. -func checkNodeRunning(ctx context.Context, c serverpb.AdminClient) error { - // Send a no-op Drain request. - stream, err := c.Drain(ctx, &serverpb.DrainRequest{ - On: nil, - Shutdown: false, - }) - if err != nil { - return errors.Wrap(err, "Failed to connect to the node: error sending drain request") - } - // Ignore errors from the stream. We've managed to connect to the node above, - // and that's all that this function is interested in. - for { - if _, err := stream.Recv(); err != nil { - if server.IsWaitingForInit(err) { - return err - } - if err != io.EOF { - log.Warningf(ctx, "unexpected error from no-op Drain request: %s", err) - } - break - } - } - return nil -} - -// doShutdown attempts to trigger a server shutdown. When given an empty -// onModes slice, it's a hard shutdown. -// -// errTryHardShutdown is returned if the caller should do a hard-shutdown. -func doShutdown(ctx context.Context, c serverpb.AdminClient, onModes []int32) error { - // We want to distinguish between the case in which we can't even connect to - // the server (in which case we don't want our caller to try to come back with - // a hard retry) and the case in which an attempt to shut down fails (times - // out, or perhaps drops the connection while waiting). To that end, we first - // run a noop DrainRequest. If that fails, we give up. - if err := checkNodeRunning(ctx, c); err != nil { - if server.IsWaitingForInit(err) { - return fmt.Errorf("node cannot be shut down before it has been initialized") - } - if grpcutil.IsClosedConnection(err) { - return nil - } - return err - } - // Send a drain request and continue reading until the connection drops (which - // then counts as a success, for the connection dropping is likely the result - // of the Stopper having reached the final stages of shutdown). - stream, err := c.Drain(ctx, &serverpb.DrainRequest{ - On: onModes, - Shutdown: true, - }) - if err != nil { - // This most likely means that we shut down successfully. Note that - // sometimes the connection can be shut down even before a DrainResponse gets - // sent back to us, so we don't require a response on the stream (see - // #14184). - if grpcutil.IsClosedConnection(err) { - return nil - } - return errors.Wrap(err, "Error sending drain request") - } - for { - if _, err := stream.Recv(); err != nil { - if grpcutil.IsClosedConnection(err) { - return nil - } - // Unexpected error; the caller should try again (and harder). - return errTryHardShutdown{err} - } - } -} - -type errTryHardShutdown struct{ error } - -// runQuit accesses the quit shutdown path. -func runQuit(cmd *cobra.Command, args []string) (err error) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - defer func() { - if err == nil { - fmt.Println("ok") - } - }() - onModes := make([]int32, len(server.GracefulDrainModes)) - for i, m := range server.GracefulDrainModes { - onModes[i] = int32(m) - } - - c, finish, err := getAdminClient(ctx, serverCfg) - if err != nil { - return err - } - defer finish() - - if quitCtx.serverDecommission { - var myself []roachpb.NodeID // will remain empty, which means target yourself - if err := runDecommissionNodeImpl(ctx, c, nodeDecommissionWaitAll, myself); err != nil { - return err - } - } - errChan := make(chan error, 1) - go func() { - errChan <- doShutdown(ctx, c, onModes) - }() - select { - case err := <-errChan: - if err != nil { - if _, ok := err.(errTryHardShutdown); ok { - log.Warningf(ctx, "graceful shutdown failed: %s; proceeding with hard shutdown\n", err) - break - } - return err - } - return nil - case <-time.After(time.Minute): - log.Warningf(ctx, "timed out; proceeding with hard shutdown") - } - // Not passing drain modes tells the server to not bother and go - // straight to shutdown. We try two times just in case there is a transient error. - err = doShutdown(ctx, c, nil) - if err != nil { - log.Warningf(ctx, "hard shutdown attempt failed, retrying: %v", err) - err = doShutdown(ctx, c, nil) - } - return errors.Wrap(doShutdown(ctx, c, nil), "hard shutdown failed") -} diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 05c3127b0c34..e48ff89e3234 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -1334,7 +1334,7 @@ func TestRefreshPendingCommands(t *testing.T) { for i := 0; i < 2; i++ { wg.Add(1) go func(i int) { - mtc.stores[i].SetDraining(true) + mtc.stores[i].SetDraining(true, nil /* reporter */) wg.Done() }(i) } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 1408681aa1c2..ec92e64230f3 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -840,7 +840,7 @@ func TestRangeTransferLeaseExpirationBased(t *testing.T) { // caught up to replica0 as draining code doesn't transfer leases to // behind replicas. l.ensureLeaderAndRaftState(t, l.replica0, l.replica1Desc) - l.mtc.stores[0].SetDraining(true) + l.mtc.stores[0].SetDraining(true, nil /* reporter */) // Check that replica0 doesn't serve reads any more. pErr := l.sendRead(0) @@ -856,7 +856,7 @@ func TestRangeTransferLeaseExpirationBased(t *testing.T) { // Check that replica1 now has the lease. l.checkHasLease(t, 1) - l.mtc.stores[0].SetDraining(false) + l.mtc.stores[0].SetDraining(false, nil /* reporter */) }) // DrainTransferWithExtension verifies that a draining store waits for any @@ -889,7 +889,7 @@ func TestRangeTransferLeaseExpirationBased(t *testing.T) { // Drain node 1 with an extension in progress. go func() { - l.mtc.stores[1].SetDraining(true) + l.mtc.stores[1].SetDraining(true, nil /* reporter */) }() // Now unblock the extension. extensionSem <- struct{}{} @@ -1648,7 +1648,7 @@ func TestDrainRangeRejection(t *testing.T) { } drainingIdx := 1 - mtc.stores[drainingIdx].SetDraining(true) + mtc.stores[drainingIdx].SetDraining(true, nil /* reporter */) chgs := roachpb.MakeReplicationChanges(roachpb.ADD_REPLICA, roachpb.ReplicationTarget{ NodeID: mtc.idents[drainingIdx].NodeID, diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index cfa808cba5f5..48385579b68b 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -481,7 +481,7 @@ func (r *Replica) ReadProtectedTimestamps(ctx context.Context) { func (nl *NodeLiveness) SetDrainingInternal( ctx context.Context, liveness storagepb.Liveness, drain bool, ) error { - return nl.setDrainingInternal(ctx, liveness, drain) + return nl.setDrainingInternal(ctx, liveness, drain, nil /* reporter */) } func (nl *NodeLiveness) SetDecommissioningInternal( diff --git a/pkg/kv/kvserver/node_liveness.go b/pkg/kv/kvserver/node_liveness.go index b1f3e581d6ae..6293a73260ee 100644 --- a/pkg/kv/kvserver/node_liveness.go +++ b/pkg/kv/kvserver/node_liveness.go @@ -226,16 +226,26 @@ func (nl *NodeLiveness) sem(nodeID roachpb.NodeID) chan struct{} { // SetDraining attempts to update this node's liveness record to put itself // into the draining state. -func (nl *NodeLiveness) SetDraining(ctx context.Context, drain bool) { +// +// The reporter callback, if non-nil, is called on a best effort basis +// to report work that needed to be done and which may or may not have +// been done by the time this call returns. See the explanation in +// pkg/server/drain.go for details. +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.Self() if err != nil && err != ErrNoLivenessRecord { log.Errorf(ctx, "unexpected error getting liveness: %+v", err) } - if err := nl.setDrainingInternal(ctx, liveness, drain); err == nil { - return + err = nl.setDrainingInternal(ctx, liveness, drain, reporter) + if err != nil { + if log.V(1) { + log.Infof(ctx, "attempting to set liveness draining status to %v: %v", drain, err) + } + continue } + return } } @@ -316,7 +326,7 @@ func (nl *NodeLiveness) SetDecommissioning( } func (nl *NodeLiveness) setDrainingInternal( - ctx context.Context, liveness storagepb.Liveness, drain bool, + ctx context.Context, liveness storagepb.Liveness, drain bool, reporter func(int, string), ) error { nodeID := nl.gossip.NodeID.Get() sem := nl.sem(nodeID) @@ -339,6 +349,10 @@ func (nl *NodeLiveness) setDrainingInternal( if liveness != (storagepb.Liveness{}) { update.Liveness = liveness } + if reporter != nil && drain && !update.Draining { + // Report progress to the Drain RPC. + reporter(1, "liveness record") + } update.Draining = drain update.ignoreCache = true @@ -349,6 +363,9 @@ func (nl *NodeLiveness) setDrainingInternal( } return errors.New("failed to update liveness record") }); err != nil { + if log.V(1) { + log.Infof(ctx, "updating liveness record: %v", err) + } if err == errNodeDrainingSet { return nil } @@ -819,7 +836,11 @@ func (nl *NodeLiveness) updateLivenessAttempt( // First check the existing liveness map to avoid known conditional // put failures. if !update.ignoreCache { - if l, err := nl.GetLiveness(update.NodeID); err == nil && l != oldLiveness { + l, err := nl.GetLiveness(update.NodeID) + if err != nil && err != ErrNoLivenessRecord { + return err + } + if err == nil && l != oldLiveness { return handleCondFailed(l) } } diff --git a/pkg/kv/kvserver/node_liveness_test.go b/pkg/kv/kvserver/node_liveness_test.go index 041a5a6a2846..5942849038ef 100644 --- a/pkg/kv/kvserver/node_liveness_test.go +++ b/pkg/kv/kvserver/node_liveness_test.go @@ -646,7 +646,7 @@ func TestNodeLivenessSetDraining(t *testing.T) { t.Fatal(err) } - mtc.nodeLivenesses[drainingNodeIdx].SetDraining(ctx, true) + mtc.nodeLivenesses[drainingNodeIdx].SetDraining(ctx, true /* drain */, nil /* reporter */) // Draining node disappears from store lists. { diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 36a6e45501f1..451c0598ef51 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -456,6 +456,11 @@ func (r *Replica) leasePostApply(ctx context.Context, newLease roachpb.Lease, pe // for one and if we went on to quiesce, they wouldn't necessarily get // one otherwise (unless they ask for it, which adds latency). r.EmitMLAI() + + if leaseChangingHands && log.V(1) { + // This logging is useful to troubleshoot incomplete drains. + log.Info(ctx, "is now leaseholder") + } } // Mark the new lease in the replica's lease history. diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 55d76408a07e..e85be059ad9f 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1415,7 +1415,7 @@ func TestReplicaDrainLease(t *testing.T) { t.Fatal(pErr) } - tc.store.SetDraining(true) + tc.store.SetDraining(true, nil /* reporter */) tc.repl.mu.Lock() pErr = <-tc.repl.requestLeaseLocked(ctx, status).C() tc.repl.mu.Unlock() @@ -1423,7 +1423,7 @@ func TestReplicaDrainLease(t *testing.T) { if !ok { t.Fatalf("expected NotLeaseHolderError, not %v", pErr) } - tc.store.SetDraining(false) + tc.store.SetDraining(false, nil /* reporter */) // Newly undrained, leases work again. if _, pErr := tc.repl.redirectOnOrAcquireLease(ctx); pErr != nil { t.Fatal(pErr) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index a8e41f1f4a6f..be92bfaa9a32 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -68,9 +68,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" "github.com/google/btree" - "github.com/pkg/errors" "go.etcd.io/etcd/raft" "golang.org/x/time/rate" ) @@ -133,6 +133,27 @@ var concurrentRangefeedItersLimit = settings.RegisterPositiveIntSetting( 64, ) +// raftLeadershipTransferTimeout limits the amount of time a drain command +// waits for lease transfers. +var raftLeadershipTransferWait = func() *settings.DurationSetting { + s := settings.RegisterValidatedDurationSetting( + raftLeadershipTransferWaitKey, + "the amount of time a server waits to transfer range leases before proceeding with the rest of the shutdown process", + 5*time.Second, + func(v time.Duration) error { + if v < 0 { + return errors.Errorf("cannot set %s to a negative duration: %s", + raftLeadershipTransferWaitKey, v) + } + return nil + }, + ) + s.SetVisibility(settings.Public) + return s +}() + +const raftLeadershipTransferWaitKey = "server.shutdown.lease_transfer_wait" + // ExportRequestsLimit is the number of Export requests that can run at once. // Each extracts data from RocksDB to a temp file and then uploads it to cloud // storage. In order to not exhaust the disk or memory, or saturate the network, @@ -965,15 +986,16 @@ func (s *Store) AnnotateCtx(ctx context.Context) context.Context { return s.cfg.AmbientCtx.AnnotateCtx(ctx) } -// The maximum amount of time waited for leadership shedding before commencing -// to drain a store. -const raftLeadershipTransferWait = 5 * time.Second - // SetDraining (when called with 'true') causes incoming lease transfers to be // rejected, prevents all of the Store's Replicas from acquiring or extending // range leases, and attempts to transfer away any leases owned. // When called with 'false', returns to the normal mode of operation. -func (s *Store) SetDraining(drain bool) { +// +// The reporter callback, if non-nil, is called on a best effort basis +// to report work that needed to be done and which may or may not have +// been done by the time this call returns. See the explanation in +// pkg/server/drain.go for details. +func (s *Store) SetDraining(drain bool, reporter func(int, string)) { s.draining.Store(drain) if !drain { newStoreReplicaVisitor(s).Visit(func(r *Replica) bool { @@ -988,23 +1010,50 @@ func (s *Store) SetDraining(drain bool) { var wg sync.WaitGroup ctx := logtags.AddTag(context.Background(), "drain", nil) - transferAllAway := func() int { + transferAllAway := func(transferCtx context.Context) int { // Limit the number of concurrent lease transfers. const leaseTransferConcurrency = 100 sem := quotapool.NewIntPool("Store.SetDraining", leaseTransferConcurrency) - // Incremented for every lease or Raft leadership transfer attempted. We try - // to send both the lease and the Raft leaders away, but this may not - // reliably work. Instead, we run the surrounding retry loop until there are - // no leaders/leases left (ignoring single-replica or uninitialized Raft - // groups). + + // Incremented for every lease or Raft leadership transfer + // attempted. We try to send both the lease and the Raft leaders + // away, but this may not reliably work. Instead, we run the + // surrounding retry loop until there are no leaders/leases left + // (ignoring single-replica or uninitialized Raft groups). var numTransfersAttempted int32 newStoreReplicaVisitor(s).Visit(func(r *Replica) bool { + // + // We need to be careful about the case where the ctx has been canceled + // prior to the call to (*Stopper).RunLimitedAsyncTask(). In that case, + // the goroutine is not even spawned. However, we don't want to + // mis-count the missing goroutine as the lack of transfer attempted. + // So what we do here is immediately increase numTransfersAttempted + // to count this replica, and then decrease it when it is known + // below that there is nothing to transfer (not lease holder and + // not raft leader). + atomic.AddInt32(&numTransfersAttempted, 1) wg.Add(1) if err := s.stopper.RunLimitedAsyncTask( r.AnnotateCtx(ctx), "storage.Store: draining replica", sem, true, /* wait */ func(ctx context.Context) { defer wg.Done() + select { + case <-transferCtx.Done(): + // Context canceled: the timeout loop has decided we've + // done enough draining + // (server.shutdown.lease_transfer_wait). + // + // We need this check here because each call of + // transferAllAway() traverses all stores/replicas without + // checking for the timeout otherwise. + if log.V(1) { + log.Infof(ctx, "lease transfer aborted due to exceeded timeout") + } + return + default: + } + r.mu.Lock() r.mu.draining = true status := r.raftStatusRLocked() @@ -1039,8 +1088,17 @@ func (s *Store) SetDraining(drain bool) { drainingLease.OwnedBy(s.StoreID()) && r.IsLeaseValid(drainingLease, s.Clock().Now()) - if needsLeaseTransfer || needsRaftTransfer { - atomic.AddInt32(&numTransfersAttempted, 1) + if !needsLeaseTransfer && !needsRaftTransfer { + if log.V(1) { + // This logging is useful to troubleshoot incomplete drains. + log.Info(ctx, "not moving out") + } + atomic.AddInt32(&numTransfersAttempted, -1) + return + } + if log.V(1) { + // This logging is useful to troubleshoot incomplete drains. + log.Infof(ctx, "trying to move replica out: lease transfer = %v, raft transfer = %v", needsLeaseTransfer, needsRaftTransfer) } if needsLeaseTransfer { @@ -1089,31 +1147,59 @@ func (s *Store) SetDraining(drain bool) { return int(numTransfersAttempted) } - transferAllAway() + // Give all replicas at least one chance to transfer. + // If we don't do that, then it's possible that a configured + // value for raftLeadershipTransferWait is too low to iterate + // through all the replicas at least once, and the drain + // condition on the remaining value will never be reached. + if numRemaining := transferAllAway(ctx); numRemaining > 0 { + // Report progress to the Drain RPC. + if reporter != nil { + reporter(numRemaining, "range lease iterations") + } + } else { + // No more work to do. + return + } + + // We've seen all the replicas once. Now we're going to iterate + // until they're all gone, up to the configured timeout. + transferTimeout := raftLeadershipTransferWait.Get(&s.cfg.Settings.SV) - if err := contextutil.RunWithTimeout(ctx, "wait for raft leadership transfer", raftLeadershipTransferWait, + if err := contextutil.RunWithTimeout(ctx, "wait for raft leadership transfer", transferTimeout, func(ctx context.Context) error { opts := retry.Options{ InitialBackoff: 10 * time.Millisecond, MaxBackoff: time.Second, Multiplier: 2, } - // Avoid retry.ForDuration because of https://github.com/cockroachdb/cockroach/issues/25091. everySecond := log.Every(time.Second) - return retry.WithMaxAttempts(ctx, opts, 10000, func() error { - if numRemaining := transferAllAway(); numRemaining > 0 { - err := errors.Errorf("waiting for %d replicas to transfer their lease away", numRemaining) + var err error + // Avoid retry.ForDuration because of https://github.com/cockroachdb/cockroach/issues/25091. + for r := retry.StartWithCtx(ctx, opts); r.Next(); { + err = nil + if numRemaining := transferAllAway(ctx); numRemaining > 0 { + // Report progress to the Drain RPC. + if reporter != nil { + reporter(numRemaining, "range lease iterations") + } + err = errors.Errorf("waiting for %d replicas to transfer their lease away", numRemaining) if everySecond.ShouldLog() { log.Info(ctx, err) } - return err } - return nil - }) + if err == nil { + // All leases transferred. We can stop retrying. + break + } + } + // If there's an error in the context but not yet detected in + // err, take it into account here. + return errors.CombineErrors(err, ctx.Err()) }); err != nil { // You expect this message when shutting down a server in an unhealthy // cluster. If we see it on healthy ones, there's likely something to fix. - log.Warningf(ctx, "unable to drain cleanly within %s, service might briefly deteriorate: %+v", raftLeadershipTransferWait, err) + log.Warningf(ctx, "unable to drain cleanly within %s, service might briefly deteriorate: %+v", transferTimeout, err) } } diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 22ef6a9fc124..9bba56a62749 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -17,7 +17,6 @@ import ( "fmt" "io" "net/http" - "os" "sort" "strconv" "strings" @@ -1623,74 +1622,6 @@ func (s *adminServer) getStatementBundle(ctx context.Context, id int64, w http.R _, _ = io.Copy(w, &bundle) } -// Drain puts the node into the specified drain mode(s) and optionally -// instructs the process to terminate. -func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_DrainServer) error { - on := make([]serverpb.DrainMode, len(req.On)) - for i := range req.On { - on[i] = serverpb.DrainMode(req.On[i]) - } - off := make([]serverpb.DrainMode, len(req.Off)) - for i := range req.Off { - off[i] = serverpb.DrainMode(req.Off[i]) - } - - ctx := stream.Context() - _ = s.server.Undrain(ctx, off) - - nowOn, err := s.server.Drain(ctx, on) - if err != nil { - return err - } - - res := serverpb.DrainResponse{ - On: make([]int32, len(nowOn)), - } - for i := range nowOn { - res.On[i] = int32(nowOn[i]) - } - if err := stream.Send(&res); err != nil { - return err - } - - if !req.Shutdown { - return nil - } - - go func() { - // TODO(tbg): why don't we stop the stopper first? Stopping the stopper - // first seems more reasonable since grpc.Stop closes the listener right - // away (and who knows whether gRPC-goroutines are tied up in some - // stopper task somewhere). - s.server.grpc.Stop() - s.server.stopper.Stop(ctx) - }() - - select { - case <-s.server.stopper.IsStopped(): - return nil - case <-ctx.Done(): - return ctx.Err() - case <-time.After(10 * time.Second): - // This is a hack to work around the problem in - // https://github.com/cockroachdb/cockroach/issues/37425#issuecomment-494336131 - // - // There appear to be deadlock scenarios in which we don't manage to - // fully stop the grpc server (which implies closing the listener, i.e. - // seeming dead to the outside world) or don't manage to shut down the - // stopper (the evidence in #37425 is inconclusive which one it is). - // - // Other problems in this area are known, such as - // https://github.com/cockroachdb/cockroach/pull/31692 - // - // The signal-based shutdown path uses a similar time-based escape hatch. - // Until we spend (potentially lots of time to) understand and fix this - // issue, this will serve us well. - os.Exit(1) - return errors.New("unreachable") - } -} - // DecommissionStatus returns the DecommissionStatus for all or the given nodes. func (s *adminServer) DecommissionStatus( ctx context.Context, req *serverpb.DecommissionStatusRequest, diff --git a/pkg/server/drain.go b/pkg/server/drain.go new file mode 100644 index 000000000000..3ecd63d76600 --- /dev/null +++ b/pkg/server/drain.go @@ -0,0 +1,232 @@ +// 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 server + +import ( + "context" + "fmt" + "os" + "reflect" + "strings" + "time" + + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +var ( + // DeprecatedDrainParameter the special value that must be + // passed in DrainRequest.DeprecatedProbeIndicator to signal the + // drain request is not a probe. + // This variable is also used in the v20.1 "quit" client + // to provide a valid input to the request sent to + // v19.1 nodes. + // + // TODO(knz): Remove this in v20.2 and whenever the "quit" command + // is not meant to work with 19.x servers any more, whichever comes + // later. + DeprecatedDrainParameter = []int32{0, 1} + + queryWait = settings.RegisterPublicDurationSetting( + "server.shutdown.query_wait", + "the server will wait for at least this amount of time for active queries to finish", + 10*time.Second, + ) + + drainWait = settings.RegisterPublicDurationSetting( + "server.shutdown.drain_wait", + "the amount of time a server waits in an unready state before proceeding with the rest "+ + "of the shutdown process", + 0*time.Second, + ) +) + +// Drain puts the node into the specified drain mode(s) and optionally +// instructs the process to terminate. +// This method is part of the serverpb.AdminClient interface. +func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_DrainServer) error { + ctx := stream.Context() + ctx = s.server.AnnotateCtx(ctx) + + doDrain := req.DoDrain + if len(req.DeprecatedProbeIndicator) > 0 { + // Pre-20.1 behavior. + // TODO(knz): Remove this condition in 20.2. + doDrain = true + if !reflect.DeepEqual(req.DeprecatedProbeIndicator, DeprecatedDrainParameter) { + return status.Errorf(codes.InvalidArgument, "Invalid drain request parameter.") + } + } + + log.Infof(ctx, "drain request received with doDrain = %v, shutdown = %v", doDrain, req.Shutdown) + + res := serverpb.DrainResponse{} + if doDrain { + remaining, info, err := s.server.Drain(ctx) + if err != nil { + log.Errorf(ctx, "drain failed: %v", err) + return err + } + res.DrainRemainingIndicator = remaining + res.DrainRemainingDescription = info + } + if s.server.isDraining() { + res.DeprecatedDrainStatus = DeprecatedDrainParameter + res.IsDraining = true + } + + if err := stream.Send(&res); err != nil { + return err + } + + if !req.Shutdown { + if doDrain { + // The condition "if doDrain" is because we don't need an info + // message for just a probe. + log.Infof(ctx, "drain request completed without server shutdown") + } + return nil + } + + go func() { + // TODO(tbg): why don't we stop the stopper first? Stopping the stopper + // first seems more reasonable since grpc.Stop closes the listener right + // away (and who knows whether gRPC-goroutines are tied up in some + // stopper task somewhere). + s.server.grpc.Stop() + s.server.stopper.Stop(ctx) + }() + + select { + case <-s.server.stopper.IsStopped(): + return nil + case <-ctx.Done(): + return ctx.Err() + case <-time.After(10 * time.Second): + // This is a hack to work around the problem in + // https://github.com/cockroachdb/cockroach/issues/37425#issuecomment-494336131 + // + // There appear to be deadlock scenarios in which we don't manage to + // fully stop the grpc server (which implies closing the listener, i.e. + // seeming dead to the outside world) or don't manage to shut down the + // stopper (the evidence in #37425 is inconclusive which one it is). + // + // Other problems in this area are known, such as + // https://github.com/cockroachdb/cockroach/pull/31692 + // + // The signal-based shutdown path uses a similar time-based escape hatch. + // Until we spend (potentially lots of time to) understand and fix this + // issue, this will serve us well. + os.Exit(1) + return errors.New("unreachable") + } +} + +// Drain idempotently activates the draining mode. +// Note: new code should not be taught to use this method +// directly. Use the Drain() RPC instead with a suitably crafted +// DrainRequest. +// +// On failure, the system may be in a partially drained +// state; the client should either continue calling Drain() or shut +// down the server. +// +// The reporter function, if non-nil, is called for each +// packet of load shed away from the server during the drain. +// +// TODO(knz): This method is currently exported for use by the +// shutdown code in cli/start.go; however, this is a mis-design. The +// start code should use the Drain() RPC like quit does. +func (s *Server) Drain(ctx context.Context) (remaining uint64, info string, err error) { + reports := make(map[string]int) + var mu syncutil.Mutex + reporter := func(howMany int, what string) { + if howMany > 0 { + mu.Lock() + reports[what] += howMany + mu.Unlock() + } + } + defer func() { + // Detail the counts based on the collected reports. + var descBuf strings.Builder + comma := "" + for what, howMany := range reports { + remaining += uint64(howMany) + fmt.Fprintf(&descBuf, "%s%s: %d", comma, what, howMany) + comma = ", " + } + info = descBuf.String() + log.Infof(ctx, "drain remaining: %d", remaining) + if info != "" { + log.Infof(ctx, "drain details: %s", info) + } + }() + + if err := s.doDrain(ctx, reporter); err != nil { + return 0, "", err + } + + return +} + +func (s *Server) doDrain(ctx context.Context, reporter func(int, string)) error { + // First drain all clients and SQL leases. + if err := s.drainClients(ctx, reporter); err != nil { + return err + } + // Finally, mark the node as draining in liveness and drain the + // range leases. + return s.drainNode(ctx, reporter) +} + +// isDraining returns true if either clients are being drained +// or one of the stores on the node is not accepting replicas. +func (s *Server) isDraining() bool { + return s.pgServer.IsDraining() || s.node.IsDraining() +} + +// drainClients starts draining the SQL layer. +func (s *Server) drainClients(ctx context.Context, reporter func(int, string)) error { + // Mark the server as draining in a way that probes to + // /health?ready=1 will notice. + s.grpc.setMode(modeDraining) + // Wait for drainUnreadyWait. This will fail load balancer checks and + // delay draining so that client traffic can move off this node. + time.Sleep(drainWait.Get(&s.st.SV)) + + // Disable incoming SQL clients up to the queryWait timeout. + drainMaxWait := queryWait.Get(&s.st.SV) + if err := s.pgServer.Drain(drainMaxWait, reporter); err != nil { + return err + } + // Stop ongoing SQL execution up to the queryWait timeout. + s.distSQLServer.Drain(ctx, drainMaxWait, reporter) + + // Drain the SQL leases. This must be done after the pgServer has + // given sessions a chance to finish ongoing work. + s.leaseMgr.SetDraining(true /* drain */, reporter) + + // Done. This executes the defers set above to drain SQL leases. + return nil +} + +// drainNode initiates the draining mode for the node, which +// starts draining range leases. +func (s *Server) drainNode(ctx context.Context, reporter func(int, string)) error { + s.nodeLiveness.SetDraining(ctx, true /* drain */, reporter) + return s.node.SetDraining(true /* drain */, reporter) +} diff --git a/pkg/server/drain_test.go b/pkg/server/drain_test.go new file mode 100644 index 000000000000..c512f2702506 --- /dev/null +++ b/pkg/server/drain_test.go @@ -0,0 +1,245 @@ +// 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 server_test + +import ( + "context" + "io" + "reflect" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/grpcutil" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/errors" + "github.com/kr/pretty" +) + +// TestDrain tests the Drain RPC. +func TestDrain(t *testing.T) { + defer leaktest.AfterTest(t)() + doTestDrain(t, true /* newInterface */) +} + +// TestDrainLegacy tests the Drain RPC using the pre-20.1 probe signaling. +// TODO(knz): Remove this test when compatibility with pre-20.1 nodes +// is dropped. +func TestDrainLegacy(t *testing.T) { + defer leaktest.AfterTest(t)() + doTestDrain(t, false /* newInterface */) +} + +// doTestDrain runs the drain test. +// The parameter newInterface indicates whether to use the pre-20.1 +// protocol based on "drain modes" or the post-20.1 protocol +// using discrete fields on the request object. +func doTestDrain(tt *testing.T, newInterface bool) { + t := newTestDrainContext(tt, newInterface) + defer t.Close() + + // Issue a probe. We're not draining yet, so the probe should + // reflect that. + resp := t.sendProbe() + t.assertDraining(resp, false) + t.assertRemaining(resp, false) + + // Issue a drain without shutdown, so we can probe more afterwards. + resp = t.sendDrainNoShutdown() + t.assertDraining(resp, true) + t.assertRemaining(resp, true) + + // Issue another probe. This checks that the server is still running + // (i.e. Shutdown: false was effective) and also that the draining + // status is still properly reported. + resp = t.sendProbe() + t.assertDraining(resp, true) + // probe-only has no remaining. + t.assertRemaining(resp, false) + + // Issue another drain. Verify that the remaining is zero (i.e. complete). + resp = t.sendDrainNoShutdown() + t.assertDraining(resp, true) + t.assertRemaining(resp, false) + + // Now issue a drain request without drain but with shutdown. + // We're expecting the node to be shut down after that. + resp = t.sendShutdown() + if resp != nil { + t.assertDraining(resp, true) + t.assertRemaining(resp, false) + } + + // Now expect the server to be shut down. + testutils.SucceedsSoon(t, func() error { + _, err := t.c.Drain(context.TODO(), &serverpb.DrainRequest{Shutdown: false}) + if grpcutil.IsClosedConnection(err) { + return nil + } + return errors.Newf("server not yet refusing RPC, got %v", err) + }) +} + +type testDrainContext struct { + *testing.T + tc *testcluster.TestCluster + newInterface bool + c serverpb.AdminClient + connCloser func() +} + +func newTestDrainContext(t *testing.T, newInterface bool) *testDrainContext { + tc := &testDrainContext{ + T: t, + newInterface: newInterface, + tc: testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + // We need to start the cluster insecure in order to not + // care about TLS settings for the RPC client connection. + ServerArgs: base.TestServerArgs{ + Insecure: true, + }, + }), + } + + // We'll have the RPC talk to the first node. + var err error + tc.c, tc.connCloser, err = getAdminClientForServer(context.TODO(), + tc.tc, 0 /* serverIdx */) + if err != nil { + tc.Close() + t.Fatal(err) + } + + return tc +} + +func (t *testDrainContext) Close() { + if t.connCloser != nil { + t.connCloser() + } + t.tc.Stopper().Stop(context.TODO()) +} + +func (t *testDrainContext) sendProbe() *serverpb.DrainResponse { + return t.drainRequest(false /* drain */, false /* shutdown */) +} + +func (t *testDrainContext) sendDrainNoShutdown() *serverpb.DrainResponse { + return t.drainRequest(true /* drain */, false /* shutdown */) +} + +func (t *testDrainContext) drainRequest(drain, shutdown bool) *serverpb.DrainResponse { + // Issue a simple drain probe. + req := &serverpb.DrainRequest{Shutdown: shutdown} + + if drain { + if t.newInterface { + req.DoDrain = true + } else { + req.DeprecatedProbeIndicator = server.DeprecatedDrainParameter + } + } + + drainStream, err := t.c.Drain(context.TODO(), req) + if err != nil { + t.Fatal(err) + } + resp, err := t.getDrainResponse(drainStream) + if err != nil { + t.Fatal(err) + } + return resp +} + +func (t *testDrainContext) sendShutdown() *serverpb.DrainResponse { + req := &serverpb.DrainRequest{Shutdown: true} + drainStream, err := t.c.Drain(context.TODO(), req) + if err != nil { + t.Fatal(err) + } + resp, err := t.getDrainResponse(drainStream) + if err != nil { + // It's possible we're getting "connection reset by peer" or some + // gRPC initialization failure because the server is shutting + // down. Tolerate that. + t.Logf("RPC error: %v", err) + } + return resp +} + +func (t *testDrainContext) assertDraining(resp *serverpb.DrainResponse, drain bool) { + if resp.IsDraining != drain { + t.Fatalf("expected draining %v, got %v", drain, resp.IsDraining) + } + // Check that the deprecated status field is compatible with expectation. + // TODO(knz): Remove this test when compatibility with pre-20.1 nodes + // is dropped. + if drain { + if !reflect.DeepEqual(resp.DeprecatedDrainStatus, server.DeprecatedDrainParameter) { + t.Fatalf("expected compat drain status, got %# v", pretty.Formatter(resp)) + } + } else { + if len(resp.DeprecatedDrainStatus) > 0 { + t.Fatalf("expected no compat drain status, got %# v", pretty.Formatter(resp)) + } + } +} + +func (t *testDrainContext) assertRemaining(resp *serverpb.DrainResponse, remaining bool) { + if actualRemaining := (resp.DrainRemainingIndicator > 0); remaining != actualRemaining { + t.Fatalf("expected remaining %v, got %v", remaining, actualRemaining) + } +} + +func (t *testDrainContext) getDrainResponse( + stream serverpb.Admin_DrainClient, +) (*serverpb.DrainResponse, error) { + resp, err := stream.Recv() + if err != nil { + return nil, err + } + unexpected, err := stream.Recv() + if err != io.EOF { + if unexpected != nil { + t.Fatalf("unexpected additional response: %# v // %v", pretty.Formatter(unexpected), err) + } + if err == nil { + err = errors.New("unexpected response") + } + return nil, err + } + return resp, nil +} + +func getAdminClientForServer( + ctx context.Context, tc *testcluster.TestCluster, serverIdx int, +) (c serverpb.AdminClient, closer func(), err error) { + stopper := stop.NewStopper() // stopper for the client. + // Retrieve some parameters to initialize the client RPC context. + cfg := tc.Server(0).RPCContext().Config + execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) + rpcContext := rpc.NewContext( + log.AmbientContext{Tracer: execCfg.Settings.Tracer}, + cfg, execCfg.Clock, stopper, execCfg.Settings, + ) + conn, err := rpcContext.GRPCUnvalidatedDial(tc.Server(serverIdx).ServingRPCAddr()).Connect(ctx) + if err != nil { + return nil, nil, err + } + return serverpb.NewAdminClient(conn), func() { stopper.Stop(ctx) }, nil +} diff --git a/pkg/server/grpc_server.go b/pkg/server/grpc_server.go index 1383038aa15f..22c9c39e650b 100644 --- a/pkg/server/grpc_server.go +++ b/pkg/server/grpc_server.go @@ -15,6 +15,7 @@ import ( "sync/atomic" "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/errors" "google.golang.org/grpc" "google.golang.org/grpc/codes" grpcstatus "google.golang.org/grpc/status" @@ -97,6 +98,6 @@ func (s *grpcServer) waitingForInitError(methodName string) error { // IsWaitingForInit checks whether the provided error is because the node is // still waiting for initialization. func IsWaitingForInit(err error) bool { - s, ok := grpcstatus.FromError(err) + s, ok := grpcstatus.FromError(errors.UnwrapAll(err)) return ok && s.Code() == codes.Unavailable && strings.Contains(err.Error(), "node waiting for init") } diff --git a/pkg/server/node.go b/pkg/server/node.go index 8c05fda1c0f7..087824d3caef 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -527,9 +527,13 @@ func (n *Node) IsDraining() bool { } // SetDraining sets the draining mode on all of the node's underlying stores. -func (n *Node) SetDraining(drain bool) error { +// The reporter callback, if non-nil, is called on a best effort basis +// to report work that needed to be done and which may or may not have +// been done by the time this call returns. See the explanation in +// pkg/server/drain.go for details. +func (n *Node) SetDraining(drain bool, reporter func(int, string)) error { return n.stores.VisitStores(func(s *kvserver.Store) error { - s.SetDraining(drain) + s.SetDraining(drain, reporter) return nil }) } diff --git a/pkg/server/server.go b/pkg/server/server.go index 7d4579430b2e..4e3756318e3a 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -105,23 +105,6 @@ var ( // Allocation pool for gzipResponseWriters. gzipResponseWriterPool sync.Pool - // GracefulDrainModes is the standard succession of drain modes entered - // for a graceful shutdown. - GracefulDrainModes = []serverpb.DrainMode{serverpb.DrainMode_CLIENT, serverpb.DrainMode_LEASES} - - queryWait = settings.RegisterPublicDurationSetting( - "server.shutdown.query_wait", - "the server will wait for at least this amount of time for active queries to finish", - 10*time.Second, - ) - - drainWait = settings.RegisterPublicDurationSetting( - "server.shutdown.drain_wait", - "the amount of time a server waits in an unready state before proceeding with the rest "+ - "of the shutdown process", - 0*time.Second, - ) - forwardClockJumpCheckEnabled = settings.RegisterPublicBoolSetting( "server.clock.forward_jump_check_enabled", "if enabled, forward clock jumps > max_offset/2 will cause a panic", @@ -2119,85 +2102,6 @@ func (s *Server) bootstrapCluster(ctx context.Context, bootstrapVersion roachpb. }) } -func (s *Server) doDrain( - ctx context.Context, modes []serverpb.DrainMode, setTo bool, -) ([]serverpb.DrainMode, error) { - for _, mode := range modes { - switch mode { - case serverpb.DrainMode_CLIENT: - if setTo { - s.grpc.setMode(modeDraining) - // Wait for drainUnreadyWait. This will fail load balancer checks and - // delay draining so that client traffic can move off this node. - time.Sleep(drainWait.Get(&s.st.SV)) - } - if err := func() error { - if !setTo { - // Execute this last. - defer func() { s.grpc.setMode(modeOperational) }() - } - // Since enabling the lease manager's draining mode will prevent - // the acquisition of new leases, the switch must be made after - // the pgServer has given sessions a chance to finish ongoing - // work. - defer s.leaseMgr.SetDraining(setTo) - - if !setTo { - s.distSQLServer.Undrain(ctx) - s.pgServer.Undrain() - return nil - } - - drainMaxWait := queryWait.Get(&s.st.SV) - if err := s.pgServer.Drain(drainMaxWait); err != nil { - return err - } - s.distSQLServer.Drain(ctx, drainMaxWait) - return nil - }(); err != nil { - return nil, err - } - case serverpb.DrainMode_LEASES: - s.nodeLiveness.SetDraining(ctx, setTo) - if err := s.node.SetDraining(setTo); err != nil { - return nil, err - } - default: - return nil, errors.Errorf("unknown drain mode: %s", mode) - } - } - var nowOn []serverpb.DrainMode - if s.pgServer.IsDraining() { - nowOn = append(nowOn, serverpb.DrainMode_CLIENT) - } - if s.node.IsDraining() { - nowOn = append(nowOn, serverpb.DrainMode_LEASES) - } - return nowOn, nil -} - -// Drain idempotently activates the given DrainModes on the Server in the order -// in which they are supplied. -// For example, Drain is typically called with [CLIENT,LEADERSHIP] before -// terminating the process for graceful shutdown. -// On success, returns all active drain modes after carrying out the request. -// On failure, the system may be in a partially drained state and should be -// recovered by calling Undrain() with the same (or a larger) slice of modes. -func (s *Server) Drain(ctx context.Context, on []serverpb.DrainMode) ([]serverpb.DrainMode, error) { - return s.doDrain(ctx, on, true /* setTo */) -} - -// Undrain idempotently deactivates the given DrainModes on the Server in the -// order in which they are supplied. -// On success, returns any remaining active drain modes. -func (s *Server) Undrain(ctx context.Context, off []serverpb.DrainMode) []serverpb.DrainMode { - nowActive, err := s.doDrain(ctx, off, false) - if err != nil { - panic(fmt.Sprintf("error returned to Undrain: %s", err)) - } - return nowActive -} - // Decommission idempotently sets the decommissioning flag for specified nodes. func (s *Server) Decommission(ctx context.Context, setTo bool, nodeIDs []roachpb.NodeID) error { eventLogger := sql.MakeEventLogger(s.execCfg) diff --git a/pkg/server/serverpb/admin.pb.go b/pkg/server/serverpb/admin.pb.go index 8e5976cb6f55..a3bfbd48c36f 100644 --- a/pkg/server/serverpb/admin.pb.go +++ b/pkg/server/serverpb/admin.pb.go @@ -73,34 +73,7 @@ func (x ZoneConfigurationLevel) String() string { return proto.EnumName(ZoneConfigurationLevel_name, int32(x)) } func (ZoneConfigurationLevel) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_admin_c459f6b997f47a7f, []int{0} -} - -type DrainMode int32 - -const ( - // CLIENT instructs the server to refuse new SQL clients. - // TODO(tschottdorf): also terminate existing clients in a graceful manner. - DrainMode_CLIENT DrainMode = 0 - // LEADERSHIP instructs the server to gracefully let all its Replicas' - // range leases expire. - DrainMode_LEASES DrainMode = 1 -) - -var DrainMode_name = map[int32]string{ - 0: "CLIENT", - 1: "LEASES", -} -var DrainMode_value = map[string]int32{ - "CLIENT": 0, - "LEASES": 1, -} - -func (x DrainMode) String() string { - return proto.EnumName(DrainMode_name, int32(x)) -} -func (DrainMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_admin_c459f6b997f47a7f, []int{1} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{0} } // DatabasesRequest requests a list of databases. @@ -111,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_c459f6b997f47a7f, []int{0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{0} } func (m *DatabasesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -145,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_c459f6b997f47a7f, []int{1} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{1} } func (m *DatabasesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -181,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_c459f6b997f47a7f, []int{2} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{2} } func (m *DatabaseDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -228,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_c459f6b997f47a7f, []int{3} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{3} } func (m *DatabaseDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -264,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_c459f6b997f47a7f, []int{3, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{3, 0} } func (m *DatabaseDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -301,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_c459f6b997f47a7f, []int{4} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{4} } func (m *TableDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -358,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_c459f6b997f47a7f, []int{5} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{5} } func (m *TableDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -395,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_c459f6b997f47a7f, []int{5, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{5, 0} } func (m *TableDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -439,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_c459f6b997f47a7f, []int{5, 1} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{5, 1} } func (m *TableDetailsResponse_Column) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -485,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_c459f6b997f47a7f, []int{5, 2} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{5, 2} } func (m *TableDetailsResponse_Index) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,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_c459f6b997f47a7f, []int{6} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{6} } func (m *TableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -577,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_c459f6b997f47a7f, []int{7} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{7} } func (m *TableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -615,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_c459f6b997f47a7f, []int{7, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{7, 0} } func (m *TableStatsResponse_MissingNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -649,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_c459f6b997f47a7f, []int{8} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{8} } func (m *NonTableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -688,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_c459f6b997f47a7f, []int{9} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{9} } func (m *NonTableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -721,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_c459f6b997f47a7f, []int{10} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{10} } func (m *UsersRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -756,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_c459f6b997f47a7f, []int{11} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{11} } func (m *UsersResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -790,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_c459f6b997f47a7f, []int{11, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{11, 0} } func (m *UsersResponse_User) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -837,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_c459f6b997f47a7f, []int{12} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{12} } func (m *EventsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -872,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_c459f6b997f47a7f, []int{13} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{13} } func (m *EventsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -917,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_c459f6b997f47a7f, []int{13, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{13, 0} } func (m *EventsResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -953,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_c459f6b997f47a7f, []int{14} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{14} } func (m *SetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -986,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_c459f6b997f47a7f, []int{15} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{15} } func (m *SetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1021,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_c459f6b997f47a7f, []int{16} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{16} } func (m *GetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1058,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_c459f6b997f47a7f, []int{17} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{17} } func (m *GetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1094,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_c459f6b997f47a7f, []int{17, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{17, 0} } func (m *GetUIDataResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1127,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_c459f6b997f47a7f, []int{18} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{18} } func (m *ClusterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1166,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_c459f6b997f47a7f, []int{19} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{19} } func (m *ClusterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1195,22 +1168,37 @@ var xxx_messageInfo_ClusterResponse proto.InternalMessageInfo // server first deactivates all the modes specified in 'off' and then activates // all those in 'on'. type DrainRequest struct { - // These are actually of type DrainMode, but grpc-gateway does not support - // proxying enum fields (yet: - // https://github.com/grpc-ecosystem/grpc-gateway/issues/5) and it fails in - // pretty dramatic ways (panics the server). - On []int32 `protobuf:"varint,1,rep,packed,name=on,proto3" json:"on,omitempty"` - Off []int32 `protobuf:"varint,2,rep,packed,name=off,proto3" json:"off,omitempty"` - // When true, terminates the process after the given drain modes have been - // activated. + // deprecated_probe_indicator works as follows: + // - if nil, it indicates that the request is a probe only and + // the server should not actually drain. This indicator + // is supported for pre-20.1 RPC clients which do not know + // about the skip_drain field below. + // - if non-nil, it must be exactly equal to the slice [0, 1]. + // Other values result in an error. When non-nil, + // it implies do_drain = true regardless of the value of the + // other field (pre-20.1 clients don't populate the other field). + // + // This field will be removed in 20.2 in favor of + // do_drain below. + DeprecatedProbeIndicator []int32 `protobuf:"varint,1,rep,packed,name=deprecated_probe_indicator,json=deprecatedProbeIndicator,proto3" json:"deprecated_probe_indicator,omitempty"` + // When true, terminates the process after the server has started draining. + // Setting both shutdown and do_drain to false causes + // the request to only operate as a probe. + // Setting do_drain to false and shutdown to true causes + // the server to shut down immediately without + // first draining. Shutdown bool `protobuf:"varint,3,opt,name=shutdown,proto3" json:"shutdown,omitempty"` + // When true, perform the drain phase. See the comment above on + // shutdown for an explanation of the interaction between the two. + // do_drain is also implied by a non-nil deprecated_probe_indicator. + DoDrain bool `protobuf:"varint,4,opt,name=do_drain,json=doDrain,proto3" json:"do_drain,omitempty"` } 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_c459f6b997f47a7f, []int{20} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{20} } func (m *DrainRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1235,17 +1223,76 @@ func (m *DrainRequest) XXX_DiscardUnknown() { var xxx_messageInfo_DrainRequest proto.InternalMessageInfo -// DrainResponse is the response to a successful DrainRequest and lists the -// modes which are activated after having processing the request. +// DrainResponse is the response to a successful DrainRequest. type DrainResponse struct { - On []int32 `protobuf:"varint,1,rep,packed,name=on,proto3" json:"on,omitempty"` + // deprecated_drain_status works as follows: + // - if the server is NOT currently draining, + // it will be set to an empty or nil slice. + // - a non-nil slice indicates the server is currently + // draining. + // + // This field exists for the benefit of 19.x clients + // and will be removed in 20.2. + DeprecatedDrainStatus []int32 `protobuf:"varint,1,rep,packed,name=deprecated_drain_status,json=deprecatedDrainStatus,proto3" json:"deprecated_drain_status,omitempty"` + // is_draining is set to true iff the server is currently draining. + // This is set to true in response to a request where skip_drain + // is false; but it can also be set to true in response + // to a probe request (!shutdown && skip_drain) if another + // drain request has been issued prior or asynchronously. + IsDraining bool `protobuf:"varint,2,opt,name=is_draining,json=isDraining,proto3" json:"is_draining,omitempty"` + // drain_remaining_indicator measures, at the time of starting to + // process the corresponding drain request, how many actions to + // fully drain the node were deemed to be necessary. Some, but not + // all, of these actions may already have been carried out by the + // time this indicator is received by the client. The client should + // issue requests until this indicator first reaches zero, which + // indicates that the node is fully drained. + // + // The API contract is the following: + // + // - upon a first Drain call with do_drain set, the remaining + // indicator will have some value >=0. If >0, it indicates that + // drain is pushing state away from the node. (What this state + // precisely means is left unspecified for this field. See below + // for details.) + // + // - upon a subsequent Drain call with do_drain set, the remaining + // indicator should have reduced in value. The drain process does best + // effort at shedding state away from the node; hopefully, all the + // state is shed away upon the first call and the progress + // indicator can be zero as early as the second call. However, + // if there was a lot of state to shed, it is possible for + // timeout to be encountered upon the first call. In that case, the + // second call will do some more work and return a non-zero value + // as well. + // + // - eventually, in an iterated sequence of DrainRequests with + // do_drain set, the remaining indicator should reduce to zero. At + // that point the client can conclude that no state is left to + // shed, and it should be safe to shut down the node with a + // DrainRequest with shutdown = true. + // + // Note that this field is left unpopulated (and thus remains at + // zero) for pre-20.1 nodes. A client can recognize this by + // observing is_draining to be false after a request with do_drain = + // true: the is_draining field is also left unpopulated by pre-20.1 + // nodes. + DrainRemainingIndicator uint64 `protobuf:"varint,3,opt,name=drain_remaining_indicator,json=drainRemainingIndicator,proto3" json:"drain_remaining_indicator,omitempty"` + // drain_remaining_description is an informal (= not + // machine-parsable) string that explains the progress of the drain + // process to human eyes. This is intended for use mainly for + // troubleshooting. + // + // The field is only populated if do_drain is true in the + // request. + DrainRemainingDescription string `protobuf:"bytes,4,opt,name=drain_remaining_description,json=drainRemainingDescription,proto3" json:"drain_remaining_description,omitempty"` } 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_c459f6b997f47a7f, []int{21} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{21} } func (m *DrainResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1280,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_c459f6b997f47a7f, []int{22} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{22} } func (m *DecommissionStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1318,7 +1365,7 @@ 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_c459f6b997f47a7f, []int{23} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{23} } func (m *DecommissionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1353,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_c459f6b997f47a7f, []int{24} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{24} } func (m *DecommissionStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1391,7 +1438,7 @@ func (m *DecommissionStatusResponse_Status) Reset() { *m = DecommissionS func (m *DecommissionStatusResponse_Status) String() string { return proto.CompactTextString(m) } func (*DecommissionStatusResponse_Status) ProtoMessage() {} func (*DecommissionStatusResponse_Status) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_c459f6b997f47a7f, []int{24, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{24, 0} } func (m *DecommissionStatusResponse_Status) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1433,7 +1480,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_c459f6b997f47a7f, []int{25} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{25} } func (m *SettingsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1467,7 +1514,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_c459f6b997f47a7f, []int{26} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{26} } func (m *SettingsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1503,7 +1550,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_c459f6b997f47a7f, []int{26, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{26, 0} } func (m *SettingsResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1556,7 +1603,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_c459f6b997f47a7f, []int{27} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{27} } func (m *HealthRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1590,7 +1637,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_c459f6b997f47a7f, []int{28} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{28} } func (m *HealthResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1623,7 +1670,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_c459f6b997f47a7f, []int{29} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{29} } func (m *LivenessRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1658,7 +1705,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_c459f6b997f47a7f, []int{30} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{30} } func (m *LivenessResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1694,7 +1741,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_c459f6b997f47a7f, []int{31} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{31} } func (m *JobsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1728,7 +1775,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_c459f6b997f47a7f, []int{32} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{32} } func (m *JobsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1781,7 +1828,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_c459f6b997f47a7f, []int{32, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{32, 0} } func (m *JobsResponse_Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1814,7 +1861,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_c459f6b997f47a7f, []int{33} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{33} } func (m *LocationsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1848,7 +1895,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_c459f6b997f47a7f, []int{34} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{34} } func (m *LocationsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1884,7 +1931,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_c459f6b997f47a7f, []int{34, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{34, 0} } func (m *LocationsResponse_Location) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1926,7 +1973,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_c459f6b997f47a7f, []int{35} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{35} } func (m *RangeLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1960,7 +2007,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_c459f6b997f47a7f, []int{36} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{36} } func (m *RangeLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2000,7 +2047,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_c459f6b997f47a7f, []int{36, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{36, 0} } func (m *RangeLogResponse_PrettyInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2034,7 +2081,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_c459f6b997f47a7f, []int{36, 1} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{36, 1} } func (m *RangeLogResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2069,7 +2116,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_c459f6b997f47a7f, []int{37} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{37} } func (m *QueryPlanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2104,7 +2151,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_c459f6b997f47a7f, []int{38} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{38} } func (m *QueryPlanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2136,7 +2183,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_c459f6b997f47a7f, []int{39} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{39} } func (m *DataDistributionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2172,7 +2219,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_c459f6b997f47a7f, []int{40} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{40} } func (m *DataDistributionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2210,7 +2257,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_c459f6b997f47a7f, []int{40, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{40, 0} } func (m *DataDistributionResponse_ZoneConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2245,7 +2292,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_c459f6b997f47a7f, []int{40, 1} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{40, 1} } func (m *DataDistributionResponse_TableInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2279,7 +2326,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_c459f6b997f47a7f, []int{40, 2} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{40, 2} } func (m *DataDistributionResponse_DatabaseInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2312,7 +2359,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_c459f6b997f47a7f, []int{41} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{41} } func (m *MetricMetadataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2346,7 +2393,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_c459f6b997f47a7f, []int{42} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{42} } func (m *MetricMetadataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2389,7 +2436,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_c459f6b997f47a7f, []int{43} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{43} } func (m *EnqueueRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2422,7 +2469,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_c459f6b997f47a7f, []int{44} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{44} } func (m *EnqueueRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2459,7 +2506,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_c459f6b997f47a7f, []int{44, 0} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{44, 0} } func (m *EnqueueRangeResponse_Details) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2492,7 +2539,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_c459f6b997f47a7f, []int{45} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{45} } func (m *ChartCatalogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2526,7 +2573,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_c459f6b997f47a7f, []int{46} + return fileDescriptor_admin_ffd4bb43c44fb7e0, []int{46} } func (m *ChartCatalogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2627,7 +2674,6 @@ func init() { proto.RegisterType((*ChartCatalogRequest)(nil), "cockroach.server.serverpb.ChartCatalogRequest") proto.RegisterType((*ChartCatalogResponse)(nil), "cockroach.server.serverpb.ChartCatalogResponse") proto.RegisterEnum("cockroach.server.serverpb.ZoneConfigurationLevel", ZoneConfigurationLevel_name, ZoneConfigurationLevel_value) - proto.RegisterEnum("cockroach.server.serverpb.DrainMode", DrainMode_name, DrainMode_value) } // Reference imports to suppress errors if they are not otherwise used. @@ -4720,10 +4766,10 @@ func (m *DrainRequest) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.On) > 0 { - dAtA10 := make([]byte, len(m.On)*10) + if len(m.DeprecatedProbeIndicator) > 0 { + dAtA10 := make([]byte, len(m.DeprecatedProbeIndicator)*10) var j9 int - for _, num1 := range m.On { + for _, num1 := range m.DeprecatedProbeIndicator { num := uint64(num1) for num >= 1<<7 { dAtA10[j9] = uint8(uint64(num)&0x7f | 0x80) @@ -4738,24 +4784,6 @@ func (m *DrainRequest) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintAdmin(dAtA, i, uint64(j9)) i += copy(dAtA[i:], dAtA10[:j9]) } - if len(m.Off) > 0 { - dAtA12 := make([]byte, len(m.Off)*10) - var j11 int - for _, num1 := range m.Off { - num := uint64(num1) - for num >= 1<<7 { - dAtA12[j11] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j11++ - } - dAtA12[j11] = uint8(num) - j11++ - } - dAtA[i] = 0x12 - i++ - i = encodeVarintAdmin(dAtA, i, uint64(j11)) - i += copy(dAtA[i:], dAtA12[:j11]) - } if m.Shutdown { dAtA[i] = 0x18 i++ @@ -4766,6 +4794,16 @@ func (m *DrainRequest) MarshalTo(dAtA []byte) (int, error) { } i++ } + if m.DoDrain { + dAtA[i] = 0x20 + i++ + if m.DoDrain { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -4784,23 +4822,44 @@ func (m *DrainResponse) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.On) > 0 { - dAtA14 := make([]byte, len(m.On)*10) - var j13 int - for _, num1 := range m.On { + if len(m.DeprecatedDrainStatus) > 0 { + dAtA12 := make([]byte, len(m.DeprecatedDrainStatus)*10) + var j11 int + for _, num1 := range m.DeprecatedDrainStatus { num := uint64(num1) for num >= 1<<7 { - dAtA14[j13] = uint8(uint64(num)&0x7f | 0x80) + dAtA12[j11] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j13++ + j11++ } - dAtA14[j13] = uint8(num) - j13++ + dAtA12[j11] = uint8(num) + j11++ } dAtA[i] = 0xa i++ - i = encodeVarintAdmin(dAtA, i, uint64(j13)) - i += copy(dAtA[i:], dAtA14[:j13]) + i = encodeVarintAdmin(dAtA, i, uint64(j11)) + i += copy(dAtA[i:], dAtA12[:j11]) + } + if m.IsDraining { + dAtA[i] = 0x10 + i++ + if m.IsDraining { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.DrainRemainingIndicator != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintAdmin(dAtA, i, uint64(m.DrainRemainingIndicator)) + } + if len(m.DrainRemainingDescription) > 0 { + dAtA[i] = 0x22 + i++ + i = encodeVarintAdmin(dAtA, i, uint64(len(m.DrainRemainingDescription))) + i += copy(dAtA[i:], m.DrainRemainingDescription) } return i, nil } @@ -4821,22 +4880,22 @@ func (m *DecommissionStatusRequest) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.NodeIDs) > 0 { - dAtA16 := make([]byte, len(m.NodeIDs)*10) - var j15 int + dAtA14 := make([]byte, len(m.NodeIDs)*10) + var j13 int for _, num1 := range m.NodeIDs { num := uint64(num1) for num >= 1<<7 { - dAtA16[j15] = uint8(uint64(num)&0x7f | 0x80) + dAtA14[j13] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j15++ + j13++ } - dAtA16[j15] = uint8(num) - j15++ + dAtA14[j13] = uint8(num) + j13++ } dAtA[i] = 0xa i++ - i = encodeVarintAdmin(dAtA, i, uint64(j15)) - i += copy(dAtA[i:], dAtA16[:j15]) + i = encodeVarintAdmin(dAtA, i, uint64(j13)) + i += copy(dAtA[i:], dAtA14[:j13]) } return i, nil } @@ -4857,22 +4916,22 @@ func (m *DecommissionRequest) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.NodeIDs) > 0 { - dAtA18 := make([]byte, len(m.NodeIDs)*10) - var j17 int + dAtA16 := make([]byte, len(m.NodeIDs)*10) + var j15 int for _, num1 := range m.NodeIDs { num := uint64(num1) for num >= 1<<7 { - dAtA18[j17] = uint8(uint64(num)&0x7f | 0x80) + dAtA16[j15] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j17++ + j15++ } - dAtA18[j17] = uint8(num) - j17++ + dAtA16[j15] = uint8(num) + j15++ } dAtA[i] = 0xa i++ - i = encodeVarintAdmin(dAtA, i, uint64(j17)) - i += copy(dAtA[i:], dAtA18[:j17]) + i = encodeVarintAdmin(dAtA, i, uint64(j15)) + i += copy(dAtA[i:], dAtA16[:j15]) } if m.Decommissioning { dAtA[i] = 0x10 @@ -5057,11 +5116,11 @@ func (m *SettingsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintAdmin(dAtA, i, uint64((&v).Size())) - n19, err := (&v).MarshalTo(dAtA[i:]) + n17, err := (&v).MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n19 + i += n17 } } return i, nil @@ -5330,21 +5389,21 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], m.Username) } if len(m.DescriptorIDs) > 0 { - dAtA21 := make([]byte, len(m.DescriptorIDs)*10) - var j20 int + dAtA19 := make([]byte, len(m.DescriptorIDs)*10) + var j18 int for _, num := range m.DescriptorIDs { for num >= 1<<7 { - dAtA21[j20] = uint8(uint64(num)&0x7f | 0x80) + dAtA19[j18] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j20++ + j18++ } - dAtA21[j20] = uint8(num) - j20++ + dAtA19[j18] = uint8(num) + j18++ } dAtA[i] = 0x2a i++ - i = encodeVarintAdmin(dAtA, i, uint64(j20)) - i += copy(dAtA[i:], dAtA21[:j20]) + i = encodeVarintAdmin(dAtA, i, uint64(j18)) + i += copy(dAtA[i:], dAtA19[:j18]) } if len(m.Status) > 0 { dAtA[i] = 0x32 @@ -5356,41 +5415,41 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Created))) - n22, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Created, dAtA[i:]) + n20, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Created, dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n20 } if m.Started != nil { dAtA[i] = 0x42 i++ i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Started))) - n23, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Started, dAtA[i:]) + n21, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Started, dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n21 } if m.Finished != nil { dAtA[i] = 0x4a i++ i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Finished))) - n24, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Finished, dAtA[i:]) + n22, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Finished, dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n22 } if m.Modified != nil { dAtA[i] = 0x52 i++ i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Modified))) - n25, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Modified, dAtA[i:]) + n23, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Modified, dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n23 } if m.FractionCompleted != 0 { dAtA[i] = 0x5d @@ -5408,11 +5467,11 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.HighwaterTimestamp))) - n26, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.HighwaterTimestamp, dAtA[i:]) + n24, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.HighwaterTimestamp, dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n24 } if len(m.HighwaterDecimal) > 0 { dAtA[i] = 0x72 @@ -5657,19 +5716,19 @@ func (m *RangeLogResponse_Event) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintAdmin(dAtA, i, uint64(m.Event.Size())) - n27, err := m.Event.MarshalTo(dAtA[i:]) + n25, err := m.Event.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n25 dAtA[i] = 0x12 i++ i = encodeVarintAdmin(dAtA, i, uint64(m.PrettyInfo.Size())) - n28, err := m.PrettyInfo.MarshalTo(dAtA[i:]) + n26, err := m.PrettyInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n26 return i, nil } @@ -5778,11 +5837,11 @@ func (m *DataDistributionResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintAdmin(dAtA, i, uint64((&v).Size())) - n29, err := (&v).MarshalTo(dAtA[i:]) + n27, err := (&v).MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n27 } } if len(m.ZoneConfigs) > 0 { @@ -5809,11 +5868,11 @@ func (m *DataDistributionResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintAdmin(dAtA, i, uint64((&v).Size())) - n30, err := (&v).MarshalTo(dAtA[i:]) + n28, err := (&v).MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n30 + i += n28 } } return i, nil @@ -5843,11 +5902,11 @@ func (m *DataDistributionResponse_ZoneConfig) MarshalTo(dAtA []byte) (int, error dAtA[i] = 0x12 i++ i = encodeVarintAdmin(dAtA, i, uint64(m.Config.Size())) - n31, err := m.Config.MarshalTo(dAtA[i:]) + n29, err := m.Config.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n29 if len(m.ConfigSQL) > 0 { dAtA[i] = 0x22 i++ @@ -5901,11 +5960,11 @@ func (m *DataDistributionResponse_TableInfo) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0x1a i++ i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.DroppedAt))) - n32, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.DroppedAt, dAtA[i:]) + n30, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.DroppedAt, dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n30 } return i, nil } @@ -5949,11 +6008,11 @@ func (m *DataDistributionResponse_DatabaseInfo) MarshalTo(dAtA []byte) (int, err dAtA[i] = 0x12 i++ i = encodeVarintAdmin(dAtA, i, uint64((&v).Size())) - n33, err := (&v).MarshalTo(dAtA[i:]) + n31, err := (&v).MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n31 } } return i, nil @@ -6016,11 +6075,11 @@ func (m *MetricMetadataResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintAdmin(dAtA, i, uint64((&v).Size())) - n34, err := (&v).MarshalTo(dAtA[i:]) + n32, err := (&v).MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n32 } } return i, nil @@ -6729,16 +6788,9 @@ func (m *DrainRequest) Size() (n int) { } var l int _ = l - if len(m.On) > 0 { - l = 0 - for _, e := range m.On { - l += sovAdmin(uint64(e)) - } - n += 1 + sovAdmin(uint64(l)) + l - } - if len(m.Off) > 0 { + if len(m.DeprecatedProbeIndicator) > 0 { l = 0 - for _, e := range m.Off { + for _, e := range m.DeprecatedProbeIndicator { l += sovAdmin(uint64(e)) } n += 1 + sovAdmin(uint64(l)) + l @@ -6746,6 +6798,9 @@ func (m *DrainRequest) Size() (n int) { if m.Shutdown { n += 2 } + if m.DoDrain { + n += 2 + } return n } @@ -6755,13 +6810,23 @@ func (m *DrainResponse) Size() (n int) { } var l int _ = l - if len(m.On) > 0 { + if len(m.DeprecatedDrainStatus) > 0 { l = 0 - for _, e := range m.On { + for _, e := range m.DeprecatedDrainStatus { l += sovAdmin(uint64(e)) } n += 1 + sovAdmin(uint64(l)) + l } + if m.IsDraining { + n += 2 + } + if m.DrainRemainingIndicator != 0 { + n += 1 + sovAdmin(uint64(m.DrainRemainingIndicator)) + } + l = len(m.DrainRemainingDescription) + if l > 0 { + n += 1 + l + sovAdmin(uint64(l)) + } return n } @@ -10821,7 +10886,7 @@ func (m *DrainRequest) Unmarshal(dAtA []byte) error { break } } - m.On = append(m.On, v) + m.DeprecatedProbeIndicator = append(m.DeprecatedProbeIndicator, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -10853,8 +10918,8 @@ func (m *DrainRequest) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.On) == 0 { - m.On = make([]int32, 0, elementCount) + if elementCount != 0 && len(m.DeprecatedProbeIndicator) == 0 { + m.DeprecatedProbeIndicator = make([]int32, 0, elementCount) } for iNdEx < postIndex { var v int32 @@ -10872,87 +10937,34 @@ func (m *DrainRequest) Unmarshal(dAtA []byte) error { break } } - m.On = append(m.On, v) + m.DeprecatedProbeIndicator = append(m.DeprecatedProbeIndicator, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field On", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedProbeIndicator", wireType) } - case 2: - if wireType == 0 { - var v int32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowAdmin - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int32(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.Off = append(m.Off, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowAdmin - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthAdmin + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Shutdown", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAdmin } - postIndex := iNdEx + packedLen - if postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - var elementCount int - var count int - for _, integer := range dAtA { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.Off) == 0 { - m.Off = make([]int32, 0, elementCount) - } - for iNdEx < postIndex { - var v int32 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowAdmin - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int32(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.Off = append(m.Off, v) + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field Off", wireType) } - case 3: + m.Shutdown = bool(v != 0) + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Shutdown", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DoDrain", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -10969,7 +10981,7 @@ func (m *DrainRequest) Unmarshal(dAtA []byte) error { break } } - m.Shutdown = bool(v != 0) + m.DoDrain = bool(v != 0) default: iNdEx = preIndex skippy, err := skipAdmin(dAtA[iNdEx:]) @@ -11037,7 +11049,7 @@ func (m *DrainResponse) Unmarshal(dAtA []byte) error { break } } - m.On = append(m.On, v) + m.DeprecatedDrainStatus = append(m.DeprecatedDrainStatus, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -11069,8 +11081,8 @@ func (m *DrainResponse) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.On) == 0 { - m.On = make([]int32, 0, elementCount) + if elementCount != 0 && len(m.DeprecatedDrainStatus) == 0 { + m.DeprecatedDrainStatus = make([]int32, 0, elementCount) } for iNdEx < postIndex { var v int32 @@ -11088,11 +11100,79 @@ func (m *DrainResponse) Unmarshal(dAtA []byte) error { break } } - m.On = append(m.On, v) + m.DeprecatedDrainStatus = append(m.DeprecatedDrainStatus, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field On", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedDrainStatus", wireType) + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsDraining", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAdmin + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } } + m.IsDraining = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DrainRemainingIndicator", wireType) + } + m.DrainRemainingIndicator = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAdmin + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DrainRemainingIndicator |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DrainRemainingDescription", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAdmin + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthAdmin + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DrainRemainingDescription = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipAdmin(dAtA[iNdEx:]) @@ -15716,264 +15796,268 @@ var ( ErrIntOverflowAdmin = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_c459f6b997f47a7f) } - -var fileDescriptor_admin_c459f6b997f47a7f = []byte{ - // 4084 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x4b, 0x70, 0x1b, 0x47, - 0x76, 0x1c, 0x80, 0xf8, 0x3d, 0x02, 0x24, 0xd8, 0xa2, 0x28, 0x08, 0xd2, 0x12, 0xf4, 0xc8, 0x5e, - 0xd3, 0xb2, 0x17, 0xb0, 0x28, 0xd9, 0xde, 0xc8, 0x76, 0x36, 0x04, 0xc9, 0xa2, 0x21, 0x53, 0xb2, - 0x35, 0xa4, 0xe4, 0x5a, 0x6d, 0xbc, 0x93, 0x01, 0xa6, 0x09, 0xce, 0x72, 0x30, 0x33, 0x9c, 0x19, - 0x50, 0xa6, 0x9d, 0xdd, 0xca, 0xee, 0x26, 0xa9, 0x9c, 0x52, 0xae, 0xdd, 0xdc, 0x5c, 0x95, 0xaa, - 0xe4, 0x90, 0x1c, 0x72, 0x89, 0x73, 0xcc, 0x21, 0x67, 0x57, 0x0e, 0xd9, 0x4d, 0xe5, 0xb2, 0xb9, - 0x70, 0x13, 0x2a, 0x87, 0xd4, 0xe6, 0x9e, 0x43, 0x2e, 0x49, 0xf5, 0x6f, 0xa6, 0x01, 0x42, 0x23, - 0x80, 0xda, 0xcd, 0x81, 0xc4, 0xf4, 0xeb, 0xee, 0xd7, 0xaf, 0xdf, 0x7b, 0xfd, 0xfa, 0x7d, 0x1a, - 0xae, 0x04, 0xd8, 0x3f, 0xc2, 0x7e, 0x83, 0xfd, 0x78, 0xed, 0x86, 0x61, 0xf6, 0x2c, 0xa7, 0xee, - 0xf9, 0x6e, 0xe8, 0xa2, 0xcb, 0x1d, 0xb7, 0x73, 0xe0, 0xbb, 0x46, 0x67, 0xbf, 0xce, 0xfa, 0xeb, - 0x62, 0x58, 0xb5, 0xd2, 0x71, 0x9d, 0x3d, 0xab, 0xdb, 0xf8, 0xd4, 0x75, 0xb0, 0xd7, 0xa6, 0x3f, - 0x6c, 0x52, 0x75, 0xf1, 0x7b, 0x6e, 0x3b, 0x68, 0x90, 0x7f, 0x5e, 0x9b, 0xfe, 0x70, 0xf8, 0xd5, - 0xe1, 0x95, 0x82, 0xd0, 0x08, 0xfb, 0xa2, 0xf7, 0x4a, 0x10, 0xba, 0xbe, 0xd1, 0xc5, 0x0d, 0xec, - 0x74, 0x2d, 0x8a, 0xb2, 0x77, 0xd4, 0xe9, 0xf0, 0xce, 0x17, 0x0f, 0x8e, 0x1a, 0x07, 0x47, 0x62, - 0x3e, 0x1b, 0xe8, 0xb5, 0x1b, 0xb6, 0x75, 0x84, 0x1d, 0x1c, 0x08, 0x14, 0xb5, 0xa7, 0x8c, 0x72, - 0xbb, 0x7c, 0xc0, 0x52, 0x18, 0x34, 0x3a, 0x46, 0x68, 0xd8, 0x6e, 0xb7, 0xd1, 0xd9, 0x37, 0xfc, - 0x50, 0xe7, 0x2d, 0xde, 0x5f, 0xe9, 0x87, 0x96, 0xdd, 0xe8, 0xe1, 0xd0, 0xb7, 0x3a, 0xfc, 0x87, - 0xf7, 0x2c, 0x74, 0xdd, 0xae, 0x4b, 0x3f, 0x1b, 0xe4, 0x4b, 0xec, 0xa8, 0xeb, 0xba, 0x5d, 0x1b, - 0x37, 0x0c, 0xcf, 0x6a, 0x18, 0x8e, 0xe3, 0x86, 0x46, 0x68, 0xb9, 0x4e, 0x44, 0x0e, 0xef, 0xa5, - 0xad, 0x76, 0x7f, 0xaf, 0x11, 0x5a, 0x3d, 0x1c, 0x84, 0x46, 0xcf, 0x63, 0x03, 0x54, 0x04, 0xe5, - 0x0d, 0x23, 0x34, 0xda, 0x46, 0x80, 0x03, 0x0d, 0x1f, 0xf6, 0x71, 0x10, 0xaa, 0x37, 0x60, 0x5e, - 0x82, 0x05, 0x9e, 0xeb, 0x04, 0x18, 0x5d, 0x85, 0x82, 0x29, 0x80, 0x15, 0x65, 0x39, 0xbd, 0x52, - 0xd0, 0x62, 0x80, 0x7a, 0x0b, 0x16, 0xc5, 0x94, 0x0d, 0x1c, 0x1a, 0x96, 0x2d, 0x90, 0xa1, 0x2a, - 0xe4, 0xc5, 0xb0, 0x8a, 0xb2, 0xac, 0xac, 0x14, 0xb4, 0xa8, 0xad, 0x7e, 0x99, 0x86, 0x4b, 0x67, - 0xa6, 0xf1, 0xf5, 0x1e, 0x42, 0xb6, 0xeb, 0x1b, 0x4e, 0xc8, 0x16, 0x9b, 0x59, 0xfd, 0x66, 0xfd, - 0xa9, 0x7a, 0x50, 0x7f, 0x0a, 0x8e, 0xfa, 0x16, 0x41, 0xd0, 0x9c, 0xfe, 0xea, 0xa4, 0x36, 0xa5, - 0x71, 0x6c, 0xa8, 0x06, 0x33, 0xa1, 0xd1, 0xb6, 0xb1, 0xee, 0x18, 0x3d, 0x1c, 0x54, 0x52, 0x74, - 0x27, 0x40, 0x41, 0xf7, 0x08, 0x04, 0xbd, 0x01, 0x25, 0x13, 0x07, 0x1d, 0xdf, 0xf2, 0x42, 0xd7, - 0xd7, 0x2d, 0xb3, 0x92, 0x5e, 0x56, 0x56, 0xd2, 0xcd, 0xf2, 0xe9, 0x49, 0xad, 0xb8, 0x11, 0x75, - 0xb4, 0x36, 0xb4, 0x62, 0x3c, 0xac, 0x65, 0xa2, 0x3b, 0x30, 0x43, 0xf4, 0x4f, 0x67, 0x2a, 0x59, - 0x99, 0x5e, 0x56, 0x56, 0x66, 0x56, 0xaf, 0x49, 0x44, 0xb3, 0x8e, 0x3a, 0xd3, 0xd5, 0xfa, 0x23, - 0xd7, 0xc1, 0xeb, 0x14, 0xc2, 0xe9, 0x83, 0x4f, 0x23, 0x08, 0xfa, 0x18, 0xe6, 0x25, 0x5c, 0xba, - 0x8d, 0x8f, 0xb0, 0x5d, 0xc9, 0x2c, 0x2b, 0x2b, 0xb3, 0xab, 0x37, 0x12, 0xd8, 0x10, 0xe3, 0xec, - 0xfb, 0x54, 0x0b, 0xb6, 0xc9, 0x44, 0x6d, 0x2e, 0xc6, 0x4c, 0x01, 0xd5, 0xb7, 0x21, 0x43, 0x39, - 0x83, 0x10, 0x4c, 0xf7, 0x03, 0xec, 0x73, 0xb9, 0xd0, 0x6f, 0xb4, 0x04, 0xe0, 0xf9, 0xd6, 0x91, - 0x65, 0xe3, 0x6e, 0xcc, 0x9e, 0x18, 0xa2, 0x6e, 0xc1, 0x85, 0x5d, 0xc2, 0xac, 0xf1, 0xc5, 0x8c, - 0x16, 0x20, 0x43, 0xf9, 0x5b, 0x49, 0xd1, 0x0e, 0xd6, 0x50, 0xff, 0x28, 0x0f, 0x0b, 0x83, 0x98, - 0xb8, 0xe4, 0x77, 0x86, 0x24, 0xff, 0x46, 0xc2, 0x96, 0x47, 0x21, 0x18, 0x29, 0xf6, 0x87, 0x90, - 0xeb, 0xb8, 0x76, 0xbf, 0xe7, 0xb0, 0x3d, 0xcd, 0xac, 0xbe, 0x39, 0x29, 0xd6, 0x75, 0x3a, 0x9d, - 0xa3, 0x15, 0xc8, 0xd0, 0x03, 0xc8, 0x59, 0x8e, 0x89, 0x3f, 0xc1, 0x41, 0x25, 0x7d, 0x3e, 0x6a, - 0x5b, 0x64, 0xba, 0x40, 0xcb, 0x71, 0x11, 0x2d, 0xf5, 0x0d, 0xa7, 0x4b, 0x54, 0xa0, 0xef, 0x84, - 0x54, 0x9b, 0xd2, 0x1a, 0x50, 0xd0, 0x3a, 0x81, 0xa0, 0x5b, 0xb0, 0xd8, 0xf1, 0xb1, 0x11, 0x62, - 0x9d, 0x69, 0x33, 0xb1, 0x63, 0xb8, 0x87, 0x9d, 0x90, 0xea, 0x49, 0x41, 0x5b, 0x60, 0xbd, 0x74, - 0xc5, 0x1d, 0xd1, 0x37, 0xac, 0xa4, 0xd9, 0x5f, 0xbb, 0x92, 0xe6, 0x7e, 0x5d, 0x4a, 0x7a, 0xf6, - 0x18, 0xe6, 0xc7, 0x39, 0x86, 0xcf, 0xa5, 0xdb, 0xd5, 0x7f, 0x50, 0x20, 0xcb, 0xc4, 0x4c, 0xa6, - 0x13, 0x03, 0x21, 0xa6, 0x93, 0x6f, 0x02, 0x0b, 0x8f, 0x3d, 0xa1, 0xc6, 0xf4, 0x9b, 0xe8, 0xbd, - 0xd3, 0xb7, 0x6d, 0xaa, 0xde, 0xc4, 0x50, 0xe4, 0xb5, 0xa8, 0x8d, 0xae, 0x91, 0x2d, 0xec, 0x19, - 0x7d, 0x3b, 0xd4, 0x8f, 0x0c, 0xbb, 0x8f, 0xa9, 0x18, 0x0b, 0x84, 0x60, 0x0a, 0x7c, 0x48, 0x60, - 0xe8, 0x26, 0x5c, 0xec, 0x62, 0x07, 0x33, 0x5e, 0xe8, 0xf8, 0x13, 0xcf, 0xc7, 0x41, 0x60, 0xb9, - 0x8e, 0x90, 0x63, 0xdc, 0xb9, 0x19, 0xf5, 0xa1, 0x45, 0xc8, 0xee, 0x5b, 0xa6, 0x89, 0x1d, 0x2a, - 0xc2, 0xbc, 0xc6, 0x5b, 0xd5, 0x2f, 0x15, 0xc8, 0x50, 0x7d, 0x1a, 0x49, 0xff, 0x22, 0x64, 0xfb, - 0x8e, 0x75, 0xd8, 0x67, 0x3b, 0xc8, 0x6b, 0xbc, 0x85, 0xca, 0x90, 0x0e, 0xf0, 0x21, 0xb3, 0x73, - 0x1a, 0xf9, 0x24, 0x23, 0x99, 0x82, 0x73, 0x92, 0x79, 0x8b, 0x5e, 0x02, 0x96, 0x8f, 0x3b, 0x61, - 0x4c, 0x60, 0x0c, 0x40, 0x15, 0xc8, 0x91, 0x1b, 0xcf, 0x72, 0xba, 0x9c, 0x2c, 0xd1, 0x24, 0x5c, - 0xb2, 0x7a, 0x9e, 0x6d, 0x75, 0xac, 0x90, 0xaa, 0x48, 0x5e, 0x8b, 0xda, 0xea, 0x26, 0xcc, 0x47, - 0x5a, 0xfa, 0x1c, 0xe6, 0xe4, 0xcb, 0x34, 0x20, 0x19, 0x0f, 0x37, 0x26, 0x43, 0x07, 0x49, 0x39, - 0x73, 0x90, 0xae, 0x41, 0xc9, 0xc7, 0x84, 0x14, 0x83, 0x0f, 0x49, 0xd1, 0x21, 0x45, 0x0e, 0x64, - 0x83, 0xbe, 0x06, 0xe0, 0xb8, 0xa6, 0x40, 0xc2, 0x18, 0x55, 0x20, 0x10, 0xd6, 0xbd, 0x06, 0x19, - 0x72, 0xfe, 0x02, 0x6e, 0xf5, 0x5f, 0x92, 0xd5, 0x9f, 0xb9, 0x00, 0x75, 0xe1, 0x51, 0xd4, 0xef, - 0x3e, 0x5c, 0x5f, 0xa7, 0x24, 0xf2, 0x23, 0xc5, 0x66, 0x22, 0x13, 0x4a, 0x3d, 0x2b, 0x08, 0x2c, - 0xa7, 0xab, 0x13, 0xbc, 0x41, 0x25, 0x43, 0xad, 0xc9, 0x6f, 0x3d, 0xcb, 0x9a, 0x0c, 0xec, 0xb6, - 0x7e, 0x97, 0xa1, 0xb8, 0xe7, 0x9a, 0x98, 0xa3, 0x2f, 0xf6, 0x62, 0x50, 0x40, 0xac, 0x86, 0xe1, - 0x79, 0xbe, 0xfb, 0x89, 0xd5, 0x23, 0xa6, 0xc3, 0xb4, 0x82, 0x03, 0xbd, 0x7d, 0x1c, 0xe2, 0x80, - 0x0a, 0x6c, 0x5a, 0x5b, 0x90, 0x7a, 0x37, 0xac, 0xe0, 0xa0, 0x49, 0xfa, 0xaa, 0x1f, 0xc1, 0x8c, - 0x84, 0x18, 0x5d, 0x83, 0x1c, 0x65, 0x86, 0x65, 0x32, 0xd1, 0x34, 0xe1, 0xf4, 0xa4, 0x96, 0x25, - 0x5d, 0xad, 0x0d, 0x2d, 0x4b, 0xba, 0x5a, 0x26, 0x61, 0x2b, 0xf6, 0x7d, 0xd7, 0xd7, 0x7b, 0x38, - 0x08, 0x8c, 0xae, 0x10, 0x56, 0x91, 0x02, 0xef, 0x32, 0x98, 0xba, 0x08, 0x0b, 0xf7, 0x5c, 0xe7, - 0x8c, 0xf4, 0xd5, 0x9f, 0x29, 0x70, 0x71, 0xa8, 0x83, 0x8b, 0xf3, 0xdb, 0x30, 0x4f, 0x3c, 0x18, - 0x3d, 0xc0, 0xbe, 0x85, 0x03, 0x9d, 0x71, 0x5d, 0xa1, 0x5c, 0xff, 0xc6, 0x44, 0xac, 0xd2, 0xe6, - 0x08, 0x9e, 0x1d, 0x8a, 0x86, 0x76, 0xa0, 0xef, 0x00, 0xb2, 0x9c, 0x10, 0xfb, 0x8e, 0x61, 0xeb, - 0xfd, 0x00, 0x73, 0xdc, 0xa9, 0xf3, 0xe0, 0x2e, 0x0b, 0x44, 0x0f, 0x02, 0xd6, 0xa3, 0xce, 0x42, - 0xf1, 0x41, 0x80, 0xfd, 0x68, 0x87, 0x3f, 0x80, 0x12, 0x6f, 0xf3, 0x8d, 0xb5, 0x20, 0x43, 0x4c, - 0x94, 0xb8, 0xf3, 0x92, 0x16, 0x1c, 0x98, 0x48, 0x5b, 0x42, 0x95, 0x28, 0x86, 0xaa, 0x0a, 0xd3, - 0x04, 0x48, 0xce, 0x10, 0x01, 0x48, 0x66, 0x20, 0x6a, 0xab, 0x3f, 0x56, 0xa0, 0xb4, 0x79, 0x84, - 0x9d, 0xf8, 0xc4, 0x09, 0xe3, 0xa6, 0x48, 0xc6, 0xed, 0x0a, 0x14, 0x42, 0xc3, 0xef, 0xe2, 0x90, - 0xc8, 0x9a, 0x9d, 0x8b, 0x3c, 0x03, 0xb4, 0x4c, 0x72, 0x0c, 0x6d, 0xab, 0x67, 0xb1, 0xe3, 0x90, - 0xd1, 0x58, 0x03, 0xbd, 0x0a, 0xf3, 0x7d, 0xc7, 0xc7, 0xa6, 0xd1, 0x09, 0xb1, 0xa9, 0x63, 0xba, - 0x04, 0x3d, 0x16, 0x79, 0xad, 0x1c, 0x77, 0xb0, 0xa5, 0xd5, 0x5f, 0xa5, 0x60, 0x56, 0x50, 0xc1, - 0xf9, 0x70, 0x17, 0xb2, 0x7c, 0x12, 0x63, 0x44, 0x23, 0x81, 0x11, 0x83, 0x53, 0x59, 0x53, 0x5c, - 0xfb, 0x0c, 0x49, 0xf5, 0x4f, 0x52, 0x90, 0xa1, 0x70, 0xd4, 0x84, 0x42, 0xe4, 0xfb, 0x72, 0x8d, - 0xa9, 0xd6, 0x99, 0x77, 0x5c, 0x17, 0xde, 0x71, 0x7d, 0x57, 0x8c, 0x68, 0xe6, 0x09, 0x9a, 0xcf, - 0x7f, 0x59, 0x53, 0xb4, 0x78, 0x1a, 0x31, 0x03, 0x14, 0xaf, 0x2e, 0x5d, 0x03, 0x05, 0x0a, 0xd9, - 0x25, 0xec, 0x7a, 0x45, 0x66, 0x17, 0xf3, 0x1a, 0x8b, 0xa7, 0x27, 0xb5, 0xfc, 0x2e, 0x63, 0xd9, - 0x86, 0xc4, 0xbc, 0x55, 0x20, 0x06, 0xc6, 0xf5, 0x43, 0x72, 0xe0, 0x2d, 0x93, 0x5d, 0xf0, 0xcd, - 0xb9, 0xd3, 0x93, 0xda, 0x8c, 0x26, 0xe0, 0xad, 0x0d, 0x6d, 0x26, 0x1a, 0xd4, 0x32, 0x89, 0x84, - 0x2c, 0x67, 0xcf, 0xe5, 0x76, 0x97, 0x7e, 0x93, 0x25, 0x99, 0x11, 0x27, 0x48, 0xc8, 0x19, 0x2e, - 0xb2, 0x25, 0x1f, 0x50, 0x20, 0x59, 0x92, 0x75, 0xb7, 0x4c, 0xf5, 0x6f, 0x14, 0x28, 0xef, 0xe0, - 0xf0, 0x41, 0x8b, 0x78, 0xcb, 0x42, 0xea, 0xdf, 0x06, 0x38, 0xc0, 0xc7, 0xec, 0x7a, 0x12, 0x2c, - 0xbf, 0x9d, 0xc0, 0xf2, 0x61, 0x04, 0xf5, 0xf7, 0xf1, 0x31, 0xbd, 0xc7, 0x82, 0x4d, 0x27, 0xf4, - 0x8f, 0xb5, 0xc2, 0x81, 0x68, 0x57, 0xdf, 0x81, 0xd9, 0xc1, 0x4e, 0x72, 0xcf, 0x1c, 0xe0, 0x63, - 0xae, 0x61, 0xe4, 0x93, 0xe8, 0x10, 0xbb, 0x19, 0x09, 0x2f, 0x8b, 0x1a, 0x6b, 0xdc, 0x4e, 0x7d, - 0x53, 0x51, 0x2f, 0xc0, 0xbc, 0xb4, 0x16, 0x93, 0xb0, 0xfa, 0x75, 0x28, 0x6f, 0x0d, 0xef, 0x00, - 0xc1, 0xf4, 0x01, 0x3e, 0x16, 0x21, 0x09, 0xfd, 0x56, 0x7f, 0x96, 0x82, 0xf9, 0xad, 0xe1, 0xd9, - 0xe8, 0xf7, 0x46, 0xec, 0xf5, 0xed, 0x84, 0xbd, 0x9e, 0xc1, 0x30, 0xb4, 0x59, 0xae, 0x6a, 0xd2, - 0x96, 0xf7, 0x20, 0xc3, 0x2e, 0xf5, 0x68, 0x5f, 0x8a, 0xb4, 0x2f, 0xb4, 0x05, 0x45, 0xdb, 0x08, - 0x42, 0xbd, 0xef, 0x99, 0x46, 0x88, 0x4d, 0x6e, 0x5b, 0xc6, 0xd3, 0xc2, 0x19, 0x32, 0xf3, 0x01, - 0x9b, 0x58, 0xf5, 0xc6, 0x60, 0xed, 0x7b, 0x32, 0x6b, 0x67, 0x56, 0x57, 0x27, 0xda, 0x28, 0x45, - 0x2d, 0x8b, 0xa3, 0x0c, 0xb3, 0xeb, 0x76, 0x3f, 0x08, 0xb1, 0x2f, 0x2c, 0xd8, 0x17, 0x0a, 0xcc, - 0x45, 0x20, 0xce, 0xe1, 0xd7, 0x00, 0x3a, 0x0c, 0x14, 0x5f, 0x0e, 0xa5, 0xd3, 0x93, 0x5a, 0x81, - 0x0f, 0x6c, 0x6d, 0x68, 0x05, 0x3e, 0xa0, 0x65, 0x12, 0x53, 0x11, 0x9f, 0x01, 0xec, 0x10, 0x33, - 0x6a, 0x72, 0xcf, 0xa4, 0x1c, 0x75, 0x6c, 0x32, 0x38, 0xfa, 0x06, 0x20, 0x4c, 0x8c, 0xaa, 0xe7, - 0x5b, 0x01, 0x8e, 0x46, 0x33, 0x8f, 0x6b, 0x3e, 0xee, 0xe1, 0xc3, 0xd5, 0x6d, 0x28, 0x6e, 0xf8, - 0x86, 0xe5, 0x08, 0x2d, 0x99, 0x85, 0x94, 0xeb, 0x50, 0x99, 0x67, 0xb4, 0x94, 0xeb, 0x10, 0x7e, - 0xb9, 0x7b, 0x7b, 0xd4, 0x05, 0xcc, 0x68, 0xe4, 0x93, 0x58, 0xcb, 0x60, 0xbf, 0x1f, 0x9a, 0xee, - 0x63, 0x47, 0x38, 0x72, 0xa2, 0xad, 0xd6, 0xa0, 0xc4, 0xb1, 0xf1, 0x8d, 0x0e, 0xa1, 0x53, 0x3f, - 0x85, 0xcb, 0x1b, 0xb8, 0xe3, 0xf6, 0xe8, 0x65, 0xeb, 0x3a, 0x3b, 0x34, 0xa9, 0x20, 0xd6, 0xfe, - 0x18, 0xf2, 0xfc, 0xbe, 0x64, 0x5a, 0x97, 0x69, 0x36, 0x4f, 0x4f, 0x6a, 0x39, 0x76, 0x61, 0x06, - 0xff, 0x73, 0x52, 0xbb, 0xd9, 0xb5, 0xc2, 0xfd, 0x7e, 0xbb, 0xde, 0x71, 0x7b, 0x8d, 0x48, 0x4a, - 0x66, 0x3b, 0xfe, 0x6e, 0x78, 0x07, 0xdd, 0x06, 0xfd, 0xf2, 0xda, 0x75, 0x7e, 0xd1, 0xe6, 0xd8, - 0x45, 0x1b, 0xa8, 0x7f, 0xae, 0xc0, 0x05, 0x79, 0xf1, 0xff, 0x9f, 0x65, 0xd1, 0x0a, 0xcc, 0x99, - 0xd2, 0xaa, 0xc4, 0xe9, 0x63, 0xb2, 0x1b, 0x06, 0xab, 0x4f, 0x52, 0x50, 0x1d, 0xc5, 0x1d, 0xce, - 0xcb, 0x47, 0x90, 0x65, 0x49, 0x18, 0x1e, 0x98, 0xbd, 0x93, 0x14, 0xe8, 0x3f, 0x15, 0x4d, 0x9d, - 0x35, 0x85, 0xf9, 0x67, 0x18, 0xab, 0xff, 0xa9, 0x40, 0x96, 0x75, 0xa0, 0x47, 0x83, 0x5e, 0x4b, - 0xa6, 0xb9, 0x16, 0x7b, 0x2d, 0xe7, 0x65, 0x86, 0x70, 0x76, 0x2e, 0x41, 0xce, 0x0a, 0x74, 0xdb, - 0x3a, 0x8a, 0x3c, 0x6b, 0x2b, 0xd8, 0xb6, 0x8e, 0xf0, 0x59, 0xe7, 0x32, 0x3d, 0xc2, 0xb9, 0x1c, - 0xc1, 0xc9, 0xe9, 0x91, 0x9c, 0xa4, 0x5e, 0x31, 0xd1, 0x43, 0x32, 0x24, 0xc3, 0x74, 0x54, 0xb4, - 0x55, 0x0d, 0xe6, 0x76, 0x70, 0x48, 0x8e, 0x4c, 0x90, 0x60, 0x1a, 0x87, 0xee, 0x67, 0x6e, 0x0b, - 0x53, 0xc3, 0xf7, 0x33, 0x33, 0x2d, 0xea, 0x3f, 0xa7, 0xe8, 0x95, 0xc1, 0x91, 0x72, 0x79, 0xe9, - 0x93, 0x5f, 0x19, 0x03, 0x08, 0x9e, 0x69, 0x45, 0x0f, 0x46, 0x5a, 0xd1, 0x82, 0xb0, 0xa2, 0xa3, - 0xa2, 0xb0, 0x65, 0x98, 0x11, 0x51, 0x20, 0x89, 0x4c, 0xd2, 0xb4, 0x4b, 0x06, 0x91, 0x88, 0xc6, - 0xeb, 0xb7, 0x6d, 0xab, 0xc3, 0x79, 0xcb, 0x5b, 0x55, 0x77, 0x0c, 0x53, 0xba, 0x35, 0x68, 0x4a, - 0x6f, 0x4c, 0xb2, 0xd9, 0x33, 0x96, 0xf4, 0x25, 0x28, 0xbd, 0x87, 0x0d, 0x3b, 0xdc, 0x17, 0x52, - 0x5a, 0x80, 0x8c, 0x8f, 0x0d, 0x93, 0xad, 0x98, 0xd7, 0x58, 0x83, 0x18, 0x5c, 0x31, 0x8c, 0x5f, - 0x7e, 0xf3, 0x30, 0xb7, 0xcd, 0x73, 0x8d, 0xc2, 0x06, 0xff, 0x2a, 0x05, 0xe5, 0x18, 0xc6, 0xe5, - 0xf3, 0x01, 0x80, 0xc8, 0x49, 0x46, 0xf2, 0x79, 0x45, 0x22, 0xf9, 0xe0, 0xa8, 0x2e, 0x12, 0x94, - 0xf5, 0x28, 0x41, 0x59, 0x17, 0x68, 0x44, 0xa0, 0x1f, 0xa3, 0x40, 0x3f, 0x51, 0x20, 0xcf, 0xce, - 0x13, 0x16, 0x67, 0x34, 0x29, 0x2c, 0x19, 0x26, 0x88, 0x9f, 0x4c, 0x21, 0xee, 0xb7, 0x09, 0xfe, - 0x1f, 0xfd, 0xf2, 0x7c, 0x07, 0x2e, 0xa2, 0xa3, 0xea, 0x41, 0x69, 0x00, 0xaf, 0x2c, 0xb6, 0x0c, - 0x13, 0x5b, 0x4b, 0x16, 0xdb, 0xec, 0xea, 0xcd, 0x67, 0xf2, 0x80, 0x2c, 0x22, 0xa8, 0xe7, 0xd6, - 0x45, 0x12, 0x9c, 0x07, 0x33, 0x77, 0xdc, 0x76, 0x20, 0x89, 0x8d, 0xb9, 0xbf, 0x8a, 0xec, 0xfe, - 0x2e, 0x4a, 0xc6, 0x8c, 0x06, 0xce, 0xac, 0x85, 0x6e, 0x72, 0xa5, 0x4d, 0x53, 0x52, 0x6a, 0x32, - 0xfb, 0x0e, 0xed, 0x3a, 0x4d, 0x50, 0xb3, 0x64, 0x75, 0x9d, 0x78, 0x92, 0x4c, 0xab, 0xd5, 0x7f, - 0xca, 0x42, 0x91, 0x2d, 0xc9, 0x45, 0xbb, 0x09, 0xd3, 0x64, 0x14, 0x17, 0xea, 0xab, 0x09, 0x42, - 0x90, 0xa7, 0x91, 0x06, 0x17, 0x2b, 0x9d, 0x5e, 0xfd, 0xef, 0x0c, 0xa4, 0xef, 0xb8, 0x6d, 0xb4, - 0x08, 0x29, 0x6e, 0x0d, 0xd3, 0xcd, 0xec, 0xe9, 0x49, 0x2d, 0xd5, 0xda, 0xd0, 0x52, 0x96, 0x79, - 0xce, 0x13, 0x26, 0x87, 0x1b, 0xd3, 0x83, 0xe1, 0x06, 0x72, 0x61, 0x76, 0x20, 0x99, 0xc3, 0xc2, - 0xdb, 0x52, 0xf3, 0xbd, 0xd3, 0x93, 0x5a, 0x49, 0xce, 0xe6, 0x8c, 0x7f, 0x2f, 0x05, 0x87, 0x36, - 0xf9, 0x6b, 0x1b, 0x01, 0xae, 0xb7, 0x36, 0xb4, 0x92, 0x9c, 0x05, 0x0a, 0x24, 0x39, 0x64, 0x07, - 0xe4, 0x70, 0x1b, 0x72, 0x2c, 0x31, 0x66, 0xd2, 0x3c, 0x44, 0xb2, 0xf7, 0x35, 0x4d, 0x3d, 0x2f, - 0x31, 0x81, 0xcc, 0x0d, 0x42, 0xc3, 0x27, 0x73, 0xf3, 0xe3, 0xce, 0xe5, 0x13, 0xd0, 0x3b, 0x90, - 0xdf, 0xb3, 0x1c, 0x2b, 0xd8, 0xc7, 0x66, 0xa5, 0x30, 0xe6, 0xe4, 0x68, 0x06, 0x99, 0xdd, 0x73, - 0x4d, 0x6b, 0xcf, 0xc2, 0x66, 0x05, 0xc6, 0x9d, 0x2d, 0x66, 0x10, 0xd7, 0x69, 0xcf, 0x37, 0x68, - 0x8a, 0x46, 0xef, 0xb8, 0x3d, 0xcf, 0xc6, 0x64, 0x0b, 0x33, 0xcb, 0xca, 0x4a, 0x4a, 0x9b, 0x17, - 0x3d, 0xeb, 0xa2, 0x83, 0x28, 0x36, 0x0d, 0xd2, 0x2b, 0x45, 0x66, 0x75, 0x69, 0x03, 0xdd, 0x87, - 0x0b, 0xfb, 0x56, 0x77, 0xff, 0xb1, 0x41, 0x9c, 0xbb, 0x38, 0x90, 0x2a, 0x8d, 0x49, 0x0d, 0x8a, - 0x26, 0x47, 0x3d, 0xe4, 0x2a, 0x8a, 0x51, 0x9a, 0xb8, 0x63, 0xf5, 0x0c, 0xbb, 0x32, 0x4b, 0x17, - 0x2d, 0x47, 0x1d, 0x1b, 0x0c, 0x8e, 0x5e, 0x82, 0x59, 0xbf, 0xef, 0x90, 0x9b, 0x4e, 0xe7, 0x82, - 0x9d, 0xa3, 0x23, 0x4b, 0x1c, 0xca, 0x6f, 0xf9, 0xab, 0x50, 0x88, 0x33, 0xa1, 0x65, 0x16, 0xa0, - 0x45, 0x00, 0x15, 0x41, 0x79, 0xdb, 0xed, 0xb0, 0x02, 0x89, 0xb0, 0xa1, 0xff, 0xab, 0xc0, 0xbc, - 0x04, 0x8c, 0xf2, 0x0c, 0x05, 0x5b, 0x00, 0xc7, 0x48, 0x43, 0x9f, 0x41, 0x10, 0x41, 0xc4, 0xf5, - 0x16, 0x61, 0xab, 0xfe, 0xa9, 0x02, 0x79, 0xd1, 0x8b, 0x5e, 0x80, 0x22, 0xe9, 0xb1, 0xad, 0xf0, - 0x58, 0x8f, 0x6f, 0x9d, 0x19, 0x01, 0x7b, 0x1f, 0x1f, 0x93, 0x9d, 0x47, 0x43, 0x62, 0x7b, 0x56, - 0xd0, 0x4a, 0x02, 0xca, 0x2e, 0xcb, 0x2a, 0xe4, 0x6d, 0x23, 0xb4, 0xc2, 0xbe, 0xc9, 0xac, 0x8c, - 0xa2, 0x45, 0x6d, 0xc2, 0x15, 0xdb, 0x75, 0xba, 0xac, 0x73, 0x9a, 0x76, 0xc6, 0x00, 0xb5, 0x09, - 0x73, 0x9a, 0xe1, 0x74, 0xf1, 0xb6, 0xdb, 0x15, 0xc6, 0xed, 0x32, 0xe4, 0x59, 0xd6, 0x4c, 0xd8, - 0x07, 0x2d, 0x47, 0xdb, 0x72, 0xd8, 0x9f, 0x92, 0xec, 0x9e, 0xfa, 0x5f, 0x69, 0x28, 0xc7, 0x48, - 0xa2, 0x9b, 0x48, 0xc4, 0xf2, 0xec, 0xd6, 0x48, 0xba, 0x38, 0x87, 0x27, 0x8f, 0x8c, 0xe6, 0xff, - 0x51, 0x01, 0xf8, 0xd0, 0xc7, 0x61, 0x78, 0xdc, 0x22, 0xc1, 0xef, 0x0b, 0x50, 0xe4, 0xa1, 0x94, - 0x4e, 0x4e, 0xbf, 0x60, 0x1e, 0x87, 0x11, 0xb3, 0x42, 0x36, 0xe2, 0xe0, 0xc7, 0xac, 0x9b, 0xb1, - 0x2d, 0xe7, 0xe0, 0xc7, 0xb4, 0xeb, 0x1a, 0x94, 0x0c, 0xd3, 0xc4, 0xa6, 0xce, 0x9d, 0x31, 0x6e, - 0xd3, 0x8a, 0x14, 0xa8, 0x31, 0x18, 0x7a, 0x19, 0xe6, 0x7c, 0xdc, 0x73, 0x8f, 0xa4, 0x61, 0xcc, - 0xb6, 0xcd, 0x72, 0xb0, 0x18, 0xb8, 0x08, 0x59, 0x1f, 0x1b, 0x41, 0x94, 0x16, 0xe5, 0x2d, 0x54, - 0x81, 0x9c, 0xc9, 0x12, 0xfe, 0xdc, 0x12, 0x89, 0x66, 0xf5, 0xef, 0x14, 0x91, 0x9a, 0xb8, 0x03, - 0x19, 0xba, 0x41, 0x9e, 0x96, 0xa8, 0x3f, 0xf3, 0xa2, 0x12, 0xcc, 0x92, 0x79, 0xc4, 0x50, 0xa0, - 0x8f, 0x61, 0xc6, 0xa3, 0x1c, 0xd2, 0x69, 0xae, 0x80, 0x79, 0x2c, 0x6f, 0x4e, 0xc2, 0xf8, 0x98, - 0xc1, 0xc2, 0x17, 0xf0, 0x22, 0xc8, 0x9d, 0xe9, 0xbc, 0x52, 0x4e, 0xa9, 0x2b, 0x50, 0xbe, 0xdf, - 0xc7, 0xfe, 0xf1, 0x87, 0xb6, 0xe1, 0x48, 0xf7, 0xe1, 0x21, 0x81, 0x09, 0x67, 0x8d, 0x36, 0x54, - 0x0f, 0xe6, 0xa5, 0x91, 0x5c, 0x2f, 0xbe, 0x03, 0x57, 0x4c, 0x2b, 0x08, 0x83, 0x43, 0x5b, 0xf7, - 0xf6, 0x8f, 0x03, 0xab, 0x63, 0xd8, 0x3a, 0x1d, 0xae, 0x7b, 0xb6, 0xe1, 0xf0, 0xb8, 0xf1, 0xea, - 0xe9, 0x49, 0xad, 0xb2, 0x61, 0x05, 0xe1, 0xce, 0xfd, 0xed, 0x0f, 0xf9, 0xa8, 0x18, 0x55, 0x85, - 0x23, 0x38, 0xd3, 0xa3, 0x5e, 0x66, 0x25, 0x45, 0x32, 0xd3, 0xb7, 0xda, 0xfd, 0x30, 0x8e, 0x88, - 0xd4, 0xbf, 0x05, 0xa8, 0x9c, 0xed, 0xe3, 0x44, 0x79, 0x50, 0x12, 0x19, 0x66, 0xc6, 0x3a, 0x76, - 0xea, 0x37, 0x9f, 0x51, 0x76, 0x1c, 0x85, 0x2b, 0xaa, 0x47, 0x12, 0x96, 0xc9, 0x4e, 0x6e, 0xd1, - 0x94, 0x3a, 0x50, 0x0f, 0x8a, 0x52, 0x01, 0x45, 0xd4, 0x8f, 0x36, 0xce, 0xb3, 0x60, 0x5c, 0x54, - 0x19, 0x70, 0xaa, 0x67, 0xe2, 0xa2, 0x4a, 0x50, 0xfd, 0x42, 0x01, 0x88, 0xc7, 0x11, 0x85, 0x65, - 0xd9, 0x28, 0x2e, 0x30, 0xde, 0x42, 0x6b, 0x90, 0xe5, 0xd5, 0xa1, 0xd4, 0xa4, 0xd5, 0x21, 0x3e, - 0x91, 0xa6, 0x01, 0x58, 0x51, 0x28, 0x38, 0xb4, 0xd9, 0x79, 0xe1, 0x69, 0x00, 0x0a, 0xdd, 0xb9, - 0xbf, 0xad, 0x15, 0xd8, 0x80, 0x9d, 0x43, 0xfb, 0xce, 0x74, 0x3e, 0x5d, 0x9e, 0xae, 0xfe, 0x71, - 0x1a, 0x0a, 0x34, 0x93, 0x4a, 0x59, 0xf3, 0x95, 0x02, 0x95, 0x81, 0xc0, 0x49, 0x6f, 0x1f, 0xeb, - 0x71, 0xf8, 0x46, 0xf8, 0xf4, 0xd1, 0x79, 0xf8, 0x14, 0xad, 0x50, 0xd7, 0xa4, 0x08, 0xac, 0x79, - 0x4c, 0xdd, 0x4b, 0x93, 0xb1, 0xee, 0xad, 0xf3, 0x3a, 0xa7, 0x0b, 0xfe, 0x08, 0x9c, 0xe8, 0x45, - 0x98, 0x95, 0xcb, 0x64, 0x51, 0x22, 0xb5, 0x18, 0xcb, 0xa6, 0x65, 0xa2, 0x6f, 0x01, 0x98, 0xbe, - 0xeb, 0x79, 0xd8, 0xd4, 0x0d, 0x16, 0x25, 0x8e, 0x73, 0xab, 0x16, 0xf8, 0x9c, 0xb5, 0xb0, 0xba, - 0x05, 0x97, 0x9f, 0xba, 0xa5, 0x11, 0xbe, 0xf1, 0x40, 0xe2, 0x2d, 0x2d, 0xb9, 0xb9, 0xd5, 0x1f, - 0xa6, 0xa0, 0x28, 0xeb, 0x2f, 0x0a, 0x81, 0x55, 0xc7, 0xe5, 0x53, 0xf1, 0xc1, 0xf3, 0x9e, 0x8a, - 0x58, 0x12, 0x03, 0x41, 0x60, 0x28, 0xa0, 0xd5, 0xcf, 0x60, 0x76, 0x70, 0xc8, 0x88, 0xb8, 0x6c, - 0x67, 0x30, 0x2e, 0x7b, 0xf7, 0xb9, 0x34, 0x62, 0x80, 0x07, 0x4a, 0xfc, 0x02, 0x22, 0x89, 0x80, - 0x87, 0x83, 0x04, 0xfc, 0xce, 0xf3, 0x72, 0x45, 0xa6, 0xe1, 0x07, 0x50, 0x1e, 0x3e, 0xd5, 0x23, - 0x28, 0xd8, 0x1d, 0xa4, 0xe0, 0xb7, 0x9f, 0xcf, 0x78, 0x48, 0xeb, 0xdf, 0x99, 0xce, 0xa7, 0xca, - 0x69, 0xf5, 0x12, 0x5c, 0xbc, 0x4b, 0xdf, 0xa0, 0xdc, 0xc5, 0xa1, 0x61, 0xc6, 0x69, 0x57, 0xf5, - 0x5f, 0x15, 0x58, 0x1c, 0xee, 0xe1, 0x96, 0xd4, 0x80, 0x7c, 0x8f, 0xc3, 0xb8, 0xba, 0x7c, 0x2b, - 0x81, 0xac, 0xd1, 0x48, 0xea, 0x02, 0x20, 0xab, 0x47, 0x84, 0xb6, 0xfa, 0xbb, 0x50, 0x1a, 0x18, - 0x30, 0x82, 0x33, 0x6f, 0x0c, 0x72, 0x46, 0x0e, 0xb9, 0xfa, 0xa1, 0x65, 0xd7, 0xf9, 0xd3, 0x9a, - 0x68, 0x61, 0x29, 0xd2, 0xfb, 0x69, 0x0a, 0x2e, 0x6c, 0x3a, 0x87, 0x7d, 0xdc, 0xc7, 0xf4, 0x76, - 0x14, 0x57, 0xdc, 0x6f, 0x32, 0x85, 0xc4, 0xae, 0xcf, 0xc8, 0xb9, 0x63, 0x0d, 0xf4, 0x5d, 0xc9, - 0x0f, 0xa3, 0x65, 0x96, 0xe6, 0xfa, 0xe9, 0x49, 0x2d, 0x47, 0xa9, 0xa2, 0x6b, 0xde, 0x9a, 0x68, - 0x4d, 0x3e, 0x2f, 0x76, 0xe6, 0xae, 0xc3, 0x7c, 0x70, 0x60, 0x79, 0x7a, 0xb0, 0xef, 0xf6, 0x6d, - 0x53, 0x67, 0x14, 0xf0, 0xe4, 0x13, 0xe9, 0xd8, 0xa1, 0xf0, 0xfb, 0x04, 0xac, 0xfe, 0x55, 0x0a, - 0x16, 0x06, 0xb9, 0xc2, 0xe5, 0x7d, 0x3f, 0x76, 0x71, 0x98, 0xb8, 0xdf, 0x4a, 0xaa, 0xd9, 0x8c, - 0xc0, 0x50, 0x17, 0x4f, 0x22, 0x22, 0xdf, 0xe8, 0xef, 0x15, 0xc8, 0x71, 0xe0, 0x6f, 0x94, 0xeb, - 0xef, 0x0e, 0x79, 0xa8, 0x2f, 0x25, 0xd5, 0xf9, 0x7c, 0xa3, 0x83, 0xa9, 0xc7, 0x25, 0xfc, 0xd1, - 0x38, 0x54, 0x4a, 0x4b, 0xa1, 0x92, 0x7a, 0x11, 0x2e, 0xac, 0xef, 0x1b, 0x7e, 0xb8, 0xce, 0xde, - 0x75, 0x89, 0x13, 0xf3, 0x08, 0x16, 0x06, 0xc1, 0x9c, 0x7d, 0x4d, 0xc8, 0xf1, 0x17, 0x60, 0x9c, - 0x7d, 0xaa, 0x44, 0x44, 0x18, 0xd4, 0xc5, 0xf3, 0x30, 0x3a, 0x79, 0x87, 0x95, 0xda, 0xa3, 0x57, - 0x28, 0xac, 0xef, 0xfa, 0xfb, 0xb0, 0x38, 0xfa, 0x5d, 0x05, 0x9a, 0x81, 0xdc, 0x83, 0x7b, 0xef, - 0xdf, 0xfb, 0xe0, 0xa3, 0x7b, 0xe5, 0x29, 0xd2, 0x58, 0xdf, 0x7e, 0xb0, 0xb3, 0xbb, 0xa9, 0x95, - 0x15, 0x54, 0x84, 0xfc, 0xc6, 0xda, 0xee, 0x5a, 0x73, 0x6d, 0x67, 0xb3, 0x9c, 0x42, 0x05, 0xc8, - 0xec, 0xae, 0x35, 0xb7, 0x37, 0xcb, 0xe9, 0xeb, 0xd7, 0xa0, 0x40, 0xb3, 0xdd, 0x77, 0x5d, 0x13, - 0x23, 0x80, 0xec, 0xfa, 0x76, 0x6b, 0xf3, 0xde, 0x6e, 0x79, 0x8a, 0x7c, 0x6f, 0x6f, 0xae, 0xed, - 0x6c, 0xee, 0x94, 0x95, 0xd5, 0x9f, 0x54, 0x21, 0xb3, 0x66, 0xf6, 0x2c, 0x07, 0x85, 0x90, 0xa1, - 0x15, 0x49, 0xf4, 0xf2, 0xb3, 0x6b, 0x96, 0x94, 0x13, 0xd5, 0x95, 0x71, 0x8b, 0x9b, 0x6a, 0xe5, - 0x47, 0xff, 0xf2, 0x1f, 0x3f, 0x4d, 0x21, 0x54, 0x6e, 0xe8, 0xf4, 0x49, 0x60, 0xe3, 0xe8, 0x46, - 0x83, 0x16, 0x39, 0xd1, 0x1f, 0x2a, 0x50, 0x88, 0x1e, 0xa9, 0xa1, 0x57, 0xc7, 0x78, 0x1c, 0x16, - 0x2d, 0xff, 0xda, 0x78, 0x83, 0x39, 0x09, 0x57, 0x29, 0x09, 0x8b, 0x68, 0x41, 0x22, 0x21, 0x7a, - 0xf7, 0x86, 0xfe, 0x42, 0x81, 0xb9, 0xa1, 0xd7, 0x67, 0xe8, 0xc6, 0x24, 0x2f, 0xd5, 0x18, 0x49, - 0xab, 0x93, 0x3f, 0x6e, 0x53, 0x5f, 0xa6, 0x84, 0xbd, 0x80, 0x6a, 0xa3, 0x08, 0x6b, 0x7c, 0x26, - 0x3e, 0xbf, 0x8f, 0xfe, 0x5a, 0x81, 0xa2, 0xfc, 0xf2, 0x08, 0xd5, 0xc7, 0x7e, 0xa2, 0xc4, 0xa8, - 0x6b, 0x4c, 0xf8, 0xa4, 0x49, 0x7d, 0x93, 0x92, 0xf6, 0x3a, 0xaa, 0x3f, 0x83, 0xb4, 0x06, 0xbd, - 0xef, 0x83, 0xc6, 0x67, 0xf4, 0x97, 0x52, 0x0a, 0x71, 0x39, 0x1d, 0xbd, 0x36, 0x66, 0xd5, 0x9d, - 0x51, 0x39, 0x59, 0x8d, 0x5e, 0x7d, 0x87, 0xd2, 0xf8, 0x26, 0xba, 0x35, 0x19, 0x8d, 0x0d, 0xf6, - 0x5c, 0xe3, 0xcf, 0x14, 0x28, 0x0d, 0xbc, 0x50, 0x40, 0x49, 0x4c, 0x1a, 0xf5, 0xc8, 0xa1, 0xfa, - 0xfa, 0xf8, 0x13, 0x38, 0xc9, 0xcb, 0x94, 0xe4, 0x2a, 0xaa, 0x48, 0x24, 0x3b, 0xae, 0xc3, 0x08, - 0xa4, 0x44, 0x7c, 0x02, 0x59, 0x56, 0x14, 0x47, 0x2b, 0x63, 0xd4, 0xcd, 0x19, 0x1d, 0xaf, 0x8c, - 0x5d, 0x61, 0x57, 0x2f, 0x53, 0x02, 0x2e, 0xa0, 0x79, 0x89, 0x00, 0x6e, 0x0a, 0xc9, 0x79, 0x8c, - 0x0a, 0xb6, 0x89, 0xe7, 0x71, 0xb8, 0x84, 0x9c, 0x78, 0x1e, 0xcf, 0xd6, 0x80, 0xf9, 0x79, 0x54, - 0x65, 0x1a, 0xfa, 0x16, 0x11, 0xd7, 0x6d, 0xe5, 0x3a, 0xfa, 0xa1, 0x02, 0x85, 0xad, 0xb1, 0xc8, - 0xd8, 0x9a, 0x84, 0x8c, 0x33, 0x15, 0xd2, 0x91, 0xac, 0x60, 0x64, 0xa0, 0xdf, 0x87, 0x1c, 0xaf, - 0x77, 0xa2, 0x24, 0xde, 0x0e, 0xd6, 0x53, 0xab, 0xd7, 0xc7, 0x19, 0xca, 0x17, 0xaf, 0xd2, 0xc5, - 0x17, 0x10, 0x92, 0x16, 0xe7, 0x75, 0x55, 0xf4, 0x07, 0x0a, 0xe4, 0x45, 0x15, 0x02, 0x5d, 0x1f, - 0xab, 0x54, 0xc1, 0x08, 0x78, 0x75, 0x82, 0xb2, 0x86, 0x7a, 0x85, 0x52, 0x70, 0x11, 0x5d, 0x90, - 0x28, 0x08, 0xc4, 0xaa, 0x3f, 0x56, 0x20, 0xcb, 0x8a, 0x17, 0x89, 0x6a, 0x38, 0x50, 0x06, 0x49, - 0x54, 0xc3, 0xa1, 0x4a, 0xc8, 0x8b, 0x74, 0xf1, 0xa5, 0x01, 0xde, 0xef, 0xd3, 0x21, 0x8f, 0x0a, - 0x28, 0xc7, 0x3f, 0x29, 0x23, 0x44, 0x36, 0x3f, 0x91, 0x11, 0x43, 0x55, 0x95, 0x44, 0x46, 0x0c, - 0x17, 0x37, 0x46, 0x32, 0x42, 0xd4, 0x4e, 0x90, 0x07, 0xd3, 0x77, 0xdc, 0x76, 0x80, 0xbe, 0xfe, - 0xcc, 0x4c, 0x3d, 0x5b, 0xf9, 0xe5, 0x31, 0x33, 0xfa, 0xea, 0x25, 0xba, 0xea, 0x3c, 0x9a, 0x93, - 0x56, 0xfd, 0x1e, 0x59, 0x89, 0x1c, 0xc3, 0x28, 0x19, 0x99, 0xa8, 0xff, 0xc3, 0x89, 0xd0, 0x44, - 0xfd, 0x3f, 0x93, 0xdf, 0x1c, 0x79, 0x2d, 0x46, 0x39, 0x4e, 0x4a, 0x46, 0x94, 0x92, 0x49, 0x24, - 0x63, 0x38, 0x8f, 0x94, 0x48, 0xc6, 0x99, 0x54, 0xd2, 0x48, 0x32, 0x68, 0x2a, 0xc9, 0x23, 0x0b, - 0x7f, 0x17, 0x32, 0xd4, 0x93, 0x49, 0x74, 0x4d, 0xe4, 0x77, 0x02, 0x89, 0xae, 0xc9, 0xc0, 0x13, - 0x00, 0x75, 0xea, 0x75, 0x05, 0x3d, 0x86, 0xa2, 0x5c, 0x91, 0x4e, 0xbc, 0x58, 0x47, 0x94, 0xe8, - 0xab, 0x6f, 0x9c, 0xab, 0xd4, 0xad, 0x4e, 0x91, 0x13, 0x86, 0xce, 0x0e, 0x40, 0xb7, 0x26, 0xc4, - 0xf7, 0x9c, 0x54, 0xfc, 0xa5, 0x02, 0x79, 0x91, 0x3e, 0x4c, 0x3c, 0x61, 0x43, 0xe9, 0xe5, 0xc4, - 0x13, 0x36, 0x9c, 0x8f, 0x54, 0xdf, 0xa5, 0x22, 0x7e, 0x6b, 0xe0, 0x84, 0xd1, 0xf8, 0xc5, 0x76, - 0xbb, 0x8f, 0x96, 0xd1, 0xd2, 0x08, 0x70, 0xe3, 0x33, 0x11, 0x35, 0x7d, 0x9f, 0x78, 0x68, 0xe5, - 0xe1, 0xd0, 0x17, 0xad, 0x4e, 0x14, 0x27, 0x33, 0xa2, 0x6f, 0x9e, 0x23, 0xb6, 0x8e, 0x4c, 0xd5, - 0xd5, 0x21, 0x2f, 0x43, 0x37, 0x65, 0x72, 0xbe, 0x50, 0x60, 0x7e, 0xcd, 0xb6, 0x07, 0x43, 0x61, - 0xf4, 0xfa, 0x04, 0x51, 0x33, 0x23, 0xf1, 0xc6, 0xc4, 0x71, 0xb6, 0xfa, 0x02, 0x25, 0xf0, 0x0a, - 0xba, 0x2c, 0x11, 0xc8, 0x42, 0x63, 0x11, 0x6c, 0xa3, 0xcf, 0x15, 0x28, 0xca, 0x91, 0x4b, 0xa2, - 0x9a, 0x8f, 0x88, 0x7c, 0x12, 0xfd, 0xc7, 0x51, 0x21, 0x91, 0x5a, 0xa3, 0x44, 0x5d, 0x46, 0x97, - 0xe4, 0xfb, 0x8d, 0x0c, 0xe4, 0xf1, 0x0e, 0x71, 0xbf, 0x8a, 0x72, 0x24, 0x99, 0x48, 0xd2, 0x88, - 0x50, 0x3e, 0x91, 0xa4, 0x51, 0x21, 0xaa, 0x7a, 0x8d, 0x92, 0xf4, 0x35, 0x55, 0xf6, 0xbd, 0x30, - 0x1b, 0xa8, 0x53, 0x65, 0xbb, 0xad, 0x5c, 0x6f, 0x5e, 0xff, 0xea, 0xdf, 0x97, 0xa6, 0xbe, 0x3a, - 0x5d, 0x52, 0x7e, 0x7e, 0xba, 0xa4, 0xfc, 0xe2, 0x74, 0x49, 0xf9, 0xb7, 0xd3, 0x25, 0xe5, 0xf3, - 0x27, 0x4b, 0x53, 0x3f, 0x7f, 0xb2, 0x34, 0xf5, 0x8b, 0x27, 0x4b, 0x53, 0x8f, 0xf2, 0x62, 0x95, - 0x76, 0x96, 0x66, 0xf5, 0x6e, 0xfe, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x57, 0x4a, 0x7e, 0x09, - 0xfc, 0x34, 0x00, 0x00, +func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_ffd4bb43c44fb7e0) } + +var fileDescriptor_admin_ffd4bb43c44fb7e0 = []byte{ + // 4149 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x4d, 0x70, 0x1b, 0xd7, + 0x79, 0x5a, 0x80, 0xf8, 0xfb, 0x08, 0x90, 0xe0, 0x13, 0x45, 0x81, 0x90, 0x42, 0xc8, 0x2b, 0x3b, + 0xa6, 0x65, 0x07, 0xb0, 0x28, 0x59, 0x4e, 0x65, 0x39, 0x29, 0x41, 0x70, 0x64, 0xc8, 0x94, 0x6c, + 0x2d, 0x25, 0x79, 0xa2, 0xd4, 0xd9, 0x2e, 0xb0, 0x8f, 0xe0, 0x86, 0x8b, 0xdd, 0xe5, 0xee, 0x82, + 0x32, 0xed, 0x26, 0xd3, 0x24, 0x6d, 0xa7, 0xbd, 0x74, 0x3c, 0x49, 0x6f, 0x9e, 0xe9, 0x4c, 0x7b, + 0x68, 0x0f, 0xbd, 0xd4, 0x3d, 0xf6, 0xd0, 0xb3, 0xa7, 0x87, 0x26, 0x9d, 0x5e, 0xd2, 0x0b, 0xd3, + 0xd2, 0x39, 0x74, 0xd2, 0x7b, 0x0f, 0xbd, 0xb4, 0xf3, 0xfe, 0x76, 0x1f, 0x40, 0x68, 0x05, 0x50, + 0x49, 0x0e, 0x24, 0xf6, 0x7d, 0xdf, 0x7b, 0xdf, 0xfb, 0xde, 0xf7, 0xbe, 0xf7, 0xfd, 0xbd, 0x07, + 0x17, 0x02, 0xec, 0x1f, 0x60, 0xbf, 0xc1, 0x7e, 0xbc, 0x4e, 0xc3, 0x30, 0xfb, 0x96, 0x53, 0xf7, + 0x7c, 0x37, 0x74, 0xd1, 0x72, 0xd7, 0xed, 0xee, 0xf9, 0xae, 0xd1, 0xdd, 0xad, 0x33, 0x7c, 0x5d, + 0x74, 0xab, 0x56, 0xba, 0xae, 0xb3, 0x63, 0xf5, 0x1a, 0x1f, 0xbb, 0x0e, 0xf6, 0x3a, 0xf4, 0x87, + 0x0d, 0xaa, 0x2e, 0x7d, 0xd7, 0xed, 0x04, 0x0d, 0xf2, 0xcf, 0xeb, 0xd0, 0x1f, 0x0e, 0xbf, 0x38, + 0x3a, 0x53, 0x10, 0x1a, 0xe1, 0x40, 0x60, 0x2f, 0x04, 0xa1, 0xeb, 0x1b, 0x3d, 0xdc, 0xc0, 0x4e, + 0xcf, 0xa2, 0x24, 0xfb, 0x07, 0xdd, 0x2e, 0x47, 0xbe, 0xb8, 0x77, 0xd0, 0xd8, 0x3b, 0x10, 0xe3, + 0x59, 0x47, 0xaf, 0xd3, 0xb0, 0xad, 0x03, 0xec, 0xe0, 0x40, 0x90, 0xa8, 0x3d, 0xa5, 0x97, 0xdb, + 0xe3, 0x1d, 0x56, 0xc2, 0xa0, 0xd1, 0x35, 0x42, 0xc3, 0x76, 0x7b, 0x8d, 0xee, 0xae, 0xe1, 0x87, + 0x3a, 0x6f, 0x71, 0x7c, 0x65, 0x10, 0x5a, 0x76, 0xa3, 0x8f, 0x43, 0xdf, 0xea, 0xf2, 0x1f, 0x8e, + 0x59, 0xec, 0xb9, 0x3d, 0x97, 0x7e, 0x36, 0xc8, 0x97, 0x58, 0x51, 0xcf, 0x75, 0x7b, 0x36, 0x6e, + 0x18, 0x9e, 0xd5, 0x30, 0x1c, 0xc7, 0x0d, 0x8d, 0xd0, 0x72, 0x9d, 0x88, 0x1d, 0x8e, 0xa5, 0xad, + 0xce, 0x60, 0xa7, 0x11, 0x5a, 0x7d, 0x1c, 0x84, 0x46, 0xdf, 0x63, 0x1d, 0x54, 0x04, 0xe5, 0x96, + 0x11, 0x1a, 0x1d, 0x23, 0xc0, 0x81, 0x86, 0xf7, 0x07, 0x38, 0x08, 0xd5, 0xab, 0xb0, 0x20, 0xc1, + 0x02, 0xcf, 0x75, 0x02, 0x8c, 0x2e, 0x42, 0xc1, 0x14, 0xc0, 0x8a, 0x72, 0x29, 0xbd, 0x5a, 0xd0, + 0x62, 0x80, 0x7a, 0x1d, 0x96, 0xc4, 0x90, 0x16, 0x0e, 0x0d, 0xcb, 0x16, 0xc4, 0x50, 0x15, 0xf2, + 0xa2, 0x5b, 0x45, 0xb9, 0xa4, 0xac, 0x16, 0xb4, 0xa8, 0xad, 0x7e, 0x9e, 0x86, 0xf3, 0x27, 0x86, + 0xf1, 0xf9, 0x1e, 0x41, 0xb6, 0xe7, 0x1b, 0x4e, 0xc8, 0x26, 0x9b, 0x5d, 0xfb, 0x7a, 0xfd, 0xa9, + 0x7a, 0x50, 0x7f, 0x0a, 0x8d, 0xfa, 0x6d, 0x42, 0xa0, 0x39, 0xf3, 0xc5, 0x51, 0xed, 0x8c, 0xc6, + 0xa9, 0xa1, 0x1a, 0xcc, 0x86, 0x46, 0xc7, 0xc6, 0xba, 0x63, 0xf4, 0x71, 0x50, 0x49, 0xd1, 0x95, + 0x00, 0x05, 0xdd, 0x23, 0x10, 0xf4, 0x06, 0x94, 0x4c, 0x1c, 0x74, 0x7d, 0xcb, 0x0b, 0x5d, 0x5f, + 0xb7, 0xcc, 0x4a, 0xfa, 0x92, 0xb2, 0x9a, 0x6e, 0x96, 0x8f, 0x8f, 0x6a, 0xc5, 0x56, 0x84, 0x68, + 0xb7, 0xb4, 0x62, 0xdc, 0xad, 0x6d, 0xa2, 0x3b, 0x30, 0x4b, 0xf4, 0x4f, 0x67, 0x2a, 0x59, 0x99, + 0xb9, 0xa4, 0xac, 0xce, 0xae, 0x5d, 0x96, 0x98, 0x66, 0x88, 0x3a, 0xd3, 0xd5, 0xfa, 0x63, 0xd7, + 0xc1, 0x1b, 0x14, 0xc2, 0xf9, 0x83, 0x8f, 0x23, 0x08, 0xfa, 0x10, 0x16, 0x24, 0x5a, 0xba, 0x8d, + 0x0f, 0xb0, 0x5d, 0xc9, 0x5c, 0x52, 0x56, 0xe7, 0xd6, 0xae, 0x26, 0x88, 0x21, 0xa6, 0x39, 0xf0, + 0xa9, 0x16, 0x6c, 0x91, 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, 0x20, 0xc2, 0x9a, 0x7c, 0x9b, 0xd1, + 0x22, 0x64, 0xa8, 0x7c, 0x2b, 0x29, 0x8a, 0x60, 0x0d, 0xf5, 0x8f, 0xf3, 0xb0, 0x38, 0x4c, 0x89, + 0xef, 0xfc, 0xf6, 0xc8, 0xce, 0xbf, 0x91, 0xb0, 0xe4, 0x71, 0x04, 0xc6, 0x6e, 0xfb, 0x23, 0xc8, + 0x75, 0x5d, 0x7b, 0xd0, 0x77, 0xd8, 0x9a, 0x66, 0xd7, 0x6e, 0x4c, 0x4b, 0x75, 0x83, 0x0e, 0xe7, + 0x64, 0x05, 0x31, 0xf4, 0x10, 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, 0x41, 0x20, 0xe8, 0x3a, 0x2c, 0x75, 0x7d, 0x6c, 0x84, 0x58, + 0x67, 0xda, 0x4c, 0xec, 0x18, 0xee, 0x63, 0x27, 0xa4, 0x7a, 0x52, 0xd0, 0x16, 0x19, 0x96, 0xce, + 0xb8, 0x2d, 0x70, 0xa3, 0x4a, 0x9a, 0xfd, 0xb5, 0x2b, 0x69, 0xee, 0xd7, 0xa5, 0xa4, 0x27, 0x8f, + 0x61, 0x7e, 0x92, 0x63, 0xf8, 0x5c, 0xba, 0x5d, 0xfd, 0x27, 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, 0x65, 0xb2, 0x84, 0x1d, 0x63, + 0x60, 0x87, 0xfa, 0x81, 0x61, 0x0f, 0x30, 0xdd, 0xc6, 0x02, 0x61, 0x98, 0x02, 0x1f, 0x11, 0x18, + 0xba, 0x06, 0xe7, 0x7a, 0xd8, 0xc1, 0x4c, 0x16, 0x3a, 0xfe, 0xc8, 0xf3, 0x71, 0x10, 0x58, 0xae, + 0x23, 0xf6, 0x31, 0x46, 0x6e, 0x46, 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, 0xf1, 0x2c, 0xa7, 0xc7, 0xd9, 0x12, 0x4d, 0x22, 0x25, + 0xab, 0xef, 0xd9, 0x56, 0xd7, 0x0a, 0xa9, 0x8a, 0xe4, 0xb5, 0xa8, 0xad, 0x6e, 0xc2, 0x42, 0xa4, + 0xa5, 0xcf, 0x61, 0x4e, 0x3e, 0x4f, 0x03, 0x92, 0xe9, 0x70, 0x63, 0x32, 0x72, 0x90, 0x94, 0x13, + 0x07, 0xe9, 0x32, 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, 0x75, 0xc8, 0x90, + 0xf3, 0x17, 0x70, 0xab, 0xff, 0x92, 0xac, 0xfe, 0x2c, 0x04, 0xa8, 0x8b, 0x88, 0xa2, 0x7e, 0xf7, + 0xd1, 0xc6, 0x06, 0x65, 0x91, 0x1f, 0x29, 0x36, 0x12, 0x99, 0x50, 0xea, 0x5b, 0x41, 0x60, 0x39, + 0x3d, 0x9d, 0xd0, 0x0d, 0x2a, 0x19, 0x6a, 0x4d, 0x7e, 0xe7, 0x59, 0xd6, 0x64, 0x68, 0xb5, 0xf5, + 0xbb, 0x8c, 0xc4, 0x3d, 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, 0x0c, 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, 0x32, 0x98, 0xba, 0x04, 0x8b, 0xf7, 0x5c, 0xe7, 0xc4, + 0xee, 0xab, 0x3f, 0x55, 0xe0, 0xdc, 0x08, 0x82, 0x6f, 0xe7, 0xb7, 0x60, 0x81, 0x44, 0x30, 0x7a, + 0x80, 0x7d, 0x0b, 0x07, 0x3a, 0x93, 0xba, 0x42, 0xa5, 0xfe, 0xb5, 0xa9, 0x44, 0xa5, 0xcd, 0x13, + 0x3a, 0xdb, 0x94, 0x0c, 0x45, 0xa0, 0x6f, 0x03, 0xb2, 0x9c, 0x10, 0xfb, 0x8e, 0x61, 0xeb, 0x83, + 0x00, 0x73, 0xda, 0xa9, 0xd3, 0xd0, 0x2e, 0x0b, 0x42, 0x0f, 0x03, 0x86, 0x51, 0xe7, 0xa0, 0xf8, + 0x30, 0xc0, 0x7e, 0xb4, 0xc2, 0xef, 0x43, 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, 0x14, 0x28, 0x6d, 0x1e, 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, 0x57, 0x29, 0x98, 0x13, 0x5c, 0x70, 0x39, + 0xdc, 0x85, 0x2c, 0x1f, 0xc4, 0x04, 0xd1, 0x48, 0x10, 0xc4, 0xf0, 0x50, 0xd6, 0x14, 0x6e, 0x9f, + 0x11, 0xa9, 0xfe, 0x69, 0x0a, 0x32, 0x14, 0x8e, 0x9a, 0x50, 0x88, 0x62, 0x5f, 0xae, 0x31, 0xd5, + 0x3a, 0x8b, 0x8e, 0xeb, 0x22, 0x3a, 0xae, 0x3f, 0x10, 0x3d, 0x9a, 0x79, 0x42, 0xe6, 0xd3, 0x5f, + 0xd4, 0x14, 0x2d, 0x1e, 0x46, 0xcc, 0x00, 0xa5, 0xab, 0x4b, 0x6e, 0xa0, 0x40, 0x21, 0x0f, 0x88, + 0xb8, 0x5e, 0x91, 0xc5, 0xc5, 0xa2, 0xc6, 0xe2, 0xf1, 0x51, 0x2d, 0xff, 0x80, 0x89, 0xac, 0x25, + 0x09, 0x6f, 0x0d, 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, 0x43, 0x0a, 0x24, 0x53, 0x32, 0x74, 0xdb, 0x54, 0xff, 0x4e, 0x81, 0xf2, 0x36, 0x0e, 0x1f, + 0xb6, 0x49, 0xb4, 0x2c, 0x76, 0xfd, 0x5b, 0x00, 0x7b, 0xf8, 0x90, 0xb9, 0x27, 0x21, 0xf2, 0x9b, + 0x09, 0x22, 0x1f, 0x25, 0x50, 0x7f, 0x17, 0x1f, 0x52, 0x3f, 0x16, 0x6c, 0x3a, 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, 0xae, 0xa8, + 0x67, 0x61, 0x41, 0x9a, 0x8b, 0xed, 0xb0, 0xfa, 0x55, 0x28, 0xdf, 0x1e, 0x5d, 0x01, 0x82, 0x99, + 0x3d, 0x7c, 0x28, 0x52, 0x12, 0xfa, 0xad, 0xfe, 0x34, 0x05, 0x0b, 0xb7, 0x47, 0x47, 0xa3, 0xdf, + 0x1f, 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, 0x87, 0x6c, 0x60, + 0xd5, 0x9b, 0x40, 0xb4, 0xef, 0xc8, 0xa2, 0x9d, 0x5d, 0x5b, 0x9b, 0x6a, 0xa1, 0x94, 0xb4, 0xbc, + 0x1d, 0x65, 0x98, 0xdb, 0xb0, 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, 0x32, 0x38, 0xfa, 0x1a, 0x20, 0x4c, 0x8c, 0xaa, 0xe7, 0x5b, 0x01, + 0x8e, 0x7a, 0xb3, 0x88, 0x6b, 0x21, 0xc6, 0xf0, 0xee, 0xea, 0x9f, 0x29, 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, 0xe2, 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, 0x2a, 0x50, 0xe2, 0xbc, 0x70, 0x39, 0xdd, 0x80, 0xf3, + 0x12, 0x33, 0x74, 0xa8, 0xce, 0x0a, 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, 0x1b, 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, 0x3d, 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, 0xa9, 0xc0, 0x59, 0x79, 0xf2, 0xdf, 0xce, 0xb4, + 0x68, 0x15, 0xe6, 0x4d, 0x69, 0xd6, 0x78, 0x4f, 0x46, 0xc1, 0xea, 0x97, 0x29, 0xa8, 0x8e, 0x93, + 0x0e, 0x57, 0x88, 0xc7, 0x90, 0xe5, 0xfb, 0xcf, 0x92, 0xd3, 0x5b, 0x49, 0xc5, 0x8e, 0xa7, 0x92, + 0xa9, 0xb3, 0xa6, 0x70, 0x81, 0x8c, 0x62, 0xf5, 0xbf, 0x14, 0xc8, 0x72, 0xfd, 0x79, 0x3c, 0x1c, + 0xb9, 0x65, 0x9a, 0xeb, 0x71, 0xe4, 0x76, 0x5a, 0x61, 0x88, 0x80, 0xef, 0x3c, 0xe4, 0xac, 0x40, + 0xb7, 0xad, 0x83, 0x28, 0xbb, 0xb0, 0x82, 0x2d, 0xeb, 0x00, 0x9f, 0x0c, 0xb0, 0xd3, 0x63, 0x02, + 0xec, 0x31, 0x92, 0x9c, 0x19, 0x2b, 0x49, 0x9a, 0x19, 0x88, 0x03, 0x90, 0x61, 0x47, 0x51, 0xb4, + 0x55, 0x0d, 0xe6, 0xb7, 0x71, 0x48, 0xcc, 0x46, 0x90, 0xe0, 0x1e, 0x46, 0x62, 0x14, 0xee, 0x0f, + 0x52, 0xa3, 0x31, 0x0a, 0x33, 0xaf, 0xea, 0xbf, 0xa6, 0xa8, 0xdb, 0xe4, 0x44, 0xf9, 0x7e, 0xe9, + 0xd3, 0xbb, 0xcd, 0x21, 0x02, 0xcf, 0xf4, 0x24, 0x7b, 0x63, 0x3d, 0x49, 0x41, 0x78, 0x92, 0x71, + 0x99, 0xe8, 0x25, 0x98, 0x95, 0xcf, 0x6b, 0x9a, 0xa2, 0x64, 0x10, 0xc9, 0xea, 0xbc, 0x41, 0xc7, + 0xb6, 0xba, 0x5c, 0xb6, 0xbc, 0x55, 0x75, 0x27, 0x70, 0x27, 0xb7, 0x87, 0xdd, 0xc9, 0xd5, 0x69, + 0x16, 0x7b, 0xc2, 0x9b, 0xbc, 0x04, 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, 0x6f, 0xf1, 0x7a, 0xab, 0xf0, 0x43, 0xbf, 0x4a, 0x41, 0x39, 0x86, 0xf1, 0xfd, 0x79, 0x0f, + 0x40, 0xd4, 0x65, 0xa3, 0xfd, 0x79, 0x45, 0x62, 0x79, 0xef, 0xa0, 0x2e, 0x8a, 0xb4, 0xf5, 0xa8, + 0x48, 0x5b, 0x17, 0x64, 0x44, 0xb1, 0x23, 0x26, 0x81, 0x7e, 0xac, 0x40, 0x9e, 0x9d, 0x27, 0x2c, + 0xce, 0x68, 0x52, 0x6a, 0x36, 0xca, 0x10, 0x3f, 0x99, 0x62, 0xbb, 0xdf, 0x22, 0xf4, 0x7f, 0xf8, + 0x8b, 0xd3, 0x1d, 0xb8, 0x88, 0x8f, 0xaa, 0x07, 0xa5, 0x21, 0xba, 0xf2, 0xb6, 0x65, 0xd8, 0xb6, + 0xb5, 0xe5, 0x6d, 0x9b, 0x5b, 0xbb, 0xf6, 0x4c, 0x19, 0x90, 0x49, 0x04, 0xf7, 0xdc, 0xba, 0x48, + 0x1b, 0xe7, 0xc1, 0xec, 0x1d, 0xb7, 0x13, 0x48, 0xdb, 0xc6, 0x52, 0x00, 0x45, 0x4e, 0x01, 0x96, + 0x24, 0x63, 0x46, 0x8b, 0x07, 0xac, 0x85, 0xae, 0x71, 0xa5, 0x4d, 0x53, 0x56, 0x6a, 0xb2, 0xf8, + 0xf6, 0xed, 0x3a, 0x2d, 0xd2, 0xb3, 0x82, 0x7d, 0x9d, 0x44, 0xd3, 0x4c, 0xab, 0xd5, 0x7f, 0xc9, + 0x42, 0x91, 0x4d, 0xc9, 0xb7, 0x76, 0x13, 0x66, 0x48, 0x2f, 0xbe, 0xa9, 0xaf, 0x26, 0x6c, 0x82, + 0x3c, 0x8c, 0x34, 0xf8, 0xb6, 0xd2, 0xe1, 0xd5, 0xff, 0xc9, 0x40, 0xfa, 0x8e, 0xdb, 0x41, 0x4b, + 0x90, 0xe2, 0xd6, 0x30, 0xdd, 0xcc, 0x1e, 0x1f, 0xd5, 0x52, 0xed, 0x96, 0x96, 0xb2, 0xcc, 0x53, + 0x9e, 0x30, 0x39, 0xe5, 0x9a, 0x19, 0x4e, 0xb9, 0x90, 0x0b, 0x73, 0x43, 0x05, 0x2d, 0x96, 0xe2, + 0x97, 0x9a, 0xef, 0x1c, 0x1f, 0xd5, 0x4a, 0x72, 0x45, 0x6b, 0x72, 0xbf, 0x14, 0xec, 0xdb, 0xe4, + 0xaf, 0x63, 0x04, 0xb8, 0xde, 0x6e, 0x69, 0x25, 0xb9, 0x12, 0x16, 0x48, 0xfb, 0x90, 0x1d, 0xda, + 0x87, 0x9b, 0x90, 0x63, 0xc5, 0x41, 0x93, 0xd6, 0x62, 0x92, 0x23, 0xd0, 0x19, 0x1a, 0x7d, 0x8a, + 0x01, 0x64, 0x6c, 0x10, 0x1a, 0x3e, 0x19, 0x9b, 0x9f, 0x74, 0x2c, 0x1f, 0x80, 0x6e, 0x41, 0x7e, + 0xc7, 0x72, 0xac, 0x60, 0x17, 0x9b, 0x95, 0xc2, 0x84, 0x83, 0xa3, 0x11, 0x64, 0x74, 0xdf, 0x35, + 0xad, 0x1d, 0x0b, 0x9b, 0x15, 0x98, 0x74, 0xb4, 0x18, 0x41, 0xc2, 0xc7, 0x1d, 0xdf, 0xa0, 0x65, + 0x2a, 0xbd, 0xeb, 0xf6, 0x3d, 0x1b, 0x93, 0x25, 0xcc, 0x5e, 0x52, 0x56, 0x53, 0xda, 0x82, 0xc0, + 0x6c, 0x08, 0x04, 0x51, 0x6c, 0x5a, 0xa8, 0xa8, 0x14, 0x99, 0xd5, 0xa5, 0x0d, 0x74, 0x1f, 0xce, + 0xee, 0x5a, 0xbd, 0xdd, 0x27, 0x06, 0x09, 0x70, 0xe3, 0x64, 0xb2, 0x34, 0x21, 0x37, 0x28, 0x1a, + 0x1c, 0x61, 0x88, 0x2b, 0x8a, 0x49, 0x9a, 0xb8, 0x6b, 0xf5, 0x0d, 0xbb, 0x32, 0x47, 0x27, 0x2d, + 0x47, 0x88, 0x16, 0x83, 0xa3, 0x97, 0x60, 0xce, 0x1f, 0x38, 0x34, 0x32, 0xe3, 0x1b, 0x3b, 0x4f, + 0x7b, 0x96, 0x38, 0x94, 0x7b, 0xf9, 0x8b, 0x50, 0x88, 0xab, 0xc1, 0x65, 0x96, 0xa4, 0x46, 0x00, + 0x15, 0x41, 0x79, 0xcb, 0xed, 0xb2, 0x4b, 0x22, 0x61, 0x43, 0xff, 0x4f, 0x81, 0x05, 0x09, 0x18, + 0xd5, 0x5a, 0x0a, 0xb6, 0x00, 0x4e, 0x50, 0x8a, 0x3f, 0x41, 0x20, 0x82, 0x08, 0xf7, 0x16, 0x51, + 0xab, 0xfe, 0xb9, 0x02, 0x79, 0x81, 0x45, 0x2f, 0x40, 0x91, 0x60, 0x6c, 0x2b, 0x3c, 0xd4, 0x63, + 0xaf, 0x33, 0x2b, 0x60, 0xef, 0xe2, 0x43, 0xb2, 0xf2, 0xa8, 0x4b, 0x6c, 0xcf, 0x0a, 0x5a, 0x49, + 0x40, 0x99, 0xb3, 0xac, 0x42, 0xde, 0x36, 0x42, 0x2b, 0x1c, 0x98, 0xcc, 0xca, 0x28, 0x5a, 0xd4, + 0x26, 0x52, 0xb1, 0x5d, 0xa7, 0xc7, 0x90, 0x33, 0x14, 0x19, 0x03, 0xd4, 0x26, 0xcc, 0x6b, 0x86, + 0xd3, 0xc3, 0x5b, 0x6e, 0x4f, 0x18, 0xb7, 0x65, 0xc8, 0xb3, 0xca, 0xa1, 0xb0, 0x0f, 0x5a, 0x8e, + 0xb6, 0xe5, 0xd2, 0x47, 0x4a, 0xb2, 0x7b, 0xea, 0x7f, 0xa7, 0xa1, 0x1c, 0x13, 0x89, 0x3c, 0x91, + 0xa8, 0x67, 0x30, 0xaf, 0x91, 0xe4, 0x38, 0x47, 0x07, 0x8f, 0xad, 0x68, 0xfc, 0xb3, 0x02, 0xf0, + 0xbe, 0x8f, 0xc3, 0xf0, 0xb0, 0xed, 0xec, 0xb8, 0x44, 0x78, 0x3c, 0x9d, 0xa4, 0xd1, 0xba, 0x10, + 0x1e, 0x87, 0x11, 0xb3, 0x42, 0x16, 0xe2, 0xe0, 0x27, 0x0c, 0xcd, 0xc4, 0x96, 0x73, 0xf0, 0x13, + 0x8a, 0xba, 0x0c, 0x25, 0xc3, 0x34, 0xb1, 0xa9, 0xf3, 0x60, 0x8c, 0xdb, 0xb4, 0x22, 0x05, 0x6a, + 0x0c, 0x86, 0x5e, 0x86, 0x79, 0x1f, 0xf7, 0xdd, 0x03, 0xa9, 0x1b, 0xb3, 0x6d, 0x73, 0x1c, 0x2c, + 0x3a, 0x2e, 0x41, 0xd6, 0xc7, 0x46, 0x10, 0x95, 0x86, 0x79, 0x0b, 0x55, 0x20, 0x67, 0xb2, 0x4b, + 0x0f, 0x6e, 0x89, 0x44, 0xb3, 0xfa, 0x0f, 0x8a, 0x28, 0xcf, 0xdc, 0x81, 0x0c, 0x5d, 0x20, 0x2f, + 0xcd, 0xd4, 0x9f, 0xe9, 0xa8, 0x84, 0xb0, 0x64, 0x19, 0x31, 0x12, 0xe8, 0x43, 0x98, 0xf5, 0xa8, + 0x84, 0x74, 0x5a, 0x2f, 0x61, 0x11, 0xcb, 0x8d, 0x69, 0x04, 0x1f, 0x0b, 0x58, 0xc4, 0x02, 0x5e, + 0x04, 0xb9, 0x33, 0x93, 0x57, 0xca, 0x29, 0x75, 0x15, 0xca, 0xf7, 0x07, 0xd8, 0x3f, 0x7c, 0xdf, + 0x36, 0x1c, 0xc9, 0x1f, 0xee, 0x13, 0x98, 0x08, 0xd6, 0x68, 0x43, 0xf5, 0x60, 0x41, 0xea, 0xc9, + 0xf5, 0xe2, 0xdb, 0x70, 0xc1, 0xb4, 0x82, 0x30, 0xd8, 0xb7, 0x75, 0x6f, 0xf7, 0x30, 0xb0, 0xba, + 0x86, 0xad, 0xd3, 0xee, 0xba, 0x67, 0x1b, 0x0e, 0xcf, 0x9d, 0x2f, 0x1e, 0x1f, 0xd5, 0x2a, 0x2d, + 0x2b, 0x08, 0xb7, 0xef, 0x6f, 0xbd, 0xcf, 0x7b, 0xc5, 0xa4, 0x2a, 0x9c, 0xc0, 0x09, 0x8c, 0xba, + 0xcc, 0xae, 0x55, 0xc9, 0x48, 0xdf, 0xea, 0x0c, 0xc2, 0x38, 0x23, 0x52, 0xff, 0x1e, 0xa0, 0x72, + 0x12, 0xc7, 0x99, 0xf2, 0xa0, 0x24, 0xaa, 0xec, 0x4c, 0x74, 0xec, 0xd4, 0x6f, 0x3e, 0xe3, 0xea, + 0x75, 0x1c, 0xad, 0xe8, 0x4e, 0x96, 0x88, 0x4c, 0x0e, 0x72, 0x8b, 0xa6, 0x84, 0x40, 0x7d, 0x28, + 0x4a, 0x97, 0x48, 0xe2, 0x0e, 0xad, 0x75, 0x9a, 0x09, 0xe3, 0x8b, 0xa5, 0xa1, 0xa0, 0x7a, 0x36, + 0xbe, 0x58, 0x0a, 0xaa, 0x9f, 0x29, 0x00, 0x71, 0x3f, 0xa2, 0xb0, 0xac, 0x22, 0xc7, 0x37, 0x8c, + 0xb7, 0xd0, 0x3a, 0x64, 0xf9, 0x0d, 0x59, 0x6a, 0xda, 0x1b, 0x32, 0x3e, 0x90, 0x96, 0x42, 0xd8, + 0xc5, 0x58, 0xb0, 0x6f, 0xb3, 0xf3, 0xc2, 0x4b, 0x21, 0x14, 0xba, 0x7d, 0x7f, 0x4b, 0x2b, 0xb0, + 0x0e, 0xdb, 0xfb, 0xf6, 0x9d, 0x99, 0x7c, 0xba, 0x3c, 0x53, 0xfd, 0x93, 0x34, 0x14, 0x68, 0x35, + 0x99, 0x8a, 0xe6, 0x0b, 0x05, 0x2a, 0x43, 0x89, 0x93, 0xde, 0x39, 0xd4, 0xe3, 0xf4, 0x8d, 0xc8, + 0xe9, 0x83, 0xd3, 0xc8, 0x29, 0x9a, 0xa1, 0xae, 0x49, 0x19, 0x58, 0xf3, 0x90, 0x86, 0x97, 0x26, + 0x13, 0xdd, 0x9b, 0xa7, 0x0d, 0x4e, 0x17, 0xfd, 0x31, 0x34, 0xd1, 0x8b, 0x30, 0x27, 0x5f, 0x15, + 0x46, 0xc5, 0xe4, 0x62, 0xbc, 0x37, 0x6d, 0x13, 0x7d, 0x13, 0xc0, 0xf4, 0x5d, 0xcf, 0xc3, 0xa6, + 0x6e, 0xb0, 0x2c, 0x71, 0x12, 0xaf, 0x5a, 0xe0, 0x63, 0xd6, 0xc3, 0xea, 0x6d, 0x58, 0x7e, 0xea, + 0x92, 0xc6, 0xc4, 0xc6, 0x43, 0xc5, 0xc7, 0xb4, 0x14, 0xe6, 0x56, 0x7f, 0x90, 0x82, 0xa2, 0xac, + 0xbf, 0x28, 0x04, 0xf6, 0x42, 0x40, 0x3e, 0x15, 0xef, 0x3d, 0xef, 0xa9, 0x88, 0x77, 0x62, 0x28, + 0x09, 0x0c, 0x05, 0xb4, 0xfa, 0x09, 0xcc, 0x0d, 0x77, 0x19, 0x93, 0x97, 0x6d, 0x0f, 0xe7, 0x65, + 0x6f, 0x3f, 0x97, 0x46, 0x0c, 0xc9, 0x40, 0x89, 0x5f, 0x81, 0x24, 0x31, 0xf0, 0x68, 0x98, 0x81, + 0xdf, 0x7d, 0x5e, 0xa9, 0xc8, 0x3c, 0x7c, 0x1f, 0xca, 0xa3, 0xa7, 0x7a, 0x0c, 0x07, 0x0f, 0x86, + 0x39, 0xf8, 0xc6, 0xf3, 0x19, 0x0f, 0x69, 0x7e, 0x5e, 0xbf, 0x3b, 0x0f, 0xe7, 0xee, 0xd2, 0x77, + 0x38, 0x77, 0x71, 0x68, 0x98, 0x71, 0xe9, 0x59, 0xfd, 0x77, 0x05, 0x96, 0x46, 0x31, 0xdc, 0x92, + 0x1a, 0x90, 0xef, 0x73, 0x18, 0x57, 0x97, 0x6f, 0x26, 0xb0, 0x35, 0x9e, 0x48, 0x5d, 0x00, 0x64, + 0xf5, 0x88, 0xc8, 0x56, 0x7f, 0x0f, 0x4a, 0x43, 0x1d, 0xc6, 0x48, 0xe6, 0x8d, 0x61, 0xc9, 0xc8, + 0x29, 0xd7, 0x20, 0xb4, 0xec, 0x3a, 0x7f, 0x5e, 0x14, 0x4d, 0x2c, 0x65, 0x7a, 0x3f, 0x49, 0xc1, + 0xd9, 0x4d, 0x67, 0x7f, 0x80, 0x07, 0x98, 0x7a, 0x47, 0xe1, 0xe2, 0x7e, 0x93, 0x25, 0x24, 0xe6, + 0x3e, 0xa3, 0xe0, 0x8e, 0x35, 0xd0, 0x77, 0xa4, 0x38, 0x8c, 0x5e, 0x35, 0x35, 0x37, 0x8e, 0x8f, + 0x6a, 0x39, 0xca, 0x15, 0x9d, 0xf3, 0xfa, 0x54, 0x73, 0xf2, 0x71, 0x71, 0x30, 0x77, 0x05, 0x16, + 0x82, 0x3d, 0xcb, 0xd3, 0x83, 0x5d, 0x77, 0x60, 0x9b, 0x3a, 0xe3, 0x80, 0x17, 0x9f, 0x08, 0x62, + 0x9b, 0xc2, 0xef, 0x13, 0xb0, 0xfa, 0x37, 0x29, 0x58, 0x1c, 0x96, 0x0a, 0xdf, 0xef, 0xfb, 0x71, + 0x88, 0xc3, 0xb6, 0xfb, 0xcd, 0xa4, 0x7b, 0xab, 0x31, 0x14, 0xea, 0xe2, 0x59, 0x48, 0x14, 0x1b, + 0xfd, 0xa3, 0x02, 0x39, 0x0e, 0xfc, 0x8d, 0x4a, 0xfd, 0xed, 0x91, 0x08, 0xf5, 0xa5, 0xa4, 0xbb, + 0x4e, 0xdf, 0xe8, 0x62, 0x1a, 0x71, 0x89, 0x78, 0x34, 0x4e, 0x95, 0xd2, 0x52, 0xaa, 0xa4, 0x9e, + 0x83, 0xb3, 0x1b, 0xbb, 0x86, 0x1f, 0x6e, 0xb0, 0xb7, 0x6d, 0xe2, 0xc4, 0x3c, 0x86, 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, 0xe1, 0xbd, 0x77, 0xef, 0xbd, 0xf7, 0xc1, 0xbd, 0xf2, 0x19, + 0xd2, 0xd8, 0xd8, 0x7a, 0xb8, 0xfd, 0x60, 0x53, 0x2b, 0x2b, 0xa8, 0x08, 0xf9, 0xd6, 0xfa, 0x83, + 0xf5, 0xe6, 0xfa, 0xf6, 0x66, 0x39, 0x85, 0x0a, 0x90, 0x79, 0xb0, 0xde, 0xdc, 0xda, 0x2c, 0xa7, + 0xd7, 0x7e, 0x5c, 0x85, 0xcc, 0xba, 0xd9, 0xb7, 0x1c, 0x14, 0x42, 0x86, 0x5e, 0xb8, 0xa2, 0x97, + 0x9f, 0x7d, 0x25, 0x4b, 0x17, 0x59, 0x5d, 0x9d, 0xf4, 0xee, 0x56, 0xad, 0xfc, 0xf0, 0xdf, 0x7e, + 0xf9, 0x93, 0x14, 0x42, 0xe5, 0x86, 0x4e, 0x5f, 0x3c, 0x36, 0x0e, 0xae, 0x36, 0xe8, 0x1d, 0x2e, + 0xfa, 0x23, 0x05, 0x0a, 0xd1, 0x1b, 0x3c, 0xf4, 0xea, 0x04, 0x6f, 0xdf, 0xa2, 0xe9, 0x5f, 0x9b, + 0xac, 0x33, 0x67, 0xe1, 0x22, 0x65, 0x61, 0x09, 0x2d, 0x4a, 0x2c, 0x44, 0xcf, 0xfa, 0xd0, 0x5f, + 0x29, 0x30, 0x3f, 0xf2, 0xb8, 0x0e, 0x5d, 0x9d, 0xe6, 0x21, 0x1e, 0x63, 0x69, 0x6d, 0xfa, 0xb7, + 0x7b, 0xea, 0xcb, 0x94, 0xb1, 0x17, 0x50, 0x6d, 0x1c, 0x63, 0x8d, 0x4f, 0xc4, 0xe7, 0xf7, 0xd0, + 0xdf, 0x2a, 0x50, 0x94, 0x1f, 0x56, 0xa1, 0xfa, 0xc4, 0x2f, 0xb0, 0x18, 0x77, 0x8d, 0x29, 0x5f, + 0x6c, 0xa9, 0x37, 0x28, 0x6b, 0xaf, 0xa3, 0xfa, 0x33, 0x58, 0x6b, 0x50, 0x57, 0x1e, 0x34, 0x3e, + 0xa1, 0xbf, 0x94, 0x53, 0x88, 0x5f, 0x0b, 0xa0, 0xd7, 0x26, 0x7c, 0x54, 0xc0, 0xb8, 0x9c, 0xee, + 0x09, 0x82, 0x7a, 0x8b, 0xf2, 0x78, 0x03, 0x5d, 0x9f, 0x8e, 0xc7, 0x06, 0x7b, 0x8d, 0xf2, 0x17, + 0x0a, 0x94, 0x86, 0x1e, 0x60, 0xa0, 0x24, 0x21, 0x8d, 0x7b, 0xc3, 0x51, 0x7d, 0x7d, 0xf2, 0x01, + 0x9c, 0xe5, 0x4b, 0x94, 0xe5, 0x2a, 0xaa, 0x48, 0x2c, 0x3b, 0xae, 0xc3, 0x18, 0xa4, 0x4c, 0x7c, + 0x04, 0x59, 0x76, 0xe7, 0x8f, 0x56, 0x27, 0x78, 0x16, 0xc0, 0xf8, 0x78, 0x65, 0xe2, 0x07, 0x04, + 0xea, 0x32, 0x65, 0xe0, 0x2c, 0x5a, 0x90, 0x18, 0xe0, 0x56, 0x8e, 0x9c, 0xc7, 0xe8, 0x3e, 0x3a, + 0xf1, 0x3c, 0x8e, 0xde, 0x90, 0x27, 0x9e, 0xc7, 0x93, 0x57, 0xdc, 0xfc, 0x3c, 0xaa, 0x32, 0x0f, + 0x03, 0x8b, 0x6c, 0xd7, 0x4d, 0xe5, 0x0a, 0xfa, 0x81, 0x02, 0x85, 0xdb, 0x13, 0xb1, 0x71, 0x7b, + 0x1a, 0x36, 0x4e, 0x5c, 0x00, 0x8f, 0x15, 0x05, 0x63, 0x03, 0xfd, 0x01, 0xe4, 0xf8, 0x75, 0x2e, + 0x4a, 0x92, 0xed, 0xf0, 0x75, 0x71, 0xf5, 0xca, 0x24, 0x5d, 0xf9, 0xe4, 0x55, 0x3a, 0xf9, 0x22, + 0x42, 0xd2, 0xe4, 0xfc, 0xda, 0x18, 0xfd, 0xa1, 0x02, 0x79, 0x71, 0xc1, 0x80, 0xae, 0x4c, 0x74, + 0x0b, 0xc1, 0x18, 0x78, 0x75, 0x8a, 0x1b, 0x0b, 0xf5, 0x02, 0xe5, 0xe0, 0x1c, 0x3a, 0x2b, 0x71, + 0x10, 0x88, 0x59, 0x7f, 0xa4, 0x40, 0x96, 0xdd, 0x4b, 0x24, 0xaa, 0xe1, 0xd0, 0x0d, 0x47, 0xa2, + 0x1a, 0x8e, 0x5c, 0x72, 0xbc, 0x48, 0x27, 0x5f, 0x19, 0x92, 0xfd, 0x2e, 0xed, 0xf2, 0xb8, 0x80, + 0x72, 0xfc, 0x93, 0x0a, 0x42, 0x14, 0xea, 0x13, 0x05, 0x31, 0x72, 0x61, 0x92, 0x28, 0x88, 0xd1, + 0x7b, 0x8b, 0xb1, 0x82, 0x10, 0xd7, 0x22, 0xc8, 0x83, 0x99, 0x3b, 0x6e, 0x27, 0x40, 0x5f, 0x7d, + 0x66, 0x11, 0x9e, 0xcd, 0xfc, 0xf2, 0x84, 0xc5, 0x7a, 0xf5, 0x3c, 0x9d, 0x75, 0x01, 0xcd, 0x4b, + 0xb3, 0x7e, 0x97, 0xcc, 0x44, 0x8e, 0x61, 0x54, 0x67, 0x4c, 0xd4, 0xff, 0xd1, 0x1a, 0x67, 0xa2, + 0xfe, 0x9f, 0x28, 0x5d, 0x8e, 0x75, 0x8b, 0x51, 0xf9, 0x92, 0xb2, 0x11, 0x55, 0x5b, 0x12, 0xd9, + 0x18, 0x2d, 0x11, 0x25, 0xb2, 0x71, 0xa2, 0x4a, 0x34, 0x96, 0x0d, 0x5a, 0x25, 0xf2, 0xc8, 0xc4, + 0xdf, 0x81, 0x0c, 0xbd, 0xf9, 0x4f, 0x0c, 0x4d, 0xe4, 0x57, 0x10, 0x89, 0xa1, 0xc9, 0xd0, 0x13, + 0x05, 0xf5, 0xcc, 0xeb, 0x0a, 0x7a, 0x02, 0x45, 0xf9, 0xb2, 0x39, 0xd1, 0xb1, 0x8e, 0xb9, 0x7d, + 0xaf, 0xbe, 0x71, 0xaa, 0x5b, 0x6c, 0xf5, 0x0c, 0x39, 0x61, 0xe8, 0x64, 0x07, 0x74, 0x7d, 0x4a, + 0x7a, 0xcf, 0xc9, 0xc5, 0x5f, 0x2b, 0x90, 0x17, 0x95, 0xc1, 0xc4, 0x13, 0x36, 0x52, 0x39, 0x4e, + 0x3c, 0x61, 0xa3, 0xa5, 0x46, 0xf5, 0x6d, 0xba, 0xc5, 0x6f, 0x0e, 0x9d, 0x30, 0x9a, 0x9a, 0xd8, + 0x6e, 0xef, 0xf1, 0x25, 0xb4, 0x32, 0x06, 0xdc, 0xf8, 0x44, 0x24, 0x44, 0xdf, 0x23, 0x11, 0x5a, + 0x79, 0x34, 0xab, 0x45, 0x6b, 0x53, 0xa5, 0xc0, 0x8c, 0xe9, 0x6b, 0xa7, 0x48, 0x9b, 0x23, 0x53, + 0x75, 0x71, 0x24, 0xca, 0xd0, 0x4d, 0x99, 0x9d, 0xcf, 0x14, 0x58, 0x58, 0xb7, 0xed, 0xe1, 0x2c, + 0x17, 0xbd, 0x3e, 0x45, 0x42, 0xcc, 0x58, 0xbc, 0x3a, 0x75, 0x0a, 0xad, 0xbe, 0x40, 0x19, 0xbc, + 0x80, 0x96, 0x25, 0x06, 0x59, 0xd6, 0x2b, 0xf2, 0x68, 0xf4, 0xa9, 0x02, 0x45, 0x39, 0x29, 0x49, + 0x54, 0xf3, 0x31, 0x49, 0x4d, 0x62, 0xfc, 0x38, 0x2e, 0xdb, 0x51, 0x6b, 0x94, 0xa9, 0x65, 0x74, + 0x5e, 0xf6, 0x6f, 0xa4, 0x23, 0x4f, 0x65, 0x48, 0xf8, 0x55, 0x94, 0x93, 0xc4, 0x44, 0x96, 0xc6, + 0x64, 0xe9, 0x89, 0x2c, 0x8d, 0xcb, 0x3e, 0xd5, 0xcb, 0x94, 0xa5, 0xaf, 0xa8, 0x72, 0xec, 0x85, + 0x59, 0x47, 0x9d, 0x2a, 0xdb, 0x4d, 0xe5, 0x4a, 0xf3, 0xca, 0x17, 0xff, 0xb9, 0x72, 0xe6, 0x8b, + 0xe3, 0x15, 0xe5, 0x67, 0xc7, 0x2b, 0xca, 0xcf, 0x8f, 0x57, 0x94, 0xff, 0x38, 0x5e, 0x51, 0x3e, + 0xfd, 0x72, 0xe5, 0xcc, 0xcf, 0xbe, 0x5c, 0x39, 0xf3, 0xf3, 0x2f, 0x57, 0xce, 0x3c, 0xce, 0x8b, + 0x59, 0x3a, 0x59, 0x5a, 0xb0, 0xbb, 0xf6, 0xff, 0x01, 0x00, 0x00, 0xff, 0xff, 0x36, 0xdd, 0x4f, + 0x35, 0xdb, 0x35, 0x00, 0x00, } diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index 6afb20a2b89c..a6c20a8a208c 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -335,34 +335,101 @@ message ClusterResponse { bool enterprise_enabled = 3; } -enum DrainMode { - // CLIENT instructs the server to refuse new SQL clients. - // TODO(tschottdorf): also terminate existing clients in a graceful manner. - CLIENT = 0; - // LEADERSHIP instructs the server to gracefully let all its Replicas' - // range leases expire. - LEASES = 1; -} - // DrainRequest requests the server to enter the specified draining mode. The // server first deactivates all the modes specified in 'off' and then activates // all those in 'on'. message DrainRequest { - // These are actually of type DrainMode, but grpc-gateway does not support - // proxying enum fields (yet: - // https://github.com/grpc-ecosystem/grpc-gateway/issues/5) and it fails in - // pretty dramatic ways (panics the server). - repeated int32 on = 1; - repeated int32 off = 2; - // When true, terminates the process after the given drain modes have been - // activated. + // deprecated_probe_indicator works as follows: + // - if nil, it indicates that the request is a probe only and + // the server should not actually drain. This indicator + // is supported for pre-20.1 RPC clients which do not know + // about the skip_drain field below. + // - if non-nil, it must be exactly equal to the slice [0, 1]. + // Other values result in an error. When non-nil, + // it implies do_drain = true regardless of the value of the + // other field (pre-20.1 clients don't populate the other field). + // + // This field will be removed in 20.2 in favor of + // do_drain below. + repeated int32 deprecated_probe_indicator = 1; + reserved 2; + // When true, terminates the process after the server has started draining. + // Setting both shutdown and do_drain to false causes + // the request to only operate as a probe. + // Setting do_drain to false and shutdown to true causes + // the server to shut down immediately without + // first draining. bool shutdown = 3; + // When true, perform the drain phase. See the comment above on + // shutdown for an explanation of the interaction between the two. + // do_drain is also implied by a non-nil deprecated_probe_indicator. + bool do_drain = 4; } -// DrainResponse is the response to a successful DrainRequest and lists the -// modes which are activated after having processing the request. +// DrainResponse is the response to a successful DrainRequest. message DrainResponse { - repeated int32 on = 1; + // deprecated_drain_status works as follows: + // - if the server is NOT currently draining, + // it will be set to an empty or nil slice. + // - a non-nil slice indicates the server is currently + // draining. + // + // This field exists for the benefit of 19.x clients + // and will be removed in 20.2. + repeated int32 deprecated_drain_status = 1; + // is_draining is set to true iff the server is currently draining. + // This is set to true in response to a request where skip_drain + // is false; but it can also be set to true in response + // to a probe request (!shutdown && skip_drain) if another + // drain request has been issued prior or asynchronously. + bool is_draining = 2; + // drain_remaining_indicator measures, at the time of starting to + // process the corresponding drain request, how many actions to + // fully drain the node were deemed to be necessary. Some, but not + // all, of these actions may already have been carried out by the + // time this indicator is received by the client. The client should + // issue requests until this indicator first reaches zero, which + // indicates that the node is fully drained. + // + // The API contract is the following: + // + // - upon a first Drain call with do_drain set, the remaining + // indicator will have some value >=0. If >0, it indicates that + // drain is pushing state away from the node. (What this state + // precisely means is left unspecified for this field. See below + // for details.) + // + // - upon a subsequent Drain call with do_drain set, the remaining + // indicator should have reduced in value. The drain process does best + // effort at shedding state away from the node; hopefully, all the + // state is shed away upon the first call and the progress + // indicator can be zero as early as the second call. However, + // if there was a lot of state to shed, it is possible for + // timeout to be encountered upon the first call. In that case, the + // second call will do some more work and return a non-zero value + // as well. + // + // - eventually, in an iterated sequence of DrainRequests with + // do_drain set, the remaining indicator should reduce to zero. At + // that point the client can conclude that no state is left to + // shed, and it should be safe to shut down the node with a + // DrainRequest with shutdown = true. + // + // Note that this field is left unpopulated (and thus remains at + // zero) for pre-20.1 nodes. A client can recognize this by + // observing is_draining to be false after a request with do_drain = + // true: the is_draining field is also left unpopulated by pre-20.1 + // nodes. + uint64 drain_remaining_indicator = 3; + + // drain_remaining_description is an informal (= not + // machine-parsable) string that explains the progress of the drain + // process to human eyes. This is intended for use mainly for + // troubleshooting. + // + // The field is only populated if do_drain is true in the + // request. + string drain_remaining_description = 4; } // DecommissionStatusRequest requests the decommissioning status for the diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 5226fa7cccd9..8e25f3bb0d13 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -468,6 +468,11 @@ func (ts *TestServer) RPCAddr() string { return ts.cfg.Addr } +// DrainClients exports the drainClients() method for use by tests. +func (ts *TestServer) DrainClients(ctx context.Context) error { + return ts.drainClients(ctx, nil /* reporter */) +} + // SQLAddr returns the server's listening SQL address. // Note: use ServingSQLAddr() instead unless there is a specific reason not to. func (ts *TestServer) SQLAddr() string { diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index bd4d8de64c74..7b324d59e8c9 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -105,7 +105,9 @@ func (ds *ServerImpl) Start() { // Drain changes the node's draining state through gossip and drains the // server's flowRegistry. See flowRegistry.Drain for more details. -func (ds *ServerImpl) Drain(ctx context.Context, flowDrainWait time.Duration) { +func (ds *ServerImpl) Drain( + ctx context.Context, flowDrainWait time.Duration, reporter func(int, string), +) { if err := ds.setDraining(true); err != nil { log.Warningf(ctx, "unable to gossip distsql draining state: %s", err) } @@ -120,16 +122,7 @@ func (ds *ServerImpl) Drain(ctx context.Context, flowDrainWait time.Duration) { // wait a minimum time for the draining state to be gossiped. minWait = 0 } - ds.flowRegistry.Drain(flowWait, minWait) -} - -// Undrain changes the node's draining state through gossip and undrains the -// server's flowRegistry. See flowRegistry.Undrain for more details. -func (ds *ServerImpl) Undrain(ctx context.Context) { - ds.flowRegistry.Undrain() - if err := ds.setDraining(false); err != nil { - log.Warningf(ctx, "unable to gossip distsql draining state: %s", err) - } + ds.flowRegistry.Drain(flowWait, minWait, reporter) } // setDraining changes the node's draining state through gossip to the provided diff --git a/pkg/sql/distsql/sync_flow_after_drain_test.go b/pkg/sql/distsql/sync_flow_after_drain_test.go index f6cb0754df17..8d0e8b8e0c06 100644 --- a/pkg/sql/distsql/sync_flow_after_drain_test.go +++ b/pkg/sql/distsql/sync_flow_after_drain_test.go @@ -40,7 +40,8 @@ func TestSyncFlowAfterDrain(t *testing.T) { cfg := s.DistSQLServer().(*ServerImpl).ServerConfig distSQLSrv := NewServer(ctx, cfg) - distSQLSrv.flowRegistry.Drain(time.Duration(0) /* flowDrainWait */, time.Duration(0) /* minFlowDrainWait */) + distSQLSrv.flowRegistry.Drain( + time.Duration(0) /* flowDrainWait */, time.Duration(0) /* minFlowDrainWait */, nil /* reporter */) // We create some flow; it doesn't matter what. req := execinfrapb.SetupFlowRequest{Version: execinfra.Version} diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index 8fb1f36f2a54..35b49d94b252 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -576,7 +576,7 @@ func TestDistSQLDrainingHosts(t *testing.T) { // Drain the second node and expect the query to be planned on only the // first node. distServer := tc.Server(1).DistSQLServer().(*distsql.ServerImpl) - distServer.Drain(ctx, 0 /* flowDrainWait */) + distServer.Drain(ctx, 0 /* flowDrainWait */, nil /* reporter */) expectPlan([][]string{{"https://cockroachdb.github.io/distsqlplan/decode.html#eJyUkM9Kw0AYxO8-xTKnVlba9LgnS60QqElNIgolyJp8hEC6G_cPKCHvLkkErVDR4843M79hO9jXBgLpdrfdZMybht0m8R07bJ_2u3UYsdlNmGbp_W7OPi2F9srNLueTT_mjzcGhdEmRPJKFOCBAztEaXZC12gxSNxrC8g1iyVGr1rtBzjkKbQiig6tdQxDI5EtDCcmSzGIJjpKcrJuxtjX1UZr364EJjrSVygp2BY7YO8EirQh5z6G9--q3TlYEEfT87xvWVWWokk6bRXA6YRM_RNlzEj-ms_lZ1uo_rIRsq5WlE8655mWfc1BZ0fSnVntT0N7oYsRMz3jMjUJJ1k3XYHqEajoNA7-Hg1_Dqx_hvL_4CAAA__-ln7ge"}}) diff --git a/pkg/sql/flowinfra/flow_registry.go b/pkg/sql/flowinfra/flow_registry.go index 762afa35622d..62a7ef5d344e 100644 --- a/pkg/sql/flowinfra/flow_registry.go +++ b/pkg/sql/flowinfra/flow_registry.go @@ -341,7 +341,14 @@ func (fr *FlowRegistry) waitForFlowLocked( // waited for. However, this is fine since there should be no local flows // running when the FlowRegistry drains as the draining logic starts with // draining all client connections to a node. -func (fr *FlowRegistry) Drain(flowDrainWait time.Duration, minFlowDrainWait time.Duration) { +// +// The reporter callback, if non-nil, is called on a best effort basis +// to report work that needed to be done and which may or may not have +// been done by the time this call returns. See the explanation in +// pkg/server/drain.go for details. +func (fr *FlowRegistry) Drain( + flowDrainWait time.Duration, minFlowDrainWait time.Duration, reporter func(int, string), +) { allFlowsDone := make(chan struct{}, 1) start := timeutil.Now() stopWaiting := false @@ -370,8 +377,6 @@ func (fr *FlowRegistry) Drain(flowDrainWait time.Duration, minFlowDrainWait time fr.Unlock() }() - // If the flow registry is empty, wait minFlowDrainWait for any incoming flows - // to register. fr.Lock() if len(fr.flows) == 0 { fr.Unlock() @@ -383,6 +388,10 @@ func (fr *FlowRegistry) Drain(flowDrainWait time.Duration, minFlowDrainWait time return } } + if reporter != nil { + // Report progress to the Drain RPC. + reporter(len(fr.flows), "distSQL execution flows") + } go func() { select { diff --git a/pkg/sql/flowinfra/flow_registry_test.go b/pkg/sql/flowinfra/flow_registry_test.go index 78b9179ff4c3..18f2fa84911e 100644 --- a/pkg/sql/flowinfra/flow_registry_test.go +++ b/pkg/sql/flowinfra/flow_registry_test.go @@ -396,7 +396,7 @@ func TestFlowRegistryDrain(t *testing.T) { registerFlow(t, id) drainDone := make(chan struct{}) go func() { - reg.Drain(math.MaxInt64 /* flowDrainWait */, 0 /* minFlowDrainWait */) + reg.Drain(math.MaxInt64 /* flowDrainWait */, 0 /* minFlowDrainWait */, nil /* reporter */) drainDone <- struct{}{} }() // Be relatively sure that the FlowRegistry is draining. @@ -409,7 +409,7 @@ func TestFlowRegistryDrain(t *testing.T) { // DrainTimeout verifies that Drain returns once the timeout expires. t.Run("DrainTimeout", func(t *testing.T) { registerFlow(t, id) - reg.Drain(0 /* flowDrainWait */, 0 /* minFlowDrainWait */) + reg.Drain(0 /* flowDrainWait */, 0 /* minFlowDrainWait */, nil /* reporter */) reg.UnregisterFlow(id) reg.Undrain() }) @@ -420,7 +420,7 @@ func TestFlowRegistryDrain(t *testing.T) { registerFlow(t, id) drainDone := make(chan struct{}) go func() { - reg.Drain(math.MaxInt64 /* flowDrainWait */, 0 /* minFlowDrainWait */) + reg.Drain(math.MaxInt64 /* flowDrainWait */, 0 /* minFlowDrainWait */, nil /* reporter */) drainDone <- struct{}{} }() // Be relatively sure that the FlowRegistry is draining. @@ -463,7 +463,7 @@ func TestFlowRegistryDrain(t *testing.T) { } defer func() { reg.testingRunBeforeDrainSleep = nil }() go func() { - reg.Drain(math.MaxInt64 /* flowDrainWait */, 0 /* minFlowDrainWait */) + reg.Drain(math.MaxInt64 /* flowDrainWait */, 0 /* minFlowDrainWait */, nil /* reporter */) drainDone <- struct{}{} }() if err := <-errChan; err != nil { @@ -491,7 +491,7 @@ func TestFlowRegistryDrain(t *testing.T) { minFlowDrainWait := 10 * time.Millisecond start := timeutil.Now() go func() { - reg.Drain(math.MaxInt64 /* flowDrainWait */, minFlowDrainWait) + reg.Drain(math.MaxInt64 /* flowDrainWait */, minFlowDrainWait, nil /* reporter */) drainDone <- struct{}{} }() // Be relatively sure that the FlowRegistry is draining. diff --git a/pkg/sql/lease.go b/pkg/sql/lease.go index c86ed253cd87..b1ad354266e8 100644 --- a/pkg/sql/lease.go +++ b/pkg/sql/lease.go @@ -1734,7 +1734,12 @@ func (m *LeaseManager) isDraining() bool { // SetDraining (when called with 'true') removes all inactive leases. Any leases // that are active will be removed once the lease's reference count drops to 0. -func (m *LeaseManager) SetDraining(drain bool) { +// +// The reporter callback, if non-nil, is called on a best effort basis +// to report work that needed to be done and which may or may not have +// been done by the time this call returns. See the explanation in +// pkg/server/drain.go for details. +func (m *LeaseManager) SetDraining(drain bool, reporter func(int, string)) { m.draining.Store(drain) if !drain { return @@ -1749,6 +1754,10 @@ func (m *LeaseManager) SetDraining(drain bool) { for _, l := range leases { releaseLease(l, m) } + if reporter != nil { + // Report progress through the Drain RPC. + reporter(len(leases), "table leases") + } } } diff --git a/pkg/sql/lease_test.go b/pkg/sql/lease_test.go index 3266aa3f7ed9..caa1bb5d0de1 100644 --- a/pkg/sql/lease_test.go +++ b/pkg/sql/lease_test.go @@ -480,8 +480,8 @@ func TestLeaseManagerDrain(testingT *testing.T) { // starts draining. l1RemovalTracker := leaseRemovalTracker.TrackRemoval(l1) - t.nodes[1].SetDraining(true) - t.nodes[2].SetDraining(true) + t.nodes[1].SetDraining(true, nil /* reporter */) + t.nodes[2].SetDraining(true, nil /* reporter */) // Leases cannot be acquired when in draining mode. if _, _, err := t.acquire(1, descID); !testutils.IsError(err, "cannot acquire lease when draining") { @@ -504,7 +504,7 @@ func TestLeaseManagerDrain(testingT *testing.T) { { // Check that leases with a refcount of 0 are correctly kept in the // store once the drain mode has been exited. - t.nodes[1].SetDraining(false) + t.nodes[1].SetDraining(false, nil /* reporter */) l1, _ := t.mustAcquire(1, descID) t.mustRelease(1, l1, nil) t.expectLeases(descID, "/1/1") diff --git a/pkg/sql/pgwire/helpers_test.go b/pkg/sql/pgwire/helpers_test.go index 92325928d74e..4533370616c6 100644 --- a/pkg/sql/pgwire/helpers_test.go +++ b/pkg/sql/pgwire/helpers_test.go @@ -16,7 +16,7 @@ import ( ) func (s *Server) DrainImpl(drainWait time.Duration, cancelWait time.Duration) error { - return s.drainImpl(drainWait, cancelWait) + return s.drainImpl(drainWait, cancelWait, nil /* reporter */) } // OverwriteCancelMap overwrites all active connections' context.CancelFuncs so diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index 2755a7811b28..32d7cd05a480 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -32,7 +32,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -96,19 +95,13 @@ func TestPGWireDrainClient(t *testing.T) { t.Fatal(err) } - on := []serverpb.DrainMode{serverpb.DrainMode_CLIENT} // Draining runs in a separate goroutine since it won't return until the // connection with an ongoing transaction finishes. errChan := make(chan error) go func() { defer close(errChan) errChan <- func() error { - if now, err := s.(*server.TestServer).Drain(ctx, on); err != nil { - return err - } else if !reflect.DeepEqual(on, now) { - return errors.Errorf("expected drain modes %v, got %v", on, now) - } - return nil + return s.(*server.TestServer).DrainClients(ctx) }() }() @@ -133,8 +126,8 @@ func TestPGWireDrainClient(t *testing.T) { } } - if now := s.(*server.TestServer).Undrain(ctx, on); len(now) != 0 { - t.Fatalf("unexpected active drain modes: %v", now) + if !s.(*server.TestServer).PGServer().IsDraining() { + t.Fatal("server should be draining, but is not") } } diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index 0d59b6668194..e4f4edc137fc 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -317,8 +317,13 @@ func (s *Server) Metrics() (res []interface{}) { // The RFC on drain modes has more information regarding the specifics of // what will happen to connections in different states: // https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20160425_drain_modes.md -func (s *Server) Drain(drainWait time.Duration) error { - return s.drainImpl(drainWait, cancelMaxWait) +// +// The reporter callback, if non-nil, is called on a best effort basis +// to report work that needed to be done and which may or may not have +// been done by the time this call returns. See the explanation in +// pkg/server/drain.go for details. +func (s *Server) Drain(drainWait time.Duration, reporter func(int, string)) error { + return s.drainImpl(drainWait, cancelMaxWait, reporter) } // Undrain switches the server back to the normal mode of operation in which @@ -339,7 +344,21 @@ func (s *Server) setDrainingLocked(drain bool) bool { return true } -func (s *Server) drainImpl(drainWait time.Duration, cancelWait time.Duration) error { +// drainImpl drains the SQL clients. +// +// The drainWait duration is used to wait on clients to +// self-disconnect after their session has been canceled. The +// cancelWait is used to wait after the drainWait timer has expired +// and there are still clients connected, and their context.Context is +// canceled. +// +// The reporter callback, if non-nil, is called on a best effort basis +// to report work that needed to be done and which may or may not have +// been done by the time this call returns. See the explanation in +// pkg/server/drain.go for details. +func (s *Server) drainImpl( + drainWait time.Duration, cancelWait time.Duration, reporter func(int, string), +) error { // This anonymous function returns a copy of s.mu.connCancelMap if there are // any active connections to cancel. We will only attempt to cancel // connections that were active at the moment the draining switch happened. @@ -366,6 +385,10 @@ func (s *Server) drainImpl(drainWait time.Duration, cancelWait time.Duration) er if len(connCancelMap) == 0 { return nil } + if reporter != nil { + // Report progress to the Drain RPC. + reporter(len(connCancelMap), "SQL clients") + } // Spin off a goroutine that waits for all connections to signal that they // are done and reports it on allConnsDone. The main goroutine signals this