From 69d5283f274f4e5b6e2b20f317678b61b7c10a0d Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 17 Feb 2020 21:16:23 +0100 Subject: [PATCH 01/12] storage: make the lease transfer timeout on shutdown configurable This is one step in a series of changes to make `cockroach quit` more friendly with clusters containing many ranges. For reference, here's the timeline of a drain below. This patch makes the delay in (D) configurable, where it was not configurable previously. A separate commit will look into the hard delay at (E). This patch uses a cluster setting instead of a more clever approach based on extra parameters passed via the client request, or some other cleverness based on the number of ranges active, so as to make the patch lightweight and readily backportable. (Customers want this in 19.1/19.2 to enable no-downtime upgrade to 20.1). ``` adminServer server pgwire distsql storage - | ----------->o server.Undrain() ----------->- server.Drain() -> doDrain() . | . - CLIENT drain starts . - grpc.setMode(draining) . . . = (A) sleep server.shutdown.drain_wait (def 0s) . . . ------>- start draining pgwire clients . . | wait for drain of clients or server.shutdown.query_wait timeout (def 10s) . . . (B) . . - cancel remaining SQL sessions . -<------ . | . ------------>- ds.setDraining() . . | start draining distsql flows . . . wait for flows to drain, with minimum 1s andm max server.shutdown.query_wait (def 10s) . . . (C) . . - cancel remaining flows . -<------------ . | . - drain SQL table leases (performs KV requests) . | . - LEASES drain starts . | . ---------------------->o nodeLiveness.SetDraining() . ---------------------->- for each Store s, SetDraining() . . | . . - attempt transfering 100 leases . . | . . - with exponential backoff (20ms->1s) repeat attempt to transfer 100 leases . . .(D) try for at most server.shutdown.lease_transfer_wait (def 5s) <- NOW CONFIGURABLE . . . . -<---------------------- stop attempting lease transfers -<----------- - stop here if client did not request Shutdown | - grpc.Stop, stopper.Stop() . start of 10 second timeout (non-configurable) . (E) - os.Exit after timeout expires ``` Release note (general change): Prior to this patch, the phase of server shutdown responsible for range lease transfers to other nodes had a hard timeout of 5 seconds. This patch makes this timeout configurable via the new cluster setting `server.shutdown.lease_transfer_wait`. Release note (backward-incompatible change): Prior to this patch, the phase of server shutdown responsible for range lease transfers to other nodes would give up after 10000 attempts of transferring replica leases away, regardless of the value of `server.shutdown.lease_transfer_wait`. The limit of 10000 attempts has been removed, so that now only the maximum duration `server.shutdown.lease_transfer_wait` applies. --- docs/generated/settings/settings.html | 1 + pkg/storage/store.go | 91 +++++++++++++++++++++------ 2 files changed, 74 insertions(+), 18 deletions(-) diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 086565ecc8ac..5ec6141a3d4d 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -90,6 +90,7 @@ server.rangelog.ttlduration720h0m0sif nonzero, range log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours. server.remote_debugging.modestringlocalset to enable remote debugging, localhost-only or disable (any, local, off) server.shutdown.drain_waitduration0sthe amount of time a server waits in an unready state before proceeding with the rest of the shutdown process +server.shutdown.lease_transfer_waitduration5sthe amount of time a server waits to transfer range leases before proceeding with the rest of the shutdown process server.shutdown.query_waitduration10sthe server will wait for at least this amount of time for active queries to finish server.time_until_store_deadduration5m0sthe time after which if there is no new gossiped information about a store, it is considered dead server.web_session_timeoutduration168h0m0sthe duration that a newly created web session will be valid diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 758be2d28280..5ddc9f3b1568 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -62,9 +62,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" ) @@ -136,6 +136,23 @@ var concurrentRangefeedItersLimit = settings.RegisterPositiveIntSetting( 64, ) +// raftLeadershipTransferTimeout limits the amount of time a drain command +// waits for lease transfers. +var raftLeadershipTransferWait = 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 + }, +) + +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, @@ -970,10 +987,6 @@ 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. @@ -993,7 +1006,7 @@ 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. sem := make(chan struct{}, 100) // Incremented for every lease or Raft leadership transfer attempted. We try @@ -1003,12 +1016,38 @@ func (s *Store) SetDraining(drain bool) { // 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() @@ -1043,8 +1082,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 { @@ -1093,31 +1141,38 @@ func (s *Store) SetDraining(drain bool) { return int(numTransfersAttempted) } - transferAllAway() + 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 { + 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) } } From e3143122dec97f504f6f78d2700a8d9203c48e3e Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 1 Apr 2020 22:04:57 +0200 Subject: [PATCH 02/12] cli: move the quit code to its own file Release note: None --- pkg/cli/quit.go | 178 +++++++++++++++++++++++++++++++++++++++++++++++ pkg/cli/start.go | 155 ----------------------------------------- 2 files changed, 178 insertions(+), 155 deletions(-) create mode 100644 pkg/cli/quit.go diff --git a/pkg/cli/quit.go b/pkg/cli/quit.go new file mode 100644 index 000000000000..bfaa0958ab5b --- /dev/null +++ b/pkg/cli/quit.go @@ -0,0 +1,178 @@ +// 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" + "time" + + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/util/grpcutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/spf13/cobra" +) + +// getAdminClient returns an AdminClient and a closure that must be invoked +// to free associated resources. +func getAdminClient(ctx context.Context) (serverpb.AdminClient, func(), error) { + conn, _, finish, err := getClientGRPCConn(ctx) + 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) + if err != nil { + return err + } + defer finish() + + if quitCtx.serverDecommission { + var myself []string // 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/cli/start.go b/pkg/cli/start.go index 643856d3f0f1..2bc07760f6fb 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" @@ -34,7 +33,6 @@ import ( "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/engine" @@ -1260,156 +1258,3 @@ func getClientGRPCConn(ctx context.Context) (*grpc.ClientConn, *hlc.Clock, func( } 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) (serverpb.AdminClient, func(), error) { - conn, _, finish, err := getClientGRPCConn(ctx) - 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) - if err != nil { - return err - } - defer finish() - - if quitCtx.serverDecommission { - var myself []string // 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") -} From 0f86a9193136bba714827d66692423782c8cb842 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 1 Apr 2020 22:08:17 +0200 Subject: [PATCH 03/12] cli/quit: reorder the functions for better readability Release note: None --- pkg/cli/quit.go | 150 ++++++++++++++++++++++++------------------------ 1 file changed, 75 insertions(+), 75 deletions(-) diff --git a/pkg/cli/quit.go b/pkg/cli/quit.go index bfaa0958ab5b..071dcc87a0e5 100644 --- a/pkg/cli/quit.go +++ b/pkg/cli/quit.go @@ -24,16 +24,6 @@ import ( "github.com/spf13/cobra" ) -// getAdminClient returns an AdminClient and a closure that must be invoked -// to free associated resources. -func getAdminClient(ctx context.Context) (serverpb.AdminClient, func(), error) { - conn, _, finish, err := getClientGRPCConn(ctx) - 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", @@ -47,31 +37,58 @@ completed, the server exits. 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, - }) +// 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) if err != nil { - return errors.Wrap(err, "Failed to connect to the node: error sending drain request") + return err } - // 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) + defer finish() + + if quitCtx.serverDecommission { + var myself []string // 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 } - break + return err } + return nil + case <-time.After(time.Minute): + log.Warningf(ctx, "timed out; proceeding with hard shutdown") } - return nil + // 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") } // doShutdown attempts to trigger a server shutdown. When given an empty @@ -123,56 +140,39 @@ func doShutdown(ctx context.Context, c serverpb.AdminClient, onModes []int32) er 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) +// 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 err - } - defer finish() - - if quitCtx.serverDecommission { - var myself []string // will remain empty, which means target yourself - if err := runDecommissionNodeImpl(ctx, c, nodeDecommissionWaitAll, myself); err != nil { - return err - } + return errors.Wrap(err, "Failed to connect to the node: error sending drain request") } - 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 + // 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 } - return err + if err != io.EOF { + log.Warningf(ctx, "unexpected error from no-op Drain request: %s", err) + } + break } - 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) + return nil +} + +// getAdminClient returns an AdminClient and a closure that must be invoked +// to free associated resources. +func getAdminClient(ctx context.Context) (serverpb.AdminClient, func(), error) { + conn, _, finish, err := getClientGRPCConn(ctx) if err != nil { - log.Warningf(ctx, "hard shutdown attempt failed, retrying: %v", err) - err = doShutdown(ctx, c, nil) + return nil, nil, errors.Wrap(err, "Failed to connect to the node") } - return errors.Wrap(doShutdown(ctx, c, nil), "hard shutdown failed") + return serverpb.NewAdminClient(conn), finish, nil } From 9edb00736d3074e09f7b2492b401cf6a19a1842a Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 17 Feb 2020 22:45:21 +0100 Subject: [PATCH 04/12] cli: make the drain wait configurable + new 'cockroach node drain' This is another step in a series of changes to make `cockroach quit` more friendly with clusters containing many ranges, and enabling graceful shutdowns when a separate service manager is responsible for stopping processes. This patch focuses on the client behavior, without considering the behavior server-side. For reference, here's the timeline of a `quit` call below. This patch introduces two things: - it makes the delay in step (X) configurable with `--drain-wait`, where it was not configurable previously. The default remains at 1 minute. - it makes it possible to drain without quitting with a new client command `cockroach node drain`, so that a DBA can look at the admin UI and satisfy themselves that the rest of the cluster has picked up the load. This patch reuses the existing Drain() RPC parameters, so that a `cockroach quit` or `cockroach node drain` client with this patch can be used with a server running a previous version. This is aiming to facilitate upgrades to the latest version on large clusters. ``` quit | - start decommission... . wait for decommission to complete (no timeout) . - end of decommission. | - start draining (no shutdown request)... . (X) wait for drain to complete or timeout specified with --drain-wait (NEW! def. 1m) . - timeout expired or drain completed. | - if 'cockroach node drain' (NEW!) was specified, stop here. | - send shutdown request (with no drain request) - if 1st shutdown fail, try again. | - end. ``` Additionally, the patch simplifies the drain/shutdown code as follows: - it now uses context.WithTimeout / contextutil.RunWithTimeout instead of its own bespoke goroutine / time.After / select dance. (The previous code was implemented before gprc supported context cancellation.) - it separates the code that sends a drain request from the code that sends a shutdown request, thereby dramatically reducing the number of conditionals. - it provides more friendly error messages in case of timeouts and when attempting operations when the cluster has not been initialized yet. Release note (backward-incompatible change): The textual error and warning messages displayed by `cockroach quit` under various circumstances have been updated. Meanwhile, the message "`ok`" remains as indicator that the operation has likely succeeded. Release note (cli change): The time that `cockroach quit` waits client-side for the node to drain (remove existing clients and push range leases away) is now configurable via the command-line flag `--drain-wait`. Note that separate server-side timeouts also apply separately, check the `server.shutdown.*` cluster settings for details. Release note (cli change): It is now possible to drain a node without shutting down the process, using `cockroach node drain`. This makes it easier to integrate with service managers and orchestration: it now becomes safe to issue `cockroach node drain` and then separately stop the service via a process manager or orchestrator. Without this new mode, there is a risk to misconfigure the service manager to auto-restart the node after it shuts down via `quit`, in a way that's surprising or unwanted. The new command `node drain` also recognizes the new `--drain-wait` flag. --- pkg/cli/cli_test.go | 4 +- pkg/cli/cliflags/flags.go | 7 ++ pkg/cli/context.go | 9 +- pkg/cli/flags.go | 7 ++ pkg/cli/node.go | 41 ++++++- pkg/cli/quit.go | 215 ++++++++++++++++++++-------------- pkg/server/admin.go | 46 ++++++-- pkg/server/grpc_server.go | 3 +- pkg/server/server.go | 10 +- pkg/sql/pgwire/pgwire_test.go | 6 +- 10 files changed, 239 insertions(+), 109 deletions(-) diff --git a/pkg/cli/cli_test.go b/pkg/cli/cli_test.go index 98f2c1c66791..9f652b07a91f 100644 --- a/pkg/cli/cli_test.go +++ b/pkg/cli/cli_test.go @@ -1592,12 +1592,12 @@ 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 user get, set, list and remove users (deprecated) 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 demo open a demo sql shell diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index 8e192e1f577f..139cfbd1b682 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -843,6 +843,13 @@ If specified, decommissions the node and waits for it to rebalance before shutting down the node.`, } + 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 1d07bf39508d..c215ec0e7dc4 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -121,6 +121,7 @@ func initCLIDefaults() { startCtx.inBackground = false quitCtx.serverDecommission = false + quitCtx.drainWait = time.Minute nodeCtx.nodeDecommissionWait = nodeDecommissionWaitAll nodeCtx.statusShowRanges = false @@ -310,10 +311,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/flags.go b/pkg/cli/flags.go index 3e603e1a244d..c73bde9b7c99 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -542,6 +542,13 @@ func init() { // Quit command. BoolFlag(quitCmd.Flags(), &quitCtx.serverDecommission, cliflags.Decommission, quitCtx.serverDecommission) + // 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/node.go b/pkg/cli/node.go index 09c8d5543ed3..d38eddbc0f2f 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -434,18 +434,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) + 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 index 071dcc87a0e5..6f3b66cde5a9 100644 --- a/pkg/cli/quit.go +++ b/pkg/cli/quit.go @@ -18,6 +18,7 @@ import ( "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" @@ -27,11 +28,17 @@ import ( // quitCmd command shuts down the node server. var quitCmd = &cobra.Command{ Use: "quit", - Short: "drain and shutdown node\n", + Short: "drain and shut down a 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. +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), @@ -42,129 +49,163 @@ 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") } }() - onModes := make([]int32, len(server.GracefulDrainModes)) - for i, m := range server.GracefulDrainModes { - onModes[i] = int32(m) - } + // Establish a RPC connection. c, finish, err := getAdminClient(ctx) 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 []string // 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 + log.Warningf(ctx, "%v", err) + if server.IsWaitingForInit(err) { + err = errors.New("node cannot be decommissioned before it has been initialized") } 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) + + 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, err := doDrain(ctx, c) + if hardError { + return err + } + 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, nil) + _, err = doShutdown(ctx, c) } - return errors.Wrap(doShutdown(ctx, c, nil), "hard shutdown failed") + return errors.Wrap(err, "hard shutdown failed") } -// doShutdown attempts to trigger a server shutdown. When given an empty -// onModes slice, it's a hard shutdown. +// doDrain calls a graceful drain. // -// 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 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 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, 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 hardError, err +} + +func doDrainNoTimeout(ctx context.Context, c serverpb.AdminClient) (hardError bool, err error) { + defer func() { if server.IsWaitingForInit(err) { - return fmt.Errorf("node cannot be shut down before it has been initialized") + log.Infof(ctx, "%v", err) + err = errors.New("node cannot be drained before it has been initialized") } - if grpcutil.IsClosedConnection(err) { - return nil - } - return err + }() + + // modes is the set of drain flags to pass to the Drain() RPC. + modes := make([]int32, len(server.GracefulDrainModes)) + for i, m := range server.GracefulDrainModes { + modes[i] = int32(m) } - // 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, - }) + + // Send a drain request with the drain bit set and the shutdown bit + // unset. + stream, err := c.Drain(ctx, &serverpb.DrainRequest{On: modes}) 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") + return true, 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} + _, err := stream.Recv() + if err == io.EOF { + // Done. + break + } + if err != nil { + // Unexpected error. + log.Infof(ctx, "graceful shutdown failed: %v", err) + return false, err } + // Iterate until end of stream, which indicates the drain is + // complete. } + return false, nil } -type errTryHardShutdown struct{ error } +// 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) { + // 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 + } + }() -// 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 + // 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 != io.EOF { - log.Warningf(ctx, "unexpected error from no-op Drain request: %s", err) + if err != nil { + return err } - break } + }) + if _, ok := err.(*contextutil.TimeoutError); !ok { + hardError = true } - return nil + return hardError, err } // getAdminClient returns an AdminClient and a closure that must be invoked diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 280630371bd4..65ba0c456e15 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -1471,23 +1471,47 @@ func (s *adminServer) QueryPlan( // 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 { + // `cockroach quit` sends a probe to check that the server is able + // to process a Drain request, prior to sending an actual drain + // request. + isProbe := len(req.On) == 0 && len(req.Off) == 0 && !req.Shutdown + + ctx := stream.Context() + 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) + if !isProbe { + var buf strings.Builder + comma := "" + for i := range req.On { + on[i] = serverpb.DrainMode(req.On[i]) + fmt.Fprintf(&buf, "%sshutdown: %s", comma, on[i].String()) + comma = ", " + } + for i := range req.Off { + off[i] = serverpb.DrainMode(req.Off[i]) + fmt.Fprintf(&buf, "%sresume: %s", comma, off[i].String()) + comma = ", " + } + // Report the event to the log file for troubleshootability. + log.Infof(ctx, "drain request received (%s), process shutdown: %v", buf.String(), req.Shutdown) + } else { + log.Infof(ctx, "received request for drain status") + } - nowOn, err := s.server.Drain(ctx, on) + nowOn, err := s.server.Undrain(ctx, off) if err != nil { return err } + if len(on) > 0 { + nowOn, err = s.server.Drain(ctx, on) + if err != nil { + return err + } + } + // Report the current status to the client. res := serverpb.DrainResponse{ On: make([]int32, len(nowOn)), } @@ -1499,6 +1523,10 @@ func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_Dr } if !req.Shutdown { + if !isProbe { + // We don't need an info message for just a probe. + log.Infof(ctx, "drain request completed without server shutdown") + } return 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/server.go b/pkg/server/server.go index 61bcc33c4def..23a5f9beae8d 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -1990,12 +1990,10 @@ func (s *Server) Drain(ctx context.Context, on []serverpb.DrainMode) ([]serverpb // 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 +func (s *Server) Undrain( + ctx context.Context, off []serverpb.DrainMode, +) ([]serverpb.DrainMode, error) { + return s.doDrain(ctx, off, false /* setTo */) } // Decommission idempotently sets the decommissioning flag for specified nodes. diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index aaa6b365a3d0..1ae5e6a1ec4a 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -275,7 +275,11 @@ func TestPGWireDrainClient(t *testing.T) { } } - if now := s.(*server.TestServer).Undrain(ctx, on); len(now) != 0 { + now, err := s.(*server.TestServer).Undrain(ctx, on) + if err != nil { + t.Fatal(err) + } + if len(now) != 0 { t.Fatalf("unexpected active drain modes: %v", now) } } From b6a8bf7940ea59cfed547adbe113c6773cb8ffde Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 30 Mar 2020 14:54:31 +0200 Subject: [PATCH 05/12] server: move the drain code to its own file Release note: None --- pkg/server/admin.go | 97 ------------------- pkg/server/drain.go | 218 +++++++++++++++++++++++++++++++++++++++++++ pkg/server/server.go | 94 ------------------- 3 files changed, 218 insertions(+), 191 deletions(-) create mode 100644 pkg/server/drain.go diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 65ba0c456e15..c2052652f206 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -15,7 +15,6 @@ import ( "context" "encoding/json" "fmt" - "os" "sort" "strconv" "strings" @@ -1468,102 +1467,6 @@ func (s *adminServer) QueryPlan( }, nil } -// 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 { - // `cockroach quit` sends a probe to check that the server is able - // to process a Drain request, prior to sending an actual drain - // request. - isProbe := len(req.On) == 0 && len(req.Off) == 0 && !req.Shutdown - - ctx := stream.Context() - - on := make([]serverpb.DrainMode, len(req.On)) - off := make([]serverpb.DrainMode, len(req.Off)) - - if !isProbe { - var buf strings.Builder - comma := "" - for i := range req.On { - on[i] = serverpb.DrainMode(req.On[i]) - fmt.Fprintf(&buf, "%sshutdown: %s", comma, on[i].String()) - comma = ", " - } - for i := range req.Off { - off[i] = serverpb.DrainMode(req.Off[i]) - fmt.Fprintf(&buf, "%sresume: %s", comma, off[i].String()) - comma = ", " - } - // Report the event to the log file for troubleshootability. - log.Infof(ctx, "drain request received (%s), process shutdown: %v", buf.String(), req.Shutdown) - } else { - log.Infof(ctx, "received request for drain status") - } - - nowOn, err := s.server.Undrain(ctx, off) - if err != nil { - return err - } - if len(on) > 0 { - nowOn, err = s.server.Drain(ctx, on) - if err != nil { - return err - } - } - - // Report the current status to the client. - 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 { - if !isProbe { - // 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") - } -} - // 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..92fb2a6aa076 --- /dev/null +++ b/pkg/server/drain.go @@ -0,0 +1,218 @@ +// 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" + "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/errors" +) + +var ( + // GracefulDrainModes is the standard succession of drain modes entered + // for a graceful shutdown. + GracefulDrainModes = []serverpb.DrainMode{serverpb.DrainMode_CLIENT, serverpb.DrainMode_LEASES} + + queryWait = settings.RegisterDurationSetting( + "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.RegisterDurationSetting( + "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 { + // `cockroach quit` sends a probe to check that the server is able + // to process a Drain request, prior to sending an actual drain + // request. + isProbe := len(req.On) == 0 && len(req.Off) == 0 && !req.Shutdown + + ctx := stream.Context() + ctx = s.server.AnnotateCtx(ctx) + + on := make([]serverpb.DrainMode, len(req.On)) + off := make([]serverpb.DrainMode, len(req.Off)) + + if !isProbe { + var buf strings.Builder + comma := "" + for i := range req.On { + on[i] = serverpb.DrainMode(req.On[i]) + fmt.Fprintf(&buf, "%sshutdown: %s", comma, on[i].String()) + comma = ", " + } + for i := range req.Off { + off[i] = serverpb.DrainMode(req.Off[i]) + fmt.Fprintf(&buf, "%sresume: %s", comma, off[i].String()) + comma = ", " + } + // Report the event to the log file for troubleshootability. + log.Infof(ctx, "drain request received (%s), process shutdown: %v", buf.String(), req.Shutdown) + } else { + log.Infof(ctx, "received request for drain status") + } + + nowOn, err := s.server.Undrain(ctx, off) + if err != nil { + return err + } + if len(on) > 0 { + nowOn, err = s.server.Drain(ctx, on) + if err != nil { + return err + } + } + + // Report the current status to the client. + 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 { + if !isProbe { + // 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 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, error) { + return s.doDrain(ctx, off, false /* setTo */) +} + +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 +} diff --git a/pkg/server/server.go b/pkg/server/server.go index 23a5f9beae8d..5de36dbd517f 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -90,23 +90,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.RegisterDurationSetting( - "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.RegisterDurationSetting( - "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.RegisterBoolSetting( "server.clock.forward_jump_check_enabled", "if enabled, forward clock jumps > max_offset/2 will cause a panic", @@ -1919,83 +1902,6 @@ func (s *Server) bootstrapCluster(ctx context.Context) error { }) } -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, error) { - return s.doDrain(ctx, off, false /* setTo */) -} - // 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) From f219cd098366affc492c68e9774bd023f6aaa1b6 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 30 Mar 2020 15:06:16 +0200 Subject: [PATCH 06/12] server: split the body of doDrain() into separate functions Release justification: low risk, high benefit changes to existing functionality Release note: None --- pkg/server/drain.go | 79 +++++++++++++++++++++++++++------------------ 1 file changed, 47 insertions(+), 32 deletions(-) diff --git a/pkg/server/drain.go b/pkg/server/drain.go index 92fb2a6aa076..b6e657d7d066 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -166,41 +166,12 @@ func (s *Server) doDrain( 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 { + if err := s.drainClients(ctx, setTo); err != nil { return nil, err } + case serverpb.DrainMode_LEASES: - s.nodeLiveness.SetDraining(ctx, setTo) - if err := s.node.SetDraining(setTo); err != nil { + if err := s.drainRangeLeases(ctx, setTo); err != nil { return nil, err } default: @@ -216,3 +187,47 @@ func (s *Server) doDrain( } return nowOn, nil } + +// drainClients starts draining the SQL layer if setTo is true, and +// stops draining SQL if setTo is false. +func (s *Server) drainClients(ctx context.Context, setTo bool) error { + 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)) + } else { + // We're disableing the drain. Make the server + // operational again at the end. + defer func() { s.grpc.setMode(modeOperational) }() + } + // Since enabling the SQL table 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 { + // Re-enable client connections. + s.distSQLServer.Undrain(ctx) + s.pgServer.Undrain() + return nil + } + + // Disable incoming SQL clients up to the queryWait timeout. + drainMaxWait := queryWait.Get(&s.st.SV) + if err := s.pgServer.Drain(drainMaxWait); err != nil { + return err + } + // Stop ongoing SQL execution up to the queryWait timeout. + s.distSQLServer.Drain(ctx, drainMaxWait) + + // Done. This executes the defers set above to (un)drain SQL leases. + return nil +} + +// drainRangeLeases starts draining range leases if setTo is true, and stops draining leases if setTo is false. +func (s *Server) drainRangeLeases(ctx context.Context, setTo bool) error { + s.nodeLiveness.SetDraining(ctx, setTo) + return s.node.SetDraining(setTo) +} From 6d746a23d87a3dd74402c803e8046f5546bab33a Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 30 Mar 2020 16:17:37 +0200 Subject: [PATCH 07/12] server: simplify/rationalize the Drain API Prior to this patch, the Drain RPC was providing much untested flexibility to clients of the RPC: fields on the DrainRequest message that had a lot of possible values, but where only a few of the possible values ever made sense. This flexibility was not tested and I have reasons to suspect that many combinations of parameters would not work. See https://github.com/cockroachdb/cockroach/issues/46388 for details and a discussion. This patch removes the untested / unused flexibility and simplifies the code accordingly: - for requests: - the `Off` field is removed. It had never been used anyway. - the `On` field was only used ever with values `nil` (probe only = RPC health check) and `[0, 1]`. it is now renamed to `DeprecatedProbeIndicator` and the server now asserts that a non-`nil` input is exactly `[0, 1]`. It is still recognized for compatibility with `cockroach quit` from previous versions. - a new bool field `DoDrain` replaces `On` to indicate whether the request should do a graceful drain or not. - for responses: - the `On` field was only used ever with values `nil` (no drain in progress) and `[0, 1]` (drain currently in progress). The field is renamed `DeprecatedDrainStatus` for compatibility with hypothetical clients from previous versions (AFAIK, no client ever checked that field - maybe we could remove it altogether). - a new bool `IsDraining` reports the current draining status of the server. This patch also introduces a server-side unit test for the Drain RPC, which was missing previously. Release note: None --- pkg/cli/quit.go | 11 +- pkg/cli/start.go | 2 +- pkg/server/drain.go | 193 +++--- pkg/server/drain_test.go | 200 ++++++ pkg/server/serverpb/admin.pb.go | 1041 ++++++++++++++--------------- pkg/server/serverpb/admin.proto | 60 +- pkg/server/testserver.go | 5 + pkg/sql/distsql/server.go | 9 - pkg/sql/pgwire/pgwire_test.go | 17 +- pkg/storage/node_liveness.go | 15 +- pkg/storage/node_liveness_test.go | 2 +- 11 files changed, 851 insertions(+), 704 deletions(-) create mode 100644 pkg/server/drain_test.go diff --git a/pkg/cli/quit.go b/pkg/cli/quit.go index 6f3b66cde5a9..8cd8e6ea303f 100644 --- a/pkg/cli/quit.go +++ b/pkg/cli/quit.go @@ -134,15 +134,12 @@ func doDrainNoTimeout(ctx context.Context, c serverpb.AdminClient) (hardError bo } }() - // modes is the set of drain flags to pass to the Drain() RPC. - modes := make([]int32, len(server.GracefulDrainModes)) - for i, m := range server.GracefulDrainModes { - modes[i] = int32(m) - } - // Send a drain request with the drain bit set and the shutdown bit // unset. - stream, err := c.Drain(ctx, &serverpb.DrainRequest{On: modes}) + stream, err := c.Drain(ctx, &serverpb.DrainRequest{ + DeprecatedProbeIndicator: server.DeprecatedDrainParameter, + DoDrain: true, + }) if err != nil { return true, errors.Wrap(err, "error sending drain request") } diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 2bc07760f6fb..2b22f26f8f8c 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -896,7 +896,7 @@ 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 { + if err := s.Drain(drainCtx); err != nil { log.Warning(drainCtx, err) } stopper.Stop(drainCtx) diff --git a/pkg/server/drain.go b/pkg/server/drain.go index b6e657d7d066..d2b8faf315bd 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -12,21 +12,30 @@ package server import ( "context" - "fmt" "os" - "strings" + "reflect" "time" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) var ( - // GracefulDrainModes is the standard succession of drain modes entered - // for a graceful shutdown. - GracefulDrainModes = []serverpb.DrainMode{serverpb.DrainMode_CLIENT, serverpb.DrainMode_LEASES} + // 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.RegisterDurationSetting( "server.shutdown.query_wait", @@ -46,61 +55,41 @@ var ( // 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 { - // `cockroach quit` sends a probe to check that the server is able - // to process a Drain request, prior to sending an actual drain - // request. - isProbe := len(req.On) == 0 && len(req.Off) == 0 && !req.Shutdown - ctx := stream.Context() ctx = s.server.AnnotateCtx(ctx) - on := make([]serverpb.DrainMode, len(req.On)) - off := make([]serverpb.DrainMode, len(req.Off)) - - if !isProbe { - var buf strings.Builder - comma := "" - for i := range req.On { - on[i] = serverpb.DrainMode(req.On[i]) - fmt.Fprintf(&buf, "%sshutdown: %s", comma, on[i].String()) - comma = ", " - } - for i := range req.Off { - off[i] = serverpb.DrainMode(req.Off[i]) - fmt.Fprintf(&buf, "%sresume: %s", comma, off[i].String()) - comma = ", " + 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.") } - // Report the event to the log file for troubleshootability. - log.Infof(ctx, "drain request received (%s), process shutdown: %v", buf.String(), req.Shutdown) - } else { - log.Infof(ctx, "received request for drain status") } - nowOn, err := s.server.Undrain(ctx, off) - if err != nil { - return err - } - if len(on) > 0 { - nowOn, err = s.server.Drain(ctx, on) - if err != nil { + log.Infof(ctx, "drain request received with doDrain = %v, shutdown = %v", doDrain, req.Shutdown) + + if doDrain { + if err := s.server.Drain(ctx); err != nil { + log.Errorf(ctx, "drain failed: %v", err) return err } } - - // Report the current status to the client. - res := serverpb.DrainResponse{ - On: make([]int32, len(nowOn)), - } - for i := range nowOn { - res.On[i] = int32(nowOn[i]) + res := serverpb.DrainResponse{} + if s.server.isDraining() { + res.DeprecatedDrainStatus = DeprecatedDrainParameter + res.IsDraining = true } + if err := stream.Send(&res); err != nil { return err } if !req.Shutdown { - if !isProbe { - // We don't need an info message for just a probe. + 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 @@ -140,79 +129,48 @@ func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_Dr } } -// 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 */) +// 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. +// +// 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) error { + // First drain all clients and SQL leases. + if err := s.drainClients(ctx); err != nil { + return err + } + // Finally, mark the node as draining in liveness and drain the + // range leases. + return s.drainNode(ctx) } -// 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, error) { - return s.doDrain(ctx, off, false /* setTo */) +// 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() } -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 err := s.drainClients(ctx, setTo); err != nil { - return nil, err - } - - case serverpb.DrainMode_LEASES: - if err := s.drainRangeLeases(ctx, 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 -} +// drainClients starts draining the SQL layer. +func (s *Server) drainClients(ctx context.Context) 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)) -// drainClients starts draining the SQL layer if setTo is true, and -// stops draining SQL if setTo is false. -func (s *Server) drainClients(ctx context.Context, setTo bool) error { - 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)) - } else { - // We're disableing the drain. Make the server - // operational again at the end. - defer func() { s.grpc.setMode(modeOperational) }() - } // Since enabling the SQL table 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 { - // Re-enable client connections. - s.distSQLServer.Undrain(ctx) - s.pgServer.Undrain() - return nil - } + defer s.leaseMgr.SetDraining(true /* drain */) // Disable incoming SQL clients up to the queryWait timeout. drainMaxWait := queryWait.Get(&s.st.SV) @@ -222,12 +180,19 @@ func (s *Server) drainClients(ctx context.Context, setTo bool) error { // Stop ongoing SQL execution up to the queryWait timeout. s.distSQLServer.Drain(ctx, drainMaxWait) - // Done. This executes the defers set above to (un)drain SQL leases. + // Done. This executes the defers set above to drain SQL leases. return nil } -// drainRangeLeases starts draining range leases if setTo is true, and stops draining leases if setTo is false. -func (s *Server) drainRangeLeases(ctx context.Context, setTo bool) error { - s.nodeLiveness.SetDraining(ctx, setTo) - return s.node.SetDraining(setTo) +// drainNode initiates the draining mode for the node, which +// starts draining range leases. +func (s *Server) drainNode(ctx context.Context) error { + s.nodeLiveness.SetDraining(ctx, true /* drain */) + return s.node.SetDraining(true /* drain */) +} + +// stopDrain should be called prior to successive invocations of +// drainNode(), otherwise the drain call would deadlock. +func (s *Server) stopDrain(ctx context.Context) error { + return s.node.SetDraining(false /* drain */) } diff --git a/pkg/server/drain_test.go b/pkg/server/drain_test.go new file mode 100644 index 000000000000..7de01b5d56e3 --- /dev/null +++ b/pkg/server/drain_test.go @@ -0,0 +1,200 @@ +// 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(t *testing.T, newInterface bool) { + 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, + }, + }) + defer tc.Stopper().Stop(context.TODO()) + + ctx := context.TODO() + + // We'll have the RPC talk to the first node. + c, finish, err := getAdminClientForServer(ctx, tc, 0 /* serverIdx */) + if err != nil { + t.Fatal(err) + } + defer finish() + + // Issue a probe. We're not draining yet, so the probe should + // reflect that. + checkDrainProbe(ctx, t, c, false /* expectedDrainStatus */) + + // Issue a drain without shutdown, so we can probe more afterwards. + req := &serverpb.DrainRequest{Shutdown: false} + if newInterface { + req.DoDrain = true + } else { + req.DeprecatedProbeIndicator = server.DeprecatedDrainParameter + } + drainStream, err := c.Drain(ctx, req) + if err != nil { + t.Fatal(err) + } + resp, err := getDrainResponse(t, drainStream) + if err != nil { + t.Fatal(err) + } + checkDrainStatus(t, resp, true /* expectedDrainStatus */) + + // 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. + checkDrainProbe(ctx, t, c, true /* expectedDrainStatus */) + + // Now issue a drain request without drain but with shutdown. + // We're expecting the node to be shut down after that. + req = &serverpb.DrainRequest{Shutdown: true} + drainStream, err = c.Drain(ctx, req) + if err != nil { + t.Fatal(err) + } + resp, err = getDrainResponse(t, 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) + } + if resp != nil { + checkDrainStatus(t, resp, true /* expectedDrainStatus */) + } + + // Now expect the server to be shut down. + testutils.SucceedsSoon(t, func() error { + _, err := c.Drain(ctx, &serverpb.DrainRequest{Shutdown: false}) + if grpcutil.IsClosedConnection(err) { + return nil + } + return errors.Newf("server not yet refusing RPC, got %v", err) + }) +} + +// checkDrainProbe issues a drain probe and asserts that the +// server is alive. It also asserts that its drain status +// is the one expected. +func checkDrainProbe( + ctx context.Context, t *testing.T, c serverpb.AdminClient, expectedDrainStatus bool, +) { + // Issue a simple drain probe. This should always succeed, + // and report the server is not currently draining. + req := &serverpb.DrainRequest{Shutdown: false} + drainStream, err := c.Drain(ctx, req) + if err != nil { + t.Fatal(err) + } + resp, err := getDrainResponse(t, drainStream) + if err != nil { + t.Fatal(err) + } + checkDrainStatus(t, resp, expectedDrainStatus) +} + +func checkDrainStatus(t *testing.T, resp *serverpb.DrainResponse, expectedDrainStatus bool) { + if resp.IsDraining != expectedDrainStatus { + t.Fatalf("expected node drain status to be %v, got %# v", expectedDrainStatus, pretty.Formatter(resp)) + } + + // 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 expectedDrainStatus { + 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 getDrainResponse( + t *testing.T, 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.Version, + ) + 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/serverpb/admin.pb.go b/pkg/server/serverpb/admin.pb.go index adcb0c55c7d7..8780b577daf2 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_058b4f1680004f39, []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_058b4f1680004f39, []int{1} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{0} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{1} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{2} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{3} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{3, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{4} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{5} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{5, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{5, 1} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{5, 2} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{6} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{7} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{7, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{8} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{9} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{10} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{11} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{11, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{12} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{13} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{13, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{14} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{15} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{16} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{17} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{17, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{18} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{19} + return fileDescriptor_admin_2d30d36d58fa713c, []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_058b4f1680004f39, []int{20} + return fileDescriptor_admin_2d30d36d58fa713c, []int{20} } func (m *DrainRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1235,17 +1223,30 @@ 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"` } 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_058b4f1680004f39, []int{21} + return fileDescriptor_admin_2d30d36d58fa713c, []int{21} } func (m *DrainResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1280,7 +1281,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_058b4f1680004f39, []int{22} + return fileDescriptor_admin_2d30d36d58fa713c, []int{22} } func (m *DecommissionStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1318,7 +1319,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_058b4f1680004f39, []int{23} + return fileDescriptor_admin_2d30d36d58fa713c, []int{23} } func (m *DecommissionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1353,7 +1354,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_058b4f1680004f39, []int{24} + return fileDescriptor_admin_2d30d36d58fa713c, []int{24} } func (m *DecommissionStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1391,7 +1392,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_058b4f1680004f39, []int{24, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []int{24, 0} } func (m *DecommissionStatusResponse_Status) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1427,7 +1428,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_058b4f1680004f39, []int{25} + return fileDescriptor_admin_2d30d36d58fa713c, []int{25} } func (m *SettingsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1461,7 +1462,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_058b4f1680004f39, []int{26} + return fileDescriptor_admin_2d30d36d58fa713c, []int{26} } func (m *SettingsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1496,7 +1497,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_058b4f1680004f39, []int{26, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []int{26, 0} } func (m *SettingsResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1549,7 +1550,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_058b4f1680004f39, []int{27} + return fileDescriptor_admin_2d30d36d58fa713c, []int{27} } func (m *HealthRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1583,7 +1584,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_058b4f1680004f39, []int{28} + return fileDescriptor_admin_2d30d36d58fa713c, []int{28} } func (m *HealthResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1616,7 +1617,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_058b4f1680004f39, []int{29} + return fileDescriptor_admin_2d30d36d58fa713c, []int{29} } func (m *LivenessRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1651,7 +1652,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_058b4f1680004f39, []int{30} + return fileDescriptor_admin_2d30d36d58fa713c, []int{30} } func (m *LivenessResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1687,7 +1688,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_058b4f1680004f39, []int{31} + return fileDescriptor_admin_2d30d36d58fa713c, []int{31} } func (m *JobsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1721,7 +1722,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_058b4f1680004f39, []int{32} + return fileDescriptor_admin_2d30d36d58fa713c, []int{32} } func (m *JobsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1774,7 +1775,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_058b4f1680004f39, []int{32, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []int{32, 0} } func (m *JobsResponse_Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1807,7 +1808,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_058b4f1680004f39, []int{33} + return fileDescriptor_admin_2d30d36d58fa713c, []int{33} } func (m *LocationsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1841,7 +1842,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_058b4f1680004f39, []int{34} + return fileDescriptor_admin_2d30d36d58fa713c, []int{34} } func (m *LocationsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1877,7 +1878,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_058b4f1680004f39, []int{34, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []int{34, 0} } func (m *LocationsResponse_Location) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1919,7 +1920,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_058b4f1680004f39, []int{35} + return fileDescriptor_admin_2d30d36d58fa713c, []int{35} } func (m *RangeLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1953,7 +1954,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_058b4f1680004f39, []int{36} + return fileDescriptor_admin_2d30d36d58fa713c, []int{36} } func (m *RangeLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1993,7 +1994,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_058b4f1680004f39, []int{36, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []int{36, 0} } func (m *RangeLogResponse_PrettyInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2027,7 +2028,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_058b4f1680004f39, []int{36, 1} + return fileDescriptor_admin_2d30d36d58fa713c, []int{36, 1} } func (m *RangeLogResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2062,7 +2063,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_058b4f1680004f39, []int{37} + return fileDescriptor_admin_2d30d36d58fa713c, []int{37} } func (m *QueryPlanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2097,7 +2098,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_058b4f1680004f39, []int{38} + return fileDescriptor_admin_2d30d36d58fa713c, []int{38} } func (m *QueryPlanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2129,7 +2130,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_058b4f1680004f39, []int{39} + return fileDescriptor_admin_2d30d36d58fa713c, []int{39} } func (m *DataDistributionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2165,7 +2166,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_058b4f1680004f39, []int{40} + return fileDescriptor_admin_2d30d36d58fa713c, []int{40} } func (m *DataDistributionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2203,7 +2204,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_058b4f1680004f39, []int{40, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []int{40, 0} } func (m *DataDistributionResponse_ZoneConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2238,7 +2239,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_058b4f1680004f39, []int{40, 1} + return fileDescriptor_admin_2d30d36d58fa713c, []int{40, 1} } func (m *DataDistributionResponse_TableInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2272,7 +2273,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_058b4f1680004f39, []int{40, 2} + return fileDescriptor_admin_2d30d36d58fa713c, []int{40, 2} } func (m *DataDistributionResponse_DatabaseInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2305,7 +2306,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_058b4f1680004f39, []int{41} + return fileDescriptor_admin_2d30d36d58fa713c, []int{41} } func (m *MetricMetadataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2339,7 +2340,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_058b4f1680004f39, []int{42} + return fileDescriptor_admin_2d30d36d58fa713c, []int{42} } func (m *MetricMetadataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2382,7 +2383,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_058b4f1680004f39, []int{43} + return fileDescriptor_admin_2d30d36d58fa713c, []int{43} } func (m *EnqueueRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2415,7 +2416,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_058b4f1680004f39, []int{44} + return fileDescriptor_admin_2d30d36d58fa713c, []int{44} } func (m *EnqueueRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2452,7 +2453,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_058b4f1680004f39, []int{44, 0} + return fileDescriptor_admin_2d30d36d58fa713c, []int{44, 0} } func (m *EnqueueRangeResponse_Details) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2485,7 +2486,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_058b4f1680004f39, []int{45} + return fileDescriptor_admin_2d30d36d58fa713c, []int{45} } func (m *ChartCatalogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2519,7 +2520,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_058b4f1680004f39, []int{46} + return fileDescriptor_admin_2d30d36d58fa713c, []int{46} } func (m *ChartCatalogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2620,7 +2621,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. @@ -4713,10 +4713,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) @@ -4731,24 +4731,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++ @@ -4759,6 +4741,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 } @@ -4777,23 +4769,33 @@ 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++ } return i, nil } @@ -4814,22 +4816,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 } @@ -4850,22 +4852,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 @@ -5040,11 +5042,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 @@ -5303,21 +5305,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 @@ -5329,41 +5331,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 @@ -5381,11 +5383,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 @@ -5630,19 +5632,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 } @@ -5751,11 +5753,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 { @@ -5782,11 +5784,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 @@ -5816,11 +5818,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++ @@ -5874,11 +5876,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 } @@ -5922,11 +5924,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 @@ -5989,11 +5991,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 @@ -6702,16 +6704,9 @@ func (m *DrainRequest) Size() (n int) { } var l int _ = l - if len(m.On) > 0 { + if len(m.DeprecatedProbeIndicator) > 0 { l = 0 - for _, e := range m.On { - l += sovAdmin(uint64(e)) - } - n += 1 + sovAdmin(uint64(l)) + l - } - if len(m.Off) > 0 { - l = 0 - for _, e := range m.Off { + for _, e := range m.DeprecatedProbeIndicator { l += sovAdmin(uint64(e)) } n += 1 + sovAdmin(uint64(l)) + l @@ -6719,6 +6714,9 @@ func (m *DrainRequest) Size() (n int) { if m.Shutdown { n += 2 } + if m.DoDrain { + n += 2 + } return n } @@ -6728,13 +6726,16 @@ 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 + } return n } @@ -10788,7 +10789,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 { @@ -10820,8 +10821,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 @@ -10839,87 +10840,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 { @@ -10936,7 +10884,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:]) @@ -11004,7 +10952,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 { @@ -11036,8 +10984,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 @@ -11055,11 +11003,31 @@ 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) default: iNdEx = preIndex skippy, err := skipAdmin(dAtA[iNdEx:]) @@ -15643,262 +15611,263 @@ var ( ErrIntOverflowAdmin = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_058b4f1680004f39) } +func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_2d30d36d58fa713c) } -var fileDescriptor_admin_058b4f1680004f39 = []byte{ - // 4050 bytes of a gzipped FileDescriptorProto +var fileDescriptor_admin_2d30d36d58fa713c = []byte{ + // 4077 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x4d, 0x73, 0x1b, 0xc9, - 0x75, 0x1c, 0x80, 0xf8, 0x7a, 0x04, 0x48, 0xb0, 0x45, 0x51, 0x10, 0x24, 0x13, 0xdc, 0x91, 0xe5, - 0xa5, 0xb5, 0x6b, 0x60, 0x45, 0x69, 0x77, 0x1d, 0xad, 0x36, 0x0e, 0x41, 0xb2, 0xb8, 0xd0, 0x52, - 0xda, 0xd5, 0x90, 0xd2, 0x96, 0xe5, 0xac, 0x27, 0x03, 0x4c, 0x13, 0x1c, 0x73, 0x30, 0x03, 0xce, - 0x0c, 0xa8, 0xe5, 0x6e, 0xec, 0x8a, 0xed, 0x54, 0x2a, 0xa7, 0xd4, 0x96, 0xed, 0x9b, 0xab, 0x52, - 0x95, 0x54, 0x2a, 0xa9, 0x24, 0xa7, 0xdc, 0x92, 0x1c, 0x72, 0xde, 0xca, 0xc1, 0x76, 0x55, 0x2e, - 0xce, 0x21, 0x74, 0xc2, 0xcd, 0x21, 0x95, 0x1f, 0x90, 0x43, 0x0e, 0x49, 0xaa, 0xbf, 0x66, 0x1a, - 0xe0, 0x68, 0x04, 0x50, 0xde, 0x1c, 0x48, 0x4c, 0xbf, 0xee, 0x7e, 0xfd, 0xfa, 0xbd, 0xd7, 0xaf, - 0xdf, 0x47, 0xc3, 0x15, 0x1f, 0x7b, 0x47, 0xd8, 0x6b, 0xb0, 0x9f, 0x7e, 0xbb, 0x61, 0x98, 0x3d, - 0xcb, 0xa9, 0xf7, 0x3d, 0x37, 0x70, 0xd1, 0xe5, 0x8e, 0xdb, 0x39, 0xf0, 0x5c, 0xa3, 0xb3, 0x5f, - 0x67, 0xfd, 0x75, 0x31, 0xac, 0x3a, 0xdf, 0x71, 0x9d, 0x3d, 0xab, 0xdb, 0xf8, 0xd8, 0x75, 0x30, - 0x1b, 0x5d, 0x5d, 0xfc, 0x8e, 0xdb, 0xf6, 0x1b, 0xe4, 0x5f, 0xbf, 0x4d, 0x7f, 0x38, 0xfc, 0xea, - 0xe8, 0x12, 0x7e, 0x60, 0x04, 0x03, 0xd1, 0xab, 0xfa, 0x81, 0xeb, 0x19, 0x5d, 0xdc, 0xc0, 0x4e, - 0xd7, 0x72, 0xc4, 0x4f, 0xbf, 0xdd, 0xe8, 0x1d, 0x75, 0x3a, 0x7c, 0xcc, 0xb2, 0x18, 0xc3, 0x7f, - 0xfb, 0xed, 0x86, 0x6d, 0x1d, 0x61, 0x07, 0xfb, 0x02, 0xcb, 0x95, 0x98, 0x11, 0x6e, 0x97, 0x77, - 0x2e, 0x05, 0x7e, 0xa3, 0x63, 0x04, 0x86, 0xed, 0x76, 0x1b, 0x9d, 0x7d, 0xc3, 0x0b, 0x74, 0xde, - 0xe2, 0xfd, 0x95, 0x41, 0x60, 0xd9, 0x8d, 0x1e, 0x0e, 0x3c, 0xab, 0xc3, 0x7f, 0x78, 0xcf, 0x42, - 0xd7, 0xed, 0xba, 0xf4, 0xb3, 0x41, 0xbe, 0xc4, 0x86, 0xba, 0xae, 0xdb, 0xb5, 0x71, 0xc3, 0xe8, - 0x5b, 0x0d, 0xc3, 0x71, 0xdc, 0xc0, 0x08, 0x2c, 0xd7, 0x11, 0xa4, 0xd4, 0x78, 0x2f, 0x6d, 0xb5, - 0x07, 0x7b, 0x8d, 0xc0, 0xea, 0x61, 0x3f, 0x30, 0x7a, 0x7d, 0x36, 0x40, 0x45, 0x50, 0xde, 0x30, - 0x02, 0xa3, 0x6d, 0xf8, 0xd8, 0xd7, 0xf0, 0xe1, 0x00, 0xfb, 0x81, 0x7a, 0x13, 0xe6, 0x25, 0x98, - 0xdf, 0x77, 0x1d, 0x1f, 0xa3, 0xab, 0x50, 0x30, 0x05, 0xb0, 0xa2, 0x2c, 0xa7, 0x57, 0x0a, 0x5a, - 0x04, 0x50, 0x6f, 0xc3, 0xa2, 0x98, 0xb2, 0x81, 0x03, 0xc3, 0xb2, 0x05, 0x32, 0x54, 0x85, 0xbc, - 0x18, 0x56, 0x51, 0x96, 0x95, 0x95, 0x82, 0x16, 0xb6, 0xd5, 0xbf, 0x4a, 0xc3, 0xa5, 0x33, 0xd3, - 0xf8, 0x7a, 0x8f, 0x21, 0xdb, 0xf5, 0x0c, 0x27, 0x60, 0x8b, 0xcd, 0xac, 0x7e, 0xbd, 0xfe, 0x4c, - 0xf9, 0xd7, 0x9f, 0x81, 0xa3, 0xbe, 0x45, 0x10, 0x34, 0xa7, 0x3f, 0x3b, 0xa9, 0x4d, 0x69, 0x1c, - 0x1b, 0xaa, 0xc1, 0x4c, 0x60, 0xb4, 0x6d, 0xac, 0x3b, 0x46, 0x0f, 0xfb, 0x95, 0x14, 0xdd, 0x09, - 0x50, 0xd0, 0x03, 0x02, 0x41, 0xaf, 0x43, 0xc9, 0xc4, 0x7e, 0xc7, 0xb3, 0xfa, 0x81, 0xeb, 0xe9, - 0x96, 0x59, 0x49, 0x2f, 0x2b, 0x2b, 0xe9, 0x66, 0xf9, 0xf4, 0xa4, 0x56, 0xdc, 0x08, 0x3b, 0x5a, - 0x1b, 0x5a, 0x31, 0x1a, 0xd6, 0x32, 0xd1, 0x3a, 0xcc, 0x10, 0xf5, 0xd3, 0x99, 0x2a, 0x56, 0xa6, - 0x97, 0x95, 0x95, 0x99, 0xd5, 0xab, 0x12, 0xd1, 0xac, 0xa3, 0xfe, 0xc4, 0x75, 0xf0, 0x3a, 0xfd, - 0xe4, 0x84, 0xc1, 0xc7, 0x21, 0x04, 0x7d, 0x08, 0xf3, 0x12, 0x12, 0xdd, 0xc6, 0x47, 0xd8, 0xae, - 0x64, 0x96, 0x95, 0x95, 0xd9, 0xd5, 0x9b, 0x09, 0xfb, 0x8f, 0x70, 0x0e, 0x3c, 0x2a, 0xfe, 0x6d, - 0x32, 0x51, 0x9b, 0x8b, 0x30, 0x53, 0x40, 0xf5, 0x2d, 0xc8, 0x50, 0x96, 0x20, 0x04, 0xd3, 0x03, - 0x1f, 0x7b, 0x5c, 0x20, 0xf4, 0x1b, 0x2d, 0x01, 0xf4, 0x3d, 0xeb, 0xc8, 0xb2, 0x71, 0x37, 0xe2, - 0x4b, 0x04, 0x51, 0xb7, 0xe0, 0xc2, 0x2e, 0xe1, 0xd2, 0xf8, 0xf2, 0x45, 0x0b, 0x90, 0xa1, 0x8c, - 0xad, 0xa4, 0x68, 0x07, 0x6b, 0xa8, 0xff, 0x93, 0x83, 0x85, 0x61, 0x4c, 0x5c, 0xe4, 0x3b, 0x23, - 0x22, 0x7f, 0x3d, 0x61, 0xcb, 0x71, 0x08, 0x62, 0xe5, 0xfd, 0x18, 0x72, 0x1d, 0xd7, 0x1e, 0xf4, - 0x1c, 0xb6, 0xa7, 0x99, 0xd5, 0x37, 0x26, 0xc5, 0xba, 0x4e, 0xa7, 0x73, 0xb4, 0x02, 0x19, 0x7a, - 0x04, 0x39, 0xcb, 0x31, 0xf1, 0x47, 0xd8, 0xaf, 0xa4, 0xcf, 0x47, 0x6d, 0x8b, 0x4c, 0x17, 0x68, - 0x39, 0x2e, 0xa2, 0x9e, 0x9e, 0xe1, 0x74, 0x89, 0x0a, 0x0c, 0x9c, 0x80, 0xaa, 0x51, 0x5a, 0x03, - 0x0a, 0x5a, 0x27, 0x10, 0x74, 0x1b, 0x16, 0x3b, 0x1e, 0x36, 0x02, 0xac, 0x33, 0x35, 0x26, 0xf6, - 0x0b, 0xf7, 0xb0, 0x13, 0x50, 0x3d, 0x29, 0x68, 0x0b, 0xac, 0x97, 0xae, 0xb8, 0x23, 0xfa, 0x46, - 0xb5, 0x33, 0xfb, 0xeb, 0xd3, 0xce, 0xdc, 0xaf, 0x4b, 0x3b, 0xcf, 0x1e, 0xbc, 0xfc, 0x38, 0x07, - 0xef, 0x85, 0x94, 0xba, 0xfa, 0x0f, 0x0a, 0x64, 0x99, 0x7c, 0xc9, 0x74, 0x62, 0x12, 0xc4, 0x74, - 0xf2, 0x4d, 0x60, 0xc1, 0x71, 0x5f, 0xe8, 0x2f, 0xfd, 0x26, 0x0a, 0xef, 0x0c, 0x6c, 0x9b, 0xea, - 0x35, 0x31, 0x0d, 0x79, 0x2d, 0x6c, 0xa3, 0x6b, 0x64, 0x0b, 0x7b, 0xc6, 0xc0, 0x0e, 0xf4, 0x23, - 0xc3, 0x1e, 0x60, 0x2a, 0xbf, 0x02, 0x21, 0x98, 0x02, 0x1f, 0x13, 0x18, 0xba, 0x05, 0x17, 0xbb, - 0xd8, 0xc1, 0x8c, 0x17, 0x3a, 0xfe, 0xa8, 0xef, 0x61, 0xdf, 0xb7, 0x5c, 0x47, 0x08, 0x30, 0xea, - 0xdc, 0x0c, 0xfb, 0xd0, 0x22, 0x64, 0xf7, 0x2d, 0xd3, 0xc4, 0x0e, 0x95, 0x5d, 0x5e, 0xe3, 0xad, - 0xea, 0xdf, 0x28, 0x90, 0xa1, 0x8a, 0x14, 0x4b, 0xff, 0x22, 0x64, 0x07, 0x8e, 0x75, 0x38, 0x60, - 0x3b, 0xc8, 0x6b, 0xbc, 0x85, 0xca, 0x90, 0xf6, 0xf1, 0x21, 0xb3, 0x6c, 0x1a, 0xf9, 0x24, 0x23, - 0x99, 0x66, 0x73, 0x92, 0x79, 0x8b, 0x9a, 0x7d, 0xcb, 0xc3, 0x9d, 0x20, 0x22, 0x30, 0x02, 0xa0, - 0x0a, 0xe4, 0xc8, 0x1d, 0x67, 0x39, 0x5d, 0x4e, 0x96, 0x68, 0x12, 0x2e, 0x59, 0xbd, 0xbe, 0x6d, - 0x75, 0xac, 0x80, 0xaa, 0x48, 0x5e, 0x0b, 0xdb, 0xea, 0x26, 0xcc, 0x87, 0xea, 0xf9, 0x02, 0x76, - 0xe4, 0x6f, 0xd3, 0x80, 0x64, 0x3c, 0xdc, 0x8a, 0x8c, 0x9c, 0x20, 0xe5, 0xcc, 0x09, 0xba, 0x06, - 0x25, 0x0f, 0x13, 0x52, 0x0c, 0x3e, 0x24, 0x45, 0x87, 0x14, 0x39, 0x90, 0x0d, 0xfa, 0x12, 0x80, - 0xe3, 0x9a, 0x02, 0x09, 0x63, 0x54, 0x81, 0x40, 0x58, 0xf7, 0x3b, 0x90, 0x21, 0x07, 0xcf, 0xe7, - 0x76, 0xfe, 0x55, 0x59, 0xfd, 0xd9, 0xa5, 0x5f, 0x67, 0xbe, 0x43, 0x5d, 0xb8, 0x10, 0xf5, 0xfb, - 0x8f, 0xd7, 0xd7, 0x29, 0xa5, 0xfc, 0x64, 0x31, 0x04, 0xc8, 0x84, 0x52, 0xcf, 0xf2, 0x7d, 0xcb, - 0xe9, 0xea, 0x04, 0xbd, 0x5f, 0xc9, 0x50, 0x6b, 0xf2, 0x1b, 0xcf, 0xb3, 0x26, 0x43, 0x9b, 0xae, - 0xdf, 0x67, 0x28, 0x1e, 0xb8, 0x26, 0xe6, 0xe8, 0x8b, 0xbd, 0x08, 0xe4, 0x13, 0xab, 0x61, 0xf4, - 0xfb, 0x9e, 0xfb, 0x91, 0xd5, 0x23, 0xa6, 0xc3, 0xb4, 0xfc, 0x03, 0xbd, 0x7d, 0x1c, 0x60, 0x9f, - 0xca, 0x6d, 0x5a, 0x5b, 0x90, 0x7a, 0x37, 0x2c, 0xff, 0xa0, 0x49, 0xfa, 0xaa, 0x1f, 0xc0, 0x8c, - 0x84, 0x18, 0x5d, 0x83, 0x1c, 0xe5, 0x89, 0x65, 0x32, 0x09, 0x35, 0xe1, 0xf4, 0xa4, 0x96, 0x25, - 0x5d, 0xad, 0x0d, 0x2d, 0x4b, 0xba, 0x5a, 0x26, 0xe1, 0x2e, 0xf6, 0x3c, 0xd7, 0xd3, 0x7b, 0xd8, - 0xf7, 0x8d, 0xae, 0x90, 0x59, 0x91, 0x02, 0xef, 0x33, 0x98, 0xba, 0x08, 0x0b, 0x0f, 0x5c, 0xe7, - 0x8c, 0x12, 0xa8, 0x3f, 0x57, 0xe0, 0xe2, 0x48, 0x07, 0x97, 0xea, 0x37, 0x61, 0x9e, 0xb8, 0x2e, - 0xba, 0x8f, 0x3d, 0x0b, 0xfb, 0x3a, 0x63, 0xbe, 0x42, 0x99, 0xff, 0xb5, 0x89, 0x58, 0xa5, 0xcd, - 0x11, 0x3c, 0x3b, 0x14, 0x0d, 0xed, 0x40, 0xdf, 0x02, 0x64, 0x39, 0x01, 0xf6, 0x1c, 0xc3, 0xd6, - 0x07, 0x3e, 0xe6, 0xb8, 0x53, 0xe7, 0xc1, 0x5d, 0x16, 0x88, 0x1e, 0xf9, 0xac, 0x47, 0x9d, 0x85, - 0xe2, 0x23, 0x1f, 0x7b, 0xe1, 0x0e, 0xbf, 0x07, 0x25, 0xde, 0xe6, 0x1b, 0x6b, 0x41, 0x86, 0x58, - 0x2a, 0x71, 0xe7, 0x25, 0x2d, 0x38, 0x34, 0x91, 0xb6, 0x84, 0x2a, 0x51, 0x0c, 0x55, 0x15, 0xa6, - 0x09, 0x90, 0x1c, 0x25, 0x02, 0x90, 0xac, 0x41, 0xd8, 0x56, 0x7f, 0xa8, 0x40, 0x69, 0xf3, 0x08, - 0x3b, 0xd1, 0xc1, 0x13, 0x36, 0x4e, 0x91, 0x6c, 0xdc, 0x15, 0x28, 0x04, 0x86, 0xd7, 0xc5, 0x01, - 0x91, 0x35, 0x3b, 0x1e, 0x79, 0x06, 0x68, 0x99, 0xe4, 0x34, 0xda, 0x56, 0xcf, 0x62, 0xa7, 0x22, - 0xa3, 0xb1, 0x06, 0x7a, 0x05, 0xe6, 0x07, 0x8e, 0x87, 0x4d, 0xa3, 0x13, 0x60, 0x53, 0xc7, 0x74, - 0x09, 0x7a, 0x3a, 0xf2, 0x5a, 0x39, 0xea, 0x60, 0x4b, 0xab, 0xff, 0x99, 0x82, 0x59, 0x41, 0x05, - 0xe7, 0xc3, 0x7d, 0xc8, 0xf2, 0x49, 0x8c, 0x11, 0x8d, 0x04, 0x46, 0x0c, 0x4f, 0x65, 0x4d, 0x71, - 0xed, 0x33, 0x24, 0xd5, 0x3f, 0x4c, 0x41, 0x86, 0xc2, 0x51, 0x13, 0x0a, 0xa1, 0xd3, 0xcb, 0x35, - 0xa6, 0x5a, 0x67, 0x6e, 0x71, 0x5d, 0xb8, 0xc5, 0xf5, 0x5d, 0x31, 0xa2, 0x99, 0x27, 0x68, 0x3e, - 0xfd, 0x55, 0x4d, 0xd1, 0xa2, 0x69, 0xc4, 0x1a, 0x50, 0xbc, 0xba, 0x74, 0x1b, 0x14, 0x28, 0x64, - 0x97, 0xb0, 0xeb, 0xab, 0x32, 0xbb, 0x98, 0xbb, 0x58, 0x3c, 0x3d, 0xa9, 0xe5, 0x77, 0x19, 0xcb, - 0x36, 0x24, 0xe6, 0xad, 0x02, 0xb1, 0x33, 0xae, 0x17, 0x90, 0x03, 0x6f, 0x99, 0xec, 0x82, 0x6f, - 0xce, 0x9d, 0x9e, 0xd4, 0x66, 0x34, 0x01, 0x6f, 0x6d, 0x68, 0x33, 0xe1, 0xa0, 0x96, 0x49, 0x24, - 0x64, 0x39, 0x7b, 0x2e, 0x37, 0xbf, 0xf4, 0x9b, 0x2c, 0xc9, 0x6c, 0x39, 0x41, 0x42, 0xce, 0x70, - 0x91, 0x2d, 0xf9, 0x88, 0x02, 0xc9, 0x92, 0xac, 0xbb, 0x65, 0xaa, 0x7f, 0xad, 0x40, 0x79, 0x07, - 0x07, 0x8f, 0x5a, 0xc4, 0x4d, 0x16, 0x52, 0xff, 0x26, 0xc0, 0x01, 0x3e, 0x66, 0xb7, 0x94, 0x60, - 0xf9, 0x9d, 0x04, 0x96, 0x8f, 0x22, 0xa8, 0xbf, 0x8b, 0x8f, 0xe9, 0x75, 0xe6, 0x6f, 0x3a, 0x81, - 0x77, 0xac, 0x15, 0x0e, 0x44, 0xbb, 0x7a, 0x17, 0x66, 0x87, 0x3b, 0xc9, 0x75, 0x73, 0x80, 0x8f, - 0xb9, 0x86, 0x91, 0x4f, 0xa2, 0x43, 0xec, 0x82, 0x24, 0xbc, 0x2c, 0x6a, 0xac, 0x71, 0x27, 0xf5, - 0x75, 0x45, 0xbd, 0x00, 0xf3, 0xd2, 0x5a, 0x4c, 0xc2, 0xea, 0x57, 0xa0, 0xbc, 0x35, 0xba, 0x03, - 0x04, 0xd3, 0x07, 0xf8, 0x58, 0xc4, 0x22, 0xf4, 0x5b, 0xfd, 0x79, 0x0a, 0xe6, 0xb7, 0x46, 0x67, - 0xa3, 0xdf, 0x89, 0xd9, 0xeb, 0x5b, 0x09, 0x7b, 0x3d, 0x83, 0x61, 0x64, 0xb3, 0x5c, 0xd5, 0xa4, - 0x2d, 0xef, 0x41, 0x86, 0xdd, 0xed, 0xe1, 0xbe, 0x14, 0x69, 0x5f, 0x68, 0x0b, 0x8a, 0xb6, 0xe1, - 0x07, 0xfa, 0xa0, 0x6f, 0x1a, 0x01, 0x36, 0xb9, 0x6d, 0x19, 0x4f, 0x0b, 0x67, 0xc8, 0xcc, 0x47, - 0x6c, 0x62, 0xb5, 0x3f, 0x06, 0x6b, 0xdf, 0x91, 0x59, 0x3b, 0xb3, 0xba, 0x3a, 0xd1, 0x46, 0x29, - 0x6a, 0x59, 0x1c, 0x65, 0x98, 0x5d, 0xb7, 0x07, 0x7e, 0x80, 0x3d, 0x61, 0xc1, 0x7e, 0xaa, 0xc0, - 0x5c, 0x08, 0xe2, 0x1c, 0x7e, 0x15, 0xa0, 0xc3, 0x40, 0xd1, 0xe5, 0x50, 0x3a, 0x3d, 0xa9, 0x15, - 0xf8, 0xc0, 0xd6, 0x86, 0x56, 0xe0, 0x03, 0x5a, 0x26, 0x31, 0x15, 0xd1, 0x19, 0xc0, 0x0e, 0x31, - 0xa3, 0x26, 0x77, 0x50, 0xca, 0x61, 0xc7, 0x26, 0x83, 0xa3, 0xaf, 0x01, 0xc2, 0xc4, 0xa8, 0xf6, - 0x3d, 0xcb, 0xc7, 0xe1, 0x68, 0xe6, 0x78, 0xcd, 0x47, 0x3d, 0x7c, 0xb8, 0xba, 0x0d, 0xc5, 0x0d, - 0xcf, 0xb0, 0x1c, 0xa1, 0x25, 0xb3, 0x90, 0x72, 0x1d, 0x2a, 0xf3, 0x8c, 0x96, 0x72, 0x1d, 0xc2, - 0x2f, 0x77, 0x6f, 0x8f, 0x7a, 0x82, 0x19, 0x8d, 0x7c, 0x12, 0x6b, 0xe9, 0xef, 0x0f, 0x02, 0xd3, - 0x7d, 0xea, 0x08, 0x7f, 0x4e, 0xb4, 0xd5, 0x1a, 0x94, 0x38, 0x36, 0xbe, 0xd1, 0x11, 0x74, 0xea, - 0xc7, 0x70, 0x79, 0x03, 0x77, 0xdc, 0x1e, 0xbd, 0x6c, 0x5d, 0x67, 0x87, 0x26, 0x13, 0xc4, 0xda, - 0x1f, 0x42, 0x9e, 0xdf, 0x97, 0x4c, 0xeb, 0x32, 0xcd, 0xe6, 0xe9, 0x49, 0x2d, 0xc7, 0x2e, 0x4c, - 0xff, 0xbf, 0x4f, 0x6a, 0xb7, 0xba, 0x56, 0xb0, 0x3f, 0x68, 0xd7, 0x3b, 0x6e, 0xaf, 0x11, 0x4a, - 0xc9, 0x6c, 0x47, 0xdf, 0x8d, 0xfe, 0x41, 0xb7, 0x41, 0xbf, 0xfa, 0xed, 0x3a, 0xbf, 0x68, 0x73, - 0xec, 0xa2, 0xf5, 0xd5, 0x3f, 0x56, 0xe0, 0x82, 0xbc, 0xf8, 0xff, 0xcf, 0xb2, 0x68, 0x05, 0xe6, - 0x4c, 0x69, 0x55, 0xe2, 0xfb, 0x31, 0xd9, 0x8d, 0x82, 0xd5, 0xcf, 0x53, 0x50, 0x8d, 0xe3, 0x0e, - 0xe7, 0xe5, 0x13, 0xc8, 0xb2, 0xe4, 0x0b, 0x0f, 0xcc, 0xee, 0x26, 0x45, 0xf8, 0xcf, 0x44, 0x53, - 0x67, 0x4d, 0x61, 0xfe, 0x19, 0xc6, 0xea, 0x7f, 0x28, 0x90, 0x65, 0x1d, 0xe8, 0xc9, 0xb0, 0xd7, - 0x92, 0x69, 0xae, 0x45, 0x5e, 0xcb, 0x79, 0x99, 0x21, 0x9c, 0x9d, 0x4b, 0x90, 0xb3, 0x7c, 0xdd, - 0xb6, 0x8e, 0x42, 0x07, 0xdb, 0xf2, 0xb7, 0xad, 0x23, 0x7c, 0xd6, 0xc7, 0x4c, 0xc7, 0xf8, 0x98, - 0x31, 0x9c, 0x9c, 0x8e, 0xe5, 0x24, 0x75, 0x8e, 0x89, 0x1e, 0x92, 0x21, 0x19, 0xa6, 0xa3, 0xa2, - 0xad, 0x5e, 0x87, 0xb9, 0x1d, 0x1c, 0x90, 0x23, 0xe3, 0x27, 0x99, 0xc6, 0xbf, 0x4b, 0xd1, 0x5b, - 0x80, 0x8f, 0xe3, 0x22, 0xd0, 0x27, 0xbf, 0x05, 0x86, 0x10, 0x3c, 0xd7, 0x30, 0xee, 0xc4, 0x1a, - 0xc6, 0x82, 0x30, 0x8c, 0x71, 0xf1, 0xd5, 0x32, 0xcc, 0x88, 0xf8, 0x8e, 0xc4, 0x1c, 0x69, 0xda, - 0x25, 0x83, 0xaa, 0xee, 0x18, 0x56, 0x70, 0x6b, 0xd8, 0x0a, 0xde, 0x9c, 0x64, 0x53, 0x67, 0x8c, - 0xe0, 0x75, 0x28, 0xbd, 0x83, 0x0d, 0x3b, 0xd8, 0x17, 0x0c, 0x5e, 0x80, 0x8c, 0x87, 0x0d, 0x93, - 0xad, 0x98, 0xd7, 0x58, 0x83, 0xd8, 0x4a, 0x31, 0x8c, 0xdf, 0x5b, 0xf3, 0x30, 0xb7, 0xcd, 0x73, - 0x83, 0xc2, 0x7c, 0xfe, 0x2c, 0x05, 0xe5, 0x08, 0xc6, 0xe5, 0xb0, 0x06, 0x20, 0x72, 0x88, 0xa1, - 0x1c, 0xae, 0xc4, 0xc4, 0x14, 0x62, 0xa2, 0x08, 0xce, 0xa3, 0x49, 0xe8, 0x47, 0x0a, 0xe4, 0x99, - 0xf2, 0x63, 0x71, 0xa0, 0x92, 0x62, 0x88, 0x51, 0x12, 0xf8, 0x31, 0x12, 0x82, 0x7c, 0x8b, 0xe0, - 0xff, 0xc1, 0xaf, 0xce, 0x77, 0x3a, 0x42, 0x3a, 0xaa, 0x6d, 0x28, 0x0d, 0xe1, 0x95, 0x05, 0x95, - 0x61, 0x82, 0x7a, 0x4b, 0x16, 0xd4, 0xec, 0xea, 0xf5, 0x98, 0x5d, 0x13, 0xb4, 0x82, 0x5e, 0x7e, - 0xf8, 0x25, 0xe1, 0xf4, 0x61, 0xe6, 0x9e, 0xdb, 0xf6, 0x25, 0xd1, 0x30, 0xef, 0x54, 0x91, 0xbd, - 0xd3, 0x45, 0xc9, 0xd6, 0xd0, 0xf0, 0x96, 0xb5, 0xd0, 0x2d, 0xae, 0x80, 0x69, 0xba, 0x78, 0x4d, - 0x5e, 0xfc, 0xd0, 0xae, 0xd3, 0xbc, 0x31, 0xcb, 0x21, 0xd7, 0x89, 0xa3, 0xc7, 0x34, 0x54, 0xfd, - 0x59, 0x16, 0x8a, 0x6c, 0x49, 0x2e, 0xbe, 0x4d, 0x98, 0x26, 0xa3, 0xb8, 0xe0, 0x5e, 0x49, 0x60, - 0xbb, 0x3c, 0x8d, 0x34, 0xb8, 0x20, 0xe9, 0xf4, 0xea, 0x7f, 0x65, 0x20, 0x7d, 0xcf, 0x6d, 0xa3, - 0x45, 0x48, 0x71, 0x63, 0x95, 0x6e, 0x66, 0x4f, 0x4f, 0x6a, 0xa9, 0xd6, 0x86, 0x96, 0xb2, 0xcc, - 0xf3, 0x9d, 0x96, 0xa1, 0x68, 0x60, 0x7a, 0x38, 0x1a, 0x40, 0x2e, 0xcc, 0x0e, 0xa5, 0x5c, 0x58, - 0xf4, 0x59, 0x6a, 0xbe, 0x73, 0x7a, 0x52, 0x2b, 0xc9, 0x39, 0x97, 0xf1, 0xaf, 0x0d, 0xff, 0xd0, - 0x26, 0x7f, 0x24, 0x6e, 0xaf, 0xb7, 0x36, 0xb4, 0x92, 0x9c, 0xab, 0xf1, 0x25, 0x39, 0x64, 0x87, - 0xe4, 0x70, 0x07, 0x72, 0x2c, 0x6f, 0x65, 0xd2, 0x6c, 0x41, 0xb2, 0x73, 0x34, 0x4d, 0x1d, 0x23, - 0x31, 0x81, 0xcc, 0xf5, 0x03, 0xc3, 0x23, 0x73, 0xf3, 0xe3, 0xce, 0xe5, 0x13, 0xd0, 0x5d, 0xc8, - 0xef, 0x59, 0x8e, 0xe5, 0xef, 0x63, 0xb3, 0x52, 0x18, 0x73, 0x72, 0x38, 0x83, 0xcc, 0xee, 0xb9, - 0xa6, 0xb5, 0x67, 0x61, 0xb3, 0x02, 0xe3, 0xce, 0x16, 0x33, 0x88, 0x67, 0xb3, 0xe7, 0x19, 0x34, - 0x91, 0xa2, 0x77, 0xdc, 0x5e, 0xdf, 0xc6, 0x64, 0x0b, 0x33, 0xcb, 0xca, 0x4a, 0x4a, 0x9b, 0x17, - 0x3d, 0xeb, 0xa2, 0x83, 0x28, 0x36, 0x8d, 0xa1, 0x2b, 0x45, 0x66, 0x41, 0x69, 0x03, 0x3d, 0x84, - 0x0b, 0xfb, 0x56, 0x77, 0xff, 0xa9, 0x41, 0x7c, 0xaf, 0x28, 0xce, 0x29, 0x8d, 0x49, 0x0d, 0x0a, - 0x27, 0x87, 0x3d, 0xc4, 0x3d, 0x8b, 0x50, 0x9a, 0xb8, 0x63, 0xf5, 0x0c, 0xbb, 0x32, 0x4b, 0x17, - 0x2d, 0x87, 0x1d, 0x1b, 0x0c, 0x8e, 0xae, 0xc3, 0xac, 0x37, 0x70, 0xc8, 0x45, 0xa4, 0x73, 0xc1, - 0xce, 0xd1, 0x91, 0x25, 0x0e, 0xe5, 0x97, 0xf0, 0x55, 0x28, 0x44, 0x89, 0xca, 0x32, 0x8b, 0x9f, - 0x42, 0x80, 0x8a, 0xa0, 0xbc, 0xed, 0x76, 0x58, 0xe1, 0x42, 0xd8, 0xc9, 0xff, 0x55, 0x60, 0x5e, - 0x02, 0x86, 0x69, 0x80, 0x82, 0x2d, 0x80, 0x63, 0x64, 0x89, 0xcf, 0x20, 0x08, 0x21, 0xe2, 0xaa, - 0x0a, 0xb1, 0x55, 0xff, 0x48, 0x81, 0xbc, 0xe8, 0x45, 0x2f, 0x41, 0x91, 0xf4, 0xd8, 0x56, 0x70, - 0xac, 0x47, 0x37, 0xcb, 0x8c, 0x80, 0xbd, 0x8b, 0x8f, 0xc9, 0xce, 0xc3, 0x21, 0x91, 0x05, 0x2b, - 0x68, 0x25, 0x01, 0x65, 0x17, 0x5f, 0x15, 0xf2, 0xb6, 0x11, 0x58, 0xc1, 0xc0, 0x64, 0x56, 0x46, - 0xd1, 0xc2, 0x36, 0xe1, 0x8a, 0xed, 0x3a, 0x5d, 0xd6, 0x39, 0x4d, 0x3b, 0x23, 0x80, 0xda, 0x84, - 0x39, 0xcd, 0x70, 0xba, 0x78, 0xdb, 0xed, 0x0a, 0xe3, 0x76, 0x19, 0xf2, 0x2c, 0xb7, 0x25, 0xec, - 0x83, 0x96, 0xa3, 0x6d, 0x39, 0x2a, 0x4f, 0x49, 0x76, 0x4f, 0xfd, 0x97, 0x34, 0x94, 0x23, 0x24, - 0x9c, 0x89, 0xef, 0x85, 0xa1, 0x36, 0xbb, 0x27, 0x92, 0x2e, 0xc7, 0xd1, 0xc9, 0xb1, 0xc1, 0xf6, - 0x3f, 0x2a, 0x00, 0xef, 0x7b, 0x38, 0x08, 0x8e, 0x5b, 0x24, 0x36, 0x7d, 0x09, 0x8a, 0x3c, 0xd2, - 0xd1, 0xc9, 0xe9, 0x17, 0xcc, 0xe3, 0x30, 0x62, 0x56, 0xc8, 0x46, 0x1c, 0xfc, 0x94, 0x75, 0x33, - 0xb6, 0xe5, 0x1c, 0xfc, 0x94, 0x76, 0x5d, 0x83, 0x92, 0x61, 0x9a, 0xd8, 0xd4, 0xb9, 0xaf, 0xc4, - 0x6d, 0x5a, 0x91, 0x02, 0x35, 0x06, 0x43, 0x2f, 0xc3, 0x9c, 0x87, 0x7b, 0xee, 0x91, 0x34, 0x8c, - 0xd9, 0xb6, 0x59, 0x0e, 0x16, 0x03, 0x17, 0x21, 0xeb, 0x61, 0xc3, 0x0f, 0x93, 0x97, 0xbc, 0x85, - 0x2a, 0x90, 0x33, 0x59, 0x3e, 0x9e, 0x5b, 0x22, 0xd1, 0xac, 0xfe, 0x99, 0x22, 0x32, 0x07, 0x77, - 0x21, 0x43, 0x37, 0xc8, 0xb3, 0x06, 0xcb, 0x31, 0x57, 0x93, 0x60, 0x8f, 0xcc, 0x15, 0x36, 0x09, - 0x7d, 0x08, 0x33, 0x7d, 0xca, 0x13, 0x9d, 0x06, 0xef, 0xcc, 0x0f, 0x79, 0x63, 0x12, 0x56, 0x47, - 0x2c, 0x15, 0xf7, 0x7d, 0x3f, 0x84, 0xdc, 0x9b, 0xce, 0x2b, 0xe5, 0x94, 0xba, 0x02, 0xe5, 0x87, - 0x03, 0xec, 0x1d, 0xbf, 0x6f, 0x1b, 0x8e, 0x74, 0x03, 0x1e, 0x12, 0x98, 0x70, 0xb5, 0x68, 0x43, - 0xed, 0xc3, 0xbc, 0x34, 0x92, 0x6b, 0xc2, 0xb7, 0xe0, 0x8a, 0x69, 0xf9, 0x81, 0x7f, 0x68, 0xeb, - 0xfd, 0xfd, 0x63, 0xdf, 0xea, 0x18, 0xb6, 0x4e, 0x87, 0xeb, 0x7d, 0xdb, 0x70, 0x78, 0x20, 0x77, - 0xf5, 0xf4, 0xa4, 0x56, 0xd9, 0xb0, 0xfc, 0x60, 0xe7, 0xe1, 0xf6, 0xfb, 0x7c, 0x54, 0x84, 0xaa, - 0xc2, 0x11, 0x9c, 0xe9, 0x51, 0x2f, 0xb3, 0xe2, 0x1e, 0x99, 0xe9, 0x59, 0xed, 0x41, 0x10, 0x85, - 0x28, 0xea, 0x5f, 0x02, 0x54, 0xce, 0xf6, 0x71, 0xa2, 0xfa, 0x50, 0x12, 0x99, 0x5f, 0xc6, 0x3a, - 0x76, 0xce, 0x37, 0x9f, 0x53, 0x00, 0x8c, 0xc3, 0x15, 0x56, 0x06, 0x09, 0xcb, 0x64, 0x17, 0xb5, - 0x68, 0x4a, 0x1d, 0xa8, 0x07, 0x45, 0xa9, 0xb0, 0x21, 0x0a, 0x3a, 0x1b, 0xe7, 0x59, 0x30, 0x2a, - 0x76, 0x0c, 0xb9, 0xc4, 0x33, 0x51, 0xb1, 0xc3, 0xaf, 0xfe, 0x44, 0x01, 0x88, 0xc6, 0x11, 0x15, - 0x65, 0xe9, 0x21, 0x2e, 0x30, 0xde, 0x42, 0x77, 0x20, 0xcb, 0xcb, 0x35, 0xa9, 0xb1, 0xcb, 0x35, - 0x7c, 0x06, 0x0d, 0xc8, 0x59, 0x95, 0xc6, 0x3f, 0xb4, 0xd9, 0xd1, 0xe0, 0x01, 0x39, 0x85, 0xee, - 0x3c, 0xdc, 0xd6, 0x0a, 0x6c, 0xc0, 0xce, 0xa1, 0x7d, 0x6f, 0x3a, 0x9f, 0x2e, 0x4f, 0x57, 0xff, - 0x20, 0x0d, 0x05, 0x9a, 0xd3, 0xa4, 0x3c, 0xf9, 0x4c, 0x81, 0xca, 0x50, 0x08, 0xa3, 0xb7, 0x8f, - 0xf5, 0x28, 0x90, 0x22, 0x0c, 0xfa, 0xe0, 0x3c, 0x0c, 0x0a, 0x57, 0xa8, 0x6b, 0x52, 0x2c, 0xd4, - 0x3c, 0xa6, 0xbe, 0xa3, 0xc9, 0x78, 0xf6, 0xe6, 0x79, 0x3d, 0xcf, 0x05, 0x2f, 0x06, 0x27, 0xfa, - 0x32, 0xcc, 0xca, 0x75, 0xab, 0x30, 0xa5, 0x59, 0x8c, 0x84, 0xd2, 0x32, 0xd1, 0x37, 0x00, 0x4c, - 0xcf, 0xed, 0xf7, 0xb1, 0xa9, 0x1b, 0x2c, 0x5e, 0x1b, 0xe7, 0x02, 0x2d, 0xf0, 0x39, 0x6b, 0x41, - 0x75, 0x0b, 0x2e, 0x3f, 0x73, 0x4b, 0x31, 0x8e, 0xef, 0x50, 0x0a, 0x2c, 0x2d, 0x79, 0xb4, 0xd5, - 0xef, 0xa7, 0xa0, 0x28, 0x2b, 0x2e, 0x0a, 0x80, 0x15, 0xa8, 0xe5, 0xe3, 0xf0, 0xde, 0x8b, 0x1e, - 0x87, 0x48, 0x12, 0x43, 0xb1, 0x5b, 0x20, 0xa0, 0xd5, 0x4f, 0x60, 0x76, 0x78, 0x48, 0x4c, 0x98, - 0xb5, 0x33, 0x1c, 0x66, 0xbd, 0xfd, 0x42, 0x1a, 0x31, 0xc4, 0x03, 0x25, 0x7a, 0x84, 0x90, 0x44, - 0xc0, 0xe3, 0x61, 0x02, 0x7e, 0xeb, 0x45, 0xb9, 0x22, 0xd3, 0xf0, 0x3d, 0x28, 0x8f, 0x1e, 0xe7, - 0x18, 0x0a, 0x76, 0x87, 0x29, 0xf8, 0xcd, 0x17, 0xb3, 0x1a, 0xd2, 0xfa, 0xf7, 0xa6, 0xf3, 0xa9, - 0x72, 0x5a, 0xbd, 0x04, 0x17, 0xef, 0xd3, 0x67, 0x20, 0xf7, 0x71, 0x60, 0x98, 0x51, 0x02, 0x54, - 0xfd, 0x67, 0x05, 0x16, 0x47, 0x7b, 0xb8, 0x09, 0x35, 0x20, 0xdf, 0xe3, 0x30, 0xae, 0x2e, 0xdf, - 0x48, 0x20, 0x2b, 0x1e, 0x49, 0x5d, 0x00, 0x64, 0xf5, 0x08, 0xd1, 0x56, 0x7f, 0x1b, 0x4a, 0x43, - 0x03, 0x62, 0x38, 0xf3, 0xfa, 0x30, 0x67, 0xe4, 0xe8, 0x6a, 0x10, 0x58, 0x76, 0x9d, 0xbf, 0x6e, - 0x09, 0x17, 0x96, 0x82, 0xba, 0x1f, 0xa7, 0xe0, 0xc2, 0xa6, 0x73, 0x38, 0xc0, 0x03, 0x4c, 0xaf, - 0x45, 0x71, 0xb7, 0x7d, 0x91, 0xc9, 0x1c, 0x76, 0x6f, 0x86, 0x7e, 0x1c, 0x6b, 0xa0, 0x6f, 0x4b, - 0x2e, 0x17, 0x2d, 0x78, 0x34, 0xd7, 0x4f, 0x4f, 0x6a, 0x39, 0x4a, 0x15, 0x5d, 0xf3, 0xf6, 0x44, - 0x6b, 0xf2, 0x79, 0x91, 0xdf, 0x76, 0x03, 0xe6, 0xfd, 0x03, 0xab, 0xaf, 0xfb, 0xfb, 0xee, 0xc0, - 0x36, 0x75, 0x46, 0x01, 0x4f, 0x03, 0x91, 0x8e, 0x1d, 0x0a, 0x7f, 0x48, 0xc0, 0xea, 0x9f, 0xa7, - 0x60, 0x61, 0x98, 0x2b, 0x5c, 0xde, 0x0f, 0x23, 0x6f, 0x86, 0x89, 0xfb, 0xcd, 0xa4, 0xea, 0x49, - 0x0c, 0x86, 0xba, 0x78, 0x9c, 0x10, 0xba, 0x41, 0x7f, 0xaf, 0x40, 0x8e, 0x03, 0xbf, 0x50, 0xae, - 0xbf, 0x3d, 0xe2, 0x8c, 0x5e, 0x4f, 0xaa, 0xb8, 0x79, 0x46, 0x07, 0x53, 0x57, 0x4b, 0xb8, 0x9e, - 0x51, 0x54, 0x94, 0x96, 0xa2, 0x22, 0xf5, 0x22, 0x5c, 0x58, 0xdf, 0x37, 0xbc, 0x60, 0x9d, 0x3d, - 0xad, 0x12, 0x27, 0xe6, 0x09, 0x2c, 0x0c, 0x83, 0x39, 0xfb, 0x9a, 0x90, 0xe3, 0x8f, 0xb0, 0x38, - 0xfb, 0x54, 0x89, 0x88, 0xc0, 0xaf, 0x8b, 0x17, 0x5a, 0x74, 0xf2, 0x0e, 0xab, 0x7d, 0x87, 0xef, - 0x41, 0x58, 0xdf, 0x8d, 0x77, 0x61, 0x31, 0xfe, 0xa1, 0x03, 0x9a, 0x81, 0xdc, 0xa3, 0x07, 0xef, - 0x3e, 0x78, 0xef, 0x83, 0x07, 0xe5, 0x29, 0xd2, 0x58, 0xdf, 0x7e, 0xb4, 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, 0x1b, 0xd7, 0xa0, 0x40, 0xf3, 0xce, 0xf7, 0x5d, 0x13, - 0x23, 0x80, 0xec, 0xfa, 0x76, 0x6b, 0xf3, 0xc1, 0x6e, 0x79, 0x8a, 0x7c, 0x6f, 0x6f, 0xae, 0xed, - 0x6c, 0xee, 0x94, 0x95, 0xd5, 0x1f, 0x55, 0x21, 0xb3, 0x66, 0xf6, 0x2c, 0x07, 0x05, 0x90, 0xa1, - 0xb5, 0x41, 0xf4, 0xf2, 0xf3, 0xab, 0x87, 0x94, 0x13, 0xd5, 0x95, 0x71, 0xcb, 0x8c, 0x6a, 0xe5, - 0x07, 0xff, 0xf4, 0xef, 0x3f, 0x4e, 0x21, 0x54, 0x6e, 0xe8, 0xf4, 0x35, 0x5e, 0xe3, 0xe8, 0x66, - 0x83, 0x96, 0x1b, 0xd1, 0xef, 0x2b, 0x50, 0x08, 0xdf, 0x89, 0xa1, 0x57, 0xc6, 0x78, 0x9f, 0x15, - 0x2e, 0xff, 0xea, 0x78, 0x83, 0x39, 0x09, 0x57, 0x29, 0x09, 0x8b, 0x68, 0x41, 0x22, 0x21, 0x7c, - 0x7a, 0x86, 0xfe, 0x44, 0x81, 0xb9, 0x91, 0x07, 0x60, 0xe8, 0xe6, 0x24, 0x8f, 0xc5, 0x18, 0x49, - 0xab, 0x93, 0xbf, 0x2f, 0x53, 0x5f, 0xa6, 0x84, 0xbd, 0x84, 0x6a, 0x71, 0x84, 0x35, 0x3e, 0x11, - 0x9f, 0xdf, 0x45, 0x7f, 0xa1, 0x40, 0x51, 0x7e, 0x03, 0x84, 0xea, 0x63, 0x3f, 0x16, 0x62, 0xd4, - 0x35, 0x26, 0x7c, 0x5c, 0xa4, 0xbe, 0x41, 0x49, 0x7b, 0x0d, 0xd5, 0x9f, 0x43, 0x5a, 0x83, 0xde, - 0xf7, 0x7e, 0xe3, 0x13, 0xfa, 0x4b, 0x29, 0x85, 0xa8, 0xb0, 0x8d, 0x5e, 0x1d, 0xb3, 0xfe, 0xcd, - 0xa8, 0x9c, 0xac, 0x5a, 0xae, 0xde, 0xa5, 0x34, 0xbe, 0x81, 0x6e, 0x4f, 0x46, 0x63, 0x83, 0x3d, - 0x9c, 0xf8, 0x89, 0x02, 0xa5, 0xa1, 0xb7, 0x02, 0x28, 0x89, 0x49, 0x71, 0xcf, 0x0d, 0xaa, 0xaf, - 0x8d, 0x3f, 0x81, 0x93, 0xbc, 0x4c, 0x49, 0xae, 0xa2, 0x8a, 0x44, 0xb2, 0xe3, 0x3a, 0x8c, 0x40, - 0x4a, 0xc4, 0x47, 0x90, 0x65, 0xe5, 0x69, 0xb4, 0x32, 0x46, 0x05, 0x9b, 0xd1, 0xf1, 0xd5, 0xb1, - 0x6b, 0xdd, 0xea, 0x65, 0x4a, 0xc0, 0x05, 0x34, 0x2f, 0x11, 0xc0, 0x4d, 0x21, 0x39, 0x8f, 0x61, - 0xe9, 0x34, 0xf1, 0x3c, 0x8e, 0x16, 0x73, 0x13, 0xcf, 0xe3, 0xd9, 0x6a, 0x2c, 0x3f, 0x8f, 0xaa, - 0x4c, 0xc3, 0xc0, 0x22, 0xe2, 0xba, 0xa3, 0xdc, 0x40, 0xdf, 0x57, 0xa0, 0xb0, 0x35, 0x16, 0x19, - 0x5b, 0x93, 0x90, 0x71, 0xa6, 0x56, 0x19, 0xcb, 0x0a, 0x46, 0x06, 0xfa, 0x5d, 0xc8, 0xf1, 0xca, - 0x23, 0x4a, 0xe2, 0xed, 0x70, 0x65, 0xb3, 0x7a, 0x63, 0x9c, 0xa1, 0x7c, 0xf1, 0x2a, 0x5d, 0x7c, - 0x01, 0x21, 0x69, 0x71, 0x5e, 0xe1, 0x44, 0xbf, 0xa7, 0x40, 0x5e, 0x14, 0x15, 0xd0, 0x8d, 0xb1, - 0x2a, 0x0f, 0x8c, 0x80, 0x57, 0x26, 0xa8, 0x52, 0xa8, 0x57, 0x28, 0x05, 0x17, 0xd1, 0x05, 0x89, - 0x02, 0x5f, 0xac, 0xfa, 0x43, 0x05, 0xb2, 0xac, 0x16, 0x91, 0xa8, 0x86, 0x43, 0x55, 0x8d, 0x44, - 0x35, 0x1c, 0x29, 0x6c, 0x7c, 0x99, 0x2e, 0xbe, 0x34, 0xc4, 0xfb, 0x7d, 0x3a, 0xe4, 0x49, 0x01, - 0xe5, 0xf8, 0x27, 0x65, 0x84, 0x48, 0xdc, 0x27, 0x32, 0x62, 0xa4, 0x48, 0x92, 0xc8, 0x88, 0xd1, - 0xca, 0x45, 0x2c, 0x23, 0x44, 0x61, 0x04, 0xf5, 0x61, 0xfa, 0x9e, 0xdb, 0xf6, 0xd1, 0x57, 0x9e, - 0x9b, 0x94, 0x67, 0x2b, 0xbf, 0x3c, 0x66, 0xf2, 0x5e, 0xbd, 0x44, 0x57, 0x9d, 0x47, 0x73, 0xd2, - 0xaa, 0xdf, 0x21, 0x2b, 0x91, 0x63, 0x18, 0xe6, 0x1d, 0x13, 0xf5, 0x7f, 0x34, 0xe7, 0x99, 0xa8, - 0xff, 0x67, 0x52, 0x99, 0xb1, 0xd7, 0x62, 0x98, 0xce, 0xa4, 0x64, 0x84, 0xb9, 0x98, 0x44, 0x32, - 0x46, 0x13, 0x48, 0x89, 0x64, 0x9c, 0xc9, 0x21, 0xc5, 0x92, 0x41, 0x73, 0x48, 0x7d, 0xb2, 0xf0, - 0xb7, 0x21, 0x43, 0x3d, 0x99, 0x44, 0xd7, 0x44, 0xae, 0xd8, 0x27, 0xba, 0x26, 0x43, 0xc5, 0x78, - 0x75, 0xea, 0x35, 0x05, 0x3d, 0x85, 0xa2, 0x5c, 0x1b, 0x4e, 0xbc, 0x58, 0x63, 0x8a, 0xe5, 0xd5, - 0xd7, 0xcf, 0x55, 0x74, 0x56, 0xa7, 0xc8, 0x09, 0x43, 0x67, 0x07, 0xa0, 0xdb, 0x13, 0xe2, 0x7b, - 0x41, 0x2a, 0xfe, 0x54, 0x81, 0xbc, 0xc8, 0x1b, 0x26, 0x9e, 0xb0, 0x91, 0x4c, 0x72, 0xe2, 0x09, - 0x1b, 0x4d, 0x44, 0xaa, 0x6f, 0x53, 0x11, 0xbf, 0x39, 0x74, 0xc2, 0x68, 0xfc, 0x62, 0xbb, 0xdd, - 0x27, 0xcb, 0x68, 0x29, 0x06, 0xdc, 0xf8, 0x44, 0x44, 0x4d, 0xdf, 0x25, 0x1e, 0x5a, 0x79, 0x34, - 0xf4, 0x45, 0xab, 0x13, 0xc5, 0xc9, 0x8c, 0xe8, 0x5b, 0xe7, 0x88, 0xad, 0x43, 0x53, 0x75, 0x75, - 0xc4, 0xcb, 0xd0, 0x4d, 0x99, 0x9c, 0x9f, 0x2a, 0x30, 0xbf, 0x66, 0xdb, 0xc3, 0xa1, 0x30, 0x7a, - 0x6d, 0x82, 0xa8, 0x99, 0x91, 0x78, 0x73, 0xe2, 0x38, 0x5b, 0x7d, 0x89, 0x12, 0x78, 0x05, 0x5d, - 0x96, 0x08, 0x64, 0xa1, 0xb1, 0x08, 0xb6, 0xd1, 0xa7, 0x0a, 0x14, 0xe5, 0xc8, 0x25, 0x51, 0xcd, - 0x63, 0x22, 0x9f, 0x44, 0xff, 0x31, 0x2e, 0x24, 0x52, 0x6b, 0x94, 0xa8, 0xcb, 0xe8, 0x92, 0x7c, - 0xbf, 0x91, 0x81, 0x3c, 0xde, 0x21, 0xee, 0x57, 0x51, 0x8e, 0x24, 0x13, 0x49, 0x8a, 0x09, 0xe5, - 0x13, 0x49, 0x8a, 0x0b, 0x51, 0xd5, 0x6b, 0x94, 0xa4, 0x2f, 0xa9, 0xb2, 0xef, 0x85, 0xd9, 0x40, - 0x9d, 0x2a, 0xdb, 0x1d, 0xe5, 0x46, 0xf3, 0xc6, 0x67, 0xff, 0xb6, 0x34, 0xf5, 0xd9, 0xe9, 0x92, - 0xf2, 0x8b, 0xd3, 0x25, 0xe5, 0x97, 0xa7, 0x4b, 0xca, 0xbf, 0x9e, 0x2e, 0x29, 0x9f, 0x7e, 0xbe, - 0x34, 0xf5, 0x8b, 0xcf, 0x97, 0xa6, 0x7e, 0xf9, 0xf9, 0xd2, 0xd4, 0x93, 0xbc, 0x58, 0xa5, 0x9d, - 0xa5, 0x59, 0xbd, 0x5b, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0x66, 0x87, 0x11, 0xbb, 0x77, 0x34, - 0x00, 0x00, + 0x75, 0x1a, 0x80, 0xf8, 0x7a, 0x04, 0x48, 0xb0, 0x45, 0x51, 0x20, 0x24, 0x13, 0xdc, 0x91, 0xe5, + 0xa5, 0xb5, 0x6b, 0x60, 0x45, 0x69, 0xb5, 0x8e, 0x56, 0x1b, 0x87, 0x20, 0x58, 0x5c, 0x68, 0x29, + 0xed, 0x6a, 0x28, 0x69, 0xcb, 0x72, 0xd6, 0x93, 0x01, 0xa6, 0x05, 0x8e, 0x39, 0x98, 0x01, 0x67, + 0x06, 0xd4, 0x72, 0x37, 0x76, 0xc5, 0x76, 0x2a, 0x95, 0x5c, 0x52, 0x5b, 0xb6, 0x6f, 0xae, 0x4a, + 0x55, 0x52, 0xa9, 0xa4, 0x92, 0x9c, 0x72, 0x4b, 0x72, 0xc8, 0x79, 0x2b, 0x07, 0xdb, 0x55, 0xb9, + 0x38, 0x87, 0xd0, 0x09, 0x37, 0x87, 0x54, 0x7e, 0x40, 0x0e, 0x39, 0x24, 0xa9, 0xfe, 0x9a, 0x69, + 0x80, 0xb3, 0x23, 0x80, 0xb2, 0x7d, 0x20, 0x31, 0xfd, 0x5e, 0xf7, 0xeb, 0xd7, 0xaf, 0x5f, 0xbf, + 0x7e, 0x1f, 0x0d, 0x97, 0x7c, 0xec, 0x1d, 0x62, 0xaf, 0xc1, 0x7e, 0x06, 0x9d, 0x86, 0x61, 0xf6, + 0x2d, 0xa7, 0x3e, 0xf0, 0xdc, 0xc0, 0x45, 0xcb, 0x5d, 0xb7, 0xbb, 0xef, 0xb9, 0x46, 0x77, 0xaf, + 0xce, 0xf0, 0x75, 0xd1, 0xad, 0xba, 0xd0, 0x75, 0x9d, 0xa7, 0x56, 0xaf, 0xf1, 0x91, 0xeb, 0x60, + 0xd6, 0xbb, 0xba, 0xf4, 0x2d, 0xb7, 0xe3, 0x37, 0xc8, 0xbf, 0x41, 0x87, 0xfe, 0x70, 0xf8, 0xe5, + 0xf1, 0x29, 0xfc, 0xc0, 0x08, 0x86, 0x02, 0xab, 0xfa, 0x81, 0xeb, 0x19, 0x3d, 0xdc, 0xc0, 0x4e, + 0xcf, 0x72, 0xc4, 0xcf, 0xa0, 0xd3, 0xe8, 0x1f, 0x76, 0xbb, 0xbc, 0xcf, 0xaa, 0xe8, 0xc3, 0x7f, + 0x07, 0x9d, 0x86, 0x6d, 0x1d, 0x62, 0x07, 0xfb, 0x82, 0xca, 0xa5, 0x98, 0x1e, 0x6e, 0x8f, 0x23, + 0x57, 0x02, 0xbf, 0xd1, 0x35, 0x02, 0xc3, 0x76, 0x7b, 0x8d, 0xee, 0x9e, 0xe1, 0x05, 0x3a, 0x6f, + 0x71, 0x7c, 0x65, 0x18, 0x58, 0x76, 0xa3, 0x8f, 0x03, 0xcf, 0xea, 0xf2, 0x1f, 0x8e, 0x59, 0xec, + 0xb9, 0x3d, 0x97, 0x7e, 0x36, 0xc8, 0x97, 0x58, 0x50, 0xcf, 0x75, 0x7b, 0x36, 0x6e, 0x18, 0x03, + 0xab, 0x61, 0x38, 0x8e, 0x1b, 0x18, 0x81, 0xe5, 0x3a, 0x82, 0x95, 0x1a, 0xc7, 0xd2, 0x56, 0x67, + 0xf8, 0xb4, 0x11, 0x58, 0x7d, 0xec, 0x07, 0x46, 0x7f, 0xc0, 0x3a, 0xa8, 0x08, 0xca, 0x2d, 0x23, + 0x30, 0x3a, 0x86, 0x8f, 0x7d, 0x0d, 0x1f, 0x0c, 0xb1, 0x1f, 0xa8, 0xd7, 0x61, 0x41, 0x82, 0xf9, + 0x03, 0xd7, 0xf1, 0x31, 0xba, 0x0c, 0x05, 0x53, 0x00, 0x2b, 0xca, 0x6a, 0x7a, 0xad, 0xa0, 0x45, + 0x00, 0xf5, 0x26, 0x2c, 0x89, 0x21, 0x2d, 0x1c, 0x18, 0x96, 0x2d, 0x88, 0xa1, 0x2a, 0xe4, 0x45, + 0xb7, 0x8a, 0xb2, 0xaa, 0xac, 0x15, 0xb4, 0xb0, 0xad, 0xfe, 0x75, 0x1a, 0x2e, 0x9e, 0x1a, 0xc6, + 0xe7, 0x7b, 0x0c, 0xd9, 0x9e, 0x67, 0x38, 0x01, 0x9b, 0x6c, 0x76, 0xfd, 0xab, 0xf5, 0xcf, 0xdd, + 0xff, 0xfa, 0xe7, 0xd0, 0xa8, 0x6f, 0x13, 0x02, 0xcd, 0x99, 0x4f, 0x8f, 0x6b, 0xe7, 0x34, 0x4e, + 0x0d, 0xd5, 0x60, 0x36, 0x30, 0x3a, 0x36, 0xd6, 0x1d, 0xa3, 0x8f, 0xfd, 0x4a, 0x8a, 0xae, 0x04, + 0x28, 0xe8, 0x3e, 0x81, 0xa0, 0xd7, 0xa1, 0x64, 0x62, 0xbf, 0xeb, 0x59, 0x83, 0xc0, 0xf5, 0x74, + 0xcb, 0xac, 0xa4, 0x57, 0x95, 0xb5, 0x74, 0xb3, 0x7c, 0x72, 0x5c, 0x2b, 0xb6, 0x42, 0x44, 0xbb, + 0xa5, 0x15, 0xa3, 0x6e, 0x6d, 0x13, 0x6d, 0xc2, 0x2c, 0x51, 0x3f, 0x9d, 0xa9, 0x62, 0x65, 0x66, + 0x55, 0x59, 0x9b, 0x5d, 0xbf, 0x2c, 0x31, 0xcd, 0x10, 0xf5, 0x27, 0xae, 0x83, 0x37, 0xe9, 0x27, + 0x67, 0x0c, 0x3e, 0x0a, 0x21, 0xe8, 0x03, 0x58, 0x90, 0x88, 0xe8, 0x36, 0x3e, 0xc4, 0x76, 0x25, + 0xb3, 0xaa, 0xac, 0xcd, 0xad, 0x5f, 0x4f, 0x58, 0x7f, 0x44, 0x73, 0xe8, 0xd1, 0xed, 0xdf, 0x21, + 0x03, 0xb5, 0xf9, 0x88, 0x32, 0x05, 0x54, 0xdf, 0x84, 0x0c, 0x15, 0x09, 0x42, 0x30, 0x33, 0xf4, + 0xb1, 0xc7, 0x37, 0x84, 0x7e, 0xa3, 0x15, 0x80, 0x81, 0x67, 0x1d, 0x5a, 0x36, 0xee, 0x45, 0x72, + 0x89, 0x20, 0xea, 0x36, 0x9c, 0x7f, 0x48, 0xa4, 0x34, 0xf9, 0xfe, 0xa2, 0x45, 0xc8, 0x50, 0xc1, + 0x56, 0x52, 0x14, 0xc1, 0x1a, 0xea, 0xff, 0xe6, 0x60, 0x71, 0x94, 0x12, 0xdf, 0xf2, 0xdd, 0xb1, + 0x2d, 0x7f, 0x3d, 0x61, 0xc9, 0x71, 0x04, 0x62, 0xf7, 0xfb, 0x31, 0xe4, 0xba, 0xae, 0x3d, 0xec, + 0x3b, 0x6c, 0x4d, 0xb3, 0xeb, 0xb7, 0xa6, 0xa5, 0xba, 0x49, 0x87, 0x73, 0xb2, 0x82, 0x18, 0x7a, + 0x04, 0x39, 0xcb, 0x31, 0xf1, 0x87, 0xd8, 0xaf, 0xa4, 0xcf, 0xc6, 0x6d, 0x9b, 0x0c, 0x17, 0x64, + 0x39, 0x2d, 0xa2, 0x9e, 0x9e, 0xe1, 0xf4, 0x88, 0x0a, 0x0c, 0x9d, 0x80, 0xaa, 0x51, 0x5a, 0x03, + 0x0a, 0xda, 0x24, 0x10, 0x74, 0x13, 0x96, 0xba, 0x1e, 0x36, 0x02, 0xac, 0x33, 0x35, 0x26, 0xf6, + 0x0b, 0xf7, 0xb1, 0x13, 0x50, 0x3d, 0x29, 0x68, 0x8b, 0x0c, 0x4b, 0x67, 0xdc, 0x15, 0xb8, 0x71, + 0xed, 0xcc, 0xfe, 0xf2, 0xb4, 0x33, 0xf7, 0xcb, 0xd2, 0xce, 0xd3, 0x07, 0x2f, 0x3f, 0xc9, 0xc1, + 0x7b, 0x21, 0xa5, 0xae, 0xfe, 0xa3, 0x02, 0x59, 0xb6, 0xbf, 0x64, 0x38, 0x31, 0x09, 0x62, 0x38, + 0xf9, 0x26, 0xb0, 0xe0, 0x68, 0x20, 0xf4, 0x97, 0x7e, 0x13, 0x85, 0x77, 0x86, 0xb6, 0x4d, 0xf5, + 0x9a, 0x98, 0x86, 0xbc, 0x16, 0xb6, 0xd1, 0x15, 0xb2, 0x84, 0xa7, 0xc6, 0xd0, 0x0e, 0xf4, 0x43, + 0xc3, 0x1e, 0x62, 0xba, 0x7f, 0x05, 0xc2, 0x30, 0x05, 0x3e, 0x26, 0x30, 0x74, 0x03, 0x2e, 0xf4, + 0xb0, 0x83, 0x99, 0x2c, 0x74, 0xfc, 0xe1, 0xc0, 0xc3, 0xbe, 0x6f, 0xb9, 0x8e, 0xd8, 0xc0, 0x08, + 0xb9, 0x15, 0xe2, 0xd0, 0x12, 0x64, 0xf7, 0x2c, 0xd3, 0xc4, 0x0e, 0xdd, 0xbb, 0xbc, 0xc6, 0x5b, + 0xd5, 0xbf, 0x55, 0x20, 0x43, 0x15, 0x29, 0x96, 0xff, 0x25, 0xc8, 0x0e, 0x1d, 0xeb, 0x60, 0xc8, + 0x56, 0x90, 0xd7, 0x78, 0x0b, 0x95, 0x21, 0xed, 0xe3, 0x03, 0x66, 0xd9, 0x34, 0xf2, 0x49, 0x7a, + 0x32, 0xcd, 0xe6, 0x2c, 0xf3, 0x16, 0x35, 0xfb, 0x96, 0x87, 0xbb, 0x41, 0xc4, 0x60, 0x04, 0x40, + 0x15, 0xc8, 0x91, 0x3b, 0xce, 0x72, 0x7a, 0x9c, 0x2d, 0xd1, 0x24, 0x52, 0xb2, 0xfa, 0x03, 0xdb, + 0xea, 0x5a, 0x01, 0x55, 0x91, 0xbc, 0x16, 0xb6, 0xd5, 0x2d, 0x58, 0x08, 0xd5, 0xf3, 0x05, 0xec, + 0xc8, 0xdf, 0xa5, 0x01, 0xc9, 0x74, 0xb8, 0x15, 0x19, 0x3b, 0x41, 0xca, 0xa9, 0x13, 0x74, 0x05, + 0x4a, 0x1e, 0x26, 0xac, 0x18, 0xbc, 0x4b, 0x8a, 0x76, 0x29, 0x72, 0x20, 0xeb, 0xf4, 0x05, 0x00, + 0xc7, 0x35, 0x05, 0x11, 0x26, 0xa8, 0x02, 0x81, 0x30, 0xf4, 0xdb, 0x90, 0x21, 0x07, 0xcf, 0xe7, + 0x76, 0xfe, 0x55, 0x59, 0xfd, 0xd9, 0xa5, 0x5f, 0x67, 0xbe, 0x43, 0x5d, 0xb8, 0x10, 0xf5, 0x7b, + 0x8f, 0x37, 0x37, 0x29, 0xa7, 0xfc, 0x64, 0x31, 0x02, 0xc8, 0x84, 0x52, 0xdf, 0xf2, 0x7d, 0xcb, + 0xe9, 0xe9, 0x84, 0xbc, 0x5f, 0xc9, 0x50, 0x6b, 0xf2, 0x1b, 0xcf, 0xb3, 0x26, 0x23, 0x8b, 0xae, + 0xdf, 0x63, 0x24, 0xee, 0xbb, 0x26, 0xe6, 0xe4, 0x8b, 0xfd, 0x08, 0xe4, 0x13, 0xab, 0x61, 0x0c, + 0x06, 0x9e, 0xfb, 0xa1, 0xd5, 0x27, 0xa6, 0xc3, 0xb4, 0xfc, 0x7d, 0xbd, 0x73, 0x14, 0x60, 0x9f, + 0xee, 0xdb, 0x8c, 0xb6, 0x28, 0x61, 0x5b, 0x96, 0xbf, 0xdf, 0x24, 0xb8, 0xea, 0xfb, 0x30, 0x2b, + 0x11, 0x46, 0x57, 0x20, 0x47, 0x65, 0x62, 0x99, 0x6c, 0x87, 0x9a, 0x70, 0x72, 0x5c, 0xcb, 0x12, + 0x54, 0xbb, 0xa5, 0x65, 0x09, 0xaa, 0x6d, 0x12, 0xe9, 0x62, 0xcf, 0x73, 0x3d, 0xbd, 0x8f, 0x7d, + 0xdf, 0xe8, 0x89, 0x3d, 0x2b, 0x52, 0xe0, 0x3d, 0x06, 0x53, 0x97, 0x60, 0xf1, 0xbe, 0xeb, 0x9c, + 0x52, 0x02, 0xf5, 0xa7, 0x0a, 0x5c, 0x18, 0x43, 0xf0, 0x5d, 0xfd, 0x3a, 0x2c, 0x10, 0xd7, 0x45, + 0xf7, 0xb1, 0x67, 0x61, 0x5f, 0x67, 0xc2, 0x57, 0xa8, 0xf0, 0xbf, 0x32, 0x95, 0xa8, 0xb4, 0x79, + 0x42, 0x67, 0x97, 0x92, 0xa1, 0x08, 0xf4, 0x0d, 0x40, 0x96, 0x13, 0x60, 0xcf, 0x31, 0x6c, 0x7d, + 0xe8, 0x63, 0x4e, 0x3b, 0x75, 0x16, 0xda, 0x65, 0x41, 0xe8, 0x91, 0xcf, 0x30, 0xea, 0x1c, 0x14, + 0x1f, 0xf9, 0xd8, 0x0b, 0x57, 0xf8, 0x1d, 0x28, 0xf1, 0x36, 0x5f, 0x58, 0x1b, 0x32, 0xc4, 0x52, + 0x89, 0x3b, 0x2f, 0x69, 0xc2, 0x91, 0x81, 0xb4, 0x25, 0x54, 0x89, 0x52, 0xa8, 0xaa, 0x30, 0x43, + 0x80, 0xe4, 0x28, 0x11, 0x80, 0x64, 0x0d, 0xc2, 0xb6, 0xfa, 0x7d, 0x05, 0x4a, 0x5b, 0x87, 0xd8, + 0x89, 0x0e, 0x9e, 0xb0, 0x71, 0x8a, 0x64, 0xe3, 0x2e, 0x41, 0x21, 0x30, 0xbc, 0x1e, 0x0e, 0xc8, + 0x5e, 0xb3, 0xe3, 0x91, 0x67, 0x80, 0xb6, 0x49, 0x4e, 0xa3, 0x6d, 0xf5, 0x2d, 0x76, 0x2a, 0x32, + 0x1a, 0x6b, 0xa0, 0x57, 0x60, 0x61, 0xe8, 0x78, 0xd8, 0x34, 0xba, 0x01, 0x36, 0x75, 0x4c, 0xa7, + 0xa0, 0xa7, 0x23, 0xaf, 0x95, 0x23, 0x04, 0x9b, 0x5a, 0xfd, 0xaf, 0x14, 0xcc, 0x09, 0x2e, 0xb8, + 0x1c, 0xee, 0x41, 0x96, 0x0f, 0x62, 0x82, 0x68, 0x24, 0x08, 0x62, 0x74, 0x28, 0x6b, 0x8a, 0x6b, + 0x9f, 0x11, 0xa9, 0xfe, 0x61, 0x0a, 0x32, 0x14, 0x8e, 0x9a, 0x50, 0x08, 0x9d, 0x5e, 0xae, 0x31, + 0xd5, 0x3a, 0x73, 0x8b, 0xeb, 0xc2, 0x2d, 0xae, 0x3f, 0x14, 0x3d, 0x9a, 0x79, 0x42, 0xe6, 0x93, + 0x5f, 0xd4, 0x14, 0x2d, 0x1a, 0x46, 0xac, 0x01, 0xa5, 0xab, 0x4b, 0xb7, 0x41, 0x81, 0x42, 0x1e, + 0x12, 0x71, 0x7d, 0x59, 0x16, 0x17, 0x73, 0x17, 0x8b, 0x27, 0xc7, 0xb5, 0xfc, 0x43, 0x26, 0xb2, + 0x96, 0x24, 0xbc, 0x75, 0x20, 0x76, 0xc6, 0xf5, 0x02, 0x72, 0xe0, 0x2d, 0x93, 0x5d, 0xf0, 0xcd, + 0xf9, 0x93, 0xe3, 0xda, 0xac, 0x26, 0xe0, 0xed, 0x96, 0x36, 0x1b, 0x76, 0x6a, 0x9b, 0x64, 0x87, + 0x2c, 0xe7, 0xa9, 0xcb, 0xcd, 0x2f, 0xfd, 0x26, 0x53, 0x32, 0x5b, 0x4e, 0x88, 0x90, 0x33, 0x5c, + 0x64, 0x53, 0x3e, 0xa2, 0x40, 0x32, 0x25, 0x43, 0xb7, 0x4d, 0xf5, 0x6f, 0x14, 0x28, 0xef, 0xe2, + 0xe0, 0x51, 0x9b, 0xb8, 0xc9, 0x62, 0xd7, 0xbf, 0x0e, 0xb0, 0x8f, 0x8f, 0xd8, 0x2d, 0x25, 0x44, + 0x7e, 0x3b, 0x41, 0xe4, 0xe3, 0x04, 0xea, 0xef, 0xe0, 0x23, 0x7a, 0x9d, 0xf9, 0x5b, 0x4e, 0xe0, + 0x1d, 0x69, 0x85, 0x7d, 0xd1, 0xae, 0xde, 0x81, 0xb9, 0x51, 0x24, 0xb9, 0x6e, 0xf6, 0xf1, 0x11, + 0xd7, 0x30, 0xf2, 0x49, 0x74, 0x88, 0x5d, 0x90, 0x44, 0x96, 0x45, 0x8d, 0x35, 0x6e, 0xa7, 0xbe, + 0xaa, 0xa8, 0xe7, 0x61, 0x41, 0x9a, 0x8b, 0xed, 0xb0, 0xfa, 0x25, 0x28, 0x6f, 0x8f, 0xaf, 0x00, + 0xc1, 0xcc, 0x3e, 0x3e, 0x12, 0xb1, 0x08, 0xfd, 0x56, 0x7f, 0x9a, 0x82, 0x85, 0xed, 0xf1, 0xd1, + 0xe8, 0x77, 0x62, 0xd6, 0xfa, 0x66, 0xc2, 0x5a, 0x4f, 0x51, 0x18, 0x5b, 0x2c, 0x57, 0x35, 0x69, + 0xc9, 0x4f, 0x21, 0xc3, 0xee, 0xf6, 0x70, 0x5d, 0x8a, 0xb4, 0x2e, 0xb4, 0x0d, 0x45, 0xdb, 0xf0, + 0x03, 0x7d, 0x38, 0x30, 0x8d, 0x00, 0x9b, 0xdc, 0xb6, 0x4c, 0xa6, 0x85, 0xb3, 0x64, 0xe4, 0x23, + 0x36, 0xb0, 0x3a, 0x98, 0x40, 0xb4, 0x6f, 0xcb, 0xa2, 0x9d, 0x5d, 0x5f, 0x9f, 0x6a, 0xa1, 0x94, + 0xb4, 0xbc, 0x1d, 0x65, 0x98, 0xdb, 0xb4, 0x87, 0x7e, 0x80, 0x3d, 0x61, 0xc1, 0x7e, 0xac, 0xc0, + 0x7c, 0x08, 0xe2, 0x12, 0x7e, 0x15, 0xa0, 0xcb, 0x40, 0xd1, 0xe5, 0x50, 0x3a, 0x39, 0xae, 0x15, + 0x78, 0xc7, 0x76, 0x4b, 0x2b, 0xf0, 0x0e, 0x6d, 0x93, 0x98, 0x8a, 0xe8, 0x0c, 0x60, 0x87, 0x98, + 0x51, 0x93, 0x3b, 0x28, 0xe5, 0x10, 0xb1, 0xc5, 0xe0, 0xe8, 0x2b, 0x80, 0x30, 0x31, 0xaa, 0x03, + 0xcf, 0xf2, 0x71, 0xd8, 0x9b, 0x39, 0x5e, 0x0b, 0x11, 0x86, 0x77, 0x57, 0xff, 0x48, 0x81, 0x62, + 0xcb, 0x33, 0x2c, 0x47, 0xa8, 0xc9, 0x1d, 0xa8, 0x9a, 0x78, 0xe0, 0xe1, 0x2e, 0x11, 0xa0, 0x3e, + 0xf0, 0xdc, 0x0e, 0xd6, 0x2d, 0xc7, 0xb4, 0xba, 0x46, 0xe0, 0x7a, 0x54, 0x19, 0x32, 0x5a, 0x25, + 0xea, 0xf1, 0x1e, 0xe9, 0xd0, 0x16, 0x78, 0x62, 0x4a, 0xfd, 0xbd, 0x61, 0x60, 0xba, 0xcf, 0x1c, + 0xe1, 0xec, 0x89, 0x36, 0x5a, 0x86, 0xbc, 0xe9, 0xea, 0x26, 0x99, 0x8c, 0x1b, 0xba, 0x9c, 0xe9, + 0xd2, 0xb9, 0xef, 0xce, 0xe4, 0x53, 0xe5, 0xb4, 0xba, 0x07, 0x25, 0xce, 0x0a, 0x17, 0xd3, 0x2d, + 0xb8, 0x28, 0xf1, 0x42, 0x47, 0xea, 0x2c, 0xff, 0xc0, 0x19, 0xb9, 0x10, 0xa1, 0xe9, 0xc8, 0x5d, + 0x8a, 0x24, 0x2e, 0x8d, 0xe5, 0xb3, 0xfe, 0xc4, 0xd5, 0x62, 0xa2, 0x02, 0xcb, 0x6f, 0x71, 0x88, + 0xfa, 0x11, 0x2c, 0xb7, 0x70, 0xd7, 0xed, 0xd3, 0x3b, 0xdf, 0xe5, 0xc3, 0x84, 0x04, 0x3e, 0x80, + 0x3c, 0xbf, 0xb6, 0xf9, 0x34, 0xcd, 0xe6, 0xc9, 0x71, 0x2d, 0xc7, 0xee, 0x6d, 0xff, 0x7f, 0x8e, + 0x6b, 0x37, 0x7a, 0x56, 0xb0, 0x37, 0xec, 0xd4, 0xbb, 0x6e, 0xbf, 0x11, 0x2a, 0x8b, 0xd9, 0x89, + 0xbe, 0x1b, 0x83, 0xfd, 0x5e, 0x83, 0x7e, 0x0d, 0x3a, 0x75, 0x7e, 0xdf, 0xe7, 0xd8, 0x7d, 0xef, + 0xab, 0x7f, 0xa2, 0xc0, 0x79, 0x79, 0xf2, 0x5f, 0xcf, 0xb4, 0x68, 0x0d, 0xe6, 0x4d, 0x69, 0xd6, + 0x48, 0x2e, 0xe3, 0x60, 0xf5, 0xb3, 0x14, 0x54, 0xe3, 0xa4, 0xc3, 0x37, 0xe5, 0x09, 0x64, 0xf9, + 0x1e, 0xb0, 0xf8, 0xf0, 0x4e, 0x52, 0xa2, 0xe1, 0x73, 0xc9, 0xd4, 0x59, 0x53, 0xdc, 0x42, 0x8c, + 0x62, 0xf5, 0x3f, 0x15, 0xc8, 0xf2, 0x3d, 0x7c, 0x32, 0xea, 0x3c, 0x65, 0x9a, 0x1b, 0x91, 0xf3, + 0x74, 0x56, 0x61, 0x08, 0x9f, 0xeb, 0x22, 0xe4, 0x2c, 0x5f, 0xb7, 0xad, 0xc3, 0xd0, 0xcf, 0xb7, + 0xfc, 0x1d, 0xeb, 0x10, 0x9f, 0x76, 0x75, 0xd3, 0x31, 0xae, 0x6e, 0x8c, 0x24, 0x67, 0x62, 0x25, + 0x49, 0x7d, 0x74, 0xa1, 0x84, 0x19, 0x76, 0x1a, 0x44, 0x5b, 0xbd, 0x0a, 0xf3, 0xbb, 0x38, 0x20, + 0x27, 0xd7, 0x4f, 0xb2, 0xd0, 0x7f, 0x9f, 0xa2, 0x97, 0x11, 0xef, 0xc7, 0xb7, 0x40, 0x9f, 0xfe, + 0x32, 0x1a, 0x21, 0xf0, 0x5c, 0xfb, 0xbc, 0x1b, 0x6b, 0x9f, 0x0b, 0xc2, 0x3e, 0xc7, 0x85, 0x79, + 0xab, 0x30, 0x2b, 0xc2, 0x4c, 0x12, 0xfa, 0xa4, 0x29, 0x4a, 0x06, 0x55, 0xdd, 0x09, 0x8c, 0xf1, + 0xf6, 0xa8, 0x31, 0xbe, 0x3e, 0xcd, 0xa2, 0x4e, 0xd9, 0xe2, 0xab, 0x50, 0x7a, 0x1b, 0x1b, 0x76, + 0xb0, 0x27, 0x04, 0xbc, 0x08, 0x19, 0x0f, 0x1b, 0x26, 0x9b, 0x31, 0xaf, 0xb1, 0x06, 0x31, 0xd9, + 0xa2, 0x1b, 0xbf, 0x3e, 0x17, 0x60, 0x7e, 0x87, 0xa7, 0x28, 0x85, 0x15, 0xff, 0x49, 0x0a, 0xca, + 0x11, 0x8c, 0xef, 0xc3, 0x06, 0x80, 0x48, 0x65, 0x86, 0xfb, 0x70, 0x29, 0x26, 0xb4, 0x11, 0x03, + 0x45, 0x8e, 0x20, 0x1a, 0x84, 0x7e, 0xa0, 0x40, 0x9e, 0x29, 0x3f, 0x16, 0x07, 0x2a, 0x29, 0x94, + 0x19, 0x67, 0x81, 0x1f, 0x23, 0xb1, 0x91, 0x6f, 0x12, 0xfa, 0xdf, 0xfb, 0xc5, 0xd9, 0x4e, 0x47, + 0xc8, 0x47, 0xb5, 0x03, 0xa5, 0x11, 0xba, 0xf2, 0x46, 0x65, 0xd8, 0x46, 0xbd, 0x29, 0x6f, 0xd4, + 0xdc, 0xfa, 0xd5, 0x98, 0x55, 0x13, 0xb2, 0x82, 0x5f, 0x7e, 0xf8, 0xa5, 0xcd, 0x19, 0xc0, 0xec, + 0x5d, 0xb7, 0xe3, 0x4b, 0x5b, 0xc3, 0x9c, 0x64, 0x45, 0x76, 0x92, 0x97, 0x24, 0x5b, 0x43, 0xa3, + 0x6c, 0xd6, 0x42, 0x37, 0xb8, 0x02, 0xa6, 0xe9, 0xe4, 0x35, 0x79, 0xf2, 0x03, 0xbb, 0x4e, 0xd3, + 0xd7, 0x2c, 0x95, 0x5d, 0x27, 0xfe, 0x26, 0xd3, 0x50, 0xf5, 0x27, 0x59, 0x28, 0xb2, 0x29, 0xf9, + 0xf6, 0x6d, 0xc1, 0x0c, 0xe9, 0xc5, 0x37, 0xee, 0x95, 0x04, 0xb1, 0xcb, 0xc3, 0x48, 0x83, 0x6f, + 0x24, 0x1d, 0x5e, 0xfd, 0xef, 0x0c, 0xa4, 0xef, 0xba, 0x1d, 0xb4, 0x04, 0x29, 0x6e, 0xac, 0xd2, + 0xcd, 0xec, 0xc9, 0x71, 0x2d, 0xd5, 0x6e, 0x69, 0x29, 0xcb, 0x3c, 0xdb, 0x69, 0x19, 0x09, 0x4a, + 0x66, 0x46, 0x83, 0x12, 0xe4, 0xc2, 0xdc, 0x48, 0xe6, 0x87, 0x05, 0xc1, 0xa5, 0xe6, 0xdb, 0x27, + 0xc7, 0xb5, 0x92, 0x9c, 0xfa, 0x99, 0xfc, 0xda, 0xf0, 0x0f, 0x6c, 0xf2, 0xd7, 0x31, 0x7c, 0x5c, + 0x6f, 0xb7, 0xb4, 0x92, 0x9c, 0x32, 0xf2, 0xa5, 0x7d, 0xc8, 0x8e, 0xec, 0xc3, 0x6d, 0xc8, 0xb1, + 0xf4, 0x99, 0x49, 0x93, 0x16, 0xc9, 0x3e, 0xda, 0x0c, 0xf5, 0xcf, 0xc4, 0x00, 0x32, 0xd6, 0x0f, + 0x0c, 0x8f, 0x8c, 0xcd, 0x4f, 0x3a, 0x96, 0x0f, 0x40, 0x77, 0x20, 0xff, 0xd4, 0x72, 0x2c, 0x7f, + 0x0f, 0x9b, 0x95, 0xc2, 0x84, 0x83, 0xc3, 0x11, 0x64, 0x74, 0xdf, 0x35, 0xad, 0xa7, 0x16, 0x36, + 0x2b, 0x30, 0xe9, 0x68, 0x31, 0x82, 0x38, 0x58, 0x4f, 0x3d, 0x83, 0xe6, 0x73, 0xf4, 0xae, 0xdb, + 0x1f, 0xd8, 0x98, 0x2c, 0x61, 0x76, 0x55, 0x59, 0x4b, 0x69, 0x0b, 0x02, 0xb3, 0x29, 0x10, 0x44, + 0xb1, 0x69, 0x28, 0x5f, 0x29, 0x32, 0x0b, 0x4a, 0x1b, 0xe8, 0x01, 0x9c, 0xdf, 0xb3, 0x7a, 0x7b, + 0xcf, 0x0c, 0xe2, 0x02, 0x46, 0xe1, 0x56, 0x69, 0x42, 0x6e, 0x50, 0x38, 0x38, 0xc4, 0x10, 0x2f, + 0x31, 0x22, 0x69, 0xe2, 0xae, 0xd5, 0x37, 0xec, 0xca, 0x1c, 0x9d, 0xb4, 0x1c, 0x22, 0x5a, 0x0c, + 0x8e, 0xae, 0xc2, 0x9c, 0x37, 0x74, 0xc8, 0x45, 0x24, 0x1c, 0xaa, 0x79, 0xda, 0xb3, 0xc4, 0xa1, + 0xfc, 0x12, 0xbe, 0x0c, 0x85, 0x28, 0x5f, 0x5a, 0x66, 0x61, 0x5c, 0x08, 0x50, 0x11, 0x94, 0x77, + 0xdc, 0x2e, 0xab, 0x9f, 0x08, 0x3b, 0xf9, 0x7f, 0x0a, 0x2c, 0x48, 0xc0, 0x30, 0x1b, 0x51, 0xb0, + 0x05, 0x70, 0x82, 0x64, 0xf5, 0x29, 0x02, 0x21, 0x44, 0x5c, 0x55, 0x21, 0xb5, 0xea, 0x1f, 0x2b, + 0x90, 0x17, 0x58, 0xf4, 0x12, 0x14, 0x09, 0xc6, 0xb6, 0x82, 0x23, 0x3d, 0xba, 0x59, 0x66, 0x05, + 0xec, 0x1d, 0x7c, 0x44, 0x56, 0x1e, 0x76, 0x89, 0x2c, 0x58, 0x41, 0x2b, 0x09, 0x28, 0xbb, 0xf8, + 0xaa, 0x90, 0xb7, 0x8d, 0xc0, 0x0a, 0x86, 0x26, 0xb3, 0x32, 0x8a, 0x16, 0xb6, 0x89, 0x54, 0x6c, + 0xd7, 0xe9, 0x31, 0xe4, 0x0c, 0x45, 0x46, 0x00, 0xb5, 0x09, 0xf3, 0x9a, 0xe1, 0xf4, 0xf0, 0x8e, + 0xdb, 0x13, 0xc6, 0x6d, 0x19, 0xf2, 0x2c, 0xc5, 0x26, 0xec, 0x83, 0x96, 0xa3, 0x6d, 0x39, 0x39, + 0x90, 0x92, 0xec, 0x9e, 0xfa, 0xaf, 0x69, 0x28, 0x47, 0x44, 0xb8, 0x10, 0xdf, 0x0d, 0x23, 0x7e, + 0x76, 0x4f, 0x24, 0x5d, 0x8e, 0xe3, 0x83, 0x63, 0x63, 0xfe, 0x7f, 0x52, 0x00, 0xde, 0xf3, 0x70, + 0x10, 0x1c, 0xb5, 0x49, 0x88, 0xfc, 0x12, 0x14, 0x79, 0xc0, 0xa5, 0x93, 0xd3, 0x2f, 0x84, 0xc7, + 0x61, 0xc4, 0xac, 0x90, 0x85, 0x38, 0xf8, 0x19, 0x43, 0x33, 0xb1, 0xe5, 0x1c, 0xfc, 0x8c, 0xa2, + 0xae, 0x40, 0xc9, 0x30, 0x4d, 0x6c, 0xea, 0xdc, 0x57, 0xe2, 0x36, 0xad, 0x48, 0x81, 0x1a, 0x83, + 0xa1, 0x97, 0x61, 0xde, 0xc3, 0x7d, 0xf7, 0x50, 0xea, 0xc6, 0x6c, 0xdb, 0x1c, 0x07, 0x8b, 0x8e, + 0x4b, 0x90, 0xf5, 0xb0, 0xe1, 0x87, 0x39, 0x54, 0xde, 0x42, 0x15, 0xc8, 0x99, 0xac, 0x2c, 0xc0, + 0x2d, 0x91, 0x68, 0x56, 0xff, 0x5c, 0x11, 0x09, 0x8c, 0x3b, 0x90, 0xa1, 0x0b, 0xe4, 0xc9, 0x8b, + 0xd5, 0x98, 0xab, 0x49, 0x88, 0x47, 0x96, 0x0a, 0x1b, 0x84, 0x3e, 0x80, 0xd9, 0x01, 0x95, 0x89, + 0x4e, 0x73, 0x08, 0xcc, 0x0f, 0xb9, 0x35, 0x8d, 0xa8, 0x23, 0x91, 0x8a, 0xfb, 0x7e, 0x10, 0x42, + 0xee, 0xce, 0xe4, 0x95, 0x72, 0x4a, 0x5d, 0x83, 0xf2, 0x83, 0x21, 0xf6, 0x8e, 0xde, 0xb3, 0x0d, + 0x47, 0xba, 0x01, 0x0f, 0x08, 0x4c, 0xb8, 0x5a, 0xb4, 0xa1, 0x0e, 0x60, 0x41, 0xea, 0xc9, 0x35, + 0xe1, 0x1b, 0x70, 0xc9, 0xb4, 0xfc, 0xc0, 0x3f, 0xb0, 0xf5, 0xc1, 0xde, 0x91, 0x6f, 0x75, 0x0d, + 0x5b, 0xa7, 0xdd, 0xf5, 0x81, 0x6d, 0x38, 0x3c, 0x9e, 0xbc, 0x7c, 0x72, 0x5c, 0xab, 0xb4, 0x2c, + 0x3f, 0xd8, 0x7d, 0xb0, 0xf3, 0x1e, 0xef, 0x15, 0x91, 0xaa, 0x70, 0x02, 0xa7, 0x30, 0xea, 0x32, + 0xab, 0x31, 0x92, 0x91, 0x9e, 0xd5, 0x19, 0x06, 0x51, 0x88, 0xa2, 0xfe, 0x15, 0x40, 0xe5, 0x34, + 0x8e, 0x33, 0x35, 0x80, 0x92, 0x48, 0x40, 0x33, 0xd1, 0xb1, 0x73, 0xbe, 0xf5, 0x9c, 0x3a, 0x64, + 0x1c, 0xad, 0xb0, 0x40, 0x49, 0x44, 0x26, 0xbb, 0xa8, 0x45, 0x53, 0x42, 0xa0, 0x3e, 0x14, 0xa5, + 0xfa, 0x8a, 0xa8, 0x2b, 0xb5, 0xce, 0x32, 0x61, 0x54, 0x73, 0x19, 0x71, 0x89, 0x67, 0xa3, 0x9a, + 0x8b, 0x5f, 0xfd, 0x91, 0x02, 0x10, 0xf5, 0x23, 0x2a, 0xca, 0xb2, 0x54, 0x7c, 0xc3, 0x78, 0x0b, + 0xdd, 0x86, 0x2c, 0xaf, 0x1a, 0xa5, 0x26, 0xae, 0x1a, 0xf1, 0x11, 0x34, 0x2f, 0xc0, 0x8a, 0x45, + 0xfe, 0x81, 0xcd, 0x8e, 0x06, 0xcf, 0x0b, 0x50, 0xe8, 0xee, 0x83, 0x1d, 0xad, 0xc0, 0x3a, 0xec, + 0x1e, 0xd8, 0x77, 0x67, 0xf2, 0xe9, 0xf2, 0x4c, 0xf5, 0x0f, 0xd2, 0x50, 0xa0, 0xa9, 0x55, 0x2a, + 0x93, 0x4f, 0x15, 0xa8, 0x8c, 0x84, 0x30, 0x7a, 0xe7, 0x48, 0x8f, 0x02, 0x29, 0x22, 0xa0, 0xf7, + 0xcf, 0x22, 0xa0, 0x70, 0x86, 0xba, 0x26, 0xc5, 0x42, 0xcd, 0x23, 0xea, 0x3b, 0x9a, 0x4c, 0x66, + 0x6f, 0x9c, 0xd5, 0xf3, 0x5c, 0xf4, 0x62, 0x68, 0xa2, 0x2f, 0xc2, 0x9c, 0x5c, 0x3e, 0x0b, 0x33, + 0xab, 0xc5, 0x68, 0x53, 0xda, 0x26, 0xfa, 0x1a, 0x80, 0xe9, 0xb9, 0x83, 0x01, 0x36, 0x75, 0x83, + 0xc5, 0x6b, 0x93, 0x5c, 0xa0, 0x05, 0x3e, 0x66, 0x23, 0xa8, 0x6e, 0xc3, 0xf2, 0xe7, 0x2e, 0x29, + 0xc6, 0xf1, 0x1d, 0xc9, 0xc4, 0xa5, 0x25, 0x8f, 0xb6, 0xfa, 0xdd, 0x14, 0x14, 0x65, 0xc5, 0x45, + 0x01, 0xb0, 0x3a, 0xb9, 0x7c, 0x1c, 0xde, 0x7d, 0xd1, 0xe3, 0x10, 0xed, 0xc4, 0x48, 0xec, 0x16, + 0x08, 0x68, 0xf5, 0x63, 0x98, 0x1b, 0xed, 0x12, 0x13, 0x66, 0xed, 0x8e, 0x86, 0x59, 0x6f, 0xbd, + 0x90, 0x46, 0x8c, 0xc8, 0x40, 0x89, 0xde, 0x42, 0x24, 0x31, 0xf0, 0x78, 0x94, 0x81, 0xdf, 0x7a, + 0x51, 0xa9, 0xc8, 0x3c, 0x7c, 0x07, 0xca, 0xe3, 0xc7, 0x39, 0x86, 0x83, 0x87, 0xa3, 0x1c, 0xfc, + 0xe6, 0x8b, 0x59, 0x0d, 0x69, 0x7e, 0x9e, 0xcc, 0xba, 0x08, 0x17, 0xee, 0xd1, 0xd7, 0x28, 0xf7, + 0x70, 0x60, 0x98, 0x51, 0x1e, 0x56, 0xfd, 0x17, 0x05, 0x96, 0xc6, 0x31, 0xdc, 0x84, 0x1a, 0x90, + 0xef, 0x73, 0x18, 0x57, 0x97, 0xaf, 0x25, 0xb0, 0x15, 0x4f, 0xa4, 0x2e, 0x00, 0xb2, 0x7a, 0x84, + 0x64, 0xab, 0xbf, 0x0d, 0xa5, 0x91, 0x0e, 0x31, 0x92, 0x79, 0x7d, 0x54, 0x32, 0x72, 0x74, 0x35, + 0x0c, 0x2c, 0xbb, 0xce, 0x1f, 0xd9, 0x84, 0x13, 0x4b, 0x41, 0xdd, 0x0f, 0x53, 0x70, 0x7e, 0xcb, + 0x39, 0x18, 0xe2, 0x21, 0xa6, 0xd7, 0xa2, 0xb8, 0xdb, 0x7e, 0x95, 0xc9, 0x1c, 0x76, 0x6f, 0x86, + 0x7e, 0x1c, 0x6b, 0xa0, 0x6f, 0x4a, 0x2e, 0x17, 0xad, 0xbb, 0x34, 0x37, 0x4f, 0x8e, 0x6b, 0x39, + 0xca, 0x15, 0x9d, 0xf3, 0xe6, 0x54, 0x73, 0xf2, 0x71, 0x91, 0xdf, 0x76, 0x0d, 0x16, 0xfc, 0x7d, + 0x6b, 0xa0, 0xfb, 0x7b, 0xee, 0xd0, 0x36, 0x75, 0xc6, 0x01, 0x4f, 0x03, 0x11, 0xc4, 0x2e, 0x85, + 0x3f, 0x20, 0x60, 0xf5, 0x2f, 0x52, 0xb0, 0x38, 0x2a, 0x15, 0xbe, 0xdf, 0x0f, 0x22, 0x6f, 0x86, + 0x6d, 0xf7, 0x1b, 0x49, 0x45, 0x9c, 0x18, 0x0a, 0x75, 0xf1, 0x46, 0x22, 0x74, 0x83, 0xfe, 0x41, + 0x81, 0x1c, 0x07, 0xfe, 0x4a, 0xa5, 0xfe, 0xd6, 0x98, 0x33, 0x7a, 0x35, 0xa9, 0xf0, 0xe7, 0x19, + 0x5d, 0x4c, 0x5d, 0x2d, 0xe1, 0x7a, 0x46, 0x51, 0x51, 0x5a, 0x8a, 0x8a, 0xd4, 0x0b, 0x70, 0x7e, + 0x73, 0xcf, 0xf0, 0x82, 0x4d, 0xf6, 0xc2, 0x4b, 0x9c, 0x98, 0x27, 0xb0, 0x38, 0x0a, 0xe6, 0xe2, + 0x6b, 0x42, 0x8e, 0xbf, 0x05, 0xe3, 0xe2, 0x53, 0x25, 0x26, 0x02, 0xbf, 0x2e, 0x1e, 0x8a, 0xd1, + 0xc1, 0xbb, 0xac, 0x04, 0x1f, 0x3e, 0x4b, 0x61, 0xb8, 0x6b, 0xef, 0xc0, 0x52, 0xfc, 0x7b, 0x0b, + 0x34, 0x0b, 0xb9, 0x47, 0xf7, 0xdf, 0xb9, 0xff, 0xee, 0xfb, 0xf7, 0xcb, 0xe7, 0x48, 0x63, 0x73, + 0xe7, 0xd1, 0xee, 0xc3, 0x2d, 0xad, 0xac, 0xa0, 0x22, 0xe4, 0x5b, 0x1b, 0x0f, 0x37, 0x9a, 0x1b, + 0xbb, 0x5b, 0xe5, 0x14, 0x2a, 0x40, 0xe6, 0xe1, 0x46, 0x73, 0x67, 0xab, 0x9c, 0x5e, 0xff, 0x41, + 0x15, 0x32, 0x1b, 0x66, 0xdf, 0x72, 0x50, 0x00, 0x19, 0x5a, 0x7d, 0x44, 0x2f, 0x3f, 0xbf, 0x3e, + 0x49, 0x17, 0x59, 0x5d, 0x9b, 0xb4, 0x90, 0xa9, 0x56, 0xbe, 0xf7, 0xcf, 0xff, 0xf1, 0xc3, 0x14, + 0x42, 0xe5, 0x86, 0x4e, 0xdf, 0xfb, 0x35, 0x0e, 0xaf, 0x37, 0x68, 0x41, 0x13, 0xfd, 0xbe, 0x02, + 0x85, 0xf0, 0x25, 0x1a, 0x7a, 0x65, 0x82, 0x17, 0x60, 0xe1, 0xf4, 0xaf, 0x4e, 0xd6, 0x99, 0xb3, + 0x70, 0x99, 0xb2, 0xb0, 0x84, 0x16, 0x25, 0x16, 0xc2, 0xc7, 0x6d, 0xe8, 0x4f, 0x15, 0x98, 0x1f, + 0x7b, 0x62, 0x86, 0xae, 0x4f, 0xf3, 0x1c, 0x8d, 0xb1, 0xb4, 0x3e, 0xfd, 0x0b, 0x36, 0xf5, 0x65, + 0xca, 0xd8, 0x4b, 0xa8, 0x16, 0xc7, 0x58, 0xe3, 0x63, 0xf1, 0xf9, 0x6d, 0xf4, 0x97, 0x0a, 0x14, + 0xe5, 0x57, 0x46, 0xa8, 0x3e, 0xf1, 0x73, 0x24, 0xc6, 0x5d, 0x63, 0xca, 0xe7, 0x4b, 0xea, 0x2d, + 0xca, 0xda, 0x6b, 0xa8, 0xfe, 0x1c, 0xd6, 0x1a, 0xf4, 0x2a, 0xf7, 0x1b, 0x1f, 0xd3, 0x5f, 0xca, + 0x29, 0x44, 0xa5, 0x73, 0xf4, 0xea, 0x84, 0x15, 0x76, 0xc6, 0xe5, 0x74, 0xf5, 0x78, 0xf5, 0x0e, + 0xe5, 0xf1, 0x16, 0xba, 0x39, 0x1d, 0x8f, 0x0d, 0xf6, 0x34, 0xe3, 0x47, 0x0a, 0x94, 0x46, 0x5e, + 0x23, 0xa0, 0x24, 0x21, 0xc5, 0x3d, 0x68, 0xa8, 0xbe, 0x36, 0xf9, 0x00, 0xce, 0xf2, 0x2a, 0x65, + 0xb9, 0x8a, 0x2a, 0x12, 0xcb, 0x8e, 0xeb, 0x30, 0x06, 0x29, 0x13, 0x1f, 0x42, 0x96, 0x15, 0xc0, + 0xd1, 0xda, 0x04, 0x35, 0x72, 0xc6, 0xc7, 0x97, 0x27, 0xae, 0xa6, 0xab, 0xcb, 0x94, 0x81, 0xf3, + 0x68, 0x41, 0x62, 0x80, 0x5b, 0x39, 0x72, 0x1e, 0xc3, 0xe2, 0x6c, 0xe2, 0x79, 0x1c, 0x2f, 0x17, + 0x27, 0x9e, 0xc7, 0xd3, 0xf5, 0x5e, 0x7e, 0x1e, 0x55, 0x99, 0x87, 0xa1, 0x45, 0xb6, 0xeb, 0xb6, + 0x72, 0x0d, 0x7d, 0x57, 0x81, 0xc2, 0xf6, 0x44, 0x6c, 0x6c, 0x4f, 0xc3, 0xc6, 0xa9, 0x6a, 0x68, + 0xac, 0x28, 0x18, 0x1b, 0xe8, 0x77, 0x21, 0xc7, 0x6b, 0x9b, 0x28, 0x49, 0xb6, 0xa3, 0xb5, 0xd3, + 0xea, 0xb5, 0x49, 0xba, 0xf2, 0xc9, 0xab, 0x74, 0xf2, 0x45, 0x84, 0xa4, 0xc9, 0x79, 0x0d, 0x15, + 0xfd, 0x9e, 0x02, 0x79, 0x51, 0x2f, 0x40, 0xd7, 0x26, 0x2a, 0x2a, 0x30, 0x06, 0x5e, 0x99, 0xa2, + 0x00, 0xa1, 0x5e, 0xa2, 0x1c, 0x5c, 0x40, 0xe7, 0x25, 0x0e, 0x7c, 0x31, 0xeb, 0xf7, 0x15, 0xc8, + 0xb2, 0x32, 0x43, 0xa2, 0x1a, 0x8e, 0x14, 0x2c, 0x12, 0xd5, 0x70, 0xac, 0x66, 0xf1, 0x45, 0x3a, + 0xf9, 0xca, 0x88, 0xec, 0xf7, 0x68, 0x97, 0x27, 0x05, 0x94, 0xe3, 0x9f, 0x54, 0x10, 0x22, 0x27, + 0x9f, 0x28, 0x88, 0xb1, 0xfa, 0x47, 0xa2, 0x20, 0xc6, 0x8b, 0x12, 0xb1, 0x82, 0x10, 0x35, 0x0f, + 0x34, 0x80, 0x99, 0xbb, 0x6e, 0xc7, 0x47, 0x5f, 0x7a, 0x6e, 0xbe, 0x9d, 0xcd, 0xfc, 0xf2, 0x84, + 0x79, 0x79, 0xf5, 0x22, 0x9d, 0x75, 0x01, 0xcd, 0x4b, 0xb3, 0x7e, 0x8b, 0xcc, 0x44, 0x8e, 0x61, + 0x98, 0x52, 0x4c, 0xd4, 0xff, 0xf1, 0x74, 0x66, 0xa2, 0xfe, 0x9f, 0xca, 0x52, 0xc6, 0x5e, 0x8b, + 0x61, 0xa6, 0x92, 0xb2, 0x11, 0xa6, 0x59, 0x12, 0xd9, 0x18, 0xcf, 0x0d, 0x25, 0xb2, 0x71, 0x2a, + 0x3d, 0x14, 0xcb, 0x06, 0x4d, 0x0f, 0x0d, 0xc8, 0xc4, 0xdf, 0x84, 0x0c, 0xad, 0x83, 0x27, 0xba, + 0x26, 0xf2, 0x93, 0x80, 0x44, 0xd7, 0x64, 0xa4, 0x60, 0xaf, 0x9e, 0x7b, 0x4d, 0x41, 0xcf, 0xa0, + 0x28, 0x97, 0x7d, 0x13, 0x2f, 0xd6, 0x98, 0x3a, 0x78, 0xf5, 0xf5, 0x33, 0xd5, 0x93, 0xd5, 0x73, + 0xe4, 0x84, 0xa1, 0xd3, 0x1d, 0xd0, 0xcd, 0x29, 0xe9, 0xbd, 0x20, 0x17, 0x7f, 0xa6, 0x40, 0x5e, + 0xa4, 0x04, 0x13, 0x4f, 0xd8, 0x58, 0x92, 0x38, 0xf1, 0x84, 0x8d, 0xe7, 0x18, 0xd5, 0xb7, 0xe8, + 0x16, 0xbf, 0x31, 0x72, 0xc2, 0x68, 0x68, 0x62, 0xbb, 0xbd, 0x27, 0xab, 0x68, 0x25, 0x06, 0xdc, + 0xf8, 0x58, 0x04, 0x44, 0xdf, 0x26, 0x1e, 0x5a, 0x79, 0x3c, 0xaa, 0x45, 0xeb, 0x53, 0x85, 0xc0, + 0x8c, 0xe9, 0x1b, 0x67, 0x08, 0x9b, 0x43, 0x53, 0x75, 0x79, 0xcc, 0xcb, 0xd0, 0x4d, 0x99, 0x9d, + 0x1f, 0x2b, 0xb0, 0xb0, 0x61, 0xdb, 0xa3, 0x51, 0x2e, 0x7a, 0x6d, 0x8a, 0x80, 0x98, 0xb1, 0x78, + 0x7d, 0xea, 0x10, 0x5a, 0x7d, 0x89, 0x32, 0x78, 0x09, 0x2d, 0x4b, 0x0c, 0xb2, 0xa8, 0x57, 0xc4, + 0xd1, 0xe8, 0x13, 0x05, 0x8a, 0x72, 0x50, 0x92, 0xa8, 0xe6, 0x31, 0x41, 0x4d, 0xa2, 0xff, 0x18, + 0x17, 0xed, 0xa8, 0x35, 0xca, 0xd4, 0x32, 0xba, 0x28, 0xdf, 0x6f, 0xa4, 0x23, 0x0f, 0x65, 0x88, + 0xfb, 0x55, 0x94, 0x83, 0xc4, 0x44, 0x96, 0x62, 0xa2, 0xf4, 0x44, 0x96, 0xe2, 0xa2, 0x4f, 0xf5, + 0x0a, 0x65, 0xe9, 0x0b, 0xaa, 0xec, 0x7b, 0x61, 0xd6, 0x51, 0xa7, 0xca, 0x76, 0x5b, 0xb9, 0xd6, + 0xbc, 0xf6, 0xe9, 0xbf, 0xaf, 0x9c, 0xfb, 0xf4, 0x64, 0x45, 0xf9, 0xd9, 0xc9, 0x8a, 0xf2, 0xf3, + 0x93, 0x15, 0xe5, 0xdf, 0x4e, 0x56, 0x94, 0x4f, 0x3e, 0x5b, 0x39, 0xf7, 0xb3, 0xcf, 0x56, 0xce, + 0xfd, 0xfc, 0xb3, 0x95, 0x73, 0x4f, 0xf2, 0x62, 0x96, 0x4e, 0x96, 0x26, 0xec, 0x6e, 0xfc, 0x7f, + 0x00, 0x00, 0x00, 0xff, 0xff, 0x78, 0x8e, 0x1f, 0x5f, 0xd9, 0x34, 0x00, 0x00, } diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index 99b643af6ebc..4d872036d041 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -335,34 +335,54 @@ 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; } // DecommissionStatusRequest requests the decommissioning status for the diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 568252e94821..b282dc6f8e64 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -450,6 +450,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) +} + // 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 c9ce9a45415d..788b078ee643 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -123,15 +123,6 @@ func (ds *ServerImpl) Drain(ctx context.Context, flowDrainWait time.Duration) { 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) - } -} - // setDraining changes the node's draining state through gossip to the provided // state. func (ds *ServerImpl) setDraining(drain bool) error { diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index 1ae5e6a1ec4a..a2650a2f5c76 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -33,7 +33,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/securitytest" "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" @@ -238,19 +237,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) }() }() @@ -275,12 +268,8 @@ func TestPGWireDrainClient(t *testing.T) { } } - now, err := s.(*server.TestServer).Undrain(ctx, on) - if err != nil { - t.Fatal(err) - } - if 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/storage/node_liveness.go b/pkg/storage/node_liveness.go index 8a95a6109abb..7b53d43e3659 100644 --- a/pkg/storage/node_liveness.go +++ b/pkg/storage/node_liveness.go @@ -233,9 +233,14 @@ func (nl *NodeLiveness) SetDraining(ctx context.Context, drain bool) { 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) + if err != nil { + if log.V(1) { + log.Infof(ctx, "attempting to set liveness draining status to %v: %v", drain, err) + } + continue } + return } } @@ -349,6 +354,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 } @@ -847,6 +855,9 @@ func (nl *NodeLiveness) updateLivenessAttempt( // put failures. if !update.ignoreCache { l, err := nl.GetLiveness(update.NodeID) + if err != nil && err != ErrNoLivenessRecord { + return err + } if err == nil && (oldLiveness == nil || *l != *oldLiveness) { return handleCondFailed(*l) } diff --git a/pkg/storage/node_liveness_test.go b/pkg/storage/node_liveness_test.go index cd07c459142f..8602323eda81 100644 --- a/pkg/storage/node_liveness_test.go +++ b/pkg/storage/node_liveness_test.go @@ -643,7 +643,7 @@ func TestNodeLivenessSetDraining(t *testing.T) { t.Fatal(err) } - mtc.nodeLivenesses[drainingNodeIdx].SetDraining(ctx, true) + mtc.nodeLivenesses[drainingNodeIdx].SetDraining(ctx, true /* drain */) // Draining node disappears from store lists. { From b9f838564122948d268b845ea10334c161fe5c24 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 30 Mar 2020 21:38:40 +0200 Subject: [PATCH 08/12] server, *: provide details about the progress of a drain Prior to this patch, it was impossible for the client of the Drain RPC to know how much was left to do after a Drain completes. In particular, it was impossible to determine whether a timeout was encountered and some load could not be shed. To improve upon this situation, this patch introduces a new field "remaining indicator" in DrainResponse. This value remains non-zero whenever the Drain has work to do or has performed work; its magnitude indicates "how much work" is being performed during the drain and is specified to decrease in successive drain invocations. When it reaches 0, the drain is known to have completed fully. Example output: Before: ``` $ cockroach quit ok ``` After: ``` $ cockroach quit node is draining; remaining: 12 node is draining; remaining: 0 (complete) ok ``` Or alternatively: ``` $ ./cockroach node drain node is draining; remaining: 12 node is draining; remaining: 0 (complete) ok ``` As shown in these examples, `node drain` and `quit` now iterate through drain requests until the remaining indicator reaches zero. The total number of iterations remains controlled by `--drain-wait` in aggregate (0 = no limit). Details are also provided in the log, which can be enabled with an opt-in explicit `--logtostderr=INFO` or some of the other log flags. For example: ``` $ ./cockroach quit --logtostderr=INFO I200331 23:02:05.146857 84 cli/start.go:1434 drain details: SQL clients: 1, table leases: 1, liveness record: 1, range leases: 24 node is shutting down; remaining: 27 node is shutting down; remaining: 0 (complete) I200331 23:02:05.148888 1 util/stop/stopper.go:539 quiescing ok ``` The unit test for this ascertains that: - a first drain request on a test cluster returns a non-zero progress indicator. (There's always at least one unit of work - in the typical case that's the liveness record.) - a second drain request returns a zero progress indicator. (Because under normal circumstances, the timeouts are never reached for an empty test clusters so the graceful drain should always be able to complete in the first request.) Release note (cli change): The commands `cockroach quit` and `cockroach node drain` now reports a "work remaining" metric on their standard error stream. The value reduces until it reaches 0 to indicate that the graceful shutdown has completed server-side. An operator can now rely on `cockroach node drain` to obtain confidence of a graceful shutdown prior to terminating the server process. Release note (backward-incompatible change): `cockroach quit` now prints out progress details on its standard error stream, even when `--logtostderr` is not specified. Previously, nothing was printed on standard error. Scripts who wish to ignore this output can redirect the stderr stream. --- .../test_missing_log_output.tcl | 2 +- pkg/cli/node.go | 2 +- pkg/cli/quit.go | 104 ++- pkg/cli/start.go | 22 +- pkg/server/drain.go | 72 +- pkg/server/drain_test.go | 179 ++-- pkg/server/node.go | 8 +- pkg/server/serverpb/admin.pb.go | 765 ++++++++++-------- pkg/server/serverpb/admin.proto | 47 ++ pkg/server/testserver.go | 2 +- pkg/sql/distsql/server.go | 6 +- pkg/sql/distsql/sync_flow_after_drain_test.go | 3 +- pkg/sql/distsql_physical_planner_test.go | 2 +- pkg/sql/flowinfra/flow_registry.go | 15 +- pkg/sql/flowinfra/flow_registry_test.go | 10 +- pkg/sql/lease.go | 11 +- pkg/sql/lease_test.go | 6 +- pkg/sql/pgwire/helpers_test.go | 2 +- pkg/sql/pgwire/server.go | 29 +- pkg/storage/client_raft_test.go | 2 +- pkg/storage/client_replica_test.go | 8 +- pkg/storage/helpers_test.go | 2 +- pkg/storage/node_liveness.go | 15 +- pkg/storage/node_liveness_test.go | 2 +- pkg/storage/replica_test.go | 4 +- pkg/storage/store.go | 28 +- 26 files changed, 872 insertions(+), 476 deletions(-) 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 d38eddbc0f2f..4c12fe0256db 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -466,7 +466,7 @@ func runDrain(cmd *cobra.Command, args []string) (err error) { } defer finish() - _, err = doDrain(ctx, c) + _, _, err = doDrain(ctx, c) return err } diff --git a/pkg/cli/quit.go b/pkg/cli/quit.go index 8cd8e6ea303f..890508c340f9 100644 --- a/pkg/cli/quit.go +++ b/pkg/cli/quit.go @@ -82,11 +82,15 @@ func runQuit(cmd *cobra.Command, args []string) (err error) { // 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, err := doDrain(ctx, c) + 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") } @@ -107,7 +111,9 @@ func drainAndShutdown(ctx context.Context, c serverpb.AdminClient) (err error) { // 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 bool, err error) { +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 { @@ -115,18 +121,19 @@ func doDrain(ctx context.Context, c serverpb.AdminClient) (hardError bool, err e } err = contextutil.RunWithTimeout(ctx, "drain", quitCtx.drainWait, func(ctx context.Context) (err error) { - hardError, err = doDrainNoTimeout(ctx, c) + 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 hardError, err + return } -func doDrainNoTimeout(ctx context.Context, c serverpb.AdminClient) (hardError bool, err error) { +func doDrainNoTimeout( + ctx context.Context, c serverpb.AdminClient, +) (hardError, remainingWork bool, err error) { defer func() { if server.IsWaitingForInit(err) { log.Infof(ctx, "%v", err) @@ -134,30 +141,75 @@ func doDrainNoTimeout(ctx context.Context, c serverpb.AdminClient) (hardError bo } }() - // 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 { - return true, errors.Wrap(err, "error sending drain request") - } + remainingWork = true for { - _, err := stream.Recv() - if err == io.EOF { - // Done. - break - } + // 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 { - // Unexpected error. - log.Infof(ctx, "graceful shutdown failed: %v", err) - return false, err + 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 } - // Iterate until end of stream, which indicates the drain is - // complete. + // 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, nil + return false, remainingWork, nil } // doShutdown attempts to trigger a server shutdown *without* diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 2b22f26f8f8c..7e70209becf0 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -896,9 +896,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); 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) }() diff --git a/pkg/server/drain.go b/pkg/server/drain.go index d2b8faf315bd..9359cadb889d 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -12,13 +12,16 @@ 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" @@ -70,13 +73,16 @@ func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_Dr log.Infof(ctx, "drain request received with doDrain = %v, shutdown = %v", doDrain, req.Shutdown) + res := serverpb.DrainResponse{} if doDrain { - if err := s.server.Drain(ctx); err != nil { + 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 } - res := serverpb.DrainResponse{} if s.server.isDraining() { res.DeprecatedDrainStatus = DeprecatedDrainParameter res.IsDraining = true @@ -138,17 +144,53 @@ func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_Dr // 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) error { +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); err != nil { + 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) + return s.drainNode(ctx, reporter) } // isDraining returns true if either clients are being drained @@ -158,7 +200,7 @@ func (s *Server) isDraining() bool { } // drainClients starts draining the SQL layer. -func (s *Server) drainClients(ctx context.Context) error { +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) @@ -170,15 +212,15 @@ func (s *Server) drainClients(ctx context.Context) error { // 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(true /* drain */) + defer s.leaseMgr.SetDraining(true /* drain */, reporter) // Disable incoming SQL clients up to the queryWait timeout. drainMaxWait := queryWait.Get(&s.st.SV) - if err := s.pgServer.Drain(drainMaxWait); err != nil { + 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) + s.distSQLServer.Drain(ctx, drainMaxWait, reporter) // Done. This executes the defers set above to drain SQL leases. return nil @@ -186,13 +228,7 @@ func (s *Server) drainClients(ctx context.Context) error { // drainNode initiates the draining mode for the node, which // starts draining range leases. -func (s *Server) drainNode(ctx context.Context) error { - s.nodeLiveness.SetDraining(ctx, true /* drain */) - return s.node.SetDraining(true /* drain */) -} - -// stopDrain should be called prior to successive invocations of -// drainNode(), otherwise the drain call would deadlock. -func (s *Server) stopDrain(ctx context.Context) error { - return s.node.SetDraining(false /* drain */) +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 index 7de01b5d56e3..c16827cf2cc7 100644 --- a/pkg/server/drain_test.go +++ b/pkg/server/drain_test.go @@ -49,72 +49,45 @@ func TestDrainLegacy(t *testing.T) { // 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(t *testing.T, newInterface bool) { - 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, - }, - }) - defer tc.Stopper().Stop(context.TODO()) - - ctx := context.TODO() - - // We'll have the RPC talk to the first node. - c, finish, err := getAdminClientForServer(ctx, tc, 0 /* serverIdx */) - if err != nil { - t.Fatal(err) - } - defer finish() +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. - checkDrainProbe(ctx, t, c, false /* expectedDrainStatus */) + resp := t.sendProbe() + t.assertDraining(resp, false) + t.assertRemaining(resp, false) // Issue a drain without shutdown, so we can probe more afterwards. - req := &serverpb.DrainRequest{Shutdown: false} - if newInterface { - req.DoDrain = true - } else { - req.DeprecatedProbeIndicator = server.DeprecatedDrainParameter - } - drainStream, err := c.Drain(ctx, req) - if err != nil { - t.Fatal(err) - } - resp, err := getDrainResponse(t, drainStream) - if err != nil { - t.Fatal(err) - } - checkDrainStatus(t, resp, true /* expectedDrainStatus */) + 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. - checkDrainProbe(ctx, t, c, true /* expectedDrainStatus */) + 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. - req = &serverpb.DrainRequest{Shutdown: true} - drainStream, err = c.Drain(ctx, req) - if err != nil { - t.Fatal(err) - } - resp, err = getDrainResponse(t, 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) - } + resp = t.sendShutdown() if resp != nil { - checkDrainStatus(t, resp, true /* expectedDrainStatus */) + t.assertDraining(resp, true) + t.assertRemaining(resp, false) } // Now expect the server to be shut down. testutils.SucceedsSoon(t, func() error { - _, err := c.Drain(ctx, &serverpb.DrainRequest{Shutdown: false}) + _, err := t.c.Drain(context.TODO(), &serverpb.DrainRequest{Shutdown: false}) if grpcutil.IsClosedConnection(err) { return nil } @@ -122,35 +95,101 @@ func doTestDrain(t *testing.T, newInterface bool) { }) } -// checkDrainProbe issues a drain probe and asserts that the -// server is alive. It also asserts that its drain status -// is the one expected. -func checkDrainProbe( - ctx context.Context, t *testing.T, c serverpb.AdminClient, expectedDrainStatus bool, -) { - // Issue a simple drain probe. This should always succeed, - // and report the server is not currently draining. - req := &serverpb.DrainRequest{Shutdown: false} - drainStream, err := c.Drain(ctx, req) +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 := getDrainResponse(t, drainStream) + resp, err := t.getDrainResponse(drainStream) if err != nil { t.Fatal(err) } - checkDrainStatus(t, resp, expectedDrainStatus) + return resp } -func checkDrainStatus(t *testing.T, resp *serverpb.DrainResponse, expectedDrainStatus bool) { - if resp.IsDraining != expectedDrainStatus { - t.Fatalf("expected node drain status to be %v, got %# v", expectedDrainStatus, pretty.Formatter(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 expectedDrainStatus { + if drain { if !reflect.DeepEqual(resp.DeprecatedDrainStatus, server.DeprecatedDrainParameter) { t.Fatalf("expected compat drain status, got %# v", pretty.Formatter(resp)) } @@ -161,8 +200,14 @@ func checkDrainStatus(t *testing.T, resp *serverpb.DrainResponse, expectedDrainS } } -func getDrainResponse( - t *testing.T, stream serverpb.Admin_DrainClient, +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 { diff --git a/pkg/server/node.go b/pkg/server/node.go index 9c277a1dba41..371f9410a3c0 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 *storage.Store) error { - s.SetDraining(drain) + s.SetDraining(drain, reporter) return nil }) } diff --git a/pkg/server/serverpb/admin.pb.go b/pkg/server/serverpb/admin.pb.go index 8780b577daf2..ee609223dd30 100644 --- a/pkg/server/serverpb/admin.pb.go +++ b/pkg/server/serverpb/admin.pb.go @@ -73,7 +73,7 @@ func (x ZoneConfigurationLevel) String() string { return proto.EnumName(ZoneConfigurationLevel_name, int32(x)) } func (ZoneConfigurationLevel) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{0} } // DatabasesRequest requests a list of databases. @@ -84,7 +84,7 @@ func (m *DatabasesRequest) Reset() { *m = DatabasesRequest{} } func (m *DatabasesRequest) String() string { return proto.CompactTextString(m) } func (*DatabasesRequest) ProtoMessage() {} func (*DatabasesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{0} } func (m *DatabasesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -118,7 +118,7 @@ func (m *DatabasesResponse) Reset() { *m = DatabasesResponse{} } func (m *DatabasesResponse) String() string { return proto.CompactTextString(m) } func (*DatabasesResponse) ProtoMessage() {} func (*DatabasesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{1} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{1} } func (m *DatabasesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -154,7 +154,7 @@ func (m *DatabaseDetailsRequest) Reset() { *m = DatabaseDetailsRequest{} func (m *DatabaseDetailsRequest) String() string { return proto.CompactTextString(m) } func (*DatabaseDetailsRequest) ProtoMessage() {} func (*DatabaseDetailsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{2} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{2} } func (m *DatabaseDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -201,7 +201,7 @@ func (m *DatabaseDetailsResponse) Reset() { *m = DatabaseDetailsResponse func (m *DatabaseDetailsResponse) String() string { return proto.CompactTextString(m) } func (*DatabaseDetailsResponse) ProtoMessage() {} func (*DatabaseDetailsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{3} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{3} } func (m *DatabaseDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -237,7 +237,7 @@ func (m *DatabaseDetailsResponse_Grant) Reset() { *m = DatabaseDetailsRe func (m *DatabaseDetailsResponse_Grant) String() string { return proto.CompactTextString(m) } func (*DatabaseDetailsResponse_Grant) ProtoMessage() {} func (*DatabaseDetailsResponse_Grant) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{3, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{3, 0} } func (m *DatabaseDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -274,7 +274,7 @@ func (m *TableDetailsRequest) Reset() { *m = TableDetailsRequest{} } func (m *TableDetailsRequest) String() string { return proto.CompactTextString(m) } func (*TableDetailsRequest) ProtoMessage() {} func (*TableDetailsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{4} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{4} } func (m *TableDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -331,7 +331,7 @@ func (m *TableDetailsResponse) Reset() { *m = TableDetailsResponse{} } func (m *TableDetailsResponse) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse) ProtoMessage() {} func (*TableDetailsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{5} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{5} } func (m *TableDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -368,7 +368,7 @@ func (m *TableDetailsResponse_Grant) Reset() { *m = TableDetailsResponse func (m *TableDetailsResponse_Grant) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Grant) ProtoMessage() {} func (*TableDetailsResponse_Grant) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{5, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{5, 0} } func (m *TableDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -412,7 +412,7 @@ func (m *TableDetailsResponse_Column) Reset() { *m = TableDetailsRespons func (m *TableDetailsResponse_Column) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Column) ProtoMessage() {} func (*TableDetailsResponse_Column) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{5, 1} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{5, 1} } func (m *TableDetailsResponse_Column) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +458,7 @@ func (m *TableDetailsResponse_Index) Reset() { *m = TableDetailsResponse func (m *TableDetailsResponse_Index) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Index) ProtoMessage() {} func (*TableDetailsResponse_Index) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{5, 2} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{5, 2} } func (m *TableDetailsResponse_Index) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -496,7 +496,7 @@ func (m *TableStatsRequest) Reset() { *m = TableStatsRequest{} } func (m *TableStatsRequest) String() string { return proto.CompactTextString(m) } func (*TableStatsRequest) ProtoMessage() {} func (*TableStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{6} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{6} } func (m *TableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -550,7 +550,7 @@ func (m *TableStatsResponse) Reset() { *m = TableStatsResponse{} } func (m *TableStatsResponse) String() string { return proto.CompactTextString(m) } func (*TableStatsResponse) ProtoMessage() {} func (*TableStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{7} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{7} } func (m *TableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -588,7 +588,7 @@ func (m *TableStatsResponse_MissingNode) Reset() { *m = TableStatsRespon func (m *TableStatsResponse_MissingNode) String() string { return proto.CompactTextString(m) } func (*TableStatsResponse_MissingNode) ProtoMessage() {} func (*TableStatsResponse_MissingNode) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{7, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{7, 0} } func (m *TableStatsResponse_MissingNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -622,7 +622,7 @@ func (m *NonTableStatsRequest) Reset() { *m = NonTableStatsRequest{} } func (m *NonTableStatsRequest) String() string { return proto.CompactTextString(m) } func (*NonTableStatsRequest) ProtoMessage() {} func (*NonTableStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{8} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{8} } func (m *NonTableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -661,7 +661,7 @@ func (m *NonTableStatsResponse) Reset() { *m = NonTableStatsResponse{} } func (m *NonTableStatsResponse) String() string { return proto.CompactTextString(m) } func (*NonTableStatsResponse) ProtoMessage() {} func (*NonTableStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{9} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{9} } func (m *NonTableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -694,7 +694,7 @@ func (m *UsersRequest) Reset() { *m = UsersRequest{} } func (m *UsersRequest) String() string { return proto.CompactTextString(m) } func (*UsersRequest) ProtoMessage() {} func (*UsersRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{10} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{10} } func (m *UsersRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -729,7 +729,7 @@ func (m *UsersResponse) Reset() { *m = UsersResponse{} } func (m *UsersResponse) String() string { return proto.CompactTextString(m) } func (*UsersResponse) ProtoMessage() {} func (*UsersResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{11} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{11} } func (m *UsersResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -763,7 +763,7 @@ func (m *UsersResponse_User) Reset() { *m = UsersResponse_User{} } func (m *UsersResponse_User) String() string { return proto.CompactTextString(m) } func (*UsersResponse_User) ProtoMessage() {} func (*UsersResponse_User) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{11, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{11, 0} } func (m *UsersResponse_User) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -810,7 +810,7 @@ func (m *EventsRequest) Reset() { *m = EventsRequest{} } func (m *EventsRequest) String() string { return proto.CompactTextString(m) } func (*EventsRequest) ProtoMessage() {} func (*EventsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{12} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{12} } func (m *EventsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -845,7 +845,7 @@ func (m *EventsResponse) Reset() { *m = EventsResponse{} } func (m *EventsResponse) String() string { return proto.CompactTextString(m) } func (*EventsResponse) ProtoMessage() {} func (*EventsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{13} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{13} } func (m *EventsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -890,7 +890,7 @@ func (m *EventsResponse_Event) Reset() { *m = EventsResponse_Event{} } func (m *EventsResponse_Event) String() string { return proto.CompactTextString(m) } func (*EventsResponse_Event) ProtoMessage() {} func (*EventsResponse_Event) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{13, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{13, 0} } func (m *EventsResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -926,7 +926,7 @@ func (m *SetUIDataRequest) Reset() { *m = SetUIDataRequest{} } func (m *SetUIDataRequest) String() string { return proto.CompactTextString(m) } func (*SetUIDataRequest) ProtoMessage() {} func (*SetUIDataRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{14} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{14} } func (m *SetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -959,7 +959,7 @@ func (m *SetUIDataResponse) Reset() { *m = SetUIDataResponse{} } func (m *SetUIDataResponse) String() string { return proto.CompactTextString(m) } func (*SetUIDataResponse) ProtoMessage() {} func (*SetUIDataResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{15} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{15} } func (m *SetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -994,7 +994,7 @@ func (m *GetUIDataRequest) Reset() { *m = GetUIDataRequest{} } func (m *GetUIDataRequest) String() string { return proto.CompactTextString(m) } func (*GetUIDataRequest) ProtoMessage() {} func (*GetUIDataRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{16} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{16} } func (m *GetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1031,7 +1031,7 @@ func (m *GetUIDataResponse) Reset() { *m = GetUIDataResponse{} } func (m *GetUIDataResponse) String() string { return proto.CompactTextString(m) } func (*GetUIDataResponse) ProtoMessage() {} func (*GetUIDataResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{17} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{17} } func (m *GetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1067,7 +1067,7 @@ func (m *GetUIDataResponse_Value) Reset() { *m = GetUIDataResponse_Value func (m *GetUIDataResponse_Value) String() string { return proto.CompactTextString(m) } func (*GetUIDataResponse_Value) ProtoMessage() {} func (*GetUIDataResponse_Value) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{17, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{17, 0} } func (m *GetUIDataResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1100,7 +1100,7 @@ func (m *ClusterRequest) Reset() { *m = ClusterRequest{} } func (m *ClusterRequest) String() string { return proto.CompactTextString(m) } func (*ClusterRequest) ProtoMessage() {} func (*ClusterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{18} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{18} } func (m *ClusterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1139,7 +1139,7 @@ func (m *ClusterResponse) Reset() { *m = ClusterResponse{} } func (m *ClusterResponse) String() string { return proto.CompactTextString(m) } func (*ClusterResponse) ProtoMessage() {} func (*ClusterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{19} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{19} } func (m *ClusterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1198,7 +1198,7 @@ func (m *DrainRequest) Reset() { *m = DrainRequest{} } func (m *DrainRequest) String() string { return proto.CompactTextString(m) } func (*DrainRequest) ProtoMessage() {} func (*DrainRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_2d30d36d58fa713c, []int{20} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{20} } func (m *DrainRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1240,13 +1240,59 @@ type DrainResponse struct { // 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_2d30d36d58fa713c, []int{21} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{21} } func (m *DrainResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1281,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_2d30d36d58fa713c, []int{22} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{22} } func (m *DecommissionStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1319,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_2d30d36d58fa713c, []int{23} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{23} } func (m *DecommissionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1354,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_2d30d36d58fa713c, []int{24} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{24} } func (m *DecommissionStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1392,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_2d30d36d58fa713c, []int{24, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{24, 0} } func (m *DecommissionStatusResponse_Status) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1428,7 +1474,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_2d30d36d58fa713c, []int{25} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{25} } func (m *SettingsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1462,7 +1508,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_2d30d36d58fa713c, []int{26} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{26} } func (m *SettingsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1497,7 +1543,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_2d30d36d58fa713c, []int{26, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{26, 0} } func (m *SettingsResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1550,7 +1596,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_2d30d36d58fa713c, []int{27} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{27} } func (m *HealthRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1584,7 +1630,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_2d30d36d58fa713c, []int{28} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{28} } func (m *HealthResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1617,7 +1663,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_2d30d36d58fa713c, []int{29} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{29} } func (m *LivenessRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1652,7 +1698,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_2d30d36d58fa713c, []int{30} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{30} } func (m *LivenessResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1688,7 +1734,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_2d30d36d58fa713c, []int{31} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{31} } func (m *JobsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1722,7 +1768,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_2d30d36d58fa713c, []int{32} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{32} } func (m *JobsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1775,7 +1821,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_2d30d36d58fa713c, []int{32, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{32, 0} } func (m *JobsResponse_Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1808,7 +1854,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_2d30d36d58fa713c, []int{33} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{33} } func (m *LocationsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1842,7 +1888,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_2d30d36d58fa713c, []int{34} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{34} } func (m *LocationsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1878,7 +1924,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_2d30d36d58fa713c, []int{34, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{34, 0} } func (m *LocationsResponse_Location) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1920,7 +1966,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_2d30d36d58fa713c, []int{35} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{35} } func (m *RangeLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1954,7 +2000,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_2d30d36d58fa713c, []int{36} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{36} } func (m *RangeLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1994,7 +2040,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_2d30d36d58fa713c, []int{36, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{36, 0} } func (m *RangeLogResponse_PrettyInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2028,7 +2074,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_2d30d36d58fa713c, []int{36, 1} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{36, 1} } func (m *RangeLogResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2063,7 +2109,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_2d30d36d58fa713c, []int{37} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{37} } func (m *QueryPlanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2098,7 +2144,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_2d30d36d58fa713c, []int{38} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{38} } func (m *QueryPlanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2130,7 +2176,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_2d30d36d58fa713c, []int{39} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{39} } func (m *DataDistributionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2166,7 +2212,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_2d30d36d58fa713c, []int{40} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{40} } func (m *DataDistributionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2204,7 +2250,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_2d30d36d58fa713c, []int{40, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{40, 0} } func (m *DataDistributionResponse_ZoneConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2239,7 +2285,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_2d30d36d58fa713c, []int{40, 1} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{40, 1} } func (m *DataDistributionResponse_TableInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2273,7 +2319,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_2d30d36d58fa713c, []int{40, 2} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{40, 2} } func (m *DataDistributionResponse_DatabaseInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2306,7 +2352,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_2d30d36d58fa713c, []int{41} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{41} } func (m *MetricMetadataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2340,7 +2386,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_2d30d36d58fa713c, []int{42} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{42} } func (m *MetricMetadataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2383,7 +2429,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_2d30d36d58fa713c, []int{43} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{43} } func (m *EnqueueRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2416,7 +2462,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_2d30d36d58fa713c, []int{44} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{44} } func (m *EnqueueRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2453,7 +2499,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_2d30d36d58fa713c, []int{44, 0} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{44, 0} } func (m *EnqueueRangeResponse_Details) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2486,7 +2532,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_2d30d36d58fa713c, []int{45} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{45} } func (m *ChartCatalogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2520,7 +2566,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_2d30d36d58fa713c, []int{46} + return fileDescriptor_admin_52a8691f76c4ab9c, []int{46} } func (m *ChartCatalogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4797,6 +4843,17 @@ func (m *DrainResponse) MarshalTo(dAtA []byte) (int, error) { } 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 } @@ -6736,6 +6793,13 @@ func (m *DrainResponse) Size() (n int) { 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 } @@ -11028,6 +11092,54 @@ func (m *DrainResponse) Unmarshal(dAtA []byte) error { } } 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:]) @@ -15611,263 +15723,266 @@ var ( ErrIntOverflowAdmin = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_2d30d36d58fa713c) } - -var fileDescriptor_admin_2d30d36d58fa713c = []byte{ - // 4077 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x4d, 0x73, 0x1b, 0xc9, - 0x75, 0x1a, 0x80, 0xf8, 0x7a, 0x04, 0x48, 0xb0, 0x45, 0x51, 0x20, 0x24, 0x13, 0xdc, 0x91, 0xe5, - 0xa5, 0xb5, 0x6b, 0x60, 0x45, 0x69, 0xb5, 0x8e, 0x56, 0x1b, 0x87, 0x20, 0x58, 0x5c, 0x68, 0x29, - 0xed, 0x6a, 0x28, 0x69, 0xcb, 0x72, 0xd6, 0x93, 0x01, 0xa6, 0x05, 0x8e, 0x39, 0x98, 0x01, 0x67, - 0x06, 0xd4, 0x72, 0x37, 0x76, 0xc5, 0x76, 0x2a, 0x95, 0x5c, 0x52, 0x5b, 0xb6, 0x6f, 0xae, 0x4a, - 0x55, 0x52, 0xa9, 0xa4, 0x92, 0x9c, 0x72, 0x4b, 0x72, 0xc8, 0x79, 0x2b, 0x07, 0xdb, 0x55, 0xb9, - 0x38, 0x87, 0xd0, 0x09, 0x37, 0x87, 0x54, 0x7e, 0x40, 0x0e, 0x39, 0x24, 0xa9, 0xfe, 0x9a, 0x69, - 0x80, 0xb3, 0x23, 0x80, 0xb2, 0x7d, 0x20, 0x31, 0xfd, 0x5e, 0xf7, 0xeb, 0xd7, 0xaf, 0x5f, 0xbf, - 0x7e, 0x1f, 0x0d, 0x97, 0x7c, 0xec, 0x1d, 0x62, 0xaf, 0xc1, 0x7e, 0x06, 0x9d, 0x86, 0x61, 0xf6, - 0x2d, 0xa7, 0x3e, 0xf0, 0xdc, 0xc0, 0x45, 0xcb, 0x5d, 0xb7, 0xbb, 0xef, 0xb9, 0x46, 0x77, 0xaf, - 0xce, 0xf0, 0x75, 0xd1, 0xad, 0xba, 0xd0, 0x75, 0x9d, 0xa7, 0x56, 0xaf, 0xf1, 0x91, 0xeb, 0x60, - 0xd6, 0xbb, 0xba, 0xf4, 0x2d, 0xb7, 0xe3, 0x37, 0xc8, 0xbf, 0x41, 0x87, 0xfe, 0x70, 0xf8, 0xe5, - 0xf1, 0x29, 0xfc, 0xc0, 0x08, 0x86, 0x02, 0xab, 0xfa, 0x81, 0xeb, 0x19, 0x3d, 0xdc, 0xc0, 0x4e, - 0xcf, 0x72, 0xc4, 0xcf, 0xa0, 0xd3, 0xe8, 0x1f, 0x76, 0xbb, 0xbc, 0xcf, 0xaa, 0xe8, 0xc3, 0x7f, - 0x07, 0x9d, 0x86, 0x6d, 0x1d, 0x62, 0x07, 0xfb, 0x82, 0xca, 0xa5, 0x98, 0x1e, 0x6e, 0x8f, 0x23, - 0x57, 0x02, 0xbf, 0xd1, 0x35, 0x02, 0xc3, 0x76, 0x7b, 0x8d, 0xee, 0x9e, 0xe1, 0x05, 0x3a, 0x6f, - 0x71, 0x7c, 0x65, 0x18, 0x58, 0x76, 0xa3, 0x8f, 0x03, 0xcf, 0xea, 0xf2, 0x1f, 0x8e, 0x59, 0xec, - 0xb9, 0x3d, 0x97, 0x7e, 0x36, 0xc8, 0x97, 0x58, 0x50, 0xcf, 0x75, 0x7b, 0x36, 0x6e, 0x18, 0x03, - 0xab, 0x61, 0x38, 0x8e, 0x1b, 0x18, 0x81, 0xe5, 0x3a, 0x82, 0x95, 0x1a, 0xc7, 0xd2, 0x56, 0x67, - 0xf8, 0xb4, 0x11, 0x58, 0x7d, 0xec, 0x07, 0x46, 0x7f, 0xc0, 0x3a, 0xa8, 0x08, 0xca, 0x2d, 0x23, - 0x30, 0x3a, 0x86, 0x8f, 0x7d, 0x0d, 0x1f, 0x0c, 0xb1, 0x1f, 0xa8, 0xd7, 0x61, 0x41, 0x82, 0xf9, - 0x03, 0xd7, 0xf1, 0x31, 0xba, 0x0c, 0x05, 0x53, 0x00, 0x2b, 0xca, 0x6a, 0x7a, 0xad, 0xa0, 0x45, - 0x00, 0xf5, 0x26, 0x2c, 0x89, 0x21, 0x2d, 0x1c, 0x18, 0x96, 0x2d, 0x88, 0xa1, 0x2a, 0xe4, 0x45, - 0xb7, 0x8a, 0xb2, 0xaa, 0xac, 0x15, 0xb4, 0xb0, 0xad, 0xfe, 0x75, 0x1a, 0x2e, 0x9e, 0x1a, 0xc6, - 0xe7, 0x7b, 0x0c, 0xd9, 0x9e, 0x67, 0x38, 0x01, 0x9b, 0x6c, 0x76, 0xfd, 0xab, 0xf5, 0xcf, 0xdd, - 0xff, 0xfa, 0xe7, 0xd0, 0xa8, 0x6f, 0x13, 0x02, 0xcd, 0x99, 0x4f, 0x8f, 0x6b, 0xe7, 0x34, 0x4e, - 0x0d, 0xd5, 0x60, 0x36, 0x30, 0x3a, 0x36, 0xd6, 0x1d, 0xa3, 0x8f, 0xfd, 0x4a, 0x8a, 0xae, 0x04, - 0x28, 0xe8, 0x3e, 0x81, 0xa0, 0xd7, 0xa1, 0x64, 0x62, 0xbf, 0xeb, 0x59, 0x83, 0xc0, 0xf5, 0x74, - 0xcb, 0xac, 0xa4, 0x57, 0x95, 0xb5, 0x74, 0xb3, 0x7c, 0x72, 0x5c, 0x2b, 0xb6, 0x42, 0x44, 0xbb, - 0xa5, 0x15, 0xa3, 0x6e, 0x6d, 0x13, 0x6d, 0xc2, 0x2c, 0x51, 0x3f, 0x9d, 0xa9, 0x62, 0x65, 0x66, - 0x55, 0x59, 0x9b, 0x5d, 0xbf, 0x2c, 0x31, 0xcd, 0x10, 0xf5, 0x27, 0xae, 0x83, 0x37, 0xe9, 0x27, - 0x67, 0x0c, 0x3e, 0x0a, 0x21, 0xe8, 0x03, 0x58, 0x90, 0x88, 0xe8, 0x36, 0x3e, 0xc4, 0x76, 0x25, - 0xb3, 0xaa, 0xac, 0xcd, 0xad, 0x5f, 0x4f, 0x58, 0x7f, 0x44, 0x73, 0xe8, 0xd1, 0xed, 0xdf, 0x21, - 0x03, 0xb5, 0xf9, 0x88, 0x32, 0x05, 0x54, 0xdf, 0x84, 0x0c, 0x15, 0x09, 0x42, 0x30, 0x33, 0xf4, - 0xb1, 0xc7, 0x37, 0x84, 0x7e, 0xa3, 0x15, 0x80, 0x81, 0x67, 0x1d, 0x5a, 0x36, 0xee, 0x45, 0x72, - 0x89, 0x20, 0xea, 0x36, 0x9c, 0x7f, 0x48, 0xa4, 0x34, 0xf9, 0xfe, 0xa2, 0x45, 0xc8, 0x50, 0xc1, - 0x56, 0x52, 0x14, 0xc1, 0x1a, 0xea, 0xff, 0xe6, 0x60, 0x71, 0x94, 0x12, 0xdf, 0xf2, 0xdd, 0xb1, - 0x2d, 0x7f, 0x3d, 0x61, 0xc9, 0x71, 0x04, 0x62, 0xf7, 0xfb, 0x31, 0xe4, 0xba, 0xae, 0x3d, 0xec, - 0x3b, 0x6c, 0x4d, 0xb3, 0xeb, 0xb7, 0xa6, 0xa5, 0xba, 0x49, 0x87, 0x73, 0xb2, 0x82, 0x18, 0x7a, - 0x04, 0x39, 0xcb, 0x31, 0xf1, 0x87, 0xd8, 0xaf, 0xa4, 0xcf, 0xc6, 0x6d, 0x9b, 0x0c, 0x17, 0x64, - 0x39, 0x2d, 0xa2, 0x9e, 0x9e, 0xe1, 0xf4, 0x88, 0x0a, 0x0c, 0x9d, 0x80, 0xaa, 0x51, 0x5a, 0x03, - 0x0a, 0xda, 0x24, 0x10, 0x74, 0x13, 0x96, 0xba, 0x1e, 0x36, 0x02, 0xac, 0x33, 0x35, 0x26, 0xf6, - 0x0b, 0xf7, 0xb1, 0x13, 0x50, 0x3d, 0x29, 0x68, 0x8b, 0x0c, 0x4b, 0x67, 0xdc, 0x15, 0xb8, 0x71, - 0xed, 0xcc, 0xfe, 0xf2, 0xb4, 0x33, 0xf7, 0xcb, 0xd2, 0xce, 0xd3, 0x07, 0x2f, 0x3f, 0xc9, 0xc1, - 0x7b, 0x21, 0xa5, 0xae, 0xfe, 0xa3, 0x02, 0x59, 0xb6, 0xbf, 0x64, 0x38, 0x31, 0x09, 0x62, 0x38, - 0xf9, 0x26, 0xb0, 0xe0, 0x68, 0x20, 0xf4, 0x97, 0x7e, 0x13, 0x85, 0x77, 0x86, 0xb6, 0x4d, 0xf5, - 0x9a, 0x98, 0x86, 0xbc, 0x16, 0xb6, 0xd1, 0x15, 0xb2, 0x84, 0xa7, 0xc6, 0xd0, 0x0e, 0xf4, 0x43, - 0xc3, 0x1e, 0x62, 0xba, 0x7f, 0x05, 0xc2, 0x30, 0x05, 0x3e, 0x26, 0x30, 0x74, 0x03, 0x2e, 0xf4, - 0xb0, 0x83, 0x99, 0x2c, 0x74, 0xfc, 0xe1, 0xc0, 0xc3, 0xbe, 0x6f, 0xb9, 0x8e, 0xd8, 0xc0, 0x08, - 0xb9, 0x15, 0xe2, 0xd0, 0x12, 0x64, 0xf7, 0x2c, 0xd3, 0xc4, 0x0e, 0xdd, 0xbb, 0xbc, 0xc6, 0x5b, - 0xd5, 0xbf, 0x55, 0x20, 0x43, 0x15, 0x29, 0x96, 0xff, 0x25, 0xc8, 0x0e, 0x1d, 0xeb, 0x60, 0xc8, - 0x56, 0x90, 0xd7, 0x78, 0x0b, 0x95, 0x21, 0xed, 0xe3, 0x03, 0x66, 0xd9, 0x34, 0xf2, 0x49, 0x7a, - 0x32, 0xcd, 0xe6, 0x2c, 0xf3, 0x16, 0x35, 0xfb, 0x96, 0x87, 0xbb, 0x41, 0xc4, 0x60, 0x04, 0x40, - 0x15, 0xc8, 0x91, 0x3b, 0xce, 0x72, 0x7a, 0x9c, 0x2d, 0xd1, 0x24, 0x52, 0xb2, 0xfa, 0x03, 0xdb, - 0xea, 0x5a, 0x01, 0x55, 0x91, 0xbc, 0x16, 0xb6, 0xd5, 0x2d, 0x58, 0x08, 0xd5, 0xf3, 0x05, 0xec, - 0xc8, 0xdf, 0xa5, 0x01, 0xc9, 0x74, 0xb8, 0x15, 0x19, 0x3b, 0x41, 0xca, 0xa9, 0x13, 0x74, 0x05, - 0x4a, 0x1e, 0x26, 0xac, 0x18, 0xbc, 0x4b, 0x8a, 0x76, 0x29, 0x72, 0x20, 0xeb, 0xf4, 0x05, 0x00, - 0xc7, 0x35, 0x05, 0x11, 0x26, 0xa8, 0x02, 0x81, 0x30, 0xf4, 0xdb, 0x90, 0x21, 0x07, 0xcf, 0xe7, - 0x76, 0xfe, 0x55, 0x59, 0xfd, 0xd9, 0xa5, 0x5f, 0x67, 0xbe, 0x43, 0x5d, 0xb8, 0x10, 0xf5, 0x7b, - 0x8f, 0x37, 0x37, 0x29, 0xa7, 0xfc, 0x64, 0x31, 0x02, 0xc8, 0x84, 0x52, 0xdf, 0xf2, 0x7d, 0xcb, - 0xe9, 0xe9, 0x84, 0xbc, 0x5f, 0xc9, 0x50, 0x6b, 0xf2, 0x1b, 0xcf, 0xb3, 0x26, 0x23, 0x8b, 0xae, - 0xdf, 0x63, 0x24, 0xee, 0xbb, 0x26, 0xe6, 0xe4, 0x8b, 0xfd, 0x08, 0xe4, 0x13, 0xab, 0x61, 0x0c, - 0x06, 0x9e, 0xfb, 0xa1, 0xd5, 0x27, 0xa6, 0xc3, 0xb4, 0xfc, 0x7d, 0xbd, 0x73, 0x14, 0x60, 0x9f, - 0xee, 0xdb, 0x8c, 0xb6, 0x28, 0x61, 0x5b, 0x96, 0xbf, 0xdf, 0x24, 0xb8, 0xea, 0xfb, 0x30, 0x2b, - 0x11, 0x46, 0x57, 0x20, 0x47, 0x65, 0x62, 0x99, 0x6c, 0x87, 0x9a, 0x70, 0x72, 0x5c, 0xcb, 0x12, - 0x54, 0xbb, 0xa5, 0x65, 0x09, 0xaa, 0x6d, 0x12, 0xe9, 0x62, 0xcf, 0x73, 0x3d, 0xbd, 0x8f, 0x7d, - 0xdf, 0xe8, 0x89, 0x3d, 0x2b, 0x52, 0xe0, 0x3d, 0x06, 0x53, 0x97, 0x60, 0xf1, 0xbe, 0xeb, 0x9c, - 0x52, 0x02, 0xf5, 0xa7, 0x0a, 0x5c, 0x18, 0x43, 0xf0, 0x5d, 0xfd, 0x3a, 0x2c, 0x10, 0xd7, 0x45, - 0xf7, 0xb1, 0x67, 0x61, 0x5f, 0x67, 0xc2, 0x57, 0xa8, 0xf0, 0xbf, 0x32, 0x95, 0xa8, 0xb4, 0x79, - 0x42, 0x67, 0x97, 0x92, 0xa1, 0x08, 0xf4, 0x0d, 0x40, 0x96, 0x13, 0x60, 0xcf, 0x31, 0x6c, 0x7d, - 0xe8, 0x63, 0x4e, 0x3b, 0x75, 0x16, 0xda, 0x65, 0x41, 0xe8, 0x91, 0xcf, 0x30, 0xea, 0x1c, 0x14, - 0x1f, 0xf9, 0xd8, 0x0b, 0x57, 0xf8, 0x1d, 0x28, 0xf1, 0x36, 0x5f, 0x58, 0x1b, 0x32, 0xc4, 0x52, - 0x89, 0x3b, 0x2f, 0x69, 0xc2, 0x91, 0x81, 0xb4, 0x25, 0x54, 0x89, 0x52, 0xa8, 0xaa, 0x30, 0x43, - 0x80, 0xe4, 0x28, 0x11, 0x80, 0x64, 0x0d, 0xc2, 0xb6, 0xfa, 0x7d, 0x05, 0x4a, 0x5b, 0x87, 0xd8, - 0x89, 0x0e, 0x9e, 0xb0, 0x71, 0x8a, 0x64, 0xe3, 0x2e, 0x41, 0x21, 0x30, 0xbc, 0x1e, 0x0e, 0xc8, - 0x5e, 0xb3, 0xe3, 0x91, 0x67, 0x80, 0xb6, 0x49, 0x4e, 0xa3, 0x6d, 0xf5, 0x2d, 0x76, 0x2a, 0x32, - 0x1a, 0x6b, 0xa0, 0x57, 0x60, 0x61, 0xe8, 0x78, 0xd8, 0x34, 0xba, 0x01, 0x36, 0x75, 0x4c, 0xa7, - 0xa0, 0xa7, 0x23, 0xaf, 0x95, 0x23, 0x04, 0x9b, 0x5a, 0xfd, 0xaf, 0x14, 0xcc, 0x09, 0x2e, 0xb8, - 0x1c, 0xee, 0x41, 0x96, 0x0f, 0x62, 0x82, 0x68, 0x24, 0x08, 0x62, 0x74, 0x28, 0x6b, 0x8a, 0x6b, - 0x9f, 0x11, 0xa9, 0xfe, 0x61, 0x0a, 0x32, 0x14, 0x8e, 0x9a, 0x50, 0x08, 0x9d, 0x5e, 0xae, 0x31, - 0xd5, 0x3a, 0x73, 0x8b, 0xeb, 0xc2, 0x2d, 0xae, 0x3f, 0x14, 0x3d, 0x9a, 0x79, 0x42, 0xe6, 0x93, - 0x5f, 0xd4, 0x14, 0x2d, 0x1a, 0x46, 0xac, 0x01, 0xa5, 0xab, 0x4b, 0xb7, 0x41, 0x81, 0x42, 0x1e, - 0x12, 0x71, 0x7d, 0x59, 0x16, 0x17, 0x73, 0x17, 0x8b, 0x27, 0xc7, 0xb5, 0xfc, 0x43, 0x26, 0xb2, - 0x96, 0x24, 0xbc, 0x75, 0x20, 0x76, 0xc6, 0xf5, 0x02, 0x72, 0xe0, 0x2d, 0x93, 0x5d, 0xf0, 0xcd, - 0xf9, 0x93, 0xe3, 0xda, 0xac, 0x26, 0xe0, 0xed, 0x96, 0x36, 0x1b, 0x76, 0x6a, 0x9b, 0x64, 0x87, - 0x2c, 0xe7, 0xa9, 0xcb, 0xcd, 0x2f, 0xfd, 0x26, 0x53, 0x32, 0x5b, 0x4e, 0x88, 0x90, 0x33, 0x5c, - 0x64, 0x53, 0x3e, 0xa2, 0x40, 0x32, 0x25, 0x43, 0xb7, 0x4d, 0xf5, 0x6f, 0x14, 0x28, 0xef, 0xe2, - 0xe0, 0x51, 0x9b, 0xb8, 0xc9, 0x62, 0xd7, 0xbf, 0x0e, 0xb0, 0x8f, 0x8f, 0xd8, 0x2d, 0x25, 0x44, - 0x7e, 0x3b, 0x41, 0xe4, 0xe3, 0x04, 0xea, 0xef, 0xe0, 0x23, 0x7a, 0x9d, 0xf9, 0x5b, 0x4e, 0xe0, - 0x1d, 0x69, 0x85, 0x7d, 0xd1, 0xae, 0xde, 0x81, 0xb9, 0x51, 0x24, 0xb9, 0x6e, 0xf6, 0xf1, 0x11, - 0xd7, 0x30, 0xf2, 0x49, 0x74, 0x88, 0x5d, 0x90, 0x44, 0x96, 0x45, 0x8d, 0x35, 0x6e, 0xa7, 0xbe, - 0xaa, 0xa8, 0xe7, 0x61, 0x41, 0x9a, 0x8b, 0xed, 0xb0, 0xfa, 0x25, 0x28, 0x6f, 0x8f, 0xaf, 0x00, - 0xc1, 0xcc, 0x3e, 0x3e, 0x12, 0xb1, 0x08, 0xfd, 0x56, 0x7f, 0x9a, 0x82, 0x85, 0xed, 0xf1, 0xd1, - 0xe8, 0x77, 0x62, 0xd6, 0xfa, 0x66, 0xc2, 0x5a, 0x4f, 0x51, 0x18, 0x5b, 0x2c, 0x57, 0x35, 0x69, - 0xc9, 0x4f, 0x21, 0xc3, 0xee, 0xf6, 0x70, 0x5d, 0x8a, 0xb4, 0x2e, 0xb4, 0x0d, 0x45, 0xdb, 0xf0, - 0x03, 0x7d, 0x38, 0x30, 0x8d, 0x00, 0x9b, 0xdc, 0xb6, 0x4c, 0xa6, 0x85, 0xb3, 0x64, 0xe4, 0x23, - 0x36, 0xb0, 0x3a, 0x98, 0x40, 0xb4, 0x6f, 0xcb, 0xa2, 0x9d, 0x5d, 0x5f, 0x9f, 0x6a, 0xa1, 0x94, - 0xb4, 0xbc, 0x1d, 0x65, 0x98, 0xdb, 0xb4, 0x87, 0x7e, 0x80, 0x3d, 0x61, 0xc1, 0x7e, 0xac, 0xc0, - 0x7c, 0x08, 0xe2, 0x12, 0x7e, 0x15, 0xa0, 0xcb, 0x40, 0xd1, 0xe5, 0x50, 0x3a, 0x39, 0xae, 0x15, - 0x78, 0xc7, 0x76, 0x4b, 0x2b, 0xf0, 0x0e, 0x6d, 0x93, 0x98, 0x8a, 0xe8, 0x0c, 0x60, 0x87, 0x98, - 0x51, 0x93, 0x3b, 0x28, 0xe5, 0x10, 0xb1, 0xc5, 0xe0, 0xe8, 0x2b, 0x80, 0x30, 0x31, 0xaa, 0x03, - 0xcf, 0xf2, 0x71, 0xd8, 0x9b, 0x39, 0x5e, 0x0b, 0x11, 0x86, 0x77, 0x57, 0xff, 0x48, 0x81, 0x62, - 0xcb, 0x33, 0x2c, 0x47, 0xa8, 0xc9, 0x1d, 0xa8, 0x9a, 0x78, 0xe0, 0xe1, 0x2e, 0x11, 0xa0, 0x3e, - 0xf0, 0xdc, 0x0e, 0xd6, 0x2d, 0xc7, 0xb4, 0xba, 0x46, 0xe0, 0x7a, 0x54, 0x19, 0x32, 0x5a, 0x25, - 0xea, 0xf1, 0x1e, 0xe9, 0xd0, 0x16, 0x78, 0x62, 0x4a, 0xfd, 0xbd, 0x61, 0x60, 0xba, 0xcf, 0x1c, - 0xe1, 0xec, 0x89, 0x36, 0x5a, 0x86, 0xbc, 0xe9, 0xea, 0x26, 0x99, 0x8c, 0x1b, 0xba, 0x9c, 0xe9, - 0xd2, 0xb9, 0xef, 0xce, 0xe4, 0x53, 0xe5, 0xb4, 0xba, 0x07, 0x25, 0xce, 0x0a, 0x17, 0xd3, 0x2d, - 0xb8, 0x28, 0xf1, 0x42, 0x47, 0xea, 0x2c, 0xff, 0xc0, 0x19, 0xb9, 0x10, 0xa1, 0xe9, 0xc8, 0x5d, - 0x8a, 0x24, 0x2e, 0x8d, 0xe5, 0xb3, 0xfe, 0xc4, 0xd5, 0x62, 0xa2, 0x02, 0xcb, 0x6f, 0x71, 0x88, - 0xfa, 0x11, 0x2c, 0xb7, 0x70, 0xd7, 0xed, 0xd3, 0x3b, 0xdf, 0xe5, 0xc3, 0x84, 0x04, 0x3e, 0x80, - 0x3c, 0xbf, 0xb6, 0xf9, 0x34, 0xcd, 0xe6, 0xc9, 0x71, 0x2d, 0xc7, 0xee, 0x6d, 0xff, 0x7f, 0x8e, - 0x6b, 0x37, 0x7a, 0x56, 0xb0, 0x37, 0xec, 0xd4, 0xbb, 0x6e, 0xbf, 0x11, 0x2a, 0x8b, 0xd9, 0x89, - 0xbe, 0x1b, 0x83, 0xfd, 0x5e, 0x83, 0x7e, 0x0d, 0x3a, 0x75, 0x7e, 0xdf, 0xe7, 0xd8, 0x7d, 0xef, - 0xab, 0x7f, 0xa2, 0xc0, 0x79, 0x79, 0xf2, 0x5f, 0xcf, 0xb4, 0x68, 0x0d, 0xe6, 0x4d, 0x69, 0xd6, - 0x48, 0x2e, 0xe3, 0x60, 0xf5, 0xb3, 0x14, 0x54, 0xe3, 0xa4, 0xc3, 0x37, 0xe5, 0x09, 0x64, 0xf9, - 0x1e, 0xb0, 0xf8, 0xf0, 0x4e, 0x52, 0xa2, 0xe1, 0x73, 0xc9, 0xd4, 0x59, 0x53, 0xdc, 0x42, 0x8c, - 0x62, 0xf5, 0x3f, 0x15, 0xc8, 0xf2, 0x3d, 0x7c, 0x32, 0xea, 0x3c, 0x65, 0x9a, 0x1b, 0x91, 0xf3, - 0x74, 0x56, 0x61, 0x08, 0x9f, 0xeb, 0x22, 0xe4, 0x2c, 0x5f, 0xb7, 0xad, 0xc3, 0xd0, 0xcf, 0xb7, - 0xfc, 0x1d, 0xeb, 0x10, 0x9f, 0x76, 0x75, 0xd3, 0x31, 0xae, 0x6e, 0x8c, 0x24, 0x67, 0x62, 0x25, - 0x49, 0x7d, 0x74, 0xa1, 0x84, 0x19, 0x76, 0x1a, 0x44, 0x5b, 0xbd, 0x0a, 0xf3, 0xbb, 0x38, 0x20, - 0x27, 0xd7, 0x4f, 0xb2, 0xd0, 0x7f, 0x9f, 0xa2, 0x97, 0x11, 0xef, 0xc7, 0xb7, 0x40, 0x9f, 0xfe, - 0x32, 0x1a, 0x21, 0xf0, 0x5c, 0xfb, 0xbc, 0x1b, 0x6b, 0x9f, 0x0b, 0xc2, 0x3e, 0xc7, 0x85, 0x79, - 0xab, 0x30, 0x2b, 0xc2, 0x4c, 0x12, 0xfa, 0xa4, 0x29, 0x4a, 0x06, 0x55, 0xdd, 0x09, 0x8c, 0xf1, - 0xf6, 0xa8, 0x31, 0xbe, 0x3e, 0xcd, 0xa2, 0x4e, 0xd9, 0xe2, 0xab, 0x50, 0x7a, 0x1b, 0x1b, 0x76, - 0xb0, 0x27, 0x04, 0xbc, 0x08, 0x19, 0x0f, 0x1b, 0x26, 0x9b, 0x31, 0xaf, 0xb1, 0x06, 0x31, 0xd9, - 0xa2, 0x1b, 0xbf, 0x3e, 0x17, 0x60, 0x7e, 0x87, 0xa7, 0x28, 0x85, 0x15, 0xff, 0x49, 0x0a, 0xca, - 0x11, 0x8c, 0xef, 0xc3, 0x06, 0x80, 0x48, 0x65, 0x86, 0xfb, 0x70, 0x29, 0x26, 0xb4, 0x11, 0x03, - 0x45, 0x8e, 0x20, 0x1a, 0x84, 0x7e, 0xa0, 0x40, 0x9e, 0x29, 0x3f, 0x16, 0x07, 0x2a, 0x29, 0x94, - 0x19, 0x67, 0x81, 0x1f, 0x23, 0xb1, 0x91, 0x6f, 0x12, 0xfa, 0xdf, 0xfb, 0xc5, 0xd9, 0x4e, 0x47, - 0xc8, 0x47, 0xb5, 0x03, 0xa5, 0x11, 0xba, 0xf2, 0x46, 0x65, 0xd8, 0x46, 0xbd, 0x29, 0x6f, 0xd4, - 0xdc, 0xfa, 0xd5, 0x98, 0x55, 0x13, 0xb2, 0x82, 0x5f, 0x7e, 0xf8, 0xa5, 0xcd, 0x19, 0xc0, 0xec, - 0x5d, 0xb7, 0xe3, 0x4b, 0x5b, 0xc3, 0x9c, 0x64, 0x45, 0x76, 0x92, 0x97, 0x24, 0x5b, 0x43, 0xa3, - 0x6c, 0xd6, 0x42, 0x37, 0xb8, 0x02, 0xa6, 0xe9, 0xe4, 0x35, 0x79, 0xf2, 0x03, 0xbb, 0x4e, 0xd3, - 0xd7, 0x2c, 0x95, 0x5d, 0x27, 0xfe, 0x26, 0xd3, 0x50, 0xf5, 0x27, 0x59, 0x28, 0xb2, 0x29, 0xf9, - 0xf6, 0x6d, 0xc1, 0x0c, 0xe9, 0xc5, 0x37, 0xee, 0x95, 0x04, 0xb1, 0xcb, 0xc3, 0x48, 0x83, 0x6f, - 0x24, 0x1d, 0x5e, 0xfd, 0xef, 0x0c, 0xa4, 0xef, 0xba, 0x1d, 0xb4, 0x04, 0x29, 0x6e, 0xac, 0xd2, - 0xcd, 0xec, 0xc9, 0x71, 0x2d, 0xd5, 0x6e, 0x69, 0x29, 0xcb, 0x3c, 0xdb, 0x69, 0x19, 0x09, 0x4a, - 0x66, 0x46, 0x83, 0x12, 0xe4, 0xc2, 0xdc, 0x48, 0xe6, 0x87, 0x05, 0xc1, 0xa5, 0xe6, 0xdb, 0x27, - 0xc7, 0xb5, 0x92, 0x9c, 0xfa, 0x99, 0xfc, 0xda, 0xf0, 0x0f, 0x6c, 0xf2, 0xd7, 0x31, 0x7c, 0x5c, - 0x6f, 0xb7, 0xb4, 0x92, 0x9c, 0x32, 0xf2, 0xa5, 0x7d, 0xc8, 0x8e, 0xec, 0xc3, 0x6d, 0xc8, 0xb1, - 0xf4, 0x99, 0x49, 0x93, 0x16, 0xc9, 0x3e, 0xda, 0x0c, 0xf5, 0xcf, 0xc4, 0x00, 0x32, 0xd6, 0x0f, - 0x0c, 0x8f, 0x8c, 0xcd, 0x4f, 0x3a, 0x96, 0x0f, 0x40, 0x77, 0x20, 0xff, 0xd4, 0x72, 0x2c, 0x7f, - 0x0f, 0x9b, 0x95, 0xc2, 0x84, 0x83, 0xc3, 0x11, 0x64, 0x74, 0xdf, 0x35, 0xad, 0xa7, 0x16, 0x36, - 0x2b, 0x30, 0xe9, 0x68, 0x31, 0x82, 0x38, 0x58, 0x4f, 0x3d, 0x83, 0xe6, 0x73, 0xf4, 0xae, 0xdb, - 0x1f, 0xd8, 0x98, 0x2c, 0x61, 0x76, 0x55, 0x59, 0x4b, 0x69, 0x0b, 0x02, 0xb3, 0x29, 0x10, 0x44, - 0xb1, 0x69, 0x28, 0x5f, 0x29, 0x32, 0x0b, 0x4a, 0x1b, 0xe8, 0x01, 0x9c, 0xdf, 0xb3, 0x7a, 0x7b, - 0xcf, 0x0c, 0xe2, 0x02, 0x46, 0xe1, 0x56, 0x69, 0x42, 0x6e, 0x50, 0x38, 0x38, 0xc4, 0x10, 0x2f, - 0x31, 0x22, 0x69, 0xe2, 0xae, 0xd5, 0x37, 0xec, 0xca, 0x1c, 0x9d, 0xb4, 0x1c, 0x22, 0x5a, 0x0c, - 0x8e, 0xae, 0xc2, 0x9c, 0x37, 0x74, 0xc8, 0x45, 0x24, 0x1c, 0xaa, 0x79, 0xda, 0xb3, 0xc4, 0xa1, - 0xfc, 0x12, 0xbe, 0x0c, 0x85, 0x28, 0x5f, 0x5a, 0x66, 0x61, 0x5c, 0x08, 0x50, 0x11, 0x94, 0x77, - 0xdc, 0x2e, 0xab, 0x9f, 0x08, 0x3b, 0xf9, 0x7f, 0x0a, 0x2c, 0x48, 0xc0, 0x30, 0x1b, 0x51, 0xb0, - 0x05, 0x70, 0x82, 0x64, 0xf5, 0x29, 0x02, 0x21, 0x44, 0x5c, 0x55, 0x21, 0xb5, 0xea, 0x1f, 0x2b, - 0x90, 0x17, 0x58, 0xf4, 0x12, 0x14, 0x09, 0xc6, 0xb6, 0x82, 0x23, 0x3d, 0xba, 0x59, 0x66, 0x05, - 0xec, 0x1d, 0x7c, 0x44, 0x56, 0x1e, 0x76, 0x89, 0x2c, 0x58, 0x41, 0x2b, 0x09, 0x28, 0xbb, 0xf8, - 0xaa, 0x90, 0xb7, 0x8d, 0xc0, 0x0a, 0x86, 0x26, 0xb3, 0x32, 0x8a, 0x16, 0xb6, 0x89, 0x54, 0x6c, - 0xd7, 0xe9, 0x31, 0xe4, 0x0c, 0x45, 0x46, 0x00, 0xb5, 0x09, 0xf3, 0x9a, 0xe1, 0xf4, 0xf0, 0x8e, - 0xdb, 0x13, 0xc6, 0x6d, 0x19, 0xf2, 0x2c, 0xc5, 0x26, 0xec, 0x83, 0x96, 0xa3, 0x6d, 0x39, 0x39, - 0x90, 0x92, 0xec, 0x9e, 0xfa, 0xaf, 0x69, 0x28, 0x47, 0x44, 0xb8, 0x10, 0xdf, 0x0d, 0x23, 0x7e, - 0x76, 0x4f, 0x24, 0x5d, 0x8e, 0xe3, 0x83, 0x63, 0x63, 0xfe, 0x7f, 0x52, 0x00, 0xde, 0xf3, 0x70, - 0x10, 0x1c, 0xb5, 0x49, 0x88, 0xfc, 0x12, 0x14, 0x79, 0xc0, 0xa5, 0x93, 0xd3, 0x2f, 0x84, 0xc7, - 0x61, 0xc4, 0xac, 0x90, 0x85, 0x38, 0xf8, 0x19, 0x43, 0x33, 0xb1, 0xe5, 0x1c, 0xfc, 0x8c, 0xa2, - 0xae, 0x40, 0xc9, 0x30, 0x4d, 0x6c, 0xea, 0xdc, 0x57, 0xe2, 0x36, 0xad, 0x48, 0x81, 0x1a, 0x83, - 0xa1, 0x97, 0x61, 0xde, 0xc3, 0x7d, 0xf7, 0x50, 0xea, 0xc6, 0x6c, 0xdb, 0x1c, 0x07, 0x8b, 0x8e, - 0x4b, 0x90, 0xf5, 0xb0, 0xe1, 0x87, 0x39, 0x54, 0xde, 0x42, 0x15, 0xc8, 0x99, 0xac, 0x2c, 0xc0, - 0x2d, 0x91, 0x68, 0x56, 0xff, 0x5c, 0x11, 0x09, 0x8c, 0x3b, 0x90, 0xa1, 0x0b, 0xe4, 0xc9, 0x8b, - 0xd5, 0x98, 0xab, 0x49, 0x88, 0x47, 0x96, 0x0a, 0x1b, 0x84, 0x3e, 0x80, 0xd9, 0x01, 0x95, 0x89, - 0x4e, 0x73, 0x08, 0xcc, 0x0f, 0xb9, 0x35, 0x8d, 0xa8, 0x23, 0x91, 0x8a, 0xfb, 0x7e, 0x10, 0x42, - 0xee, 0xce, 0xe4, 0x95, 0x72, 0x4a, 0x5d, 0x83, 0xf2, 0x83, 0x21, 0xf6, 0x8e, 0xde, 0xb3, 0x0d, - 0x47, 0xba, 0x01, 0x0f, 0x08, 0x4c, 0xb8, 0x5a, 0xb4, 0xa1, 0x0e, 0x60, 0x41, 0xea, 0xc9, 0x35, - 0xe1, 0x1b, 0x70, 0xc9, 0xb4, 0xfc, 0xc0, 0x3f, 0xb0, 0xf5, 0xc1, 0xde, 0x91, 0x6f, 0x75, 0x0d, - 0x5b, 0xa7, 0xdd, 0xf5, 0x81, 0x6d, 0x38, 0x3c, 0x9e, 0xbc, 0x7c, 0x72, 0x5c, 0xab, 0xb4, 0x2c, - 0x3f, 0xd8, 0x7d, 0xb0, 0xf3, 0x1e, 0xef, 0x15, 0x91, 0xaa, 0x70, 0x02, 0xa7, 0x30, 0xea, 0x32, - 0xab, 0x31, 0x92, 0x91, 0x9e, 0xd5, 0x19, 0x06, 0x51, 0x88, 0xa2, 0xfe, 0x15, 0x40, 0xe5, 0x34, - 0x8e, 0x33, 0x35, 0x80, 0x92, 0x48, 0x40, 0x33, 0xd1, 0xb1, 0x73, 0xbe, 0xf5, 0x9c, 0x3a, 0x64, - 0x1c, 0xad, 0xb0, 0x40, 0x49, 0x44, 0x26, 0xbb, 0xa8, 0x45, 0x53, 0x42, 0xa0, 0x3e, 0x14, 0xa5, - 0xfa, 0x8a, 0xa8, 0x2b, 0xb5, 0xce, 0x32, 0x61, 0x54, 0x73, 0x19, 0x71, 0x89, 0x67, 0xa3, 0x9a, - 0x8b, 0x5f, 0xfd, 0x91, 0x02, 0x10, 0xf5, 0x23, 0x2a, 0xca, 0xb2, 0x54, 0x7c, 0xc3, 0x78, 0x0b, - 0xdd, 0x86, 0x2c, 0xaf, 0x1a, 0xa5, 0x26, 0xae, 0x1a, 0xf1, 0x11, 0x34, 0x2f, 0xc0, 0x8a, 0x45, - 0xfe, 0x81, 0xcd, 0x8e, 0x06, 0xcf, 0x0b, 0x50, 0xe8, 0xee, 0x83, 0x1d, 0xad, 0xc0, 0x3a, 0xec, - 0x1e, 0xd8, 0x77, 0x67, 0xf2, 0xe9, 0xf2, 0x4c, 0xf5, 0x0f, 0xd2, 0x50, 0xa0, 0xa9, 0x55, 0x2a, - 0x93, 0x4f, 0x15, 0xa8, 0x8c, 0x84, 0x30, 0x7a, 0xe7, 0x48, 0x8f, 0x02, 0x29, 0x22, 0xa0, 0xf7, - 0xcf, 0x22, 0xa0, 0x70, 0x86, 0xba, 0x26, 0xc5, 0x42, 0xcd, 0x23, 0xea, 0x3b, 0x9a, 0x4c, 0x66, - 0x6f, 0x9c, 0xd5, 0xf3, 0x5c, 0xf4, 0x62, 0x68, 0xa2, 0x2f, 0xc2, 0x9c, 0x5c, 0x3e, 0x0b, 0x33, - 0xab, 0xc5, 0x68, 0x53, 0xda, 0x26, 0xfa, 0x1a, 0x80, 0xe9, 0xb9, 0x83, 0x01, 0x36, 0x75, 0x83, - 0xc5, 0x6b, 0x93, 0x5c, 0xa0, 0x05, 0x3e, 0x66, 0x23, 0xa8, 0x6e, 0xc3, 0xf2, 0xe7, 0x2e, 0x29, - 0xc6, 0xf1, 0x1d, 0xc9, 0xc4, 0xa5, 0x25, 0x8f, 0xb6, 0xfa, 0xdd, 0x14, 0x14, 0x65, 0xc5, 0x45, - 0x01, 0xb0, 0x3a, 0xb9, 0x7c, 0x1c, 0xde, 0x7d, 0xd1, 0xe3, 0x10, 0xed, 0xc4, 0x48, 0xec, 0x16, - 0x08, 0x68, 0xf5, 0x63, 0x98, 0x1b, 0xed, 0x12, 0x13, 0x66, 0xed, 0x8e, 0x86, 0x59, 0x6f, 0xbd, - 0x90, 0x46, 0x8c, 0xc8, 0x40, 0x89, 0xde, 0x42, 0x24, 0x31, 0xf0, 0x78, 0x94, 0x81, 0xdf, 0x7a, - 0x51, 0xa9, 0xc8, 0x3c, 0x7c, 0x07, 0xca, 0xe3, 0xc7, 0x39, 0x86, 0x83, 0x87, 0xa3, 0x1c, 0xfc, - 0xe6, 0x8b, 0x59, 0x0d, 0x69, 0x7e, 0x9e, 0xcc, 0xba, 0x08, 0x17, 0xee, 0xd1, 0xd7, 0x28, 0xf7, - 0x70, 0x60, 0x98, 0x51, 0x1e, 0x56, 0xfd, 0x17, 0x05, 0x96, 0xc6, 0x31, 0xdc, 0x84, 0x1a, 0x90, - 0xef, 0x73, 0x18, 0x57, 0x97, 0xaf, 0x25, 0xb0, 0x15, 0x4f, 0xa4, 0x2e, 0x00, 0xb2, 0x7a, 0x84, - 0x64, 0xab, 0xbf, 0x0d, 0xa5, 0x91, 0x0e, 0x31, 0x92, 0x79, 0x7d, 0x54, 0x32, 0x72, 0x74, 0x35, - 0x0c, 0x2c, 0xbb, 0xce, 0x1f, 0xd9, 0x84, 0x13, 0x4b, 0x41, 0xdd, 0x0f, 0x53, 0x70, 0x7e, 0xcb, - 0x39, 0x18, 0xe2, 0x21, 0xa6, 0xd7, 0xa2, 0xb8, 0xdb, 0x7e, 0x95, 0xc9, 0x1c, 0x76, 0x6f, 0x86, - 0x7e, 0x1c, 0x6b, 0xa0, 0x6f, 0x4a, 0x2e, 0x17, 0xad, 0xbb, 0x34, 0x37, 0x4f, 0x8e, 0x6b, 0x39, - 0xca, 0x15, 0x9d, 0xf3, 0xe6, 0x54, 0x73, 0xf2, 0x71, 0x91, 0xdf, 0x76, 0x0d, 0x16, 0xfc, 0x7d, - 0x6b, 0xa0, 0xfb, 0x7b, 0xee, 0xd0, 0x36, 0x75, 0xc6, 0x01, 0x4f, 0x03, 0x11, 0xc4, 0x2e, 0x85, - 0x3f, 0x20, 0x60, 0xf5, 0x2f, 0x52, 0xb0, 0x38, 0x2a, 0x15, 0xbe, 0xdf, 0x0f, 0x22, 0x6f, 0x86, - 0x6d, 0xf7, 0x1b, 0x49, 0x45, 0x9c, 0x18, 0x0a, 0x75, 0xf1, 0x46, 0x22, 0x74, 0x83, 0xfe, 0x41, - 0x81, 0x1c, 0x07, 0xfe, 0x4a, 0xa5, 0xfe, 0xd6, 0x98, 0x33, 0x7a, 0x35, 0xa9, 0xf0, 0xe7, 0x19, - 0x5d, 0x4c, 0x5d, 0x2d, 0xe1, 0x7a, 0x46, 0x51, 0x51, 0x5a, 0x8a, 0x8a, 0xd4, 0x0b, 0x70, 0x7e, - 0x73, 0xcf, 0xf0, 0x82, 0x4d, 0xf6, 0xc2, 0x4b, 0x9c, 0x98, 0x27, 0xb0, 0x38, 0x0a, 0xe6, 0xe2, - 0x6b, 0x42, 0x8e, 0xbf, 0x05, 0xe3, 0xe2, 0x53, 0x25, 0x26, 0x02, 0xbf, 0x2e, 0x1e, 0x8a, 0xd1, - 0xc1, 0xbb, 0xac, 0x04, 0x1f, 0x3e, 0x4b, 0x61, 0xb8, 0x6b, 0xef, 0xc0, 0x52, 0xfc, 0x7b, 0x0b, - 0x34, 0x0b, 0xb9, 0x47, 0xf7, 0xdf, 0xb9, 0xff, 0xee, 0xfb, 0xf7, 0xcb, 0xe7, 0x48, 0x63, 0x73, - 0xe7, 0xd1, 0xee, 0xc3, 0x2d, 0xad, 0xac, 0xa0, 0x22, 0xe4, 0x5b, 0x1b, 0x0f, 0x37, 0x9a, 0x1b, - 0xbb, 0x5b, 0xe5, 0x14, 0x2a, 0x40, 0xe6, 0xe1, 0x46, 0x73, 0x67, 0xab, 0x9c, 0x5e, 0xff, 0x41, - 0x15, 0x32, 0x1b, 0x66, 0xdf, 0x72, 0x50, 0x00, 0x19, 0x5a, 0x7d, 0x44, 0x2f, 0x3f, 0xbf, 0x3e, - 0x49, 0x17, 0x59, 0x5d, 0x9b, 0xb4, 0x90, 0xa9, 0x56, 0xbe, 0xf7, 0xcf, 0xff, 0xf1, 0xc3, 0x14, - 0x42, 0xe5, 0x86, 0x4e, 0xdf, 0xfb, 0x35, 0x0e, 0xaf, 0x37, 0x68, 0x41, 0x13, 0xfd, 0xbe, 0x02, - 0x85, 0xf0, 0x25, 0x1a, 0x7a, 0x65, 0x82, 0x17, 0x60, 0xe1, 0xf4, 0xaf, 0x4e, 0xd6, 0x99, 0xb3, - 0x70, 0x99, 0xb2, 0xb0, 0x84, 0x16, 0x25, 0x16, 0xc2, 0xc7, 0x6d, 0xe8, 0x4f, 0x15, 0x98, 0x1f, - 0x7b, 0x62, 0x86, 0xae, 0x4f, 0xf3, 0x1c, 0x8d, 0xb1, 0xb4, 0x3e, 0xfd, 0x0b, 0x36, 0xf5, 0x65, - 0xca, 0xd8, 0x4b, 0xa8, 0x16, 0xc7, 0x58, 0xe3, 0x63, 0xf1, 0xf9, 0x6d, 0xf4, 0x97, 0x0a, 0x14, - 0xe5, 0x57, 0x46, 0xa8, 0x3e, 0xf1, 0x73, 0x24, 0xc6, 0x5d, 0x63, 0xca, 0xe7, 0x4b, 0xea, 0x2d, - 0xca, 0xda, 0x6b, 0xa8, 0xfe, 0x1c, 0xd6, 0x1a, 0xf4, 0x2a, 0xf7, 0x1b, 0x1f, 0xd3, 0x5f, 0xca, - 0x29, 0x44, 0xa5, 0x73, 0xf4, 0xea, 0x84, 0x15, 0x76, 0xc6, 0xe5, 0x74, 0xf5, 0x78, 0xf5, 0x0e, - 0xe5, 0xf1, 0x16, 0xba, 0x39, 0x1d, 0x8f, 0x0d, 0xf6, 0x34, 0xe3, 0x47, 0x0a, 0x94, 0x46, 0x5e, - 0x23, 0xa0, 0x24, 0x21, 0xc5, 0x3d, 0x68, 0xa8, 0xbe, 0x36, 0xf9, 0x00, 0xce, 0xf2, 0x2a, 0x65, - 0xb9, 0x8a, 0x2a, 0x12, 0xcb, 0x8e, 0xeb, 0x30, 0x06, 0x29, 0x13, 0x1f, 0x42, 0x96, 0x15, 0xc0, - 0xd1, 0xda, 0x04, 0x35, 0x72, 0xc6, 0xc7, 0x97, 0x27, 0xae, 0xa6, 0xab, 0xcb, 0x94, 0x81, 0xf3, - 0x68, 0x41, 0x62, 0x80, 0x5b, 0x39, 0x72, 0x1e, 0xc3, 0xe2, 0x6c, 0xe2, 0x79, 0x1c, 0x2f, 0x17, - 0x27, 0x9e, 0xc7, 0xd3, 0xf5, 0x5e, 0x7e, 0x1e, 0x55, 0x99, 0x87, 0xa1, 0x45, 0xb6, 0xeb, 0xb6, - 0x72, 0x0d, 0x7d, 0x57, 0x81, 0xc2, 0xf6, 0x44, 0x6c, 0x6c, 0x4f, 0xc3, 0xc6, 0xa9, 0x6a, 0x68, - 0xac, 0x28, 0x18, 0x1b, 0xe8, 0x77, 0x21, 0xc7, 0x6b, 0x9b, 0x28, 0x49, 0xb6, 0xa3, 0xb5, 0xd3, - 0xea, 0xb5, 0x49, 0xba, 0xf2, 0xc9, 0xab, 0x74, 0xf2, 0x45, 0x84, 0xa4, 0xc9, 0x79, 0x0d, 0x15, - 0xfd, 0x9e, 0x02, 0x79, 0x51, 0x2f, 0x40, 0xd7, 0x26, 0x2a, 0x2a, 0x30, 0x06, 0x5e, 0x99, 0xa2, - 0x00, 0xa1, 0x5e, 0xa2, 0x1c, 0x5c, 0x40, 0xe7, 0x25, 0x0e, 0x7c, 0x31, 0xeb, 0xf7, 0x15, 0xc8, - 0xb2, 0x32, 0x43, 0xa2, 0x1a, 0x8e, 0x14, 0x2c, 0x12, 0xd5, 0x70, 0xac, 0x66, 0xf1, 0x45, 0x3a, - 0xf9, 0xca, 0x88, 0xec, 0xf7, 0x68, 0x97, 0x27, 0x05, 0x94, 0xe3, 0x9f, 0x54, 0x10, 0x22, 0x27, - 0x9f, 0x28, 0x88, 0xb1, 0xfa, 0x47, 0xa2, 0x20, 0xc6, 0x8b, 0x12, 0xb1, 0x82, 0x10, 0x35, 0x0f, - 0x34, 0x80, 0x99, 0xbb, 0x6e, 0xc7, 0x47, 0x5f, 0x7a, 0x6e, 0xbe, 0x9d, 0xcd, 0xfc, 0xf2, 0x84, - 0x79, 0x79, 0xf5, 0x22, 0x9d, 0x75, 0x01, 0xcd, 0x4b, 0xb3, 0x7e, 0x8b, 0xcc, 0x44, 0x8e, 0x61, - 0x98, 0x52, 0x4c, 0xd4, 0xff, 0xf1, 0x74, 0x66, 0xa2, 0xfe, 0x9f, 0xca, 0x52, 0xc6, 0x5e, 0x8b, - 0x61, 0xa6, 0x92, 0xb2, 0x11, 0xa6, 0x59, 0x12, 0xd9, 0x18, 0xcf, 0x0d, 0x25, 0xb2, 0x71, 0x2a, - 0x3d, 0x14, 0xcb, 0x06, 0x4d, 0x0f, 0x0d, 0xc8, 0xc4, 0xdf, 0x84, 0x0c, 0xad, 0x83, 0x27, 0xba, - 0x26, 0xf2, 0x93, 0x80, 0x44, 0xd7, 0x64, 0xa4, 0x60, 0xaf, 0x9e, 0x7b, 0x4d, 0x41, 0xcf, 0xa0, - 0x28, 0x97, 0x7d, 0x13, 0x2f, 0xd6, 0x98, 0x3a, 0x78, 0xf5, 0xf5, 0x33, 0xd5, 0x93, 0xd5, 0x73, - 0xe4, 0x84, 0xa1, 0xd3, 0x1d, 0xd0, 0xcd, 0x29, 0xe9, 0xbd, 0x20, 0x17, 0x7f, 0xa6, 0x40, 0x5e, - 0xa4, 0x04, 0x13, 0x4f, 0xd8, 0x58, 0x92, 0x38, 0xf1, 0x84, 0x8d, 0xe7, 0x18, 0xd5, 0xb7, 0xe8, - 0x16, 0xbf, 0x31, 0x72, 0xc2, 0x68, 0x68, 0x62, 0xbb, 0xbd, 0x27, 0xab, 0x68, 0x25, 0x06, 0xdc, - 0xf8, 0x58, 0x04, 0x44, 0xdf, 0x26, 0x1e, 0x5a, 0x79, 0x3c, 0xaa, 0x45, 0xeb, 0x53, 0x85, 0xc0, - 0x8c, 0xe9, 0x1b, 0x67, 0x08, 0x9b, 0x43, 0x53, 0x75, 0x79, 0xcc, 0xcb, 0xd0, 0x4d, 0x99, 0x9d, - 0x1f, 0x2b, 0xb0, 0xb0, 0x61, 0xdb, 0xa3, 0x51, 0x2e, 0x7a, 0x6d, 0x8a, 0x80, 0x98, 0xb1, 0x78, - 0x7d, 0xea, 0x10, 0x5a, 0x7d, 0x89, 0x32, 0x78, 0x09, 0x2d, 0x4b, 0x0c, 0xb2, 0xa8, 0x57, 0xc4, - 0xd1, 0xe8, 0x13, 0x05, 0x8a, 0x72, 0x50, 0x92, 0xa8, 0xe6, 0x31, 0x41, 0x4d, 0xa2, 0xff, 0x18, - 0x17, 0xed, 0xa8, 0x35, 0xca, 0xd4, 0x32, 0xba, 0x28, 0xdf, 0x6f, 0xa4, 0x23, 0x0f, 0x65, 0x88, - 0xfb, 0x55, 0x94, 0x83, 0xc4, 0x44, 0x96, 0x62, 0xa2, 0xf4, 0x44, 0x96, 0xe2, 0xa2, 0x4f, 0xf5, - 0x0a, 0x65, 0xe9, 0x0b, 0xaa, 0xec, 0x7b, 0x61, 0xd6, 0x51, 0xa7, 0xca, 0x76, 0x5b, 0xb9, 0xd6, - 0xbc, 0xf6, 0xe9, 0xbf, 0xaf, 0x9c, 0xfb, 0xf4, 0x64, 0x45, 0xf9, 0xd9, 0xc9, 0x8a, 0xf2, 0xf3, - 0x93, 0x15, 0xe5, 0xdf, 0x4e, 0x56, 0x94, 0x4f, 0x3e, 0x5b, 0x39, 0xf7, 0xb3, 0xcf, 0x56, 0xce, - 0xfd, 0xfc, 0xb3, 0x95, 0x73, 0x4f, 0xf2, 0x62, 0x96, 0x4e, 0x96, 0x26, 0xec, 0x6e, 0xfc, 0x7f, - 0x00, 0x00, 0x00, 0xff, 0xff, 0x78, 0x8e, 0x1f, 0x5f, 0xd9, 0x34, 0x00, 0x00, +func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_52a8691f76c4ab9c) } + +var fileDescriptor_admin_52a8691f76c4ab9c = []byte{ + // 4119 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7a, 0xcd, 0x73, 0x1b, 0xc9, + 0x75, 0xb8, 0x06, 0x20, 0xbe, 0x1e, 0x01, 0x12, 0x6c, 0x51, 0x14, 0x08, 0xc9, 0x04, 0x77, 0x64, + 0x79, 0x69, 0xed, 0x1a, 0x58, 0x51, 0x5a, 0xad, 0x7f, 0x5a, 0xad, 0xfd, 0x23, 0x08, 0x16, 0x17, + 0x5a, 0x4a, 0xbb, 0x1a, 0x4a, 0xda, 0xb2, 0x9c, 0xf5, 0x64, 0x80, 0x69, 0x81, 0x63, 0x0e, 0x66, + 0xc0, 0x99, 0x01, 0xb5, 0xdc, 0x8d, 0x5d, 0xb1, 0x9d, 0x4a, 0x25, 0x97, 0xd4, 0x96, 0xed, 0x9b, + 0xab, 0x52, 0x95, 0x54, 0x2a, 0xa9, 0x24, 0xa7, 0xdc, 0x92, 0x1c, 0x72, 0xde, 0xca, 0xc1, 0x76, + 0x55, 0x2e, 0xce, 0x21, 0x74, 0xc2, 0xf5, 0x21, 0x95, 0x3f, 0x20, 0x87, 0x1c, 0x92, 0x54, 0x7f, + 0xcd, 0x34, 0xc0, 0xd9, 0x11, 0x40, 0xd9, 0x39, 0x90, 0x98, 0x7e, 0xaf, 0xfb, 0xf5, 0xeb, 0xd7, + 0xaf, 0x5f, 0xbf, 0x8f, 0x86, 0x4b, 0x3e, 0xf6, 0x0e, 0xb1, 0xd7, 0x60, 0x3f, 0x83, 0x4e, 0xc3, + 0x30, 0xfb, 0x96, 0x53, 0x1f, 0x78, 0x6e, 0xe0, 0xa2, 0xe5, 0xae, 0xdb, 0xdd, 0xf7, 0x5c, 0xa3, + 0xbb, 0x57, 0x67, 0xf8, 0xba, 0xe8, 0x56, 0x5d, 0xe8, 0xba, 0xce, 0x53, 0xab, 0xd7, 0xf8, 0xc8, + 0x75, 0x30, 0xeb, 0x5d, 0x5d, 0xfa, 0xb6, 0xdb, 0xf1, 0x1b, 0xe4, 0xdf, 0xa0, 0x43, 0x7f, 0x38, + 0xfc, 0xf2, 0xf8, 0x14, 0x7e, 0x60, 0x04, 0x43, 0x81, 0x55, 0xfd, 0xc0, 0xf5, 0x8c, 0x1e, 0x6e, + 0x60, 0xa7, 0x67, 0x39, 0xe2, 0x67, 0xd0, 0x69, 0xf4, 0x0f, 0xbb, 0x5d, 0xde, 0x67, 0x55, 0xf4, + 0xe1, 0xbf, 0x83, 0x4e, 0xc3, 0xb6, 0x0e, 0xb1, 0x83, 0x7d, 0x41, 0xe5, 0x52, 0x4c, 0x0f, 0xb7, + 0xc7, 0x91, 0x2b, 0x81, 0xdf, 0xe8, 0x1a, 0x81, 0x61, 0xbb, 0xbd, 0x46, 0x77, 0xcf, 0xf0, 0x02, + 0x9d, 0xb7, 0x38, 0xbe, 0x32, 0x0c, 0x2c, 0xbb, 0xd1, 0xc7, 0x81, 0x67, 0x75, 0xf9, 0x0f, 0xc7, + 0x2c, 0xf6, 0xdc, 0x9e, 0x4b, 0x3f, 0x1b, 0xe4, 0x4b, 0x2c, 0xa8, 0xe7, 0xba, 0x3d, 0x1b, 0x37, + 0x8c, 0x81, 0xd5, 0x30, 0x1c, 0xc7, 0x0d, 0x8c, 0xc0, 0x72, 0x1d, 0xc1, 0x4a, 0x8d, 0x63, 0x69, + 0xab, 0x33, 0x7c, 0xda, 0x08, 0xac, 0x3e, 0xf6, 0x03, 0xa3, 0x3f, 0x60, 0x1d, 0x54, 0x04, 0xe5, + 0x96, 0x11, 0x18, 0x1d, 0xc3, 0xc7, 0xbe, 0x86, 0x0f, 0x86, 0xd8, 0x0f, 0xd4, 0xeb, 0xb0, 0x20, + 0xc1, 0xfc, 0x81, 0xeb, 0xf8, 0x18, 0x5d, 0x86, 0x82, 0x29, 0x80, 0x15, 0x65, 0x35, 0xbd, 0x56, + 0xd0, 0x22, 0x80, 0x7a, 0x13, 0x96, 0xc4, 0x90, 0x16, 0x0e, 0x0c, 0xcb, 0x16, 0xc4, 0x50, 0x15, + 0xf2, 0xa2, 0x5b, 0x45, 0x59, 0x55, 0xd6, 0x0a, 0x5a, 0xd8, 0x56, 0xff, 0x2a, 0x0d, 0x17, 0x4f, + 0x0d, 0xe3, 0xf3, 0x3d, 0x86, 0x6c, 0xcf, 0x33, 0x9c, 0x80, 0x4d, 0x36, 0xbb, 0xfe, 0xd5, 0xfa, + 0xe7, 0xee, 0x7f, 0xfd, 0x73, 0x68, 0xd4, 0xb7, 0x09, 0x81, 0xe6, 0xcc, 0xa7, 0xc7, 0xb5, 0x73, + 0x1a, 0xa7, 0x86, 0x6a, 0x30, 0x1b, 0x18, 0x1d, 0x1b, 0xeb, 0x8e, 0xd1, 0xc7, 0x7e, 0x25, 0x45, + 0x57, 0x02, 0x14, 0x74, 0x9f, 0x40, 0xd0, 0xeb, 0x50, 0x32, 0xb1, 0xdf, 0xf5, 0xac, 0x41, 0xe0, + 0x7a, 0xba, 0x65, 0x56, 0xd2, 0xab, 0xca, 0x5a, 0xba, 0x59, 0x3e, 0x39, 0xae, 0x15, 0x5b, 0x21, + 0xa2, 0xdd, 0xd2, 0x8a, 0x51, 0xb7, 0xb6, 0x89, 0x36, 0x61, 0x96, 0xa8, 0x9f, 0xce, 0x54, 0xb1, + 0x32, 0xb3, 0xaa, 0xac, 0xcd, 0xae, 0x5f, 0x96, 0x98, 0x66, 0x88, 0xfa, 0x13, 0xd7, 0xc1, 0x9b, + 0xf4, 0x93, 0x33, 0x06, 0x1f, 0x85, 0x10, 0xf4, 0x01, 0x2c, 0x48, 0x44, 0x74, 0x1b, 0x1f, 0x62, + 0xbb, 0x92, 0x59, 0x55, 0xd6, 0xe6, 0xd6, 0xaf, 0x27, 0xac, 0x3f, 0xa2, 0x39, 0xf4, 0xe8, 0xf6, + 0xef, 0x90, 0x81, 0xda, 0x7c, 0x44, 0x99, 0x02, 0xaa, 0x6f, 0x42, 0x86, 0x8a, 0x04, 0x21, 0x98, + 0x19, 0xfa, 0xd8, 0xe3, 0x1b, 0x42, 0xbf, 0xd1, 0x0a, 0xc0, 0xc0, 0xb3, 0x0e, 0x2d, 0x1b, 0xf7, + 0x22, 0xb9, 0x44, 0x10, 0x75, 0x1b, 0xce, 0x3f, 0x24, 0x52, 0x9a, 0x7c, 0x7f, 0xd1, 0x22, 0x64, + 0xa8, 0x60, 0x2b, 0x29, 0x8a, 0x60, 0x0d, 0xf5, 0xbf, 0x73, 0xb0, 0x38, 0x4a, 0x89, 0x6f, 0xf9, + 0xee, 0xd8, 0x96, 0xbf, 0x9e, 0xb0, 0xe4, 0x38, 0x02, 0xb1, 0xfb, 0xfd, 0x18, 0x72, 0x5d, 0xd7, + 0x1e, 0xf6, 0x1d, 0xb6, 0xa6, 0xd9, 0xf5, 0x5b, 0xd3, 0x52, 0xdd, 0xa4, 0xc3, 0x39, 0x59, 0x41, + 0x0c, 0x3d, 0x82, 0x9c, 0xe5, 0x98, 0xf8, 0x43, 0xec, 0x57, 0xd2, 0x67, 0xe3, 0xb6, 0x4d, 0x86, + 0x0b, 0xb2, 0x9c, 0x16, 0x51, 0x4f, 0xcf, 0x70, 0x7a, 0x44, 0x05, 0x86, 0x4e, 0x40, 0xd5, 0x28, + 0xad, 0x01, 0x05, 0x6d, 0x12, 0x08, 0xba, 0x09, 0x4b, 0x5d, 0x0f, 0x1b, 0x01, 0xd6, 0x99, 0x1a, + 0x13, 0xfb, 0x85, 0xfb, 0xd8, 0x09, 0xa8, 0x9e, 0x14, 0xb4, 0x45, 0x86, 0xa5, 0x33, 0xee, 0x0a, + 0xdc, 0xb8, 0x76, 0x66, 0x7f, 0x7d, 0xda, 0x99, 0xfb, 0x75, 0x69, 0xe7, 0xe9, 0x83, 0x97, 0x9f, + 0xe4, 0xe0, 0xbd, 0x90, 0x52, 0x57, 0xff, 0x41, 0x81, 0x2c, 0xdb, 0x5f, 0x32, 0x9c, 0x98, 0x04, + 0x31, 0x9c, 0x7c, 0x13, 0x58, 0x70, 0x34, 0x10, 0xfa, 0x4b, 0xbf, 0x89, 0xc2, 0x3b, 0x43, 0xdb, + 0xa6, 0x7a, 0x4d, 0x4c, 0x43, 0x5e, 0x0b, 0xdb, 0xe8, 0x0a, 0x59, 0xc2, 0x53, 0x63, 0x68, 0x07, + 0xfa, 0xa1, 0x61, 0x0f, 0x31, 0xdd, 0xbf, 0x02, 0x61, 0x98, 0x02, 0x1f, 0x13, 0x18, 0xba, 0x01, + 0x17, 0x7a, 0xd8, 0xc1, 0x4c, 0x16, 0x3a, 0xfe, 0x70, 0xe0, 0x61, 0xdf, 0xb7, 0x5c, 0x47, 0x6c, + 0x60, 0x84, 0xdc, 0x0a, 0x71, 0x68, 0x09, 0xb2, 0x7b, 0x96, 0x69, 0x62, 0x87, 0xee, 0x5d, 0x5e, + 0xe3, 0xad, 0xea, 0xdf, 0x28, 0x90, 0xa1, 0x8a, 0x14, 0xcb, 0xff, 0x12, 0x64, 0x87, 0x8e, 0x75, + 0x30, 0x64, 0x2b, 0xc8, 0x6b, 0xbc, 0x85, 0xca, 0x90, 0xf6, 0xf1, 0x01, 0xb3, 0x6c, 0x1a, 0xf9, + 0x24, 0x3d, 0x99, 0x66, 0x73, 0x96, 0x79, 0x8b, 0x9a, 0x7d, 0xcb, 0xc3, 0xdd, 0x20, 0x62, 0x30, + 0x02, 0xa0, 0x0a, 0xe4, 0xc8, 0x1d, 0x67, 0x39, 0x3d, 0xce, 0x96, 0x68, 0x12, 0x29, 0x59, 0xfd, + 0x81, 0x6d, 0x75, 0xad, 0x80, 0xaa, 0x48, 0x5e, 0x0b, 0xdb, 0xea, 0x16, 0x2c, 0x84, 0xea, 0xf9, + 0x02, 0x76, 0xe4, 0x6f, 0xd3, 0x80, 0x64, 0x3a, 0xdc, 0x8a, 0x8c, 0x9d, 0x20, 0xe5, 0xd4, 0x09, + 0xba, 0x02, 0x25, 0x0f, 0x13, 0x56, 0x0c, 0xde, 0x25, 0x45, 0xbb, 0x14, 0x39, 0x90, 0x75, 0xfa, + 0x02, 0x80, 0xe3, 0x9a, 0x82, 0x08, 0x13, 0x54, 0x81, 0x40, 0x18, 0xfa, 0x6d, 0xc8, 0x90, 0x83, + 0xe7, 0x73, 0x3b, 0xff, 0xaa, 0xac, 0xfe, 0xec, 0xd2, 0xaf, 0x33, 0xdf, 0xa1, 0x2e, 0x5c, 0x88, + 0xfa, 0xbd, 0xc7, 0x9b, 0x9b, 0x94, 0x53, 0x7e, 0xb2, 0x18, 0x01, 0x64, 0x42, 0xa9, 0x6f, 0xf9, + 0xbe, 0xe5, 0xf4, 0x74, 0x42, 0xde, 0xaf, 0x64, 0xa8, 0x35, 0xf9, 0x7f, 0xcf, 0xb3, 0x26, 0x23, + 0x8b, 0xae, 0xdf, 0x63, 0x24, 0xee, 0xbb, 0x26, 0xe6, 0xe4, 0x8b, 0xfd, 0x08, 0xe4, 0x13, 0xab, + 0x61, 0x0c, 0x06, 0x9e, 0xfb, 0xa1, 0xd5, 0x27, 0xa6, 0xc3, 0xb4, 0xfc, 0x7d, 0xbd, 0x73, 0x14, + 0x60, 0x9f, 0xee, 0xdb, 0x8c, 0xb6, 0x28, 0x61, 0x5b, 0x96, 0xbf, 0xdf, 0x24, 0xb8, 0xea, 0xfb, + 0x30, 0x2b, 0x11, 0x46, 0x57, 0x20, 0x47, 0x65, 0x62, 0x99, 0x6c, 0x87, 0x9a, 0x70, 0x72, 0x5c, + 0xcb, 0x12, 0x54, 0xbb, 0xa5, 0x65, 0x09, 0xaa, 0x6d, 0x12, 0xe9, 0x62, 0xcf, 0x73, 0x3d, 0xbd, + 0x8f, 0x7d, 0xdf, 0xe8, 0x89, 0x3d, 0x2b, 0x52, 0xe0, 0x3d, 0x06, 0x53, 0x97, 0x60, 0xf1, 0xbe, + 0xeb, 0x9c, 0x52, 0x02, 0xf5, 0x67, 0x0a, 0x5c, 0x18, 0x43, 0xf0, 0x5d, 0xfd, 0x06, 0x2c, 0x10, + 0xd7, 0x45, 0xf7, 0xb1, 0x67, 0x61, 0x5f, 0x67, 0xc2, 0x57, 0xa8, 0xf0, 0xbf, 0x32, 0x95, 0xa8, + 0xb4, 0x79, 0x42, 0x67, 0x97, 0x92, 0xa1, 0x08, 0xf4, 0x4d, 0x40, 0x96, 0x13, 0x60, 0xcf, 0x31, + 0x6c, 0x7d, 0xe8, 0x63, 0x4e, 0x3b, 0x75, 0x16, 0xda, 0x65, 0x41, 0xe8, 0x91, 0xcf, 0x30, 0xea, + 0x1c, 0x14, 0x1f, 0xf9, 0xd8, 0x0b, 0x57, 0xf8, 0x5d, 0x28, 0xf1, 0x36, 0x5f, 0x58, 0x1b, 0x32, + 0xc4, 0x52, 0x89, 0x3b, 0x2f, 0x69, 0xc2, 0x91, 0x81, 0xb4, 0x25, 0x54, 0x89, 0x52, 0xa8, 0xaa, + 0x30, 0x43, 0x80, 0xe4, 0x28, 0x11, 0x80, 0x64, 0x0d, 0xc2, 0xb6, 0xfa, 0x03, 0x05, 0x4a, 0x5b, + 0x87, 0xd8, 0x89, 0x0e, 0x9e, 0xb0, 0x71, 0x8a, 0x64, 0xe3, 0x2e, 0x41, 0x21, 0x30, 0xbc, 0x1e, + 0x0e, 0xc8, 0x5e, 0xb3, 0xe3, 0x91, 0x67, 0x80, 0xb6, 0x49, 0x4e, 0xa3, 0x6d, 0xf5, 0x2d, 0x76, + 0x2a, 0x32, 0x1a, 0x6b, 0xa0, 0x57, 0x60, 0x61, 0xe8, 0x78, 0xd8, 0x34, 0xba, 0x01, 0x36, 0x75, + 0x4c, 0xa7, 0xa0, 0xa7, 0x23, 0xaf, 0x95, 0x23, 0x04, 0x9b, 0x5a, 0xfd, 0x8f, 0x14, 0xcc, 0x09, + 0x2e, 0xb8, 0x1c, 0xee, 0x41, 0x96, 0x0f, 0x62, 0x82, 0x68, 0x24, 0x08, 0x62, 0x74, 0x28, 0x6b, + 0x8a, 0x6b, 0x9f, 0x11, 0xa9, 0xfe, 0x41, 0x0a, 0x32, 0x14, 0x8e, 0x9a, 0x50, 0x08, 0x9d, 0x5e, + 0xae, 0x31, 0xd5, 0x3a, 0x73, 0x8b, 0xeb, 0xc2, 0x2d, 0xae, 0x3f, 0x14, 0x3d, 0x9a, 0x79, 0x42, + 0xe6, 0x93, 0x5f, 0xd6, 0x14, 0x2d, 0x1a, 0x46, 0xac, 0x01, 0xa5, 0xab, 0x4b, 0xb7, 0x41, 0x81, + 0x42, 0x1e, 0x12, 0x71, 0x7d, 0x59, 0x16, 0x17, 0x73, 0x17, 0x8b, 0x27, 0xc7, 0xb5, 0xfc, 0x43, + 0x26, 0xb2, 0x96, 0x24, 0xbc, 0x75, 0x20, 0x76, 0xc6, 0xf5, 0x02, 0x72, 0xe0, 0x2d, 0x93, 0x5d, + 0xf0, 0xcd, 0xf9, 0x93, 0xe3, 0xda, 0xac, 0x26, 0xe0, 0xed, 0x96, 0x36, 0x1b, 0x76, 0x6a, 0x9b, + 0x64, 0x87, 0x2c, 0xe7, 0xa9, 0xcb, 0xcd, 0x2f, 0xfd, 0x26, 0x53, 0x32, 0x5b, 0x4e, 0x88, 0x90, + 0x33, 0x5c, 0x64, 0x53, 0x3e, 0xa2, 0x40, 0x32, 0x25, 0x43, 0xb7, 0x4d, 0xf5, 0xaf, 0x15, 0x28, + 0xef, 0xe2, 0xe0, 0x51, 0x9b, 0xb8, 0xc9, 0x62, 0xd7, 0xbf, 0x01, 0xb0, 0x8f, 0x8f, 0xd8, 0x2d, + 0x25, 0x44, 0x7e, 0x3b, 0x41, 0xe4, 0xe3, 0x04, 0xea, 0xef, 0xe0, 0x23, 0x7a, 0x9d, 0xf9, 0x5b, + 0x4e, 0xe0, 0x1d, 0x69, 0x85, 0x7d, 0xd1, 0xae, 0xde, 0x81, 0xb9, 0x51, 0x24, 0xb9, 0x6e, 0xf6, + 0xf1, 0x11, 0xd7, 0x30, 0xf2, 0x49, 0x74, 0x88, 0x5d, 0x90, 0x44, 0x96, 0x45, 0x8d, 0x35, 0x6e, + 0xa7, 0xbe, 0xaa, 0xa8, 0xe7, 0x61, 0x41, 0x9a, 0x8b, 0xed, 0xb0, 0xfa, 0x25, 0x28, 0x6f, 0x8f, + 0xaf, 0x00, 0xc1, 0xcc, 0x3e, 0x3e, 0x12, 0xb1, 0x08, 0xfd, 0x56, 0x7f, 0x96, 0x82, 0x85, 0xed, + 0xf1, 0xd1, 0xe8, 0xb7, 0x63, 0xd6, 0xfa, 0x66, 0xc2, 0x5a, 0x4f, 0x51, 0x18, 0x5b, 0x2c, 0x57, + 0x35, 0x69, 0xc9, 0x4f, 0x21, 0xc3, 0xee, 0xf6, 0x70, 0x5d, 0x8a, 0xb4, 0x2e, 0xb4, 0x0d, 0x45, + 0xdb, 0xf0, 0x03, 0x7d, 0x38, 0x30, 0x8d, 0x00, 0x9b, 0xdc, 0xb6, 0x4c, 0xa6, 0x85, 0xb3, 0x64, + 0xe4, 0x23, 0x36, 0xb0, 0x3a, 0x98, 0x40, 0xb4, 0x6f, 0xcb, 0xa2, 0x9d, 0x5d, 0x5f, 0x9f, 0x6a, + 0xa1, 0x94, 0xb4, 0xbc, 0x1d, 0x65, 0x98, 0xdb, 0xb4, 0x87, 0x7e, 0x80, 0x3d, 0x61, 0xc1, 0x7e, + 0xa2, 0xc0, 0x7c, 0x08, 0xe2, 0x12, 0x7e, 0x15, 0xa0, 0xcb, 0x40, 0xd1, 0xe5, 0x50, 0x3a, 0x39, + 0xae, 0x15, 0x78, 0xc7, 0x76, 0x4b, 0x2b, 0xf0, 0x0e, 0x6d, 0x93, 0x98, 0x8a, 0xe8, 0x0c, 0x60, + 0x87, 0x98, 0x51, 0x93, 0x3b, 0x28, 0xe5, 0x10, 0xb1, 0xc5, 0xe0, 0xe8, 0x2b, 0x80, 0x30, 0x31, + 0xaa, 0x03, 0xcf, 0xf2, 0x71, 0xd8, 0x9b, 0x39, 0x5e, 0x0b, 0x11, 0x86, 0x77, 0x57, 0xff, 0x50, + 0x81, 0x62, 0xcb, 0x33, 0x2c, 0x47, 0xa8, 0xc9, 0x1d, 0xa8, 0x9a, 0x78, 0xe0, 0xe1, 0x2e, 0x11, + 0xa0, 0x3e, 0xf0, 0xdc, 0x0e, 0xd6, 0x2d, 0xc7, 0xb4, 0xba, 0x46, 0xe0, 0x7a, 0x54, 0x19, 0x32, + 0x5a, 0x25, 0xea, 0xf1, 0x1e, 0xe9, 0xd0, 0x16, 0x78, 0x62, 0x4a, 0xfd, 0xbd, 0x61, 0x60, 0xba, + 0xcf, 0x1c, 0xe1, 0xec, 0x89, 0x36, 0x5a, 0x86, 0xbc, 0xe9, 0xea, 0x26, 0x99, 0x8c, 0x1b, 0xba, + 0x9c, 0xe9, 0xd2, 0xb9, 0xef, 0xce, 0xe4, 0x53, 0xe5, 0xb4, 0xfa, 0x2b, 0x05, 0x4a, 0x9c, 0x17, + 0x2e, 0xa7, 0x5b, 0x70, 0x51, 0x62, 0x86, 0x0e, 0xd5, 0x59, 0x02, 0x82, 0x73, 0x72, 0x21, 0x42, + 0xd3, 0x91, 0xbb, 0x14, 0x49, 0x7c, 0x1a, 0xcb, 0x67, 0xfd, 0x89, 0xaf, 0xc5, 0x64, 0x05, 0x96, + 0xdf, 0xe2, 0x10, 0x74, 0x1b, 0x96, 0x19, 0x35, 0x0f, 0xf7, 0x19, 0x48, 0x5a, 0x64, 0x9a, 0x5e, + 0xf1, 0x17, 0x4d, 0xc6, 0x0a, 0xc7, 0x47, 0x6b, 0xfc, 0x1a, 0x5c, 0x1a, 0x1f, 0x2b, 0x1c, 0x6c, + 0xe2, 0xf4, 0x31, 0x7f, 0x70, 0x79, 0x74, 0x74, 0x2b, 0xea, 0xa0, 0x7e, 0x04, 0xcb, 0x2d, 0xdc, + 0x75, 0xfb, 0xd4, 0xe1, 0x70, 0x39, 0xcb, 0x42, 0xfc, 0x1f, 0x40, 0x9e, 0xfb, 0x0c, 0x7c, 0x89, + 0xcd, 0xe6, 0xc9, 0x71, 0x2d, 0xc7, 0x9c, 0x06, 0xff, 0xbf, 0x8e, 0x6b, 0x37, 0x7a, 0x56, 0xb0, + 0x37, 0xec, 0xd4, 0xbb, 0x6e, 0xbf, 0x11, 0x6a, 0xaa, 0xd9, 0x89, 0xbe, 0x1b, 0x83, 0xfd, 0x5e, + 0x83, 0x7e, 0x0d, 0x3a, 0x75, 0xee, 0x6c, 0xe4, 0x98, 0xb3, 0xe1, 0xab, 0x7f, 0xac, 0xc0, 0x79, + 0x79, 0xf2, 0xff, 0x9b, 0x69, 0xd1, 0x1a, 0xcc, 0x9b, 0xd2, 0xac, 0xd1, 0x9e, 0x8c, 0x83, 0xd5, + 0xcf, 0x52, 0x50, 0x8d, 0x93, 0x0e, 0x57, 0x88, 0x27, 0x90, 0xe5, 0xfb, 0xcf, 0x82, 0xd3, 0x3b, + 0x49, 0x59, 0x8e, 0xcf, 0x25, 0x53, 0x67, 0x4d, 0x71, 0x05, 0x32, 0x8a, 0xd5, 0x7f, 0x57, 0x20, + 0xcb, 0xf5, 0xe7, 0xc9, 0xa8, 0xe7, 0x96, 0x69, 0x6e, 0x44, 0x9e, 0xdb, 0x59, 0x85, 0x21, 0x1c, + 0xbe, 0x8b, 0x90, 0xb3, 0x7c, 0xdd, 0xb6, 0x0e, 0xc3, 0x20, 0xc3, 0xf2, 0x77, 0xac, 0x43, 0x7c, + 0xda, 0xcf, 0x4e, 0xc7, 0xf8, 0xd9, 0x31, 0x92, 0x9c, 0x89, 0x95, 0x24, 0x0d, 0x10, 0xc4, 0x01, + 0xc8, 0xb0, 0xa3, 0x28, 0xda, 0xea, 0x55, 0x98, 0xdf, 0xc5, 0x01, 0x31, 0x1b, 0x7e, 0xd2, 0xf5, + 0xf0, 0x77, 0x29, 0x7a, 0x13, 0xf2, 0x7e, 0x7c, 0x0b, 0xf4, 0xe9, 0x6f, 0xc2, 0x11, 0x02, 0xcf, + 0xbd, 0x1c, 0x76, 0x63, 0x2f, 0x87, 0x82, 0xb8, 0x1c, 0xe2, 0x62, 0xcc, 0x55, 0x98, 0x95, 0x8f, + 0x60, 0x9a, 0xa2, 0x64, 0x50, 0xd5, 0x9d, 0xe0, 0x26, 0xd8, 0x1e, 0xbd, 0x09, 0xae, 0x4f, 0xb3, + 0xa8, 0x53, 0x17, 0xc1, 0x55, 0x28, 0xbd, 0x8d, 0x0d, 0x3b, 0xd8, 0x13, 0x02, 0x5e, 0x84, 0x8c, + 0x87, 0x0d, 0x93, 0xcd, 0x98, 0xd7, 0x58, 0x83, 0xdc, 0x17, 0xa2, 0x1b, 0xbf, 0xbb, 0x17, 0x60, + 0x7e, 0x87, 0xe7, 0x47, 0xc5, 0x15, 0xf2, 0xd3, 0x14, 0x94, 0x23, 0x18, 0xdf, 0x87, 0x0d, 0x00, + 0x91, 0x47, 0x0d, 0xf7, 0xe1, 0x52, 0x4c, 0x5c, 0x25, 0x06, 0x8a, 0x04, 0x45, 0x34, 0x08, 0xfd, + 0x50, 0x81, 0x3c, 0x53, 0x7e, 0x2c, 0x0e, 0x54, 0x52, 0x1c, 0x35, 0xce, 0x02, 0x3f, 0x46, 0x62, + 0x23, 0xdf, 0x24, 0xf4, 0xbf, 0xff, 0xcb, 0xb3, 0x9d, 0x8e, 0x90, 0x8f, 0x6a, 0x07, 0x4a, 0x23, + 0x74, 0xe5, 0x8d, 0xca, 0xb0, 0x8d, 0x7a, 0x53, 0xde, 0xa8, 0xb9, 0xf5, 0xab, 0x31, 0xab, 0x26, + 0x64, 0x05, 0xbf, 0xfc, 0xf0, 0x4b, 0x9b, 0x33, 0x80, 0xd9, 0xbb, 0x6e, 0xc7, 0x97, 0xb6, 0x86, + 0x79, 0xe8, 0x8a, 0xec, 0xa1, 0x2f, 0x49, 0xb6, 0x86, 0x86, 0xf8, 0xac, 0x85, 0x6e, 0x70, 0x05, + 0x4c, 0xd3, 0xc9, 0x6b, 0xf2, 0xe4, 0x07, 0x76, 0x9d, 0xe6, 0xce, 0x59, 0x1e, 0xbd, 0x4e, 0x9c, + 0x5d, 0xa6, 0xa1, 0xea, 0x4f, 0xb3, 0x50, 0x64, 0x53, 0xf2, 0xed, 0xdb, 0x82, 0x19, 0xd2, 0x8b, + 0x6f, 0xdc, 0x2b, 0x09, 0x62, 0x97, 0x87, 0x91, 0x06, 0xdf, 0x48, 0x3a, 0xbc, 0xfa, 0x9f, 0x19, + 0x48, 0xdf, 0x75, 0x3b, 0x68, 0x09, 0x52, 0xdc, 0x58, 0xa5, 0x9b, 0xd9, 0x93, 0xe3, 0x5a, 0xaa, + 0xdd, 0xd2, 0x52, 0x96, 0x79, 0xb6, 0xd3, 0x32, 0x12, 0x11, 0xcd, 0x8c, 0x46, 0x44, 0xc8, 0x85, + 0xb9, 0x91, 0xb4, 0x13, 0x8b, 0xc0, 0x4b, 0xcd, 0xb7, 0x4f, 0x8e, 0x6b, 0x25, 0x39, 0xef, 0x34, + 0xf9, 0xb5, 0xe1, 0x1f, 0xd8, 0xe4, 0xaf, 0x63, 0xf8, 0xb8, 0xde, 0x6e, 0x69, 0x25, 0x39, 0x5f, + 0xe5, 0x4b, 0xfb, 0x90, 0x1d, 0xd9, 0x87, 0xdb, 0x90, 0x63, 0xb9, 0x3b, 0x93, 0x66, 0x4c, 0x92, + 0x1d, 0xc4, 0x19, 0xea, 0x1c, 0x8a, 0x01, 0x64, 0xac, 0x1f, 0x18, 0x1e, 0x19, 0x9b, 0x9f, 0x74, + 0x2c, 0x1f, 0x80, 0xee, 0x40, 0xfe, 0xa9, 0xe5, 0x58, 0xfe, 0x1e, 0x36, 0x2b, 0x85, 0x09, 0x07, + 0x87, 0x23, 0xc8, 0xe8, 0xbe, 0x6b, 0x5a, 0x4f, 0x2d, 0x6c, 0x56, 0x60, 0xd2, 0xd1, 0x62, 0x04, + 0xf1, 0xee, 0x9e, 0x7a, 0x06, 0x4d, 0x26, 0xe9, 0x5d, 0xb7, 0x3f, 0xb0, 0x31, 0x59, 0xc2, 0xec, + 0xaa, 0xb2, 0x96, 0xd2, 0x16, 0x04, 0x66, 0x53, 0x20, 0x88, 0x62, 0xd3, 0x3c, 0x42, 0xa5, 0xc8, + 0x2c, 0x28, 0x6d, 0xa0, 0x07, 0x70, 0x7e, 0xcf, 0xea, 0xed, 0x3d, 0x33, 0x88, 0xff, 0x19, 0xc5, + 0x7a, 0xa5, 0x09, 0xb9, 0x41, 0xe1, 0xe0, 0x10, 0x43, 0x5c, 0xd4, 0x88, 0xa4, 0x89, 0xbb, 0x56, + 0xdf, 0xb0, 0x2b, 0x73, 0x74, 0xd2, 0x72, 0x88, 0x68, 0x31, 0x38, 0xba, 0x0a, 0x73, 0xde, 0xd0, + 0xa1, 0x8e, 0x13, 0xdf, 0xd8, 0x79, 0xda, 0xb3, 0xc4, 0xa1, 0xfc, 0x12, 0xbe, 0x0c, 0x85, 0x28, + 0x59, 0x5b, 0x66, 0x31, 0x64, 0x08, 0x50, 0x11, 0x94, 0x77, 0xdc, 0x2e, 0x2b, 0xde, 0x08, 0x3b, + 0xf9, 0x3f, 0x0a, 0x2c, 0x48, 0xc0, 0x30, 0x15, 0x52, 0xb0, 0x05, 0x70, 0x82, 0x4c, 0xf9, 0x29, + 0x02, 0x21, 0x44, 0x5c, 0x55, 0x21, 0xb5, 0xea, 0x1f, 0x29, 0x90, 0x17, 0x58, 0xf4, 0x12, 0x14, + 0x09, 0xc6, 0xb6, 0x82, 0x23, 0x3d, 0xba, 0x59, 0x66, 0x05, 0xec, 0x1d, 0x7c, 0x44, 0x56, 0x1e, + 0x76, 0x89, 0x2c, 0x58, 0x41, 0x2b, 0x09, 0x28, 0xbb, 0xf8, 0xaa, 0x90, 0xb7, 0x8d, 0xc0, 0x0a, + 0x86, 0x26, 0xb3, 0x32, 0x8a, 0x16, 0xb6, 0x89, 0x54, 0x6c, 0xd7, 0xe9, 0x31, 0xe4, 0x0c, 0x45, + 0x46, 0x00, 0xb5, 0x09, 0xf3, 0x9a, 0xe1, 0xf4, 0xf0, 0x8e, 0xdb, 0x13, 0xc6, 0x6d, 0x19, 0xf2, + 0x2c, 0xbf, 0x27, 0xec, 0x83, 0x96, 0xa3, 0x6d, 0x39, 0x33, 0x91, 0x92, 0xec, 0x9e, 0xfa, 0x2f, + 0x69, 0x28, 0x47, 0x44, 0xb8, 0x10, 0xdf, 0x0d, 0xd3, 0x0d, 0xec, 0x9e, 0x48, 0xba, 0x1c, 0xc7, + 0x07, 0xc7, 0x26, 0x1c, 0xfe, 0x51, 0x01, 0x78, 0xcf, 0xc3, 0x41, 0x70, 0xd4, 0x26, 0xf1, 0xf9, + 0x4b, 0x50, 0xe4, 0xd1, 0x1e, 0x75, 0xa6, 0x85, 0xf0, 0x38, 0x8c, 0x98, 0x15, 0xb2, 0x10, 0x07, + 0x3f, 0x63, 0x68, 0x26, 0xb6, 0x9c, 0x83, 0x9f, 0x51, 0xd4, 0x15, 0x28, 0x19, 0xa6, 0x89, 0x4d, + 0x9d, 0xfb, 0x4a, 0xdc, 0xa6, 0x15, 0x29, 0x50, 0x63, 0x30, 0xf4, 0x32, 0xcc, 0x7b, 0xb8, 0xef, + 0x1e, 0x4a, 0xdd, 0x98, 0x6d, 0x9b, 0xe3, 0x60, 0xd1, 0x71, 0x09, 0xb2, 0x1e, 0x36, 0xfc, 0x30, + 0x81, 0xcb, 0x5b, 0xa8, 0x02, 0x39, 0x93, 0xd5, 0x24, 0xb8, 0x25, 0x12, 0xcd, 0xea, 0x9f, 0x29, + 0x22, 0x7b, 0x72, 0x07, 0x32, 0x74, 0x81, 0x3c, 0x73, 0xb2, 0x1a, 0x73, 0x35, 0x09, 0xf1, 0xc8, + 0x52, 0x61, 0x83, 0xd0, 0x07, 0x30, 0x3b, 0xa0, 0x32, 0xd1, 0x69, 0x02, 0x83, 0xf9, 0x21, 0xb7, + 0xa6, 0x11, 0x75, 0x24, 0x52, 0x71, 0xdf, 0x0f, 0x42, 0xc8, 0xdd, 0x99, 0xbc, 0x52, 0x4e, 0xa9, + 0x6b, 0x50, 0x7e, 0x30, 0xc4, 0xde, 0xd1, 0x7b, 0xb6, 0xe1, 0x48, 0x37, 0xe0, 0x01, 0x81, 0x09, + 0x57, 0x8b, 0x36, 0xd4, 0x01, 0x2c, 0x48, 0x3d, 0xb9, 0x26, 0x7c, 0x13, 0x2e, 0x99, 0x96, 0x1f, + 0xf8, 0x07, 0xb6, 0x3e, 0xd8, 0x3b, 0xf2, 0xad, 0xae, 0x61, 0xeb, 0xb4, 0xbb, 0x3e, 0xb0, 0x0d, + 0x87, 0x07, 0xb3, 0x97, 0x4f, 0x8e, 0x6b, 0x95, 0x96, 0xe5, 0x07, 0xbb, 0x0f, 0x76, 0xde, 0xe3, + 0xbd, 0x22, 0x52, 0x15, 0x4e, 0xe0, 0x14, 0x46, 0x5d, 0x66, 0x05, 0x4e, 0x32, 0xd2, 0xb3, 0x3a, + 0xc3, 0x20, 0x0a, 0x51, 0xd4, 0xbf, 0x04, 0xa8, 0x9c, 0xc6, 0x71, 0xa6, 0x06, 0x50, 0x12, 0xd9, + 0x6f, 0x26, 0x3a, 0x76, 0xce, 0xb7, 0x9e, 0x53, 0x04, 0x8d, 0xa3, 0x15, 0x56, 0x47, 0x89, 0xc8, + 0x64, 0x17, 0xb5, 0x68, 0x4a, 0x08, 0xd4, 0x87, 0xa2, 0x54, 0xdc, 0x11, 0x45, 0xad, 0xd6, 0x59, + 0x26, 0x8c, 0x0a, 0x3e, 0x23, 0x2e, 0xf1, 0x6c, 0x54, 0xf0, 0xf1, 0xab, 0x3f, 0x56, 0x00, 0xa2, + 0x7e, 0x44, 0x45, 0x59, 0x8a, 0x8c, 0x6f, 0x18, 0x6f, 0xa1, 0xdb, 0x90, 0xe5, 0x25, 0xab, 0xd4, + 0xc4, 0x25, 0x2b, 0x3e, 0x82, 0x26, 0x25, 0x58, 0xa5, 0xca, 0x3f, 0xb0, 0xd9, 0xd1, 0xe0, 0x49, + 0x09, 0x0a, 0xdd, 0x7d, 0xb0, 0xa3, 0x15, 0x58, 0x87, 0xdd, 0x03, 0xfb, 0xee, 0x4c, 0x3e, 0x5d, + 0x9e, 0xa9, 0xfe, 0x7e, 0x1a, 0x0a, 0x34, 0xaf, 0x4b, 0x65, 0xf2, 0xa9, 0x02, 0x95, 0x91, 0x10, + 0x46, 0xef, 0x1c, 0xe9, 0x51, 0x20, 0x45, 0x04, 0xf4, 0xfe, 0x59, 0x04, 0x14, 0xce, 0x50, 0xd7, + 0xa4, 0x58, 0xa8, 0x79, 0x44, 0x7d, 0x47, 0x93, 0xc9, 0xec, 0x8d, 0xb3, 0x7a, 0x9e, 0x8b, 0x5e, + 0x0c, 0x4d, 0xf4, 0x45, 0x98, 0x93, 0x6b, 0x77, 0x61, 0x5a, 0xb7, 0x18, 0x6d, 0x4a, 0xdb, 0x44, + 0x5f, 0x07, 0x30, 0x3d, 0x77, 0x30, 0xc0, 0xa6, 0x6e, 0xb0, 0x78, 0x6d, 0x92, 0x0b, 0xb4, 0xc0, + 0xc7, 0x6c, 0x04, 0xd5, 0x6d, 0x58, 0xfe, 0xdc, 0x25, 0xc5, 0x38, 0xbe, 0x23, 0x69, 0xc0, 0xb4, + 0xe4, 0xd1, 0x56, 0xbf, 0x97, 0x82, 0xa2, 0xac, 0xb8, 0x28, 0x00, 0x56, 0xa4, 0x97, 0x8f, 0xc3, + 0xbb, 0x2f, 0x7a, 0x1c, 0xa2, 0x9d, 0x18, 0x89, 0xdd, 0x02, 0x01, 0xad, 0x7e, 0x0c, 0x73, 0xa3, + 0x5d, 0x62, 0xc2, 0xac, 0xdd, 0xd1, 0x30, 0xeb, 0xad, 0x17, 0xd2, 0x88, 0x11, 0x19, 0x28, 0xd1, + 0x43, 0x8c, 0x24, 0x06, 0x1e, 0x8f, 0x32, 0xf0, 0xff, 0x5f, 0x54, 0x2a, 0x32, 0x0f, 0xdf, 0x85, + 0xf2, 0xf8, 0x71, 0x8e, 0xe1, 0xe0, 0xe1, 0x28, 0x07, 0x5f, 0x7b, 0x31, 0xab, 0x21, 0xcd, 0xcf, + 0x33, 0x69, 0x17, 0xe1, 0xc2, 0x3d, 0xfa, 0x14, 0xe6, 0x1e, 0x0e, 0x0c, 0x33, 0x4a, 0x02, 0xab, + 0xff, 0xac, 0xc0, 0xd2, 0x38, 0x86, 0x9b, 0x50, 0x03, 0xf2, 0x7d, 0x0e, 0xe3, 0xea, 0xf2, 0xf5, + 0x04, 0xb6, 0xe2, 0x89, 0xd4, 0x05, 0x40, 0x56, 0x8f, 0x90, 0x6c, 0xf5, 0xb7, 0xa0, 0x34, 0xd2, + 0x21, 0x46, 0x32, 0xaf, 0x8f, 0x4a, 0x46, 0x8e, 0xae, 0x86, 0x81, 0x65, 0xd7, 0xf9, 0x0b, 0x9f, + 0x70, 0x62, 0x29, 0xa8, 0xfb, 0x51, 0x0a, 0xce, 0x6f, 0x39, 0x07, 0x43, 0x3c, 0xc4, 0xf4, 0x5a, + 0x14, 0x77, 0xdb, 0x6f, 0x32, 0x99, 0xc3, 0xee, 0xcd, 0xd0, 0x8f, 0x63, 0x0d, 0xf4, 0x2d, 0xc9, + 0xe5, 0xa2, 0x45, 0x9f, 0xe6, 0xe6, 0xc9, 0x71, 0x2d, 0x47, 0xb9, 0xa2, 0x73, 0xde, 0x9c, 0x6a, + 0x4e, 0x3e, 0x2e, 0xf2, 0xdb, 0xae, 0xc1, 0x82, 0xbf, 0x6f, 0x0d, 0x74, 0x7f, 0xcf, 0x1d, 0xda, + 0xa6, 0xce, 0x38, 0xe0, 0x69, 0x20, 0x82, 0xd8, 0xa5, 0xf0, 0x07, 0x04, 0xac, 0xfe, 0x79, 0x0a, + 0x16, 0x47, 0xa5, 0xc2, 0xf7, 0xfb, 0x41, 0xe4, 0xcd, 0xb0, 0xed, 0x7e, 0x23, 0xa9, 0x82, 0x14, + 0x43, 0xa1, 0x2e, 0x1e, 0x68, 0x84, 0x6e, 0xd0, 0xdf, 0x2b, 0x90, 0xe3, 0xc0, 0xdf, 0xa8, 0xd4, + 0xdf, 0x1a, 0x73, 0x46, 0xaf, 0x26, 0x55, 0x1d, 0x3d, 0xa3, 0x8b, 0xa9, 0xab, 0x25, 0x5c, 0xcf, + 0x28, 0x2a, 0x4a, 0x4b, 0x51, 0x91, 0x7a, 0x01, 0xce, 0x6f, 0xee, 0x19, 0x5e, 0xb0, 0xc9, 0x9e, + 0x97, 0x89, 0x13, 0xf3, 0x04, 0x16, 0x47, 0xc1, 0x5c, 0x7c, 0x4d, 0xc8, 0xf1, 0x87, 0x68, 0x5c, + 0x7c, 0xaa, 0xc4, 0x44, 0xe0, 0xd7, 0xc5, 0x2b, 0x35, 0x3a, 0x78, 0x97, 0xd5, 0xff, 0xc3, 0x37, + 0x31, 0x0c, 0x77, 0xed, 0x1d, 0x58, 0x8a, 0x7f, 0xec, 0x81, 0x66, 0x21, 0xf7, 0xe8, 0xfe, 0x3b, + 0xf7, 0xdf, 0x7d, 0xff, 0x7e, 0xf9, 0x1c, 0x69, 0x6c, 0xee, 0x3c, 0xda, 0x7d, 0xb8, 0xa5, 0x95, + 0x15, 0x54, 0x84, 0x7c, 0x6b, 0xe3, 0xe1, 0x46, 0x73, 0x63, 0x77, 0xab, 0x9c, 0x42, 0x05, 0xc8, + 0x3c, 0xdc, 0x68, 0xee, 0x6c, 0x95, 0xd3, 0xeb, 0x3f, 0xac, 0x42, 0x66, 0xc3, 0xec, 0x5b, 0x0e, + 0x0a, 0x20, 0x43, 0x4b, 0x9f, 0xe8, 0xe5, 0xe7, 0x17, 0x47, 0xe9, 0x22, 0xab, 0x6b, 0x93, 0x56, + 0x51, 0xd5, 0xca, 0xf7, 0xff, 0xe9, 0x57, 0x3f, 0x4a, 0x21, 0x54, 0x6e, 0xe8, 0xf4, 0xb1, 0x61, + 0xe3, 0xf0, 0x7a, 0x83, 0x56, 0x53, 0xd1, 0xef, 0x29, 0x50, 0x08, 0x9f, 0xc1, 0xa1, 0x57, 0x26, + 0x78, 0x7e, 0x16, 0x4e, 0xff, 0xea, 0x64, 0x9d, 0x39, 0x0b, 0x97, 0x29, 0x0b, 0x4b, 0x68, 0x51, + 0x62, 0x21, 0x7c, 0x59, 0x87, 0xfe, 0x44, 0x81, 0xf9, 0xb1, 0xf7, 0x6d, 0xe8, 0xfa, 0x34, 0x6f, + 0xe1, 0x18, 0x4b, 0xeb, 0xd3, 0x3f, 0x9f, 0x53, 0x5f, 0xa6, 0x8c, 0xbd, 0x84, 0x6a, 0x71, 0x8c, + 0x35, 0x3e, 0x16, 0x9f, 0xdf, 0x41, 0x7f, 0xa1, 0x40, 0x51, 0x7e, 0xe2, 0x84, 0xea, 0x13, 0xbf, + 0x85, 0x62, 0xdc, 0x35, 0xa6, 0x7c, 0x3b, 0xa5, 0xde, 0xa2, 0xac, 0xbd, 0x86, 0xea, 0xcf, 0x61, + 0xad, 0x41, 0xaf, 0x72, 0xbf, 0xf1, 0x31, 0xfd, 0xa5, 0x9c, 0x42, 0x54, 0xb7, 0x47, 0xaf, 0x4e, + 0x58, 0xde, 0x67, 0x5c, 0x4e, 0xf7, 0x18, 0x40, 0xbd, 0x43, 0x79, 0xbc, 0x85, 0x6e, 0x4e, 0xc7, + 0x63, 0x83, 0xbd, 0x0b, 0xf9, 0xb1, 0x02, 0xa5, 0x91, 0xa7, 0x10, 0x28, 0x49, 0x48, 0x71, 0xaf, + 0x29, 0xaa, 0xaf, 0x4d, 0x3e, 0x80, 0xb3, 0xbc, 0x4a, 0x59, 0xae, 0xa2, 0x8a, 0xc4, 0xb2, 0xe3, + 0x3a, 0x8c, 0x41, 0xca, 0xc4, 0x87, 0x90, 0x65, 0xd5, 0x77, 0xb4, 0x36, 0x41, 0x81, 0x9e, 0xf1, + 0xf1, 0xe5, 0x89, 0x4b, 0xf9, 0xea, 0x32, 0x65, 0xe0, 0x3c, 0x5a, 0x90, 0x18, 0xe0, 0x56, 0x8e, + 0x9c, 0xc7, 0xb0, 0x32, 0x9c, 0x78, 0x1e, 0xc7, 0x6b, 0xd5, 0x89, 0xe7, 0xf1, 0x74, 0xb1, 0x99, + 0x9f, 0x47, 0x55, 0xe6, 0x61, 0x68, 0x91, 0xed, 0xba, 0xad, 0x5c, 0x43, 0xdf, 0x53, 0xa0, 0xb0, + 0x3d, 0x11, 0x1b, 0xdb, 0xd3, 0xb0, 0x71, 0xaa, 0x14, 0x1b, 0x2b, 0x0a, 0xc6, 0x06, 0xfa, 0x1d, + 0xc8, 0xf1, 0xc2, 0x2a, 0x4a, 0x92, 0xed, 0x68, 0xe1, 0xb6, 0x7a, 0x6d, 0x92, 0xae, 0x7c, 0xf2, + 0x2a, 0x9d, 0x7c, 0x11, 0x21, 0x69, 0x72, 0x5e, 0xc0, 0x45, 0xbf, 0xab, 0x40, 0x5e, 0xd4, 0x0b, + 0xd0, 0xb5, 0x89, 0x8a, 0x0a, 0x8c, 0x81, 0x57, 0xa6, 0x28, 0x40, 0xa8, 0x97, 0x28, 0x07, 0x17, + 0xd0, 0x79, 0x89, 0x03, 0x5f, 0xcc, 0xfa, 0x03, 0x05, 0xb2, 0xac, 0xcc, 0x90, 0xa8, 0x86, 0x23, + 0x05, 0x8b, 0x44, 0x35, 0x1c, 0xab, 0x59, 0x7c, 0x91, 0x4e, 0xbe, 0x32, 0x22, 0xfb, 0x3d, 0xda, + 0xe5, 0x49, 0x01, 0xe5, 0xf8, 0x27, 0x15, 0x84, 0xc8, 0xc9, 0x27, 0x0a, 0x62, 0xac, 0xfe, 0x91, + 0x28, 0x88, 0xf1, 0xa2, 0x44, 0xac, 0x20, 0x44, 0xcd, 0x03, 0x0d, 0x60, 0xe6, 0xae, 0xdb, 0xf1, + 0xd1, 0x97, 0x9e, 0x9b, 0x6f, 0x67, 0x33, 0xbf, 0x3c, 0x61, 0x5e, 0x5e, 0xbd, 0x48, 0x67, 0x5d, + 0x40, 0xf3, 0xd2, 0xac, 0xdf, 0x26, 0x33, 0x91, 0x63, 0x18, 0xa6, 0x14, 0x13, 0xf5, 0x7f, 0x3c, + 0x9d, 0x99, 0xa8, 0xff, 0xa7, 0xb2, 0x94, 0xb1, 0xd7, 0x62, 0x98, 0xa9, 0xa4, 0x6c, 0x84, 0x69, + 0x96, 0x44, 0x36, 0xc6, 0x73, 0x43, 0x89, 0x6c, 0x9c, 0x4a, 0x0f, 0xc5, 0xb2, 0x41, 0xd3, 0x43, + 0x03, 0x32, 0xf1, 0xb7, 0x20, 0x43, 0x6b, 0xf0, 0x89, 0xae, 0x89, 0xfc, 0x1e, 0x21, 0xd1, 0x35, + 0x19, 0x79, 0x2c, 0xa0, 0x9e, 0x7b, 0x4d, 0x41, 0xcf, 0xa0, 0x28, 0x97, 0x7d, 0x13, 0x2f, 0xd6, + 0x98, 0x3a, 0x78, 0xf5, 0xf5, 0x33, 0xd5, 0x93, 0xd5, 0x73, 0xe4, 0x84, 0xa1, 0xd3, 0x1d, 0xd0, + 0xcd, 0x29, 0xe9, 0xbd, 0x20, 0x17, 0x7f, 0xaa, 0x40, 0x5e, 0xa4, 0x04, 0x13, 0x4f, 0xd8, 0x58, + 0x92, 0x38, 0xf1, 0x84, 0x8d, 0xe7, 0x18, 0xd5, 0xb7, 0xe8, 0x16, 0xbf, 0x31, 0x72, 0xc2, 0x68, + 0x68, 0x62, 0xbb, 0xbd, 0x27, 0xab, 0x68, 0x25, 0x06, 0xdc, 0xf8, 0x58, 0x04, 0x44, 0xdf, 0x21, + 0x1e, 0x5a, 0x79, 0x3c, 0xaa, 0x45, 0xeb, 0x53, 0x85, 0xc0, 0x8c, 0xe9, 0x1b, 0x67, 0x08, 0x9b, + 0x43, 0x53, 0x75, 0x79, 0xcc, 0xcb, 0xd0, 0x4d, 0x99, 0x9d, 0x9f, 0x28, 0xb0, 0xb0, 0x61, 0xdb, + 0xa3, 0x51, 0x2e, 0x7a, 0x6d, 0x8a, 0x80, 0x98, 0xb1, 0x78, 0x7d, 0xea, 0x10, 0x5a, 0x7d, 0x89, + 0x32, 0x78, 0x09, 0x2d, 0x4b, 0x0c, 0xb2, 0xa8, 0x57, 0xc4, 0xd1, 0xe8, 0x13, 0x05, 0x8a, 0x72, + 0x50, 0x92, 0xa8, 0xe6, 0x31, 0x41, 0x4d, 0xa2, 0xff, 0x18, 0x17, 0xed, 0xa8, 0x35, 0xca, 0xd4, + 0x32, 0xba, 0x28, 0xdf, 0x6f, 0xa4, 0x23, 0x0f, 0x65, 0x88, 0xfb, 0x55, 0x94, 0x83, 0xc4, 0x44, + 0x96, 0x62, 0xa2, 0xf4, 0x44, 0x96, 0xe2, 0xa2, 0x4f, 0xf5, 0x0a, 0x65, 0xe9, 0x0b, 0xaa, 0xec, + 0x7b, 0x61, 0xd6, 0x51, 0xa7, 0xca, 0x76, 0x5b, 0xb9, 0xd6, 0xbc, 0xf6, 0xe9, 0xbf, 0xad, 0x9c, + 0xfb, 0xf4, 0x64, 0x45, 0xf9, 0xf9, 0xc9, 0x8a, 0xf2, 0x8b, 0x93, 0x15, 0xe5, 0x5f, 0x4f, 0x56, + 0x94, 0x4f, 0x3e, 0x5b, 0x39, 0xf7, 0xf3, 0xcf, 0x56, 0xce, 0xfd, 0xe2, 0xb3, 0x95, 0x73, 0x4f, + 0xf2, 0x62, 0x96, 0x4e, 0x96, 0x26, 0xec, 0x6e, 0xfc, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd4, + 0x35, 0xe9, 0x5b, 0x56, 0x35, 0x00, 0x00, } diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index 4d872036d041..3f95afb0d143 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -383,6 +383,53 @@ message DrainResponse { // 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 b282dc6f8e64..8de68850a662 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -452,7 +452,7 @@ func (ts *TestServer) RPCAddr() string { // DrainClients exports the drainClients() method for use by tests. func (ts *TestServer) DrainClients(ctx context.Context) error { - return ts.drainClients(ctx) + return ts.drainClients(ctx, nil /* reporter */) } // SQLAddr returns the server's listening SQL address. diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 788b078ee643..87674ab2c813 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,7 +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) + 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 91139cc44d91..29ef9f779f50 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 058c0d4d595f..25d083b71163 100644 --- a/pkg/sql/lease.go +++ b/pkg/sql/lease.go @@ -1700,7 +1700,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 @@ -1715,6 +1720,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 4a438ca4ca8e..605e45121c7f 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/server.go b/pkg/sql/pgwire/server.go index d0f23bc70aff..b5b9a71d1df2 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -318,8 +318,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 @@ -340,7 +345,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. @@ -367,6 +386,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 diff --git a/pkg/storage/client_raft_test.go b/pkg/storage/client_raft_test.go index e6e6b04b4c87..9f8323729178 100644 --- a/pkg/storage/client_raft_test.go +++ b/pkg/storage/client_raft_test.go @@ -1314,7 +1314,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/storage/client_replica_test.go b/pkg/storage/client_replica_test.go index 1312ea01d91c..e978d8225c07 100644 --- a/pkg/storage/client_replica_test.go +++ b/pkg/storage/client_replica_test.go @@ -827,7 +827,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) @@ -843,7 +843,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 @@ -876,7 +876,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{}{} @@ -1635,7 +1635,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/storage/helpers_test.go b/pkg/storage/helpers_test.go index 6643b5e4d506..c14e3dbc3896 100644 --- a/pkg/storage/helpers_test.go +++ b/pkg/storage/helpers_test.go @@ -473,7 +473,7 @@ func (r *Replica) UnquiesceAndWakeLeader() { 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/storage/node_liveness.go b/pkg/storage/node_liveness.go index 7b53d43e3659..3b9b6efb09f6 100644 --- a/pkg/storage/node_liveness.go +++ b/pkg/storage/node_liveness.go @@ -226,14 +226,19 @@ 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) } - err = nl.setDrainingInternal(ctx, liveness, drain) + 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) @@ -321,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) @@ -344,6 +349,10 @@ func (nl *NodeLiveness) setDrainingInternal( if liveness != nil { 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 diff --git a/pkg/storage/node_liveness_test.go b/pkg/storage/node_liveness_test.go index 8602323eda81..baa1e037c9c0 100644 --- a/pkg/storage/node_liveness_test.go +++ b/pkg/storage/node_liveness_test.go @@ -643,7 +643,7 @@ func TestNodeLivenessSetDraining(t *testing.T) { t.Fatal(err) } - mtc.nodeLivenesses[drainingNodeIdx].SetDraining(ctx, true /* drain */) + mtc.nodeLivenesses[drainingNodeIdx].SetDraining(ctx, true /* drain */, nil /* reporter */) // Draining node disappears from store lists. { diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 48df51ce89af..9dd83b64cf46 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -1484,7 +1484,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() @@ -1492,7 +1492,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/storage/store.go b/pkg/storage/store.go index 5ddc9f3b1568..18ceb5cb40ba 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -991,7 +991,12 @@ func (s *Store) AnnotateCtx(ctx context.Context) context.Context { // 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 { @@ -1141,6 +1146,23 @@ func (s *Store) SetDraining(drain bool) { return int(numTransfersAttempted) } + // 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", transferTimeout, @@ -1156,6 +1178,10 @@ func (s *Store) SetDraining(drain bool) { 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) From 6692a58388f644772ffb28f73beca15e7d5faae8 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 6 Apr 2020 20:57:56 +0200 Subject: [PATCH 09/12] server: clarify where SQL leases get drain Recommended by @tbg. Release note: None --- pkg/server/drain.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/pkg/server/drain.go b/pkg/server/drain.go index 9359cadb889d..ab0489019524 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -208,12 +208,6 @@ func (s *Server) drainClients(ctx context.Context, reporter func(int, string)) e // delay draining so that client traffic can move off this node. time.Sleep(drainWait.Get(&s.st.SV)) - // Since enabling the SQL table 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(true /* drain */, reporter) - // Disable incoming SQL clients up to the queryWait timeout. drainMaxWait := queryWait.Get(&s.st.SV) if err := s.pgServer.Drain(drainMaxWait, reporter); err != nil { @@ -222,6 +216,10 @@ func (s *Server) drainClients(ctx context.Context, reporter func(int, string)) e // 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 } From 863570933229461fe5a940531983cc204917b208 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 6 Apr 2020 23:09:26 +0200 Subject: [PATCH 10/12] storage: add some logging This is intended to aid troubleshooting faulty drains. Release note: None --- pkg/storage/replica_proposal.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/storage/replica_proposal.go b/pkg/storage/replica_proposal.go index b566f5ecc0ea..c2a465a01014 100644 --- a/pkg/storage/replica_proposal.go +++ b/pkg/storage/replica_proposal.go @@ -462,6 +462,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. From d2f3dbe6c3ea9c9a05d8a67f08c56dbad29cb4aa Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Tue, 7 Apr 2020 13:40:17 +0200 Subject: [PATCH 11/12] cli/quit: tolerate gRPC confusion when the server process exits Prior to this patch, when the server closed the connection at the end of the `Drain` RPC the gRPC client in `quit` would get confused with either of the following: ``` ERROR: hard shutdown failed: rpc error: code = Unimplemented desc = grpc: Decompressor is not installed for grpc-encoding "snappy" ``` or ``` ERROR: hard shutdown failed: rpc error: code = Internal desc = grpc: compressed flag set with identity or empty encoding ``` This patch special-cases these two errors. Release note: None --- pkg/cli/quit.go | 26 +++++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/pkg/cli/quit.go b/pkg/cli/quit.go index 890508c340f9..4c8e27724b0b 100644 --- a/pkg/cli/quit.go +++ b/pkg/cli/quit.go @@ -14,6 +14,7 @@ import ( "context" "fmt" "io" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/server" @@ -23,6 +24,8 @@ import ( "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. @@ -223,7 +226,7 @@ func doShutdown(ctx context.Context, c serverpb.AdminClient) (hardError bool, er err = errors.WithHint(err, "You can still stop the process using a service manager or a signal.") hardError = true } - if grpcutil.IsClosedConnection(err) { + 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 @@ -266,3 +269,24 @@ func getAdminClient(ctx context.Context) (serverpb.AdminClient, func(), error) { } 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 + } +} From dbc1efdd6f73dc36937c5b338d60d126bea41c15 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Sat, 21 Mar 2020 17:12:43 +0100 Subject: [PATCH 12/12] cli/quit: bump the default for --drain-wait Release note (cli change): The default value of the paramater `--drain-wait` for `cockroach quit` has been increased from 1 minute to 10 minutes, to give more time for nodes with thousands of range to migrate their leases away. --- pkg/cli/context.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cli/context.go b/pkg/cli/context.go index c215ec0e7dc4..87ec87cbeac3 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -121,7 +121,7 @@ func initCLIDefaults() { startCtx.inBackground = false quitCtx.serverDecommission = false - quitCtx.drainWait = time.Minute + quitCtx.drainWait = 10 * time.Minute nodeCtx.nodeDecommissionWait = nodeDecommissionWaitAll nodeCtx.statusShowRanges = false