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.ttl | duration | 720h0m0s | if nonzero, range log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours. |
server.remote_debugging.mode | string | local | set to enable remote debugging, localhost-only or disable (any, local, off) |
server.shutdown.drain_wait | duration | 0s | the amount of time a server waits in an unready state before proceeding with the rest of the shutdown process |
+server.shutdown.lease_transfer_wait | duration | 5s | the amount of time a server waits to transfer range leases before proceeding with the rest of the shutdown process |
server.shutdown.query_wait | duration | 10s | the server will wait for at least this amount of time for active queries to finish |
server.time_until_store_dead | duration | 5m0s | the time after which if there is no new gossiped information about a store, it is considered dead |
server.user_login.timeout | duration | 10s | timeout 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