diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index 96a8524d2de7..b322238f9fde 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -2893,6 +2893,7 @@ a table. | zone_config | [cockroach.config.zonepb.ZoneConfig](#cockroach.server.serverpb.TableDetailsResponse-cockroach.config.zonepb.ZoneConfig) | | The zone configuration in effect for this table. | [reserved](#support-status) | | zone_config_level | [ZoneConfigurationLevel](#cockroach.server.serverpb.TableDetailsResponse-cockroach.server.serverpb.ZoneConfigurationLevel) | | The level at which this object's zone configuration is set. | [reserved](#support-status) | | descriptor_id | [int64](#cockroach.server.serverpb.TableDetailsResponse-int64) | | descriptor_id is an identifier used to uniquely identify this table. It can be used to find events pertaining to this table by filtering on the 'target_id' field of events. | [reserved](#support-status) | +| configure_zone_statement | [string](#cockroach.server.serverpb.TableDetailsResponse-string) | | configure_zone_statement is the output of "SHOW ZONE CONFIGURATION FOR TABLE" for this table. It is a SQL statement that would re-configure the table's current zone if executed. | [reserved](#support-status) | diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index 9786a0db4863..0e467d5d2cf3 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -216,7 +216,7 @@ func TestGossipNodeLivenessOnLeaseChange(t *testing.T) { // liveness are not triggering gossiping. for _, s := range tc.Servers { pErr := s.Stores().VisitStores(func(store *kvserver.Store) error { - store.NodeLiveness().PauseHeartbeatLoopForTest() + store.GetStoreConfig().NodeLiveness.PauseHeartbeatLoopForTest() return nil }) if pErr != nil { diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 127d2227de88..c606144b2f79 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -1152,7 +1152,7 @@ func TestRequestsOnLaggingReplica(t *testing.T) { // Stop the heartbeats so that n1's lease can expire. log.Infof(ctx, "test: suspending heartbeats for n1") - resumeN1Heartbeats := partitionStore.NodeLiveness().PauseAllHeartbeatsForTest() + resumeN1Heartbeats := partitionStore.GetStoreConfig().NodeLiveness.PauseAllHeartbeatsForTest() // Wait until another replica campaigns and becomes leader, replacing the // partitioned one. @@ -2456,7 +2456,7 @@ func TestRaftHeartbeats(t *testing.T) { store := tc.GetFirstStoreFromServer(t, int(leaderRepl.StoreID()-1)) // Wait for several ticks to elapse. - ticksToWait := 2 * store.RaftElectionTimeoutTicks() + ticksToWait := 2 * store.GetStoreConfig().RaftElectionTimeoutTicks ticks := store.Metrics().RaftTicks.Count for targetTicks := ticks() + int64(ticksToWait); ticks() < targetTicks; { time.Sleep(time.Millisecond) @@ -2509,7 +2509,7 @@ func TestReportUnreachableHeartbeats(t *testing.T) { t.Fatal(err) } - ticksToWait := 2 * leaderStore.RaftElectionTimeoutTicks() + ticksToWait := 2 * leaderStore.GetStoreConfig().RaftElectionTimeoutTicks ticks := leaderStore.Metrics().RaftTicks.Count for targetTicks := ticks() + int64(ticksToWait); ticks() < targetTicks; { time.Sleep(time.Millisecond) @@ -2564,7 +2564,7 @@ outer: tc.RemoveVotersOrFatal(t, key, tc.Target(leaderIdx)) cb := tc.Servers[replicaIdx].RaftTransport().GetCircuitBreaker(tc.Target(replicaIdx).NodeID, rpc.DefaultClass) cb.Break() - time.Sleep(tc.GetFirstStoreFromServer(t, replicaIdx).CoalescedHeartbeatsInterval()) + time.Sleep(tc.GetFirstStoreFromServer(t, replicaIdx).GetStoreConfig().CoalescedHeartbeatsInterval) cb.Reset() tc.AddVotersOrFatal(t, key, tc.Target(leaderIdx)) continue outer @@ -4034,7 +4034,7 @@ func TestRangeQuiescence(t *testing.T) { }) defer tc.Stopper().Stop(ctx) - pauseNodeLivenessHeartbeatLoopsTC(tc) + pauseNodeLivenessHeartbeatLoops(tc) key := tc.ScratchRange(t) tc.AddVotersOrFatal(t, key, tc.Targets(1, 2)...) @@ -4065,7 +4065,7 @@ func TestRangeQuiescence(t *testing.T) { // Wait for a bunch of ticks to occur which will allow the follower time to // campaign. ticks := tc.GetFirstStoreFromServer(t, followerIdx).Metrics().RaftTicks.Count - for targetTicks := ticks() + int64(2*tc.GetFirstStoreFromServer(t, followerIdx).RaftElectionTimeoutTicks()); ticks() < targetTicks; { + for targetTicks := ticks() + int64(2*tc.GetFirstStoreFromServer(t, followerIdx).GetStoreConfig().RaftElectionTimeoutTicks); ticks() < targetTicks; { time.Sleep(time.Millisecond) } diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index 71db58dbbb07..de3add113b51 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -692,10 +692,10 @@ func forceLeaseTransferOnSubsumedRange( } return nil }) - restartHeartbeats := oldLeaseholderStore.NodeLiveness().PauseAllHeartbeatsForTest() + restartHeartbeats := oldLeaseholderStore.GetStoreConfig().NodeLiveness.PauseAllHeartbeatsForTest() defer restartHeartbeats() log.Infof(ctx, "test: paused RHS rightLeaseholder's liveness heartbeats") - time.Sleep(oldLeaseholderStore.NodeLiveness().GetLivenessThreshold()) + time.Sleep(oldLeaseholderStore.GetStoreConfig().NodeLiveness.GetLivenessThreshold()) // Send a read request from one of the followers of RHS so that it notices // that the current rightLeaseholder has stopped heartbeating. This will prompt @@ -955,7 +955,7 @@ func getCurrentMaxClosed( attempts := 0 for attempts == 0 || timeutil.Now().Before(deadline) { attempts++ - store.ClosedTimestamp().Storage.VisitDescending(target.NodeID, func(entry ctpb.Entry) (done bool) { + store.GetStoreConfig().ClosedTimestamp.Storage.VisitDescending(target.NodeID, func(entry ctpb.Entry) (done bool) { if _, ok := entry.MLAI[desc.RangeID]; ok { maxClosed = entry return true diff --git a/pkg/kv/kvserver/node_liveness_test.go b/pkg/kv/kvserver/node_liveness_test.go index c3403ff87faa..4c3bea015459 100644 --- a/pkg/kv/kvserver/node_liveness_test.go +++ b/pkg/kv/kvserver/node_liveness_test.go @@ -14,9 +14,11 @@ import ( "context" "reflect" "sort" + "strconv" "sync/atomic" "testing" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -24,8 +26,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -35,42 +40,33 @@ import ( "golang.org/x/sync/errgroup" ) -func verifyLiveness(t *testing.T, mtc *multiTestContext) { +func verifyLiveness(t *testing.T, tc *testcluster.TestCluster) { testutils.SucceedsSoon(t, func() error { - for i, nl := range mtc.nodeLivenesses { - for _, g := range mtc.gossips { - live, err := nl.IsLive(g.NodeID.Get()) - if err != nil { - return err - } else if !live { - return errors.Errorf("node %d not live", g.NodeID.Get()) - } - } - if a, e := nl.Metrics().LiveNodes.Value(), int64(len(mtc.nodeLivenesses)); a != e { - return errors.Errorf("expected node %d's LiveNodes metric to be %d; got %d", - mtc.gossips[i].NodeID.Get(), e, a) - } + for _, s := range tc.Servers { + return verifyLivenessServer(s, int64(len(tc.Servers))) } return nil }) } - -func pauseNodeLivenessHeartbeatLoops(mtc *multiTestContext) func() { - var enableFns []func() - for _, nl := range mtc.nodeLivenesses { - enableFns = append(enableFns, nl.PauseHeartbeatLoopForTest()) +func verifyLivenessServer(s *server.TestServer, numServers int64) error { + nl := s.NodeLiveness().(*liveness.NodeLiveness) + live, err := nl.IsLive(s.Gossip().NodeID.Get()) + if err != nil { + return err + } else if !live { + return errors.Errorf("node %d not live", s.Gossip().NodeID.Get()) } - return func() { - for _, fn := range enableFns { - fn() - } + if a, e := nl.Metrics().LiveNodes.Value(), numServers; a != e { + return errors.Errorf("expected node %d's LiveNodes metric to be %d; got %d", + s.Gossip().NodeID.Get(), e, a) } + return nil } -func pauseNodeLivenessHeartbeatLoopsTC(tc *testcluster.TestCluster) func() { +func pauseNodeLivenessHeartbeatLoops(tc *testcluster.TestCluster) func() { var enableFns []func() - for _, server := range tc.Servers { - enableFns = append(enableFns, server.NodeLiveness().(*liveness.NodeLiveness).PauseHeartbeatLoopForTest()) + for _, s := range tc.Servers { + enableFns = append(enableFns, s.NodeLiveness().(*liveness.NodeLiveness).PauseHeartbeatLoopForTest()) } return func() { for _, fn := range enableFns { @@ -78,22 +74,35 @@ func pauseNodeLivenessHeartbeatLoopsTC(tc *testcluster.TestCluster) func() { } } } - func TestNodeLiveness(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 3) + + ctx := context.Background() + manualClock := hlc.NewHybridManualClock() + tc := testcluster.StartTestCluster(t, 3, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) // Verify liveness of all nodes for all nodes. - verifyLiveness(t, mtc) - pauseNodeLivenessHeartbeatLoops(mtc) + verifyLiveness(t, tc) + pauseNodeLivenessHeartbeatLoops(tc) // Advance clock past the liveness threshold to verify IsLive becomes false. - mtc.manualClock.Increment(mtc.nodeLivenesses[0].GetLivenessThreshold().Nanoseconds() + 1) - for idx, nl := range mtc.nodeLivenesses { - nodeID := mtc.gossips[idx].NodeID.Get() + manualClock.Increment(tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).GetLivenessThreshold().Nanoseconds() + 1) + + for _, s := range tc.Servers { + nl := s.NodeLiveness().(*liveness.NodeLiveness) + nodeID := s.Gossip().NodeID.Get() live, err := nl.IsLive(nodeID) if err != nil { t.Error(err) @@ -109,7 +118,8 @@ func TestNodeLiveness(t *testing.T) { }) } // Trigger a manual heartbeat and verify liveness is reestablished. - for _, nl := range mtc.nodeLivenesses { + for _, s := range tc.Servers { + nl := s.NodeLiveness().(*liveness.NodeLiveness) l, ok := nl.Self() assert.True(t, ok) for { @@ -125,10 +135,11 @@ func TestNodeLiveness(t *testing.T) { t.Fatal(err) } } - verifyLiveness(t, mtc) + verifyLiveness(t, tc) // Verify metrics counts. - for i, nl := range mtc.nodeLivenesses { + for i, s := range tc.Servers { + nl := s.NodeLiveness().(*liveness.NodeLiveness) if c := nl.Metrics().HeartbeatSuccesses.Count(); c < 2 { t.Errorf("node %d: expected metrics count >= 2; got %d", (i + 1), c) } @@ -138,33 +149,52 @@ func TestNodeLiveness(t *testing.T) { func TestNodeLivenessInitialIncrement(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 1) + + stickyEngineRegistry := server.NewStickyInMemEnginesRegistry() + defer stickyEngineRegistry.CloseAllStickyInMemEngines() + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + StoreSpecs: []base.StoreSpec{ + { + InMemory: true, + StickyInMemoryEngineID: "1", + }, + }, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + StickyEngineRegistry: stickyEngineRegistry, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) // Verify liveness of all nodes for all nodes. - verifyLiveness(t, mtc) + verifyLiveness(t, tc) - liveness, ok := mtc.nodeLivenesses[0].GetLiveness(mtc.gossips[0].NodeID.Get()) + nl, ok := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).GetLiveness(tc.Servers[0].Gossip().NodeID.Get()) assert.True(t, ok) - if liveness.Epoch != 1 { - t.Errorf("expected epoch to be set to 1 initially; got %d", liveness.Epoch) + if nl.Epoch != 1 { + t.Errorf("expected epoch to be set to 1 initially; got %d", nl.Epoch) } // Restart the node and verify the epoch is incremented with initial heartbeat. - mtc.stopStore(0) - mtc.restartStore(0) - verifyEpochIncremented(t, mtc, 0) + require.NoError(t, tc.Restart()) + verifyEpochIncremented(t, tc, 0) } -func verifyEpochIncremented(t *testing.T, mtc *multiTestContext, nodeIdx int) { +func verifyEpochIncremented(t *testing.T, tc *testcluster.TestCluster, nodeIdx int) { testutils.SucceedsSoon(t, func() error { - liveness, ok := mtc.nodeLivenesses[nodeIdx].GetLiveness(mtc.gossips[nodeIdx].NodeID.Get()) + liv, ok := tc.Servers[nodeIdx].NodeLiveness().(*liveness.NodeLiveness).GetLiveness(tc.Servers[nodeIdx].Gossip().NodeID.Get()) if !ok { return errors.New("liveness not found") } - if liveness.Epoch < 2 { - return errors.Errorf("expected epoch to be >=2 on restart but was %d", liveness.Epoch) + if liv.Epoch < 2 { + return errors.Errorf("expected epoch to be >=2 on restart but was %d", liv.Epoch) } return nil }) @@ -178,18 +208,22 @@ func TestRedundantNodeLivenessHeartbeatsAvoided(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 1) - nl := mtc.nodeLivenesses[0] - nlActive, _ := mtc.storeConfig.NodeLivenessDurations() + serv, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + s := serv.(*server.TestServer) + store, err := s.Stores().GetStore(s.GetFirstStoreID()) + require.NoError(t, err) + defer s.Stopper().Stop(ctx) - // Verify liveness of all nodes for all nodes. - verifyLiveness(t, mtc) + nl := s.NodeLiveness().(*liveness.NodeLiveness) + nlActive, _ := store.GetStoreConfig().NodeLivenessDurations() + + testutils.SucceedsSoon(t, func() error { + return verifyLivenessServer(s, 1) + }) nl.PauseHeartbeatLoopForTest() enableSync := nl.PauseSynchronousHeartbeatsForTest() - liveness, ok := nl.Self() + nlSelf, ok := nl.Self() assert.True(t, ok) hbBefore := nl.Metrics().HeartbeatSuccesses.Count() require.Equal(t, int64(0), nl.Metrics().HeartbeatsInFlight.Value()) @@ -201,8 +235,8 @@ func TestRedundantNodeLivenessHeartbeatsAvoided(t *testing.T) { const herdSize = 30 for i := 0; i < herdSize; i++ { g.Go(func() error { - before := mtc.clock().Now() - if err := nl.Heartbeat(ctx, liveness); err != nil { + before := s.Clock().Now() + if err := nl.Heartbeat(ctx, nlSelf); err != nil { return err } livenessAfter, found := nl.Self() @@ -236,9 +270,9 @@ func TestRedundantNodeLivenessHeartbeatsAvoided(t *testing.T) { require.Equal(t, int64(0), nl.Metrics().HeartbeatsInFlight.Value()) // Send one more heartbeat. Should update liveness record. - liveness, ok = nl.Self() + nlSelf, ok = nl.Self() require.True(t, ok) - require.NoError(t, nl.Heartbeat(ctx, liveness)) + require.NoError(t, nl.Heartbeat(ctx, nlSelf)) require.Equal(t, hbBefore+2, nl.Metrics().HeartbeatSuccesses.Count()) require.Equal(t, int64(0), nl.Metrics().HeartbeatsInFlight.Value()) } @@ -248,27 +282,40 @@ func TestRedundantNodeLivenessHeartbeatsAvoided(t *testing.T) { func TestNodeIsLiveCallback(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 3) + + ctx := context.Background() + manualClock := hlc.NewHybridManualClock() + tc := testcluster.StartTestCluster(t, 3, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) // Verify liveness of all nodes for all nodes. - verifyLiveness(t, mtc) - pauseNodeLivenessHeartbeatLoops(mtc) + verifyLiveness(t, tc) + pauseNodeLivenessHeartbeatLoops(tc) var cbMu syncutil.Mutex cbs := map[roachpb.NodeID]struct{}{} - mtc.nodeLivenesses[0].RegisterCallback(func(l livenesspb.Liveness) { + tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).RegisterCallback(func(l livenesspb.Liveness) { cbMu.Lock() defer cbMu.Unlock() cbs[l.NodeID] = struct{}{} }) // Advance clock past the liveness threshold. - mtc.manualClock.Increment(mtc.nodeLivenesses[0].GetLivenessThreshold().Nanoseconds() + 1) + manualClock.Increment(tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).GetLivenessThreshold().Nanoseconds() + 1) // Trigger a manual heartbeat and verify callbacks for each node ID are invoked. - for _, nl := range mtc.nodeLivenesses { + for _, s := range tc.Servers { + nl := s.NodeLiveness().(*liveness.NodeLiveness) l, ok := nl.Self() assert.True(t, ok) if err := nl.Heartbeat(context.Background(), l); err != nil { @@ -279,8 +326,8 @@ func TestNodeIsLiveCallback(t *testing.T) { testutils.SucceedsSoon(t, func() error { cbMu.Lock() defer cbMu.Unlock() - for _, g := range mtc.gossips { - nodeID := g.NodeID.Get() + for _, s := range tc.Servers { + nodeID := s.Gossip().NodeID.Get() if _, ok := cbs[nodeID]; !ok { return errors.Errorf("expected IsLive callback for node %d", nodeID) } @@ -294,23 +341,36 @@ func TestNodeIsLiveCallback(t *testing.T) { func TestNodeHeartbeatCallback(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 3) + + ctx := context.Background() + manualClock := hlc.NewHybridManualClock() + expected := manualClock.UnixNano() + tc := testcluster.StartTestCluster(t, 3, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) // Verify liveness of all nodes for all nodes. - verifyLiveness(t, mtc) - pauseNodeLivenessHeartbeatLoops(mtc) + verifyLiveness(t, tc) + pauseNodeLivenessHeartbeatLoops(tc) // Verify that last update time has been set for all nodes. verifyUptimes := func() error { - expected := mtc.clock().Now() - for i, s := range mtc.stores { + for i := range tc.Servers { + s := tc.GetFirstStoreFromServer(t, i) uptm, err := s.ReadLastUpTimestamp(context.Background()) if err != nil { return errors.Wrapf(err, "error reading last up time from store %d", i) } - if a, e := uptm.WallTime, expected.WallTime; a != e { + if a, e := uptm.WallTime, expected; a < e { return errors.Errorf("store %d last uptime = %d; wanted %d", i, a, e) } } @@ -324,8 +384,10 @@ func TestNodeHeartbeatCallback(t *testing.T) { // Advance clock past the liveness threshold and force a manual heartbeat on // all node liveness objects, which should update the last up time for each // store. - mtc.manualClock.Increment(mtc.nodeLivenesses[0].GetLivenessThreshold().Nanoseconds() + 1) - for _, nl := range mtc.nodeLivenesses { + manualClock.Increment(tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).GetLivenessThreshold().Nanoseconds() + 1) + expected = manualClock.UnixNano() + for _, s := range tc.Servers { + nl := s.NodeLiveness().(*liveness.NodeLiveness) l, ok := nl.Self() assert.True(t, ok) if err := nl.Heartbeat(context.Background(), l); err != nil { @@ -349,32 +411,43 @@ func TestNodeLivenessEpochIncrement(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 2) + manualClock := hlc.NewHybridManualClock() + tc := testcluster.StartTestCluster(t, 2, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) - verifyLiveness(t, mtc) - pauseNodeLivenessHeartbeatLoops(mtc) + // Verify liveness of all nodes for all nodes. + verifyLiveness(t, tc) + pauseNodeLivenessHeartbeatLoops(tc) // First try to increment the epoch of a known-live node. - deadNodeID := mtc.gossips[1].NodeID.Get() - oldLiveness, ok := mtc.nodeLivenesses[0].GetLiveness(deadNodeID) + deadNodeID := tc.Servers[1].Gossip().NodeID.Get() + oldLiveness, ok := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).GetLiveness(deadNodeID) assert.True(t, ok) - if err := mtc.nodeLivenesses[0].IncrementEpoch( + if err := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).IncrementEpoch( ctx, oldLiveness.Liveness, ); !testutils.IsError(err, "cannot increment epoch on live node") { t.Fatalf("expected error incrementing a live node: %+v", err) } // Advance clock past liveness threshold & increment epoch. - mtc.manualClock.Increment(mtc.nodeLivenesses[0].GetLivenessThreshold().Nanoseconds() + 1) - if err := mtc.nodeLivenesses[0].IncrementEpoch(ctx, oldLiveness.Liveness); err != nil { + manualClock.Increment(tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).GetLivenessThreshold().Nanoseconds() + 1) + if err := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).IncrementEpoch(ctx, oldLiveness.Liveness); err != nil { t.Fatalf("unexpected error incrementing a non-live node: %+v", err) } // Verify that the epoch has been advanced. testutils.SucceedsSoon(t, func() error { - newLiveness, ok := mtc.nodeLivenesses[0].GetLiveness(deadNodeID) + newLiveness, ok := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).GetLiveness(deadNodeID) if !ok { return errors.New("liveness not found") } @@ -384,19 +457,19 @@ func TestNodeLivenessEpochIncrement(t *testing.T) { if newLiveness.Expiration != oldLiveness.Expiration { return errors.Errorf("expected expiration to remain unchanged") } - if live, err := mtc.nodeLivenesses[0].IsLive(deadNodeID); live || err != nil { + if live, err := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).IsLive(deadNodeID); live || err != nil { return errors.Errorf("expected dead node to remain dead after epoch increment %t: %v", live, err) } return nil }) // Verify epoch increment metric count. - if c := mtc.nodeLivenesses[0].Metrics().EpochIncrements.Count(); c != 1 { + if c := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).Metrics().EpochIncrements.Count(); c != 1 { t.Errorf("expected epoch increment == 1; got %d", c) } // Verify error on incrementing an already-incremented epoch. - if err := mtc.nodeLivenesses[0].IncrementEpoch( + if err := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).IncrementEpoch( ctx, oldLiveness.Liveness, ); !errors.Is(err, liveness.ErrEpochAlreadyIncremented) { t.Fatalf("unexpected error incrementing a non-live node: %+v", err) @@ -404,7 +477,7 @@ func TestNodeLivenessEpochIncrement(t *testing.T) { // Verify error incrementing with a too-high expectation for liveness epoch. oldLiveness.Epoch = 3 - if err := mtc.nodeLivenesses[0].IncrementEpoch( + if err := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness).IncrementEpoch( ctx, oldLiveness.Liveness, ); !testutils.IsError(err, "unexpected liveness epoch 2; expected >= 3") { t.Fatalf("expected error incrementing with a too-high expected epoch: %+v", err) @@ -416,22 +489,48 @@ func TestNodeLivenessEpochIncrement(t *testing.T) { func TestNodeLivenessRestart(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 2) + + stickyEngineRegistry := server.NewStickyInMemEnginesRegistry() + defer stickyEngineRegistry.CloseAllStickyInMemEngines() + + const numServers int = 2 + stickyServerArgs := make(map[int]base.TestServerArgs) + for i := 0; i < numServers; i++ { + stickyServerArgs[i] = base.TestServerArgs{ + StoreSpecs: []base.StoreSpec{ + { + InMemory: true, + StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10), + }, + }, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + StickyEngineRegistry: stickyEngineRegistry, + }, + }, + } + } + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 2, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: stickyServerArgs, + }) + defer tc.Stopper().Stop(ctx) // After verifying node is in liveness table, stop store. - verifyLiveness(t, mtc) - mtc.stopStore(0) + verifyLiveness(t, tc) + tc.StopServer(1) // Clear the liveness records in store 1's gossip to make sure we're // seeing the liveness record properly gossiped at store startup. var expKeys []string - for _, g := range mtc.gossips { - nodeID := g.NodeID.Get() + for _, s := range tc.Servers { + nodeID := s.Gossip().NodeID.Get() key := gossip.MakeNodeLivenessKey(nodeID) expKeys = append(expKeys, key) - if err := g.AddInfoProto(key, &livenesspb.Liveness{NodeID: nodeID}, 0); err != nil { + if err := s.Gossip().AddInfoProto(key, &livenesspb.Liveness{NodeID: nodeID}, 0); err != nil { t.Fatal(err) } } @@ -443,20 +542,21 @@ func TestNodeLivenessRestart(t *testing.T) { syncutil.Mutex keys []string } - livenessRegex := gossip.MakePrefixPattern(gossip.KeyNodeLivenessPrefix) - mtc.gossips[0].RegisterCallback(livenessRegex, func(key string, _ roachpb.Value) { - keysMu.Lock() - defer keysMu.Unlock() - for _, k := range keysMu.keys { - if k == key { - return - } - } - keysMu.keys = append(keysMu.keys, key) - }) // Restart store and verify gossip contains liveness record for nodes 1&2. - mtc.restartStore(0) + require.NoError(t, tc.RestartServerWithInspect(1, func(s *server.TestServer) { + livenessRegex := gossip.MakePrefixPattern(gossip.KeyNodeLivenessPrefix) + s.Gossip().RegisterCallback(livenessRegex, func(key string, _ roachpb.Value) { + keysMu.Lock() + defer keysMu.Unlock() + for _, k := range keysMu.keys { + if k == key { + return + } + } + keysMu.keys = append(keysMu.keys, key) + }) + })) testutils.SucceedsSoon(t, func() error { keysMu.Lock() defer keysMu.Unlock() @@ -479,26 +579,27 @@ func TestNodeLivenessRestart(t *testing.T) { func TestNodeLivenessSelf(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 1) - g := mtc.gossips[0] - - pauseNodeLivenessHeartbeatLoops(mtc) + ctx := context.Background() + serv, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + s := serv.(*server.TestServer) + defer s.Stopper().Stop(ctx) + g := s.Gossip() + nl := s.NodeLiveness().(*liveness.NodeLiveness) + nl.PauseHeartbeatLoopForTest() // Verify liveness is properly initialized. This needs to be wrapped in a // SucceedsSoon because node liveness gets initialized via an async gossip // callback. - var liveness liveness.Record + var livenessRecord liveness.Record testutils.SucceedsSoon(t, func() error { - l, ok := mtc.nodeLivenesses[0].GetLiveness(g.NodeID.Get()) + l, ok := nl.GetLiveness(g.NodeID.Get()) if !ok { return errors.New("liveness not found") } - liveness = l + livenessRecord = l return nil }) - if err := mtc.nodeLivenesses[0].Heartbeat(context.Background(), liveness.Liveness); err != nil { + if err := nl.Heartbeat(context.Background(), livenessRecord.Liveness); err != nil { t.Fatal(err) } @@ -510,7 +611,7 @@ func TestNodeLivenessSelf(t *testing.T) { atomic.AddInt32(&count, 1) }) testutils.SucceedsSoon(t, func() error { - fakeBehindLiveness := liveness + fakeBehindLiveness := livenessRecord fakeBehindLiveness.Epoch-- // almost certainly results in zero if err := g.AddInfoProto(key, &fakeBehindLiveness, 0); err != nil { @@ -523,11 +624,10 @@ func TestNodeLivenessSelf(t *testing.T) { }) // Self should not see the fake liveness, but have kept the real one. - l := mtc.nodeLivenesses[0] - lGetRec, ok := l.GetLiveness(g.NodeID.Get()) + lGetRec, ok := nl.GetLiveness(g.NodeID.Get()) require.True(t, ok) lGet := lGetRec.Liveness - lSelf, ok := l.Self() + lSelf, ok := nl.Self() assert.True(t, ok) if !reflect.DeepEqual(lGet, lSelf) { t.Errorf("expected GetLiveness() to return same value as Self(): %+v != %+v", lGet, lSelf) @@ -540,16 +640,29 @@ func TestNodeLivenessSelf(t *testing.T) { func TestNodeLivenessGetIsLiveMap(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 3) - - verifyLiveness(t, mtc) - pauseNodeLivenessHeartbeatLoops(mtc) - lMap := mtc.nodeLivenesses[0].GetIsLiveMap() - l1, _ := mtc.nodeLivenesses[0].GetLiveness(1) - l2, _ := mtc.nodeLivenesses[0].GetLiveness(2) - l3, _ := mtc.nodeLivenesses[0].GetLiveness(3) + + ctx := context.Background() + manualClock := hlc.NewHybridManualClock() + tc := testcluster.StartTestCluster(t, 3, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + verifyLiveness(t, tc) + pauseNodeLivenessHeartbeatLoops(tc) + nl := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness) + lMap := nl.GetIsLiveMap() + l1, _ := nl.GetLiveness(1) + l2, _ := nl.GetLiveness(2) + l3, _ := nl.GetLiveness(3) expectedLMap := liveness.IsLiveMap{ 1: {Liveness: l1.Liveness, IsLive: true}, 2: {Liveness: l2.Liveness, IsLive: true}, @@ -560,10 +673,10 @@ func TestNodeLivenessGetIsLiveMap(t *testing.T) { } // Advance the clock but only heartbeat node 0. - mtc.manualClock.Increment(mtc.nodeLivenesses[0].GetLivenessThreshold().Nanoseconds() + 1) + manualClock.Increment(nl.GetLivenessThreshold().Nanoseconds() + 1) var livenessRec liveness.Record testutils.SucceedsSoon(t, func() error { - lr, ok := mtc.nodeLivenesses[0].GetLiveness(mtc.gossips[0].NodeID.Get()) + lr, ok := nl.GetLiveness(tc.Servers[0].Gossip().NodeID.Get()) if !ok { return errors.New("liveness not found") } @@ -572,7 +685,7 @@ func TestNodeLivenessGetIsLiveMap(t *testing.T) { }) testutils.SucceedsSoon(t, func() error { - if err := mtc.nodeLivenesses[0].Heartbeat(context.Background(), livenessRec.Liveness); err != nil { + if err := nl.Heartbeat(context.Background(), livenessRec.Liveness); err != nil { if errors.Is(err, liveness.ErrEpochIncremented) { return err } @@ -582,10 +695,10 @@ func TestNodeLivenessGetIsLiveMap(t *testing.T) { }) // Now verify only node 0 is live. - lMap = mtc.nodeLivenesses[0].GetIsLiveMap() - l1, _ = mtc.nodeLivenesses[0].GetLiveness(1) - l2, _ = mtc.nodeLivenesses[0].GetLiveness(2) - l3, _ = mtc.nodeLivenesses[0].GetLiveness(3) + lMap = nl.GetIsLiveMap() + l1, _ = nl.GetLiveness(1) + l2, _ = nl.GetLiveness(2) + l3, _ = nl.GetLiveness(3) expectedLMap = liveness.IsLiveMap{ 1: {Liveness: l1.Liveness, IsLive: true}, 2: {Liveness: l2.Liveness, IsLive: false}, @@ -599,21 +712,33 @@ func TestNodeLivenessGetIsLiveMap(t *testing.T) { func TestNodeLivenessGetLivenesses(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 3) + ctx := context.Background() + manualClock := hlc.NewHybridManualClock() + testStartTime := manualClock.UnixNano() + tc := testcluster.StartTestCluster(t, 3, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) - verifyLiveness(t, mtc) - pauseNodeLivenessHeartbeatLoops(mtc) + verifyLiveness(t, tc) + pauseNodeLivenessHeartbeatLoops(tc) - livenesses := mtc.nodeLivenesses[0].GetLivenesses() + nl := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness) actualLMapNodes := make(map[roachpb.NodeID]struct{}) - originalExpiration := mtc.clock().PhysicalNow() + mtc.nodeLivenesses[0].GetLivenessThreshold().Nanoseconds() - for _, l := range livenesses { + originalExpiration := testStartTime + nl.GetLivenessThreshold().Nanoseconds() + for _, l := range nl.GetLivenesses() { if a, e := l.Epoch, int64(1); a != e { t.Errorf("liveness record had epoch %d, wanted %d", a, e) } - if a, e := l.Expiration.WallTime, originalExpiration; a != e { + if a, e := l.Expiration.WallTime, originalExpiration; a < e { t.Errorf("liveness record had expiration %d, wanted %d", a, e) } actualLMapNodes[l.NodeID] = struct{}{} @@ -624,32 +749,31 @@ func TestNodeLivenessGetLivenesses(t *testing.T) { } // Advance the clock but only heartbeat node 0. - mtc.manualClock.Increment(mtc.nodeLivenesses[0].GetLivenessThreshold().Nanoseconds() + 1) - var liveness liveness.Record + manualClock.Increment(nl.GetLivenessThreshold().Nanoseconds() + 1) + var livenessRecord liveness.Record testutils.SucceedsSoon(t, func() error { - livenessRec, ok := mtc.nodeLivenesses[0].GetLiveness(mtc.gossips[0].NodeID.Get()) + livenessRec, ok := nl.GetLiveness(tc.Servers[0].Gossip().NodeID.Get()) if !ok { return errors.New("liveness not found") } - liveness = livenessRec + livenessRecord = livenessRec return nil }) - if err := mtc.nodeLivenesses[0].Heartbeat(context.Background(), liveness.Liveness); err != nil { + if err := nl.Heartbeat(context.Background(), livenessRecord.Liveness); err != nil { t.Fatal(err) } // Verify that node liveness receives the change. - livenesses = mtc.nodeLivenesses[0].GetLivenesses() actualLMapNodes = make(map[roachpb.NodeID]struct{}) - for _, l := range livenesses { + for _, l := range nl.GetLivenesses() { if a, e := l.Epoch, int64(1); a != e { t.Errorf("liveness record had epoch %d, wanted %d", a, e) } expectedExpiration := originalExpiration if l.NodeID == 1 { - expectedExpiration += mtc.nodeLivenesses[0].GetLivenessThreshold().Nanoseconds() + 1 + expectedExpiration += nl.GetLivenessThreshold().Nanoseconds() + 1 } - if a, e := l.Expiration.WallTime, expectedExpiration; a != e { + if a, e := l.Expiration.WallTime, expectedExpiration; a < e { t.Errorf("liveness record had expiration %d, wanted %d", a, e) } actualLMapNodes[l.NodeID] = struct{}{} @@ -664,18 +788,29 @@ func TestNodeLivenessGetLivenesses(t *testing.T) { func TestNodeLivenessConcurrentHeartbeats(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 1) - verifyLiveness(t, mtc) - pauseNodeLivenessHeartbeatLoops(mtc) + ctx := context.Background() + manualClock := hlc.NewHybridManualClock() + serv, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }) + s := serv.(*server.TestServer) + defer s.Stopper().Stop(ctx) + + testutils.SucceedsSoon(t, func() error { + return verifyLivenessServer(s, 1) + }) + nl := s.NodeLiveness().(*liveness.NodeLiveness) + nl.PauseHeartbeatLoopForTest() const concurrency = 10 // Advance clock past the liveness threshold & concurrently heartbeat node. - nl := mtc.nodeLivenesses[0] - mtc.manualClock.Increment(nl.GetLivenessThreshold().Nanoseconds() + 1) + manualClock.Increment(nl.GetLivenessThreshold().Nanoseconds() + 1) l, ok := nl.Self() assert.True(t, ok) errCh := make(chan error, concurrency) @@ -696,19 +831,31 @@ func TestNodeLivenessConcurrentHeartbeats(t *testing.T) { func TestNodeLivenessConcurrentIncrementEpochs(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 2) - verifyLiveness(t, mtc) - pauseNodeLivenessHeartbeatLoops(mtc) + ctx := context.Background() + manualClock := hlc.NewHybridManualClock() + tc := testcluster.StartTestCluster(t, 2, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + verifyLiveness(t, tc) + pauseNodeLivenessHeartbeatLoops(tc) const concurrency = 10 // Advance the clock and this time increment epoch concurrently for node 1. - nl := mtc.nodeLivenesses[0] - mtc.manualClock.Increment(nl.GetLivenessThreshold().Nanoseconds() + 1) - l, ok := nl.GetLiveness(mtc.gossips[1].NodeID.Get()) + nl := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness) + manualClock.Increment(nl.GetLivenessThreshold().Nanoseconds() + 1) + l, ok := nl.GetLiveness(tc.Servers[1].Gossip().NodeID.Get()) assert.True(t, ok) errCh := make(chan error, concurrency) for i := 0; i < concurrency; i++ { @@ -729,16 +876,41 @@ func TestNodeLivenessConcurrentIncrementEpochs(t *testing.T) { func TestNodeLivenessSetDraining(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 3) - mtc.initGossipNetwork() - verifyLiveness(t, mtc) + stickyEngineRegistry := server.NewStickyInMemEnginesRegistry() + defer stickyEngineRegistry.CloseAllStickyInMemEngines() + + const numServers int = 3 + stickyServerArgs := make(map[int]base.TestServerArgs) + for i := 0; i < numServers; i++ { + stickyServerArgs[i] = base.TestServerArgs{ + StoreSpecs: []base.StoreSpec{ + { + InMemory: true, + StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10), + }, + }, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + StickyEngineRegistry: stickyEngineRegistry, + }, + }, + } + } ctx := context.Background() + tc := testcluster.StartTestCluster(t, numServers, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: stickyServerArgs, + }) + defer tc.Stopper().Stop(ctx) + + // Verify liveness of all nodes for all nodes. + verifyLiveness(t, tc) + drainingNodeIdx := 0 - drainingNodeID := mtc.gossips[drainingNodeIdx].NodeID.Get() + drainingNodeID := tc.Servers[0].Gossip().NodeID.Get() nodeIDAppearsInStoreList := func(id roachpb.NodeID, sl kvserver.StoreList) bool { for _, store := range sl.Stores() { @@ -751,7 +923,7 @@ func TestNodeLivenessSetDraining(t *testing.T) { // Verify success on failed update of a liveness record that already has the // given draining setting. - if err := mtc.nodeLivenesses[drainingNodeIdx].TestingSetDrainingInternal( + if err := tc.Servers[drainingNodeIdx].NodeLiveness().(*liveness.NodeLiveness).TestingSetDrainingInternal( ctx, liveness.Record{Liveness: livenesspb.Liveness{ NodeID: drainingNodeID, }}, false, @@ -759,7 +931,7 @@ func TestNodeLivenessSetDraining(t *testing.T) { t.Fatal(err) } - if err := mtc.nodeLivenesses[drainingNodeIdx].SetDraining(ctx, true /* drain */, nil /* reporter */); err != nil { + if err := tc.Servers[drainingNodeIdx].NodeLiveness().(*liveness.NodeLiveness).SetDraining(ctx, true /* drain */, nil /* reporter */); err != nil { t.Fatal(err) } @@ -769,9 +941,9 @@ func TestNodeLivenessSetDraining(t *testing.T) { // Executed in a retry loop to wait until the new liveness record has // been gossiped to the rest of the cluster. testutils.SucceedsSoon(t, func() error { - for i, sp := range mtc.storePools { - curNodeID := mtc.gossips[i].NodeID.Get() - sl, alive, _ := sp.GetStoreList() + for i, s := range tc.Servers { + curNodeID := s.Gossip().NodeID.Get() + sl, alive, _ := tc.GetFirstStoreFromServer(t, i).GetStoreConfig().StorePool.GetStoreList() if alive != expectedLive { return errors.Errorf( "expected %d live stores but got %d from node %d", @@ -794,8 +966,8 @@ func TestNodeLivenessSetDraining(t *testing.T) { // Stop and restart the store to verify that a restarted server clears the // draining field on the liveness record. - mtc.stopStore(drainingNodeIdx) - mtc.restartStore(drainingNodeIdx) + tc.StopServer(drainingNodeIdx) + require.NoError(t, tc.RestartServer(drainingNodeIdx)) // Restarted node appears once again in the store list. { @@ -803,9 +975,9 @@ func TestNodeLivenessSetDraining(t *testing.T) { // Executed in a retry loop to wait until the new liveness record has // been gossiped to the rest of the cluster. testutils.SucceedsSoon(t, func() error { - for i, sp := range mtc.storePools { - curNodeID := mtc.gossips[i].NodeID.Get() - sl, alive, _ := sp.GetStoreList() + for i, s := range tc.Servers { + curNodeID := s.Gossip().NodeID.Get() + sl, alive, _ := tc.GetFirstStoreFromServer(t, i).GetStoreConfig().StorePool.GetStoreList() if alive != expectedLive { return errors.Errorf( "expected %d live stores but got %d from node %d", @@ -836,8 +1008,7 @@ func TestNodeLivenessRetryAmbiguousResultError(t *testing.T) { var injectedErrorCount int32 injectError.Store(true) - storeCfg := kvserver.TestStoreConfig(nil) - storeCfg.TestingKnobs.EvalKnobs.TestingEvalFilter = func(args kvserverbase.FilterArgs) *roachpb.Error { + testingEvalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { if _, ok := args.Req.(*roachpb.ConditionalPutRequest); !ok { return nil } @@ -848,16 +1019,28 @@ func TestNodeLivenessRetryAmbiguousResultError(t *testing.T) { } return nil } - mtc := &multiTestContext{ - storeConfig: &storeCfg, - } - mtc.Start(t, 1) - defer mtc.Stop() + ctx := context.Background() + manualClock := hlc.NewHybridManualClock() + serv, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + Store: &kvserver.StoreTestingKnobs{ + EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ + TestingEvalFilter: testingEvalFilter, + }, + }, + }, + }) + s := serv.(*server.TestServer) + defer s.Stopper().Stop(ctx) - // Verify retry of the ambiguous result for heartbeat loop. - verifyLiveness(t, mtc) + testutils.SucceedsSoon(t, func() error { + return verifyLivenessServer(s, 1) + }) + nl := s.NodeLiveness().(*liveness.NodeLiveness) - nl := mtc.nodeLivenesses[0] l, ok := nl.Self() assert.True(t, ok) @@ -871,16 +1054,15 @@ func TestNodeLivenessRetryAmbiguousResultError(t *testing.T) { } } -func verifyNodeIsDecommissioning(t *testing.T, mtc *multiTestContext, nodeID roachpb.NodeID) { +func verifyNodeIsDecommissioning(t *testing.T, tc *testcluster.TestCluster, nodeID roachpb.NodeID) { testutils.SucceedsSoon(t, func() error { - for _, nl := range mtc.nodeLivenesses { - livenesses := nl.GetLivenesses() - for _, liveness := range livenesses { - if liveness.NodeID != nodeID { + for _, s := range tc.Servers { + for _, liv := range s.NodeLiveness().(*liveness.NodeLiveness).GetLivenesses() { + if liv.NodeID != nodeID { continue } - if !liveness.Membership.Decommissioning() { - return errors.Errorf("unexpected Membership value of %v for node %v", liveness.Membership, liveness.NodeID) + if !liv.Membership.Decommissioning() { + return errors.Errorf("unexpected Membership value of %v for node %v", liv.Membership, liv.NodeID) } } } @@ -889,16 +1071,40 @@ func verifyNodeIsDecommissioning(t *testing.T, mtc *multiTestContext, nodeID roa } func testNodeLivenessSetDecommissioning(t *testing.T, decommissionNodeIdx int) { - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 3) - mtc.initGossipNetwork() - - verifyLiveness(t, mtc) + stickyEngineRegistry := server.NewStickyInMemEnginesRegistry() + defer stickyEngineRegistry.CloseAllStickyInMemEngines() + + const numServers int = 3 + stickyServerArgs := make(map[int]base.TestServerArgs) + for i := 0; i < numServers; i++ { + stickyServerArgs[i] = base.TestServerArgs{ + StoreSpecs: []base.StoreSpec{ + { + InMemory: true, + StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10), + }, + }, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + StickyEngineRegistry: stickyEngineRegistry, + }, + }, + } + } ctx := context.Background() - callerNodeLiveness := mtc.nodeLivenesses[0] - nodeID := mtc.gossips[decommissionNodeIdx].NodeID.Get() + tc := testcluster.StartTestCluster(t, numServers, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: stickyServerArgs, + }) + defer tc.Stopper().Stop(ctx) + + // Verify liveness of all nodes for all nodes. + verifyLiveness(t, tc) + + callerNodeLiveness := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness) + nodeID := tc.Servers[decommissionNodeIdx].Gossip().NodeID.Get() // Verify success on failed update of a liveness record that already has the // given decommissioning setting. @@ -915,18 +1121,18 @@ func testNodeLivenessSetDecommissioning(t *testing.T, decommissionNodeIdx int) { ctx, nodeID, livenesspb.MembershipStatus_DECOMMISSIONING); err != nil { t.Fatal(err) } - verifyNodeIsDecommissioning(t, mtc, nodeID) + verifyNodeIsDecommissioning(t, tc, nodeID) // Stop and restart the store to verify that a restarted server retains the // decommissioning field on the liveness record. - mtc.stopStore(decommissionNodeIdx) - mtc.restartStore(decommissionNodeIdx) + tc.StopServer(decommissionNodeIdx) + require.NoError(t, tc.RestartServer(decommissionNodeIdx)) // Wait until store has restarted and published a new heartbeat to ensure not // looking at pre-restart state. Want to be sure test fails if node wiped the // decommission flag. - verifyEpochIncremented(t, mtc, decommissionNodeIdx) - verifyNodeIsDecommissioning(t, mtc, nodeID) + verifyEpochIncremented(t, tc, decommissionNodeIdx) + verifyNodeIsDecommissioning(t, tc, nodeID) } // TestNodeLivenessSetDecommissioning verifies that when decommissioning, a @@ -949,66 +1155,56 @@ func TestNodeLivenessDecommissionAbsent(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - mtc := &multiTestContext{} - defer mtc.Stop() - mtc.Start(t, 3) - mtc.initGossipNetwork() + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 3, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + }) + defer tc.Stopper().Stop(ctx) - verifyLiveness(t, mtc) + // Verify liveness of all nodes for all nodes. + verifyLiveness(t, tc) - ctx := context.Background() const goneNodeID = roachpb.NodeID(10000) + nl := tc.Servers[0].NodeLiveness().(*liveness.NodeLiveness) + nl1 := tc.Servers[1].NodeLiveness().(*liveness.NodeLiveness) + nl2 := tc.Servers[1].NodeLiveness().(*liveness.NodeLiveness) // When the node simply never existed, expect an error. - if _, err := mtc.nodeLivenesses[0].SetMembershipStatus( + if _, err := nl.SetMembershipStatus( ctx, goneNodeID, livenesspb.MembershipStatus_DECOMMISSIONING, ); !errors.Is(err, liveness.ErrMissingRecord) { t.Fatal(err) } // Pretend the node was once there but isn't gossiped anywhere. - if err := mtc.dbs[0].CPut(ctx, keys.NodeLivenessKey(goneNodeID), &livenesspb.Liveness{ + if err := tc.Servers[0].DB().CPut(ctx, keys.NodeLivenessKey(goneNodeID), &livenesspb.Liveness{ NodeID: goneNodeID, Epoch: 1, - Expiration: mtc.clock().Now().ToLegacyTimestamp(), + Expiration: tc.Servers[0].Clock().Now().ToLegacyTimestamp(), Membership: livenesspb.MembershipStatus_ACTIVE, }, nil); err != nil { t.Fatal(err) } - // Decommission from second node. - if committed, err := mtc.nodeLivenesses[1].SetMembershipStatus( - ctx, goneNodeID, livenesspb.MembershipStatus_DECOMMISSIONING); err != nil { - t.Fatal(err) - } else if !committed { - t.Fatal("no change committed") + setMembershipStatus := func(nodeLiveness *liveness.NodeLiveness, + status livenesspb.MembershipStatus, shouldCommit bool) { + if committed, err := nodeLiveness.SetMembershipStatus( + ctx, goneNodeID, status); err != nil { + t.Fatal(err) + } else { + require.Equal(t, committed, shouldCommit) + } } + + // Decommission from second node. + setMembershipStatus(nl1, livenesspb.MembershipStatus_DECOMMISSIONING, true) // Re-decommission from first node. - if committed, err := mtc.nodeLivenesses[0].SetMembershipStatus( - ctx, goneNodeID, livenesspb.MembershipStatus_DECOMMISSIONING); err != nil { - t.Fatal(err) - } else if committed { - t.Fatal("spurious change committed") - } + setMembershipStatus(nl, livenesspb.MembershipStatus_DECOMMISSIONING, false) // Recommission from first node. - if committed, err := mtc.nodeLivenesses[0].SetMembershipStatus( - ctx, goneNodeID, livenesspb.MembershipStatus_ACTIVE); err != nil { - t.Fatal(err) - } else if !committed { - t.Fatal("no change committed") - } + setMembershipStatus(nl, livenesspb.MembershipStatus_ACTIVE, true) // Decommission from second node (a second time). - if committed, err := mtc.nodeLivenesses[1].SetMembershipStatus( - ctx, goneNodeID, livenesspb.MembershipStatus_DECOMMISSIONING); err != nil { - t.Fatal(err) - } else if !committed { - t.Fatal("no change committed") - } + setMembershipStatus(nl1, livenesspb.MembershipStatus_DECOMMISSIONING, true) // Recommission from third node. - if committed, err := mtc.nodeLivenesses[2].SetMembershipStatus( - ctx, goneNodeID, livenesspb.MembershipStatus_ACTIVE); err != nil { - t.Fatal(err) - } else if !committed { - t.Fatal("no change committed") - } + setMembershipStatus(nl2, livenesspb.MembershipStatus_ACTIVE, true) } diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index b9b5f0c5ce14..7af13fb0b6d2 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -806,16 +806,6 @@ func (sc *StoreConfig) LeaseExpiration() int64 { return 2 * (sc.RangeLeaseActiveDuration() + maxOffset).Nanoseconds() } -// RaftElectionTimeoutTicks exposed for testing. -func (s *Store) RaftElectionTimeoutTicks() int { - return s.cfg.RaftElectionTimeoutTicks -} - -// CoalescedHeartbeatsInterval exposed for testing. -func (s *Store) CoalescedHeartbeatsInterval() time.Duration { - return s.cfg.CoalescedHeartbeatsInterval -} - // NewStore returns a new instance of a store. func NewStore( ctx context.Context, cfg StoreConfig, eng storage.Engine, nodeDesc *roachpb.NodeDescriptor, @@ -2332,16 +2322,6 @@ func (s *Store) Stopper() *stop.Stopper { return s.stopper } // TestingKnobs accessor. func (s *Store) TestingKnobs() *StoreTestingKnobs { return &s.cfg.TestingKnobs } -// ClosedTimestamp accessor. -func (s *Store) ClosedTimestamp() *container.Container { - return s.cfg.ClosedTimestamp -} - -// NodeLiveness accessor. -func (s *Store) NodeLiveness() *liveness.NodeLiveness { - return s.cfg.NodeLiveness -} - // IsDraining accessor. func (s *Store) IsDraining() bool { return s.draining.Load().(bool) diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 4282df706c37..2f6e18b50308 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -595,6 +595,31 @@ func (s *adminServer) TableDetails( resp.CreateTableStatement = createStmt } + // Marshal SHOW ZONE CONFIGURATION result. + rows, cols, err = s.server.sqlServer.internalExecutor.QueryWithCols( + ctx, "admin-show-zone-config", nil, /* txn */ + sessiondata.InternalExecutorOverride{User: userName}, + fmt.Sprintf("SHOW ZONE CONFIGURATION FOR TABLE %s", escQualTable)) + if s.isNotFoundError(err) { + return nil, status.Errorf(codes.NotFound, "%s", err) + } + if err != nil { + return nil, s.serverError(err) + } + { + const rawConfigSQLColName = "raw_config_sql" + if len(rows) == 1 { + scanner := makeResultScanner(cols) + var configureZoneStmt string + if err := scanner.Scan(rows[0], rawConfigSQLColName, &configureZoneStmt); err != nil { + return nil, err + } + resp.ConfigureZoneStatement = configureZoneStmt + } else { + resp.ConfigureZoneStatement = "" + } + } + var tableID descpb.ID // Query the descriptor ID and zone configuration for this table. { diff --git a/pkg/server/serverpb/admin.pb.go b/pkg/server/serverpb/admin.pb.go index b7bd77b48d60..af4a9a73c791 100644 --- a/pkg/server/serverpb/admin.pb.go +++ b/pkg/server/serverpb/admin.pb.go @@ -74,7 +74,7 @@ func (x ZoneConfigurationLevel) String() string { return proto.EnumName(ZoneConfigurationLevel_name, int32(x)) } func (ZoneConfigurationLevel) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_admin_beba7622cc4ca35f, []int{0} + return fileDescriptor_admin_065753b953171640, []int{0} } // DatabasesRequest requests a list of databases. @@ -85,7 +85,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_beba7622cc4ca35f, []int{0} + return fileDescriptor_admin_065753b953171640, []int{0} } func (m *DatabasesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -119,7 +119,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_beba7622cc4ca35f, []int{1} + return fileDescriptor_admin_065753b953171640, []int{1} } func (m *DatabasesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -155,7 +155,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_beba7622cc4ca35f, []int{2} + return fileDescriptor_admin_065753b953171640, []int{2} } func (m *DatabaseDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -204,7 +204,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_beba7622cc4ca35f, []int{3} + return fileDescriptor_admin_065753b953171640, []int{3} } func (m *DatabaseDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -240,7 +240,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_beba7622cc4ca35f, []int{3, 0} + return fileDescriptor_admin_065753b953171640, []int{3, 0} } func (m *DatabaseDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -280,7 +280,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_beba7622cc4ca35f, []int{4} + return fileDescriptor_admin_065753b953171640, []int{4} } func (m *TableDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -331,13 +331,17 @@ type TableDetailsResponse struct { // It can be used to find events pertaining to this table by filtering on // the 'target_id' field of events. DescriptorID int64 `protobuf:"varint,8,opt,name=descriptor_id,json=descriptorId,proto3" json:"descriptor_id,omitempty"` + // configure_zone_statement is the output of "SHOW ZONE CONFIGURATION FOR TABLE" + // for this table. It is a SQL statement that would re-configure the table's current + // zone if executed. + ConfigureZoneStatement string `protobuf:"bytes,9,opt,name=configure_zone_statement,json=configureZoneStatement,proto3" json:"configure_zone_statement,omitempty"` } 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_beba7622cc4ca35f, []int{5} + return fileDescriptor_admin_065753b953171640, []int{5} } func (m *TableDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -374,7 +378,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_beba7622cc4ca35f, []int{5, 0} + return fileDescriptor_admin_065753b953171640, []int{5, 0} } func (m *TableDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -418,7 +422,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_beba7622cc4ca35f, []int{5, 1} + return fileDescriptor_admin_065753b953171640, []int{5, 1} } func (m *TableDetailsResponse_Column) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -464,7 +468,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_beba7622cc4ca35f, []int{5, 2} + return fileDescriptor_admin_065753b953171640, []int{5, 2} } func (m *TableDetailsResponse_Index) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -505,7 +509,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_beba7622cc4ca35f, []int{6} + return fileDescriptor_admin_065753b953171640, []int{6} } func (m *TableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,7 +563,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_beba7622cc4ca35f, []int{7} + return fileDescriptor_admin_065753b953171640, []int{7} } func (m *TableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -597,7 +601,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_beba7622cc4ca35f, []int{7, 0} + return fileDescriptor_admin_065753b953171640, []int{7, 0} } func (m *TableStatsResponse_MissingNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -631,7 +635,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_beba7622cc4ca35f, []int{8} + return fileDescriptor_admin_065753b953171640, []int{8} } func (m *NonTableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -670,7 +674,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_beba7622cc4ca35f, []int{9} + return fileDescriptor_admin_065753b953171640, []int{9} } func (m *NonTableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -703,7 +707,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_beba7622cc4ca35f, []int{10} + return fileDescriptor_admin_065753b953171640, []int{10} } func (m *UsersRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -738,7 +742,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_beba7622cc4ca35f, []int{11} + return fileDescriptor_admin_065753b953171640, []int{11} } func (m *UsersResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -772,7 +776,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_beba7622cc4ca35f, []int{11, 0} + return fileDescriptor_admin_065753b953171640, []int{11, 0} } func (m *UsersResponse_User) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -819,7 +823,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_beba7622cc4ca35f, []int{12} + return fileDescriptor_admin_065753b953171640, []int{12} } func (m *EventsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -854,7 +858,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_beba7622cc4ca35f, []int{13} + return fileDescriptor_admin_065753b953171640, []int{13} } func (m *EventsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -899,7 +903,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_beba7622cc4ca35f, []int{13, 0} + return fileDescriptor_admin_065753b953171640, []int{13, 0} } func (m *EventsResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -935,7 +939,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_beba7622cc4ca35f, []int{14} + return fileDescriptor_admin_065753b953171640, []int{14} } func (m *SetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -968,7 +972,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_beba7622cc4ca35f, []int{15} + return fileDescriptor_admin_065753b953171640, []int{15} } func (m *SetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1003,7 +1007,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_beba7622cc4ca35f, []int{16} + return fileDescriptor_admin_065753b953171640, []int{16} } func (m *GetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1040,7 +1044,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_beba7622cc4ca35f, []int{17} + return fileDescriptor_admin_065753b953171640, []int{17} } func (m *GetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1076,7 +1080,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_beba7622cc4ca35f, []int{17, 0} + return fileDescriptor_admin_065753b953171640, []int{17, 0} } func (m *GetUIDataResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1109,7 +1113,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_beba7622cc4ca35f, []int{18} + return fileDescriptor_admin_065753b953171640, []int{18} } func (m *ClusterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1148,7 +1152,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_beba7622cc4ca35f, []int{19} + return fileDescriptor_admin_065753b953171640, []int{19} } func (m *ClusterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1196,7 +1200,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_beba7622cc4ca35f, []int{20} + return fileDescriptor_admin_065753b953171640, []int{20} } func (m *DrainRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1281,7 +1285,7 @@ func (m *DrainResponse) Reset() { *m = DrainResponse{} } func (m *DrainResponse) String() string { return proto.CompactTextString(m) } func (*DrainResponse) ProtoMessage() {} func (*DrainResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_admin_beba7622cc4ca35f, []int{21} + return fileDescriptor_admin_065753b953171640, []int{21} } func (m *DrainResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1316,7 +1320,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_beba7622cc4ca35f, []int{22} + return fileDescriptor_admin_065753b953171640, []int{22} } func (m *DecommissionStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1354,7 +1358,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_beba7622cc4ca35f, []int{23} + return fileDescriptor_admin_065753b953171640, []int{23} } func (m *DecommissionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1389,7 +1393,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_beba7622cc4ca35f, []int{24} + return fileDescriptor_admin_065753b953171640, []int{24} } func (m *DecommissionStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1428,7 +1432,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_beba7622cc4ca35f, []int{24, 0} + return fileDescriptor_admin_065753b953171640, []int{24, 0} } func (m *DecommissionStatusResponse_Status) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1470,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_beba7622cc4ca35f, []int{25} + return fileDescriptor_admin_065753b953171640, []int{25} } func (m *SettingsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1504,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_beba7622cc4ca35f, []int{26} + return fileDescriptor_admin_065753b953171640, []int{26} } func (m *SettingsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1540,7 +1544,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_beba7622cc4ca35f, []int{26, 0} + return fileDescriptor_admin_065753b953171640, []int{26, 0} } func (m *SettingsResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1595,7 +1599,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_beba7622cc4ca35f, []int{27} + return fileDescriptor_admin_065753b953171640, []int{27} } func (m *HealthRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1630,7 +1634,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_beba7622cc4ca35f, []int{28} + return fileDescriptor_admin_065753b953171640, []int{28} } func (m *HealthResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1663,7 +1667,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_beba7622cc4ca35f, []int{29} + return fileDescriptor_admin_065753b953171640, []int{29} } func (m *LivenessRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1698,7 +1702,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_beba7622cc4ca35f, []int{30} + return fileDescriptor_admin_065753b953171640, []int{30} } func (m *LivenessResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1734,7 +1738,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_beba7622cc4ca35f, []int{31} + return fileDescriptor_admin_065753b953171640, []int{31} } func (m *JobsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1768,7 +1772,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_beba7622cc4ca35f, []int{32} + return fileDescriptor_admin_065753b953171640, []int{32} } func (m *JobsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1821,7 +1825,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_beba7622cc4ca35f, []int{32, 0} + return fileDescriptor_admin_065753b953171640, []int{32, 0} } func (m *JobsResponse_Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1854,7 +1858,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_beba7622cc4ca35f, []int{33} + return fileDescriptor_admin_065753b953171640, []int{33} } func (m *LocationsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1888,7 +1892,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_beba7622cc4ca35f, []int{34} + return fileDescriptor_admin_065753b953171640, []int{34} } func (m *LocationsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1924,7 +1928,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_beba7622cc4ca35f, []int{34, 0} + return fileDescriptor_admin_065753b953171640, []int{34, 0} } func (m *LocationsResponse_Location) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1966,7 +1970,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_beba7622cc4ca35f, []int{35} + return fileDescriptor_admin_065753b953171640, []int{35} } func (m *RangeLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2000,7 +2004,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_beba7622cc4ca35f, []int{36} + return fileDescriptor_admin_065753b953171640, []int{36} } func (m *RangeLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2040,7 +2044,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_beba7622cc4ca35f, []int{36, 0} + return fileDescriptor_admin_065753b953171640, []int{36, 0} } func (m *RangeLogResponse_PrettyInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2074,7 +2078,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_beba7622cc4ca35f, []int{36, 1} + return fileDescriptor_admin_065753b953171640, []int{36, 1} } func (m *RangeLogResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2109,7 +2113,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_beba7622cc4ca35f, []int{37} + return fileDescriptor_admin_065753b953171640, []int{37} } func (m *QueryPlanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2144,7 +2148,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_beba7622cc4ca35f, []int{38} + return fileDescriptor_admin_065753b953171640, []int{38} } func (m *QueryPlanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2176,7 +2180,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_beba7622cc4ca35f, []int{39} + return fileDescriptor_admin_065753b953171640, []int{39} } func (m *DataDistributionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2212,7 +2216,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_beba7622cc4ca35f, []int{40} + return fileDescriptor_admin_065753b953171640, []int{40} } func (m *DataDistributionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2250,7 +2254,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_beba7622cc4ca35f, []int{40, 0} + return fileDescriptor_admin_065753b953171640, []int{40, 0} } func (m *DataDistributionResponse_ZoneConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2285,7 +2289,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_beba7622cc4ca35f, []int{40, 1} + return fileDescriptor_admin_065753b953171640, []int{40, 1} } func (m *DataDistributionResponse_TableInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2319,7 +2323,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_beba7622cc4ca35f, []int{40, 2} + return fileDescriptor_admin_065753b953171640, []int{40, 2} } func (m *DataDistributionResponse_DatabaseInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2352,7 +2356,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_beba7622cc4ca35f, []int{41} + return fileDescriptor_admin_065753b953171640, []int{41} } func (m *MetricMetadataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2386,7 +2390,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_beba7622cc4ca35f, []int{42} + return fileDescriptor_admin_065753b953171640, []int{42} } func (m *MetricMetadataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2429,7 +2433,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_beba7622cc4ca35f, []int{43} + return fileDescriptor_admin_065753b953171640, []int{43} } func (m *EnqueueRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2462,7 +2466,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_beba7622cc4ca35f, []int{44} + return fileDescriptor_admin_065753b953171640, []int{44} } func (m *EnqueueRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2499,7 +2503,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_beba7622cc4ca35f, []int{44, 0} + return fileDescriptor_admin_065753b953171640, []int{44, 0} } func (m *EnqueueRangeResponse_Details) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2532,7 +2536,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_beba7622cc4ca35f, []int{45} + return fileDescriptor_admin_065753b953171640, []int{45} } func (m *ChartCatalogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2566,7 +2570,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_beba7622cc4ca35f, []int{46} + return fileDescriptor_admin_065753b953171640, []int{46} } func (m *ChartCatalogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3957,6 +3961,12 @@ func (m *TableDetailsResponse) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintAdmin(dAtA, i, uint64(m.DescriptorID)) } + if len(m.ConfigureZoneStatement) > 0 { + dAtA[i] = 0x4a + i++ + i = encodeVarintAdmin(dAtA, i, uint64(len(m.ConfigureZoneStatement))) + i += copy(dAtA[i:], m.ConfigureZoneStatement) + } return i, nil } @@ -6365,6 +6375,10 @@ func (m *TableDetailsResponse) Size() (n int) { if m.DescriptorID != 0 { n += 1 + sovAdmin(uint64(m.DescriptorID)) } + l = len(m.ConfigureZoneStatement) + if l > 0 { + n += 1 + l + sovAdmin(uint64(l)) + } return n } @@ -8294,6 +8308,35 @@ func (m *TableDetailsResponse) Unmarshal(dAtA []byte) error { break } } + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConfigureZoneStatement", 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.ConfigureZoneStatement = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipAdmin(dAtA[iNdEx:]) @@ -15681,269 +15724,270 @@ var ( ErrIntOverflowAdmin = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_beba7622cc4ca35f) } - -var fileDescriptor_admin_beba7622cc4ca35f = []byte{ - // 4167 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x5d, 0x73, 0x1b, 0xd7, - 0x75, 0x5a, 0x80, 0xf8, 0x3a, 0x04, 0x48, 0xf0, 0x8a, 0xa2, 0x40, 0x48, 0x21, 0xe8, 0x55, 0x1c, - 0x33, 0xb2, 0x03, 0x48, 0x94, 0x6c, 0xa7, 0xb2, 0x9d, 0x94, 0x20, 0x38, 0x0a, 0x64, 0x4a, 0xb6, - 0x96, 0x92, 0x3c, 0x91, 0xeb, 0x6c, 0x17, 0xd8, 0x4b, 0x70, 0xc3, 0xc5, 0xee, 0x72, 0x77, 0x41, - 0x99, 0x76, 0x93, 0x69, 0x92, 0xb6, 0xd3, 0xa7, 0x8e, 0x27, 0xe9, 0xf4, 0xc5, 0x4f, 0xed, 0x43, - 0xfb, 0xd0, 0x97, 0x3a, 0x8f, 0x7d, 0xe8, 0xb3, 0xa7, 0x2f, 0x49, 0xa7, 0xd3, 0x99, 0xf4, 0x85, - 0x69, 0xe9, 0x3e, 0xb5, 0xff, 0xa0, 0x33, 0xfd, 0x98, 0xfb, 0xb5, 0x7b, 0x01, 0x42, 0x2b, 0x80, - 0x8a, 0xf3, 0x02, 0xec, 0x3d, 0xe7, 0xde, 0x73, 0xcf, 0x3d, 0xe7, 0xdc, 0x73, 0xcf, 0x39, 0xf7, - 0xc2, 0xa5, 0x00, 0xfb, 0x87, 0xd8, 0x6f, 0xb0, 0x3f, 0xaf, 0xd3, 0x30, 0xcc, 0xbe, 0xe5, 0xd4, - 0x3d, 0xdf, 0x0d, 0x5d, 0xb4, 0xdc, 0x75, 0xbb, 0xfb, 0xbe, 0x6b, 0x74, 0xf7, 0xea, 0x0c, 0x5f, - 0x17, 0xdd, 0xaa, 0x95, 0xae, 0xeb, 0xec, 0x5a, 0xbd, 0xc6, 0x47, 0xae, 0x83, 0xbd, 0x0e, 0xfd, - 0x63, 0x83, 0xaa, 0x4b, 0xdf, 0x77, 0x3b, 0x41, 0x83, 0xfc, 0x78, 0x1d, 0xfa, 0xc7, 0xe1, 0x97, - 0x47, 0x67, 0x0a, 0x42, 0x23, 0x1c, 0x08, 0xec, 0xa5, 0x20, 0x74, 0x7d, 0xa3, 0x87, 0x1b, 0xd8, - 0xe9, 0x59, 0x94, 0x64, 0xff, 0xb0, 0xdb, 0xe5, 0xc8, 0xfa, 0xfe, 0x61, 0x63, 0xff, 0x90, 0x8f, - 0xb7, 0xad, 0x43, 0xec, 0xe0, 0x20, 0x88, 0x3e, 0xbc, 0x4e, 0xf4, 0xc9, 0xfb, 0xaf, 0xca, 0xfd, - 0xc5, 0x07, 0xe9, 0xe6, 0xf6, 0x78, 0x8f, 0x95, 0x30, 0x68, 0x74, 0x8d, 0xd0, 0xb0, 0xdd, 0x5e, - 0xa3, 0xbb, 0x67, 0xf8, 0xa1, 0xce, 0x5b, 0x1c, 0x5f, 0x19, 0x84, 0x96, 0xdd, 0xe8, 0xe3, 0xd0, - 0xb7, 0xba, 0xfc, 0x8f, 0x63, 0x16, 0x7b, 0x6e, 0xcf, 0xa5, 0x9f, 0x0d, 0xf2, 0x25, 0x16, 0xd7, - 0x73, 0xdd, 0x9e, 0x8d, 0x1b, 0x86, 0x67, 0x35, 0x0c, 0xc7, 0x71, 0x43, 0x23, 0xb4, 0x5c, 0x47, - 0xf0, 0x53, 0xe3, 0x58, 0xda, 0xea, 0x0c, 0x76, 0x1b, 0xa1, 0xd5, 0xc7, 0x41, 0x68, 0xf4, 0x3d, - 0xd6, 0x41, 0x45, 0x50, 0x6e, 0x19, 0xa1, 0xd1, 0x31, 0x02, 0x1c, 0x68, 0xf8, 0x60, 0x80, 0x83, - 0x50, 0xbd, 0x0e, 0x0b, 0x12, 0x2c, 0xf0, 0x5c, 0x27, 0xc0, 0xe8, 0x32, 0x14, 0x4c, 0x01, 0xac, - 0x28, 0xab, 0xe9, 0xb5, 0x82, 0x16, 0x03, 0xd4, 0x9b, 0xb0, 0x24, 0x86, 0xb4, 0x70, 0x68, 0x58, - 0xb6, 0x20, 0x86, 0xaa, 0x90, 0x17, 0xdd, 0x2a, 0xca, 0xaa, 0xb2, 0x56, 0xd0, 0xa2, 0xb6, 0xfa, - 0x59, 0x1a, 0x2e, 0x9e, 0x1a, 0xc6, 0xe7, 0x7b, 0x04, 0xd9, 0x9e, 0x6f, 0x38, 0x21, 0x9b, 0x6c, - 0x76, 0xfd, 0x9b, 0xf5, 0xa7, 0x9a, 0x44, 0xfd, 0x29, 0x34, 0xea, 0xb7, 0x09, 0x81, 0xe6, 0xcc, - 0xe7, 0xc7, 0xb5, 0x73, 0x1a, 0xa7, 0x86, 0x6a, 0x30, 0x1b, 0x1a, 0x1d, 0x1b, 0xeb, 0x8e, 0xd1, - 0xc7, 0x41, 0x25, 0x45, 0x57, 0x02, 0x14, 0x74, 0x8f, 0x40, 0xd0, 0xab, 0x50, 0x32, 0x71, 0xd0, - 0xf5, 0x2d, 0x2f, 0x74, 0x7d, 0xdd, 0x32, 0x2b, 0xe9, 0x55, 0x65, 0x2d, 0xdd, 0x2c, 0x9f, 0x1c, - 0xd7, 0x8a, 0xad, 0x08, 0xd1, 0x6e, 0x69, 0xc5, 0xb8, 0x5b, 0xdb, 0x44, 0x77, 0x60, 0x96, 0x98, - 0xa2, 0xce, 0xac, 0xb3, 0x32, 0xb3, 0xaa, 0xac, 0xcd, 0xae, 0x5f, 0x91, 0x98, 0x66, 0x88, 0x3a, - 0x33, 0xdb, 0xfa, 0x63, 0xd7, 0xc1, 0x9b, 0x14, 0xc2, 0xf9, 0x83, 0x8f, 0x22, 0x08, 0xfa, 0x00, - 0x16, 0x24, 0x5a, 0xba, 0x8d, 0x0f, 0xb1, 0x5d, 0xc9, 0xac, 0x2a, 0x6b, 0x73, 0xeb, 0xd7, 0x13, - 0xc4, 0x10, 0xd3, 0x1c, 0xf8, 0xd4, 0x0a, 0xb6, 0xc9, 0x40, 0x6d, 0x3e, 0xa6, 0x4c, 0x01, 0xd5, - 0x37, 0x20, 0x43, 0x25, 0x83, 0x10, 0xcc, 0x0c, 0x02, 0xec, 0x73, 0xbd, 0xd0, 0x6f, 0xb4, 0x02, - 0xe0, 0xf9, 0xd6, 0xa1, 0x65, 0xe3, 0x5e, 0x2c, 0x9e, 0x18, 0xa2, 0xde, 0x86, 0xf3, 0x0f, 0x88, - 0xb0, 0x26, 0x57, 0x33, 0x5a, 0x84, 0x0c, 0x95, 0x6f, 0x25, 0x45, 0x11, 0xac, 0xa1, 0xfe, 0x71, - 0x1e, 0x16, 0x87, 0x29, 0x71, 0xcd, 0xef, 0x8c, 0x68, 0xfe, 0xd5, 0x84, 0x25, 0x8f, 0x23, 0x30, - 0x56, 0xed, 0x8f, 0x20, 0xd7, 0x75, 0xed, 0x41, 0xdf, 0x61, 0x6b, 0x9a, 0x5d, 0x7f, 0x6d, 0x5a, - 0xaa, 0x9b, 0x74, 0x38, 0x27, 0x2b, 0x88, 0xa1, 0x87, 0x90, 0xb3, 0x1c, 0x13, 0x7f, 0x88, 0x83, - 0x4a, 0xfa, 0x6c, 0xdc, 0xb6, 0xc9, 0x70, 0x41, 0x96, 0xd3, 0x22, 0x56, 0xea, 0x1b, 0x4e, 0x8f, - 0x98, 0xc0, 0xc0, 0x09, 0xa9, 0x35, 0xa5, 0x35, 0xa0, 0xa0, 0x4d, 0x02, 0x41, 0x37, 0x61, 0xa9, - 0xeb, 0x63, 0x23, 0xc4, 0x3a, 0xb3, 0x66, 0xe2, 0xd2, 0x70, 0x1f, 0x3b, 0x21, 0xb5, 0x93, 0x82, - 0xb6, 0xc8, 0xb0, 0x74, 0xc6, 0x1d, 0x81, 0x1b, 0x35, 0xd2, 0xec, 0x6f, 0xdc, 0x48, 0x73, 0xbf, - 0x29, 0x23, 0x3d, 0xbd, 0x0d, 0xf3, 0x93, 0x6c, 0xc3, 0xe7, 0xb2, 0xed, 0xea, 0x3f, 0x28, 0x90, - 0x65, 0x6a, 0x26, 0xc3, 0x89, 0x83, 0x10, 0xc3, 0xc9, 0x37, 0x81, 0x85, 0x47, 0x9e, 0x30, 0x63, - 0xfa, 0x4d, 0xec, 0xde, 0x19, 0xd8, 0x36, 0x35, 0x6f, 0xe2, 0x28, 0xf2, 0x5a, 0xd4, 0x46, 0x57, - 0xc8, 0x12, 0x76, 0x8d, 0x81, 0x1d, 0xea, 0x87, 0x86, 0x3d, 0xc0, 0x54, 0x8d, 0x05, 0xc2, 0x30, - 0x05, 0x3e, 0x22, 0x30, 0x74, 0x03, 0x2e, 0xf4, 0xb0, 0x83, 0x99, 0x2c, 0x74, 0xfc, 0xa1, 0xe7, - 0xe3, 0x20, 0xb0, 0x5c, 0x47, 0xe8, 0x31, 0x46, 0x6e, 0x45, 0x38, 0xb4, 0x04, 0xd9, 0x3d, 0xcb, - 0x34, 0xb1, 0x43, 0x55, 0x98, 0xd7, 0x78, 0xab, 0xfa, 0x99, 0x02, 0x19, 0x6a, 0x4f, 0x63, 0xf9, - 0x5f, 0x82, 0xec, 0xc0, 0xb1, 0x0e, 0x06, 0x6c, 0x05, 0x79, 0x8d, 0xb7, 0x50, 0x19, 0xd2, 0x01, - 0x3e, 0x60, 0x7e, 0x4e, 0x23, 0x9f, 0xa4, 0x27, 0x33, 0x70, 0xce, 0x32, 0x6f, 0xd1, 0x43, 0xc0, - 0xf2, 0x71, 0x37, 0x8c, 0x19, 0x8c, 0x01, 0xa8, 0x02, 0x39, 0x72, 0x96, 0x5a, 0x4e, 0x8f, 0xb3, - 0x25, 0x9a, 0x44, 0x4a, 0x56, 0xdf, 0xb3, 0xad, 0xae, 0x15, 0x52, 0x13, 0xc9, 0x6b, 0x51, 0x5b, - 0xdd, 0x82, 0x85, 0xc8, 0x4a, 0x9f, 0xc3, 0x9d, 0x7c, 0x96, 0x06, 0x24, 0xd3, 0xe1, 0xce, 0x64, - 0x64, 0x23, 0x29, 0xa7, 0x36, 0xd2, 0x15, 0x28, 0xf9, 0x98, 0xb0, 0x62, 0xf0, 0x2e, 0x29, 0xda, - 0xa5, 0xc8, 0x81, 0xac, 0xd3, 0x57, 0x00, 0x1c, 0xd7, 0x14, 0x44, 0x98, 0xa0, 0x0a, 0x04, 0xc2, - 0xd0, 0x1b, 0x90, 0x21, 0xfb, 0x2f, 0xe0, 0x5e, 0xff, 0x45, 0xd9, 0xfc, 0x59, 0x70, 0x51, 0x17, - 0xc1, 0x45, 0xfd, 0xee, 0xa3, 0xcd, 0x4d, 0xca, 0x22, 0xdf, 0x52, 0x6c, 0x24, 0x32, 0xa1, 0xd4, - 0xb7, 0x82, 0xc0, 0x72, 0x7a, 0x3a, 0xa1, 0x1b, 0x54, 0x32, 0xd4, 0x9b, 0xfc, 0xce, 0xb3, 0xbc, - 0xc9, 0xd0, 0x6a, 0xeb, 0x77, 0x19, 0x89, 0x7b, 0xae, 0x89, 0x39, 0xf9, 0x62, 0x3f, 0x06, 0x05, - 0xc4, 0x6b, 0x18, 0x9e, 0xe7, 0xbb, 0x1f, 0x5a, 0x7d, 0xe2, 0x3a, 0x4c, 0x2b, 0xd8, 0xd7, 0x3b, - 0x47, 0x21, 0x0e, 0xa8, 0xc2, 0x66, 0xb4, 0x45, 0x09, 0xdb, 0xb2, 0x82, 0xfd, 0x26, 0xc1, 0x55, - 0xdf, 0x83, 0x59, 0x89, 0x30, 0xba, 0x02, 0x39, 0x2a, 0x0c, 0xcb, 0x64, 0xaa, 0x69, 0xc2, 0xc9, - 0x71, 0x2d, 0x4b, 0x50, 0xed, 0x96, 0x96, 0x25, 0xa8, 0xb6, 0x49, 0xc4, 0x8a, 0x7d, 0xdf, 0xf5, - 0xf5, 0x3e, 0x0e, 0x02, 0xa3, 0x27, 0x94, 0x55, 0xa4, 0xc0, 0xbb, 0x0c, 0xa6, 0x2e, 0xc1, 0xe2, - 0x3d, 0xd7, 0x39, 0xa5, 0x7d, 0xf5, 0x17, 0x0a, 0x5c, 0x18, 0x41, 0x70, 0x75, 0x7e, 0x17, 0x16, - 0x48, 0x04, 0xa3, 0x07, 0xd8, 0xb7, 0x70, 0xa0, 0x33, 0xa9, 0x2b, 0x54, 0xea, 0xdf, 0x98, 0x4a, - 0x54, 0xda, 0x3c, 0xa1, 0xb3, 0x43, 0xc9, 0x50, 0x04, 0x7a, 0x1f, 0x90, 0xe5, 0x84, 0xd8, 0x77, - 0x0c, 0x5b, 0x1f, 0x04, 0x98, 0xd3, 0x4e, 0x9d, 0x85, 0x76, 0x59, 0x10, 0x7a, 0x18, 0x30, 0x8c, - 0x3a, 0x07, 0xc5, 0x87, 0x01, 0xf6, 0xa3, 0x15, 0xfe, 0x10, 0x4a, 0xbc, 0xcd, 0x17, 0xd6, 0x86, - 0x0c, 0x71, 0x51, 0xe2, 0xcc, 0x4b, 0x9a, 0x70, 0x68, 0x20, 0x6d, 0x09, 0x53, 0xa2, 0x14, 0xaa, - 0x2a, 0xcc, 0x10, 0x20, 0xd9, 0x43, 0x04, 0x20, 0xb9, 0x81, 0xa8, 0xad, 0xfe, 0x44, 0x81, 0xd2, - 0xd6, 0x21, 0x76, 0xe2, 0x1d, 0x27, 0x9c, 0x9b, 0x22, 0x39, 0xb7, 0x4b, 0x50, 0x08, 0x0d, 0xbf, - 0x87, 0x43, 0xa2, 0x6b, 0xb6, 0x2f, 0xf2, 0x0c, 0xd0, 0x36, 0xc9, 0x36, 0xb4, 0xad, 0xbe, 0xc5, - 0xb6, 0x43, 0x46, 0x63, 0x0d, 0xf4, 0x32, 0x2c, 0x0c, 0x1c, 0x1f, 0x9b, 0x46, 0x37, 0xc4, 0xa6, - 0x8e, 0xe9, 0x14, 0x74, 0x5b, 0xe4, 0xb5, 0x72, 0x8c, 0x60, 0x53, 0xab, 0xff, 0x99, 0x82, 0x39, - 0xc1, 0x05, 0x97, 0xc3, 0x5d, 0xc8, 0xf2, 0x41, 0x4c, 0x10, 0x8d, 0x04, 0x41, 0x0c, 0x0f, 0x65, - 0x4d, 0x71, 0xec, 0x33, 0x22, 0xd5, 0x3f, 0x4d, 0x41, 0x86, 0xc2, 0x51, 0x13, 0x0a, 0x51, 0xec, - 0xcb, 0x2d, 0xa6, 0x5a, 0x67, 0xd1, 0x71, 0x5d, 0x44, 0xc7, 0xf5, 0x07, 0xa2, 0x47, 0x33, 0x4f, - 0xc8, 0x7c, 0xf2, 0xeb, 0x9a, 0xa2, 0xc5, 0xc3, 0x88, 0x1b, 0xa0, 0x74, 0x75, 0xe9, 0x18, 0x28, - 0x50, 0xc8, 0x03, 0x22, 0xae, 0xaf, 0xcb, 0xe2, 0x62, 0x51, 0x63, 0xf1, 0xe4, 0xb8, 0x96, 0x7f, - 0xc0, 0x44, 0xd6, 0x92, 0x84, 0xb7, 0x0e, 0xc4, 0xc1, 0xb8, 0x7e, 0x48, 0x36, 0xbc, 0x65, 0xb2, - 0x03, 0xbe, 0x39, 0x7f, 0x72, 0x5c, 0x9b, 0xd5, 0x04, 0xbc, 0xdd, 0xd2, 0x66, 0xa3, 0x4e, 0x6d, - 0x93, 0x68, 0xc8, 0x72, 0x76, 0x5d, 0xee, 0x77, 0xe9, 0x37, 0x99, 0x92, 0x39, 0x71, 0x42, 0x84, - 0xec, 0xe1, 0x22, 0x9b, 0xf2, 0x21, 0x05, 0x92, 0x29, 0x19, 0xba, 0x6d, 0xaa, 0x7f, 0xab, 0x40, - 0x79, 0x07, 0x87, 0x0f, 0xdb, 0x24, 0x5a, 0x16, 0x5a, 0xff, 0x2e, 0xc0, 0x3e, 0x3e, 0x62, 0xc7, - 0x93, 0x10, 0xf9, 0xad, 0x04, 0x91, 0x8f, 0x12, 0xa8, 0xbf, 0x8d, 0x8f, 0xe8, 0x39, 0x16, 0x6c, - 0x39, 0xa1, 0x7f, 0xa4, 0x15, 0xf6, 0x45, 0xbb, 0xfa, 0x26, 0xcc, 0x0d, 0x23, 0xc9, 0x39, 0xb3, - 0x8f, 0x8f, 0xb8, 0x85, 0x91, 0x4f, 0x62, 0x43, 0xec, 0x64, 0x24, 0xb2, 0x2c, 0x6a, 0xac, 0x71, - 0x2b, 0xf5, 0x4d, 0x45, 0x3d, 0x0f, 0x0b, 0xd2, 0x5c, 0x4c, 0xc3, 0xea, 0xd7, 0xa0, 0x7c, 0x7b, - 0x74, 0x05, 0x08, 0x66, 0xf6, 0xf1, 0x91, 0x48, 0x49, 0xe8, 0xb7, 0xfa, 0x8b, 0x14, 0x2c, 0xdc, - 0x1e, 0x1d, 0x8d, 0x7e, 0x7f, 0xcc, 0x5a, 0xdf, 0x48, 0x58, 0xeb, 0x29, 0x0a, 0x23, 0x8b, 0xe5, - 0xa6, 0x26, 0x2d, 0x79, 0x17, 0x32, 0xec, 0x50, 0x8f, 0xd6, 0xa5, 0x48, 0xeb, 0x42, 0xb7, 0xa1, - 0x68, 0x1b, 0x41, 0xa8, 0x0f, 0x3c, 0xd3, 0x08, 0xb1, 0xc9, 0x7d, 0xcb, 0x64, 0x56, 0x38, 0x4b, - 0x46, 0x3e, 0x64, 0x03, 0xab, 0xde, 0x04, 0xa2, 0xfd, 0x8e, 0x2c, 0xda, 0xd9, 0xf5, 0xf5, 0xa9, - 0x16, 0x4a, 0x49, 0xcb, 0xea, 0x28, 0xc3, 0xdc, 0xa6, 0x3d, 0x08, 0x42, 0xec, 0x0b, 0x0f, 0xf6, - 0xa9, 0x02, 0xf3, 0x11, 0x88, 0x4b, 0xf8, 0x15, 0x80, 0x2e, 0x03, 0xc5, 0x87, 0x43, 0xe9, 0xe4, - 0xb8, 0x56, 0xe0, 0x1d, 0xdb, 0x2d, 0xad, 0xc0, 0x3b, 0xb4, 0x4d, 0xe2, 0x2a, 0xe2, 0x3d, 0x80, - 0x1d, 0xe2, 0x46, 0x4d, 0x1e, 0x99, 0x94, 0x23, 0xc4, 0x16, 0x83, 0xa3, 0x6f, 0x00, 0xc2, 0xc4, - 0xa9, 0x7a, 0xbe, 0x15, 0xe0, 0xa8, 0x37, 0x8b, 0xb8, 0x16, 0x62, 0x0c, 0xef, 0xae, 0x7a, 0x50, - 0x6c, 0xf9, 0x86, 0xe5, 0x08, 0x2b, 0xb9, 0x02, 0x25, 0xcf, 0xc7, 0xeb, 0xd7, 0xae, 0xeb, 0x7d, - 0xc3, 0xdf, 0xa7, 0x61, 0x61, 0x7a, 0x2d, 0xa3, 0x15, 0x19, 0xf0, 0x2e, 0x85, 0x11, 0x87, 0x19, - 0xec, 0x0d, 0x42, 0xd3, 0x7d, 0xe2, 0x88, 0x58, 0x4e, 0xb4, 0xd1, 0x32, 0xe4, 0x4d, 0x57, 0x37, - 0x09, 0x4d, 0xee, 0xce, 0x72, 0xa6, 0x4b, 0xa7, 0xb8, 0x33, 0x93, 0x4f, 0x95, 0xd3, 0xea, 0xcf, - 0x15, 0x28, 0xf1, 0x29, 0xe3, 0xd0, 0xc3, 0x0a, 0xd8, 0x10, 0x12, 0x12, 0xb1, 0x95, 0x81, 0x15, - 0xb4, 0x38, 0x04, 0xdd, 0x82, 0x65, 0x8a, 0xd5, 0x7d, 0xdc, 0x67, 0x20, 0xdd, 0x72, 0x4c, 0xab, - 0x6b, 0x84, 0xae, 0x4f, 0x19, 0x98, 0xd1, 0x2e, 0x9a, 0x8c, 0x24, 0xc7, 0xb7, 0x05, 0x1a, 0x7d, - 0x0b, 0x2e, 0x8d, 0x8e, 0x15, 0x71, 0x30, 0x89, 0xcd, 0x58, 0xd8, 0xb6, 0x3c, 0x3c, 0xba, 0x15, - 0x77, 0xb8, 0x33, 0x93, 0x57, 0xca, 0x29, 0xf5, 0x23, 0x58, 0x6e, 0xe1, 0xae, 0xdb, 0xa7, 0x41, - 0x82, 0xeb, 0xec, 0xd0, 0xba, 0x88, 0x90, 0xd9, 0x07, 0x90, 0xe7, 0xe7, 0x3c, 0xdb, 0x2d, 0x99, - 0x66, 0xf3, 0xe4, 0xb8, 0x96, 0x63, 0x07, 0x7d, 0xf0, 0xdf, 0xc7, 0xb5, 0x1b, 0x3d, 0x2b, 0xdc, - 0x1b, 0x74, 0xea, 0x5d, 0xb7, 0xdf, 0x88, 0xac, 0xcb, 0xec, 0xc4, 0xdf, 0x0d, 0x6f, 0xbf, 0xd7, - 0xa0, 0x5f, 0x5e, 0xa7, 0xce, 0x03, 0x84, 0x1c, 0x0b, 0x10, 0x02, 0xf5, 0x58, 0x81, 0xf3, 0xf2, - 0xe4, 0xbf, 0x9d, 0x69, 0xd1, 0x1e, 0x2c, 0x70, 0x27, 0xdd, 0xc7, 0xfd, 0x0e, 0xf6, 0x83, 0x3d, - 0xcb, 0xa3, 0xba, 0x99, 0x1b, 0x72, 0x06, 0xfb, 0x87, 0x75, 0x51, 0xbe, 0xa9, 0x47, 0x35, 0x9e, - 0xb8, 0xee, 0x53, 0xbf, 0x1b, 0x0d, 0xe6, 0x42, 0x2b, 0x33, 0xaa, 0x31, 0x5c, 0xfd, 0x8b, 0x34, - 0x54, 0xc7, 0x49, 0x97, 0x9b, 0xc7, 0x63, 0xc8, 0xb2, 0x3a, 0x14, 0x4f, 0x48, 0xdf, 0x4c, 0x2a, - 0x70, 0x3c, 0x95, 0x4c, 0x9d, 0x35, 0xc5, 0xb1, 0xc7, 0x28, 0x56, 0x3f, 0x4d, 0x41, 0x96, 0x21, - 0xd0, 0xe3, 0xe1, 0x68, 0x2d, 0xd3, 0xdc, 0x88, 0xa3, 0xb5, 0xb3, 0x0a, 0x53, 0x04, 0x79, 0x17, - 0x21, 0x67, 0x05, 0x3a, 0x91, 0x8a, 0xc8, 0x28, 0xac, 0x60, 0xdb, 0x3a, 0xc4, 0xa7, 0x83, 0xea, - 0xf4, 0x98, 0xa0, 0xfa, 0x7d, 0x00, 0x49, 0x05, 0x33, 0xcf, 0xaf, 0x02, 0x89, 0x1c, 0x4d, 0x20, - 0xc4, 0xce, 0xcb, 0xb0, 0xbd, 0x2c, 0xda, 0xaa, 0x06, 0xf3, 0x3b, 0x38, 0x24, 0xde, 0x25, 0x48, - 0x38, 0x45, 0x46, 0x42, 0x19, 0x7e, 0x6c, 0xa4, 0x46, 0x43, 0x19, 0xe6, 0x85, 0xd5, 0x7f, 0x4a, - 0xd1, 0xd3, 0x95, 0x13, 0xe5, 0x2a, 0xd6, 0xa7, 0x3f, 0x5d, 0x87, 0x08, 0x3c, 0xf3, 0xc0, 0xd9, - 0x1f, 0x7b, 0xe0, 0x14, 0xc4, 0x81, 0x33, 0x2e, 0x61, 0x5d, 0x85, 0x59, 0xd9, 0x51, 0xa4, 0x29, - 0x4a, 0x06, 0x91, 0xe4, 0xcf, 0x1b, 0x74, 0x6c, 0xab, 0xcb, 0x1d, 0x1d, 0x6f, 0x55, 0xdd, 0x09, - 0x4e, 0x9d, 0xdb, 0xc3, 0xa7, 0xce, 0xf5, 0x69, 0x16, 0x7b, 0xea, 0xd0, 0x79, 0x11, 0x4a, 0xdf, - 0xc1, 0x86, 0x1d, 0xee, 0x09, 0x2d, 0x2d, 0x42, 0xc6, 0xc7, 0x86, 0xc9, 0x66, 0xcc, 0x6b, 0xac, - 0x41, 0xce, 0x26, 0xd1, 0x8d, 0xc7, 0x09, 0x0b, 0x30, 0xbf, 0xcd, 0xed, 0x44, 0x1c, 0x57, 0xff, - 0x9b, 0x82, 0x72, 0x0c, 0x8b, 0xb2, 0x09, 0x10, 0xf6, 0x14, 0xe9, 0xe7, 0xc6, 0x14, 0x16, 0x28, - 0x08, 0x8a, 0xea, 0x48, 0x4c, 0x0c, 0xfd, 0x54, 0x81, 0x3c, 0xdb, 0x8c, 0x58, 0x6c, 0xf0, 0xa4, - 0x5c, 0x6e, 0x94, 0x35, 0xbe, 0xad, 0x85, 0xe2, 0xdf, 0x20, 0xf4, 0x7f, 0xfc, 0xeb, 0xb3, 0xed, - 0xd6, 0x88, 0x8f, 0xea, 0x47, 0x50, 0x1a, 0xa2, 0x2b, 0x2b, 0x30, 0xc3, 0x14, 0xb8, 0x23, 0x2b, - 0x70, 0x6e, 0xfd, 0xad, 0x29, 0xa4, 0x41, 0xa6, 0x13, 0xeb, 0xe0, 0x3b, 0x52, 0x52, 0xa6, 0x07, - 0xb3, 0x77, 0xdc, 0x4e, 0x20, 0xa9, 0x92, 0x65, 0x0f, 0x8a, 0x9c, 0x3d, 0x2c, 0x49, 0x3e, 0x91, - 0xd6, 0x1d, 0x58, 0x0b, 0xdd, 0xe0, 0x86, 0x9c, 0xa6, 0x4c, 0xd5, 0x64, 0x41, 0x1e, 0xd8, 0x75, - 0x5a, 0xea, 0x67, 0x65, 0xff, 0x3a, 0x09, 0xc4, 0x99, 0xa5, 0xab, 0xff, 0x92, 0x85, 0x22, 0x9b, - 0x92, 0xab, 0x7b, 0x0b, 0x66, 0x48, 0x2f, 0xae, 0xe8, 0x97, 0x13, 0xd4, 0x21, 0x0f, 0x23, 0x0d, - 0xae, 0x60, 0x3a, 0xbc, 0xfa, 0x3f, 0x19, 0x48, 0xdf, 0x71, 0x3b, 0x68, 0x09, 0x52, 0xdc, 0xa9, - 0xa6, 0x9b, 0xd9, 0x93, 0xe3, 0x5a, 0xaa, 0xdd, 0xd2, 0x52, 0x96, 0x79, 0xc6, 0x5d, 0x27, 0x67, - 0x6b, 0x33, 0xc3, 0xd9, 0x1a, 0x3a, 0x84, 0xb9, 0xa1, 0x5a, 0x18, 0xab, 0x0e, 0x94, 0x9a, 0xef, - 0x9c, 0x1c, 0xd7, 0x4a, 0x72, 0x31, 0x8c, 0x1c, 0x8f, 0xb7, 0x26, 0xb2, 0x91, 0xe0, 0xc0, 0x8e, - 0xae, 0x21, 0x08, 0x6d, 0xaf, 0x53, 0x6f, 0xb7, 0xb4, 0x92, 0x5c, 0x4b, 0x0b, 0x24, 0x75, 0x64, - 0x87, 0xd4, 0x71, 0x0b, 0x72, 0xac, 0xbc, 0x68, 0xd2, 0x6a, 0x4e, 0x72, 0x0c, 0x3b, 0x43, 0xe3, - 0x57, 0x31, 0x80, 0x8c, 0x0d, 0x42, 0xc3, 0x27, 0x63, 0xf3, 0x93, 0x8e, 0xe5, 0x03, 0xd0, 0x9b, - 0x90, 0xdf, 0xb5, 0x1c, 0x2b, 0xd8, 0xc3, 0x66, 0xa5, 0x30, 0xe1, 0xe0, 0x68, 0x04, 0x19, 0xdd, - 0x77, 0x4d, 0x6b, 0xd7, 0xc2, 0x66, 0x05, 0x26, 0x1d, 0x2d, 0x46, 0x90, 0x00, 0x74, 0xd7, 0x37, - 0x68, 0xa1, 0x4b, 0xef, 0xba, 0x7d, 0xcf, 0xc6, 0x64, 0x09, 0xb3, 0xab, 0xca, 0x5a, 0x4a, 0x5b, - 0x10, 0x98, 0x4d, 0x81, 0x20, 0xf6, 0x4d, 0x4b, 0x1d, 0x95, 0x22, 0x73, 0xc8, 0xb4, 0x81, 0xee, - 0xc3, 0xf9, 0x3d, 0xab, 0xb7, 0xf7, 0xc4, 0x20, 0x21, 0x72, 0x9c, 0x8e, 0x96, 0x26, 0xe4, 0x06, - 0x45, 0x83, 0x23, 0x0c, 0x39, 0xa5, 0x62, 0x92, 0x26, 0xee, 0x5a, 0x7d, 0xc3, 0xae, 0xcc, 0xd1, - 0x49, 0xcb, 0x11, 0xa2, 0xc5, 0xe0, 0xe8, 0x45, 0x98, 0xf3, 0x07, 0x0e, 0x8d, 0x16, 0xb9, 0x62, - 0xe7, 0x69, 0xcf, 0x12, 0x87, 0xf2, 0x98, 0xe1, 0x32, 0x14, 0xe2, 0x7a, 0x72, 0x99, 0xa5, 0xb9, - 0x11, 0x40, 0x45, 0x50, 0xde, 0x76, 0xbb, 0xec, 0x9a, 0x49, 0xb8, 0xd7, 0xff, 0x53, 0x60, 0x41, - 0x02, 0x46, 0xfe, 0xb5, 0x60, 0x0b, 0xe0, 0x04, 0xc5, 0xfc, 0x53, 0x04, 0x22, 0x88, 0x38, 0xf9, - 0x22, 0x6a, 0xd5, 0x3f, 0x53, 0x20, 0x2f, 0xb0, 0xe8, 0x05, 0x28, 0x12, 0x8c, 0x6d, 0x85, 0x47, - 0x7a, 0x7c, 0x20, 0xcd, 0x0a, 0xd8, 0xdb, 0xf8, 0x88, 0xac, 0x3c, 0xea, 0x12, 0x3b, 0xb8, 0x82, - 0x56, 0x12, 0x50, 0x76, 0x8e, 0x56, 0x21, 0x6f, 0x1b, 0xa1, 0x15, 0x0e, 0x4c, 0xe6, 0x6c, 0x14, - 0x2d, 0x6a, 0x13, 0xa9, 0xd8, 0xae, 0xd3, 0x63, 0xc8, 0x19, 0x8a, 0x8c, 0x01, 0x6a, 0x13, 0xe6, - 0x35, 0xc3, 0xe9, 0xe1, 0x6d, 0xb7, 0x27, 0x7c, 0xdc, 0x32, 0xe4, 0x59, 0xed, 0x51, 0xb8, 0x09, - 0x2d, 0x47, 0xdb, 0x72, 0xf1, 0x24, 0x25, 0xb9, 0x3f, 0xf5, 0xbf, 0xd2, 0x50, 0x8e, 0x89, 0x70, - 0x21, 0xbe, 0x13, 0x55, 0x44, 0xd8, 0x31, 0x92, 0x74, 0xa6, 0x8e, 0x0e, 0x1e, 0x5b, 0x13, 0xf9, - 0x47, 0x05, 0xe0, 0x5d, 0x1f, 0x87, 0xe1, 0x51, 0xdb, 0xd9, 0x75, 0x89, 0xf0, 0x78, 0x42, 0x4a, - 0x33, 0x08, 0x21, 0x3c, 0x0e, 0x23, 0xde, 0x85, 0x2c, 0xc4, 0xc1, 0x4f, 0x18, 0x9a, 0x89, 0x2d, - 0xe7, 0xe0, 0x27, 0x14, 0x75, 0x05, 0x4a, 0x86, 0x69, 0x62, 0x53, 0xe7, 0xa1, 0x1d, 0x77, 0x6d, - 0x45, 0x0a, 0xd4, 0x18, 0x0c, 0xbd, 0x04, 0xf3, 0x3e, 0xee, 0xbb, 0x87, 0x52, 0x37, 0xe6, 0xe2, - 0xe6, 0x38, 0x58, 0x74, 0x5c, 0x82, 0xac, 0x8f, 0x8d, 0x20, 0x2a, 0x2e, 0xf3, 0x16, 0xaa, 0x40, - 0xce, 0x64, 0xd7, 0x26, 0xdc, 0x13, 0x89, 0x66, 0xf5, 0xe7, 0x8a, 0x28, 0xf0, 0xdc, 0x81, 0x0c, - 0x5d, 0x20, 0x2f, 0xee, 0xd4, 0x9f, 0x72, 0x72, 0xf1, 0x82, 0xac, 0x24, 0x2c, 0x59, 0x46, 0x8c, - 0x04, 0xfa, 0x00, 0x66, 0x3d, 0x2a, 0x21, 0x9d, 0x56, 0x5c, 0x58, 0x30, 0xf3, 0xda, 0x34, 0x82, - 0x8f, 0x05, 0x2c, 0x82, 0x03, 0x2f, 0x82, 0xf0, 0xe4, 0x6b, 0x0d, 0xca, 0xf7, 0x07, 0xd8, 0x3f, - 0x7a, 0xd7, 0x36, 0x1c, 0xe9, 0x58, 0x3c, 0x20, 0x30, 0x11, 0xc7, 0xd1, 0x86, 0xea, 0xc1, 0x82, - 0xd4, 0x93, 0xdb, 0xc5, 0xfb, 0x70, 0xc9, 0xb4, 0x82, 0x30, 0x38, 0xb0, 0x75, 0x6f, 0xef, 0x28, - 0xb0, 0xba, 0x86, 0xad, 0xd3, 0xee, 0xba, 0x67, 0x1b, 0x0e, 0xcf, 0xbe, 0x2f, 0x9f, 0x1c, 0xd7, - 0x2a, 0x2d, 0x2b, 0x08, 0x77, 0xee, 0x6f, 0xbf, 0xcb, 0x7b, 0xc5, 0xa4, 0x2a, 0x9c, 0xc0, 0x29, - 0x8c, 0xba, 0xcc, 0x2e, 0x66, 0xc9, 0x48, 0xdf, 0xea, 0x0c, 0xc2, 0x38, 0x3f, 0x53, 0xff, 0x0e, - 0xa0, 0x72, 0x1a, 0xc7, 0x99, 0xf2, 0xa0, 0x24, 0xea, 0xf4, 0x4c, 0x74, 0x6c, 0xd7, 0x6f, 0x3d, - 0xe3, 0xf2, 0x76, 0x1c, 0xad, 0xe8, 0x56, 0x97, 0x88, 0x4c, 0x8e, 0x7f, 0x8b, 0xa6, 0x84, 0x40, - 0x7d, 0x28, 0x4a, 0xd7, 0x50, 0xe2, 0x16, 0xae, 0x75, 0x96, 0x09, 0xe3, 0xab, 0xa9, 0xa1, 0x78, - 0x7b, 0x36, 0xbe, 0x9a, 0x0a, 0xaa, 0x9f, 0x2a, 0x00, 0x71, 0x3f, 0x62, 0xb0, 0x2c, 0xef, 0xe3, - 0x0a, 0xe3, 0x2d, 0xb4, 0x01, 0x59, 0x7e, 0xc7, 0x96, 0x9a, 0xf6, 0x8e, 0x8d, 0x0f, 0xa4, 0xc5, - 0x14, 0x76, 0xb5, 0x16, 0x1c, 0xd8, 0x6c, 0xbf, 0xf0, 0x62, 0x0a, 0x85, 0xee, 0xdc, 0xdf, 0xd6, - 0x0a, 0xac, 0xc3, 0xce, 0x81, 0x7d, 0x67, 0x26, 0x9f, 0x2e, 0xcf, 0x54, 0xff, 0x24, 0x0d, 0x05, - 0x5a, 0x8f, 0xa6, 0xa2, 0xf9, 0x5c, 0x81, 0xca, 0x50, 0x1a, 0xa6, 0x77, 0x8e, 0xf4, 0x38, 0x19, - 0x24, 0x72, 0x7a, 0xef, 0x2c, 0x72, 0x8a, 0x66, 0xa8, 0x6b, 0x52, 0x3e, 0xd7, 0x3c, 0xa2, 0xf1, - 0xa6, 0xc9, 0x44, 0xf7, 0xfa, 0x59, 0xa3, 0xd5, 0x45, 0x7f, 0x0c, 0x4d, 0xf4, 0x55, 0x98, 0x93, - 0x2f, 0x1b, 0xa3, 0x72, 0x74, 0x31, 0xd6, 0x4d, 0xdb, 0x44, 0xdf, 0x06, 0x30, 0x7d, 0xd7, 0xf3, - 0xb0, 0xa9, 0x1b, 0x2c, 0xe7, 0x9c, 0xe4, 0x54, 0x2d, 0xf0, 0x31, 0x1b, 0x61, 0xf5, 0x36, 0x2c, - 0x3f, 0x75, 0x49, 0x63, 0x82, 0xe5, 0xa1, 0xf2, 0x65, 0x5a, 0x8a, 0x76, 0xab, 0x3f, 0x4a, 0x41, - 0x51, 0xb6, 0x5f, 0x14, 0x02, 0x7b, 0x63, 0x20, 0xef, 0x8a, 0x77, 0x9e, 0x77, 0x57, 0xc4, 0x9a, - 0x18, 0xca, 0x0f, 0x43, 0x01, 0xad, 0x7e, 0x0c, 0x73, 0xc3, 0x5d, 0xc6, 0xa4, 0x6c, 0x3b, 0xc3, - 0x29, 0xdb, 0x5b, 0xcf, 0x65, 0x11, 0x43, 0x32, 0x50, 0xe2, 0x77, 0x24, 0x49, 0x0c, 0x3c, 0x1a, - 0x66, 0xe0, 0x77, 0x9f, 0x57, 0x2a, 0x32, 0x0f, 0x3f, 0x84, 0xf2, 0xe8, 0xae, 0x1e, 0xc3, 0xc1, - 0x83, 0x61, 0x0e, 0xbe, 0xf5, 0x7c, 0xce, 0x43, 0x9a, 0x9f, 0xd7, 0x06, 0x2f, 0xc2, 0x85, 0xbb, - 0xf4, 0x25, 0xcf, 0x5d, 0x1c, 0x1a, 0x66, 0x5c, 0xbc, 0x56, 0xff, 0x55, 0x81, 0xa5, 0x51, 0x0c, - 0xf7, 0xa4, 0x06, 0xe4, 0xfb, 0x1c, 0xc6, 0xcd, 0xe5, 0xdb, 0x09, 0x6c, 0x8d, 0x27, 0x52, 0x17, - 0x00, 0xd9, 0x3c, 0x22, 0xb2, 0xd5, 0xdf, 0x83, 0xd2, 0x50, 0x87, 0x31, 0x92, 0x79, 0x75, 0x58, - 0x32, 0x72, 0xe6, 0x35, 0x08, 0x2d, 0xbb, 0xce, 0x1f, 0x28, 0x45, 0x13, 0x4b, 0x09, 0xdf, 0xcf, - 0x52, 0x70, 0x7e, 0xcb, 0x39, 0x18, 0xe0, 0x01, 0xa6, 0xa7, 0xa3, 0x38, 0xe2, 0xbe, 0xcc, 0x82, - 0x14, 0x3b, 0x3e, 0xa3, 0xe0, 0x8e, 0x35, 0xd0, 0xf7, 0xa4, 0x38, 0x8c, 0x5e, 0x56, 0x35, 0x37, - 0x4f, 0x8e, 0x6b, 0x39, 0xca, 0x15, 0x9d, 0xf3, 0xe6, 0x54, 0x73, 0xf2, 0x71, 0x71, 0x30, 0x77, - 0x15, 0x16, 0x82, 0x7d, 0xcb, 0xd3, 0x83, 0x3d, 0x77, 0x60, 0x9b, 0x3a, 0xe3, 0x80, 0xd5, 0x4e, - 0xe6, 0x09, 0x62, 0x87, 0xc2, 0xef, 0x13, 0xb0, 0xfa, 0xd7, 0x29, 0x58, 0x1c, 0x96, 0x0a, 0xd7, - 0xf7, 0xfd, 0x38, 0xc4, 0x61, 0xea, 0x7e, 0x3d, 0xe9, 0xe6, 0x6b, 0x0c, 0x85, 0xba, 0x78, 0x58, - 0x12, 0xc5, 0x46, 0x7f, 0xaf, 0x40, 0x8e, 0x03, 0xbf, 0x54, 0xa9, 0xbf, 0x35, 0x12, 0xa1, 0xbe, - 0x98, 0x74, 0x5b, 0xea, 0x1b, 0x5d, 0x4c, 0x23, 0x2e, 0x11, 0x8f, 0xc6, 0xa9, 0x52, 0x5a, 0x4a, - 0x95, 0xd4, 0x0b, 0x70, 0x7e, 0x73, 0xcf, 0xf0, 0xc3, 0x4d, 0x96, 0xa4, 0x8a, 0x1d, 0xf3, 0x18, - 0x16, 0x87, 0xc1, 0x5c, 0x7c, 0x4d, 0xc8, 0xf1, 0x74, 0x96, 0x8b, 0x4f, 0x95, 0x98, 0x08, 0x83, - 0xba, 0x78, 0x64, 0x47, 0x07, 0xef, 0xb0, 0x07, 0x0b, 0xd1, 0x5b, 0x1e, 0x86, 0xbb, 0xfa, 0x36, - 0x2c, 0x8d, 0x7f, 0x9d, 0x82, 0x66, 0x21, 0xf7, 0xf0, 0xde, 0xdb, 0xf7, 0xde, 0x79, 0xef, 0x5e, - 0xf9, 0x1c, 0x69, 0x6c, 0x6e, 0x3f, 0xdc, 0x79, 0xb0, 0xa5, 0x95, 0x15, 0x54, 0x84, 0x7c, 0x6b, - 0xe3, 0xc1, 0x46, 0x73, 0x63, 0x67, 0xab, 0x9c, 0x42, 0x05, 0xc8, 0x3c, 0xd8, 0x68, 0x6e, 0x6f, - 0x95, 0xd3, 0xeb, 0x3f, 0xad, 0x42, 0x66, 0xc3, 0xec, 0x5b, 0x0e, 0x0a, 0x21, 0x43, 0xaf, 0x6c, - 0xd1, 0x4b, 0xcf, 0xbe, 0xd4, 0xa5, 0x8b, 0xac, 0xae, 0x4d, 0x7a, 0xfb, 0xab, 0x56, 0x7e, 0xfc, - 0xcf, 0xff, 0xf1, 0xb3, 0x14, 0x42, 0xe5, 0x86, 0x4e, 0x9f, 0x4f, 0x36, 0x0e, 0xaf, 0x37, 0xe8, - 0x2d, 0x30, 0xfa, 0x23, 0x05, 0x0a, 0xd1, 0x2b, 0x3e, 0xf4, 0xf2, 0x04, 0xaf, 0xe7, 0xa2, 0xe9, - 0x5f, 0x99, 0xac, 0x33, 0x67, 0xe1, 0x32, 0x65, 0x61, 0x09, 0x2d, 0x4a, 0x2c, 0x44, 0x0f, 0x03, - 0xd1, 0x5f, 0x2a, 0x30, 0x3f, 0xf2, 0x3c, 0x0f, 0x5d, 0x9f, 0xe6, 0x29, 0x1f, 0x63, 0x69, 0x7d, - 0xfa, 0xd7, 0x7f, 0xea, 0x4b, 0x94, 0xb1, 0x17, 0x50, 0x6d, 0x1c, 0x63, 0x8d, 0x8f, 0xc5, 0xe7, - 0x0f, 0xd0, 0xdf, 0x28, 0x50, 0x94, 0x9f, 0x66, 0xa1, 0xfa, 0xc4, 0x6f, 0xb8, 0x18, 0x77, 0x8d, - 0x29, 0xdf, 0x7c, 0xa9, 0xaf, 0x51, 0xd6, 0xae, 0xa1, 0xfa, 0x33, 0x58, 0x6b, 0xd0, 0xa3, 0x3c, - 0x68, 0x7c, 0x4c, 0xff, 0x29, 0xa7, 0x10, 0xbf, 0x37, 0x40, 0xaf, 0x4c, 0xf8, 0x2c, 0x81, 0x71, - 0x39, 0xdd, 0x23, 0x06, 0xf5, 0x4d, 0xca, 0xe3, 0x6b, 0xe8, 0xe6, 0x74, 0x3c, 0x36, 0xd8, 0x7b, - 0x96, 0x3f, 0x57, 0xa0, 0x34, 0xf4, 0x84, 0x03, 0x25, 0x09, 0x69, 0xdc, 0x2b, 0x90, 0xea, 0xb5, - 0xc9, 0x07, 0x70, 0x96, 0x57, 0x29, 0xcb, 0x55, 0x54, 0x91, 0x58, 0x76, 0x5c, 0x87, 0x31, 0x48, - 0x99, 0xf8, 0x10, 0xb2, 0xec, 0xd5, 0x00, 0x5a, 0x9b, 0xe0, 0x61, 0x01, 0xe3, 0xe3, 0xeb, 0x13, - 0x3f, 0x41, 0x50, 0x97, 0x29, 0x03, 0xe7, 0xd1, 0x82, 0xc4, 0x00, 0xf7, 0x72, 0x64, 0x3f, 0x46, - 0x37, 0xda, 0x89, 0xfb, 0x71, 0xf4, 0x8e, 0x3d, 0x71, 0x3f, 0x9e, 0xbe, 0x24, 0xe7, 0xfb, 0x51, - 0x95, 0x79, 0x18, 0x58, 0x44, 0x5d, 0xb7, 0x94, 0xab, 0xe8, 0x47, 0x0a, 0x14, 0x6e, 0x4f, 0xc4, - 0xc6, 0xed, 0x69, 0xd8, 0x38, 0x75, 0x85, 0x3c, 0x56, 0x14, 0x8c, 0x0d, 0xf4, 0x07, 0x90, 0xe3, - 0x17, 0xc2, 0x28, 0x49, 0xb6, 0xc3, 0x17, 0xce, 0xd5, 0xab, 0x93, 0x74, 0xe5, 0x93, 0x57, 0xe9, - 0xe4, 0x8b, 0x08, 0x49, 0x93, 0xf3, 0x8b, 0x67, 0xf4, 0x87, 0x0a, 0xe4, 0xc5, 0xdd, 0x03, 0xba, - 0x3a, 0xd1, 0x05, 0x05, 0x63, 0xe0, 0xe5, 0x29, 0x2e, 0x33, 0xd4, 0x4b, 0x94, 0x83, 0x0b, 0xe8, - 0xbc, 0xc4, 0x41, 0x20, 0x66, 0xfd, 0x89, 0x02, 0x59, 0x76, 0x65, 0x91, 0x68, 0x86, 0x43, 0x97, - 0x1f, 0x89, 0x66, 0x38, 0x72, 0xff, 0xf1, 0x55, 0x3a, 0xf9, 0xca, 0x90, 0xec, 0xf7, 0x68, 0x97, - 0xc7, 0x05, 0x94, 0xe3, 0x9f, 0x54, 0x10, 0xa2, 0x5e, 0x9f, 0x28, 0x88, 0x91, 0xbb, 0x94, 0x44, - 0x41, 0x8c, 0x5e, 0x64, 0x8c, 0x15, 0x84, 0xb8, 0x34, 0x40, 0x1e, 0xcc, 0xdc, 0x71, 0x3b, 0x01, - 0xfa, 0xda, 0x33, 0x6b, 0xf1, 0x6c, 0xe6, 0x97, 0x26, 0xac, 0xd9, 0xab, 0x17, 0xe9, 0xac, 0x0b, - 0x68, 0x5e, 0x9a, 0xf5, 0xfb, 0x64, 0x26, 0xb2, 0x0d, 0xa3, 0x3a, 0x63, 0xa2, 0xfd, 0x8f, 0xd6, - 0x38, 0x13, 0xed, 0xff, 0x54, 0xe9, 0x72, 0xec, 0xb1, 0x18, 0x95, 0x2f, 0x29, 0x1b, 0x51, 0xb5, - 0x25, 0x91, 0x8d, 0xd1, 0x12, 0x51, 0x22, 0x1b, 0xa7, 0xaa, 0x44, 0x63, 0xd9, 0xa0, 0x55, 0x22, - 0x8f, 0x4c, 0xfc, 0x3d, 0xc8, 0xd0, 0xd7, 0x08, 0x89, 0xa1, 0x89, 0xfc, 0x90, 0x22, 0x31, 0x34, - 0x19, 0x7a, 0xfe, 0xa0, 0x9e, 0xbb, 0xa6, 0xa0, 0x27, 0x50, 0x94, 0xaf, 0xae, 0x13, 0x0f, 0xd6, - 0x31, 0x6f, 0x01, 0xaa, 0xaf, 0x9e, 0xe9, 0x4e, 0x5c, 0x3d, 0x47, 0x76, 0x18, 0x3a, 0xdd, 0x01, - 0xdd, 0x9c, 0x92, 0xde, 0x73, 0x72, 0xf1, 0x57, 0x0a, 0xe4, 0x45, 0x65, 0x30, 0x71, 0x87, 0x8d, - 0x54, 0x8e, 0x13, 0x77, 0xd8, 0x68, 0xa9, 0x51, 0x7d, 0x8b, 0xaa, 0xf8, 0xf5, 0xa1, 0x1d, 0x46, - 0x53, 0x13, 0xdb, 0xed, 0x3d, 0x5e, 0x45, 0x2b, 0x63, 0xc0, 0x8d, 0x8f, 0x45, 0x42, 0xf4, 0x03, - 0x12, 0xa1, 0x95, 0x47, 0xb3, 0x5a, 0xb4, 0x3e, 0x55, 0x0a, 0xcc, 0x98, 0xbe, 0x71, 0x86, 0xb4, - 0x39, 0x72, 0x55, 0x97, 0x47, 0xa2, 0x0c, 0xdd, 0x94, 0xd9, 0xf9, 0x54, 0x81, 0x85, 0x0d, 0xdb, - 0x1e, 0xce, 0x72, 0xd1, 0xb5, 0x29, 0x12, 0x62, 0xc6, 0xe2, 0xf5, 0xa9, 0x53, 0x68, 0xf5, 0x05, - 0xca, 0xe0, 0x25, 0xb4, 0x2c, 0x31, 0xc8, 0xb2, 0x5e, 0x91, 0x47, 0xa3, 0x4f, 0x14, 0x28, 0xca, - 0x49, 0x49, 0xa2, 0x99, 0x8f, 0x49, 0x6a, 0x12, 0xe3, 0xc7, 0x71, 0xd9, 0x8e, 0x5a, 0xa3, 0x4c, - 0x2d, 0xa3, 0x8b, 0xf2, 0xf9, 0x46, 0x3a, 0xf2, 0x54, 0x86, 0x84, 0x5f, 0x45, 0x39, 0x49, 0x4c, - 0x64, 0x69, 0x4c, 0x96, 0x9e, 0xc8, 0xd2, 0xb8, 0xec, 0x53, 0xbd, 0x42, 0x59, 0xfa, 0x8a, 0x2a, - 0xc7, 0x5e, 0x98, 0x75, 0xd4, 0xa9, 0xb1, 0xdd, 0x52, 0xae, 0x36, 0xaf, 0x7e, 0xfe, 0xef, 0x2b, - 0xe7, 0x3e, 0x3f, 0x59, 0x51, 0x7e, 0x79, 0xb2, 0xa2, 0xfc, 0xea, 0x64, 0x45, 0xf9, 0xb7, 0x93, - 0x15, 0xe5, 0x93, 0x2f, 0x56, 0xce, 0xfd, 0xf2, 0x8b, 0x95, 0x73, 0xbf, 0xfa, 0x62, 0xe5, 0xdc, - 0xe3, 0xbc, 0x98, 0xa5, 0x93, 0xa5, 0x05, 0xbb, 0x1b, 0xff, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x62, - 0x95, 0x69, 0x43, 0x28, 0x36, 0x00, 0x00, +func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_065753b953171640) } + +var fileDescriptor_admin_065753b953171640 = []byte{ + // 4189 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, 0xd7, + 0x4b, 0x6b, 0xd7, 0x80, 0x44, 0x69, 0x3f, 0xa2, 0xdd, 0xb5, 0x43, 0x10, 0x2c, 0x19, 0x5a, 0x4a, + 0xbb, 0x1a, 0x4a, 0xda, 0xb2, 0x36, 0xeb, 0xc9, 0x00, 0xd3, 0x04, 0xc7, 0x1c, 0xcc, 0x0c, 0x67, + 0x06, 0xd4, 0x72, 0x37, 0x76, 0xc5, 0x76, 0x2a, 0x95, 0x53, 0x6a, 0xcb, 0x4e, 0xe5, 0xb2, 0xa7, + 0xe4, 0x90, 0x1c, 0x72, 0xc9, 0xfa, 0x98, 0x43, 0x72, 0xdd, 0xca, 0xc5, 0x4e, 0xa5, 0x52, 0xe5, + 0x5c, 0xe8, 0x84, 0x9b, 0x53, 0xf2, 0x0f, 0x52, 0x95, 0x8f, 0xea, 0xaf, 0x99, 0x06, 0x08, 0x8d, + 0x00, 0xca, 0xf6, 0x05, 0x98, 0x7e, 0xaf, 0xfb, 0xf5, 0xeb, 0xd7, 0xaf, 0xdf, 0x57, 0x37, 0x5c, + 0x0a, 0xb0, 0x7f, 0x88, 0xfd, 0x06, 0xfb, 0xf3, 0x3a, 0x0d, 0xc3, 0xec, 0x5b, 0x4e, 0xdd, 0xf3, + 0xdd, 0xd0, 0x45, 0xcb, 0x5d, 0xb7, 0xbb, 0xef, 0xbb, 0x46, 0x77, 0xaf, 0xce, 0xf0, 0x75, 0xd1, + 0xad, 0x5a, 0xe9, 0xba, 0xce, 0xae, 0xd5, 0x6b, 0x7c, 0xec, 0x3a, 0xd8, 0xeb, 0xd0, 0x3f, 0x36, + 0xa8, 0xba, 0xf4, 0x3d, 0xb7, 0x13, 0x34, 0xc8, 0x8f, 0xd7, 0xa1, 0x7f, 0x1c, 0x7e, 0x79, 0x74, + 0xa6, 0x20, 0x34, 0xc2, 0x81, 0xc0, 0x5e, 0x0a, 0x42, 0xd7, 0x37, 0x7a, 0xb8, 0x81, 0x9d, 0x9e, + 0x45, 0x49, 0xf6, 0x0f, 0xbb, 0x5d, 0x8e, 0xac, 0xef, 0x1f, 0x36, 0xf6, 0x0f, 0xf9, 0x78, 0xdb, + 0x3a, 0xc4, 0x0e, 0x0e, 0x82, 0xe8, 0xc3, 0xeb, 0x44, 0x9f, 0xbc, 0xff, 0xaa, 0xdc, 0x5f, 0x7c, + 0x90, 0x6e, 0x6e, 0x8f, 0xf7, 0x58, 0x09, 0x83, 0x46, 0xd7, 0x08, 0x0d, 0xdb, 0xed, 0x35, 0xba, + 0x7b, 0x86, 0x1f, 0xea, 0xbc, 0xc5, 0xf1, 0x95, 0x41, 0x68, 0xd9, 0x8d, 0x3e, 0x0e, 0x7d, 0xab, + 0xcb, 0xff, 0x38, 0x66, 0xb1, 0xe7, 0xf6, 0x5c, 0xfa, 0xd9, 0x20, 0x5f, 0x62, 0x71, 0x3d, 0xd7, + 0xed, 0xd9, 0xb8, 0x61, 0x78, 0x56, 0xc3, 0x70, 0x1c, 0x37, 0x34, 0x42, 0xcb, 0x75, 0x04, 0x3f, + 0x35, 0x8e, 0xa5, 0xad, 0xce, 0x60, 0xb7, 0x11, 0x5a, 0x7d, 0x1c, 0x84, 0x46, 0xdf, 0x63, 0x1d, + 0x54, 0x04, 0xe5, 0x96, 0x11, 0x1a, 0x1d, 0x23, 0xc0, 0x81, 0x86, 0x0f, 0x06, 0x38, 0x08, 0xd5, + 0xeb, 0xb0, 0x20, 0xc1, 0x02, 0xcf, 0x75, 0x02, 0x8c, 0x2e, 0x43, 0xc1, 0x14, 0xc0, 0x8a, 0xb2, + 0x9a, 0x5e, 0x2b, 0x68, 0x31, 0x40, 0xbd, 0x09, 0x4b, 0x62, 0x48, 0x0b, 0x87, 0x86, 0x65, 0x0b, + 0x62, 0xa8, 0x0a, 0x79, 0xd1, 0xad, 0xa2, 0xac, 0x2a, 0x6b, 0x05, 0x2d, 0x6a, 0xab, 0x9f, 0xa7, + 0xe1, 0xe2, 0xa9, 0x61, 0x7c, 0xbe, 0x47, 0x90, 0xed, 0xf9, 0x86, 0x13, 0xb2, 0xc9, 0x66, 0xd7, + 0xdf, 0xa8, 0x3f, 0x55, 0x25, 0xea, 0x4f, 0xa1, 0x51, 0xbf, 0x4d, 0x08, 0x34, 0x67, 0xbe, 0x38, + 0xae, 0x9d, 0xd3, 0x38, 0x35, 0x54, 0x83, 0xd9, 0xd0, 0xe8, 0xd8, 0x58, 0x77, 0x8c, 0x3e, 0x0e, + 0x2a, 0x29, 0xba, 0x12, 0xa0, 0xa0, 0x7b, 0x04, 0x82, 0x5e, 0x85, 0x92, 0x89, 0x83, 0xae, 0x6f, + 0x79, 0xa1, 0xeb, 0xeb, 0x96, 0x59, 0x49, 0xaf, 0x2a, 0x6b, 0xe9, 0x66, 0xf9, 0xe4, 0xb8, 0x56, + 0x6c, 0x45, 0x88, 0x76, 0x4b, 0x2b, 0xc6, 0xdd, 0xda, 0x26, 0xba, 0x03, 0xb3, 0x44, 0x15, 0x75, + 0xa6, 0x9d, 0x95, 0x99, 0x55, 0x65, 0x6d, 0x76, 0xfd, 0x8a, 0xc4, 0x34, 0x43, 0xd4, 0x99, 0xda, + 0xd6, 0x1f, 0xbb, 0x0e, 0xde, 0xa4, 0x10, 0xce, 0x1f, 0x7c, 0x1c, 0x41, 0xd0, 0x87, 0xb0, 0x20, + 0xd1, 0xd2, 0x6d, 0x7c, 0x88, 0xed, 0x4a, 0x66, 0x55, 0x59, 0x9b, 0x5b, 0xbf, 0x9e, 0x20, 0x86, + 0x98, 0xe6, 0xc0, 0xa7, 0x5a, 0xb0, 0x4d, 0x06, 0x6a, 0xf3, 0x31, 0x65, 0x0a, 0xa8, 0xbe, 0x09, + 0x19, 0x2a, 0x19, 0x84, 0x60, 0x66, 0x10, 0x60, 0x9f, 0xef, 0x0b, 0xfd, 0x46, 0x2b, 0x00, 0x9e, + 0x6f, 0x1d, 0x5a, 0x36, 0xee, 0xc5, 0xe2, 0x89, 0x21, 0xea, 0x6d, 0x38, 0xff, 0x80, 0x08, 0x6b, + 0xf2, 0x6d, 0x46, 0x8b, 0x90, 0xa1, 0xf2, 0xad, 0xa4, 0x28, 0x82, 0x35, 0xd4, 0x7f, 0xc8, 0xc3, + 0xe2, 0x30, 0x25, 0xbe, 0xf3, 0x3b, 0x23, 0x3b, 0xff, 0x6a, 0xc2, 0x92, 0xc7, 0x11, 0x18, 0xbb, + 0xed, 0x8f, 0x20, 0xd7, 0x75, 0xed, 0x41, 0xdf, 0x61, 0x6b, 0x9a, 0x5d, 0x7f, 0x6d, 0x5a, 0xaa, + 0x9b, 0x74, 0x38, 0x27, 0x2b, 0x88, 0xa1, 0x87, 0x90, 0xb3, 0x1c, 0x13, 0x7f, 0x84, 0x83, 0x4a, + 0xfa, 0x6c, 0xdc, 0xb6, 0xc9, 0x70, 0x41, 0x96, 0xd3, 0x22, 0x5a, 0xea, 0x1b, 0x4e, 0x8f, 0xa8, + 0xc0, 0xc0, 0x09, 0xa9, 0x36, 0xa5, 0x35, 0xa0, 0xa0, 0x4d, 0x02, 0x41, 0x37, 0x61, 0xa9, 0xeb, + 0x63, 0x23, 0xc4, 0x3a, 0xd3, 0x66, 0x62, 0xd2, 0x70, 0x1f, 0x3b, 0x21, 0xd5, 0x93, 0x82, 0xb6, + 0xc8, 0xb0, 0x74, 0xc6, 0x1d, 0x81, 0x1b, 0x55, 0xd2, 0xec, 0xaf, 0x5d, 0x49, 0x73, 0xbf, 0x2e, + 0x25, 0x3d, 0x7d, 0x0c, 0xf3, 0x13, 0x1d, 0xc3, 0x37, 0x80, 0xfb, 0x87, 0x81, 0x8f, 0x75, 0xca, + 0x5f, 0x2c, 0x99, 0x02, 0x95, 0xcc, 0x52, 0x84, 0x27, 0xac, 0x44, 0xb2, 0x79, 0xae, 0x53, 0x51, + 0xfd, 0x7b, 0x05, 0xb2, 0x4c, 0x41, 0xc8, 0x70, 0x62, 0x5a, 0xc4, 0x70, 0xf2, 0x4d, 0x60, 0xe1, + 0x91, 0x27, 0x0e, 0x00, 0xfd, 0x26, 0x27, 0xc6, 0x19, 0xd8, 0x36, 0x3d, 0x18, 0xc4, 0xc4, 0xe4, + 0xb5, 0xa8, 0x8d, 0xae, 0x90, 0xc5, 0xef, 0x1a, 0x03, 0x3b, 0xd4, 0x0f, 0x0d, 0x7b, 0x80, 0xa9, + 0x02, 0x14, 0xc8, 0x52, 0x29, 0xf0, 0x11, 0x81, 0xa1, 0x1b, 0x70, 0xa1, 0x87, 0x1d, 0xcc, 0xa4, + 0xa8, 0xe3, 0x8f, 0x3c, 0x1f, 0x07, 0x81, 0xe5, 0x3a, 0x42, 0x03, 0x62, 0xe4, 0x56, 0x84, 0x43, + 0x4b, 0x90, 0xdd, 0xb3, 0x4c, 0x13, 0x3b, 0x74, 0xf3, 0xf3, 0x1a, 0x6f, 0x55, 0x3f, 0x57, 0x20, + 0x43, 0x35, 0x71, 0x2c, 0xff, 0x4b, 0x90, 0x1d, 0x38, 0xd6, 0xc1, 0x80, 0xad, 0x20, 0xaf, 0xf1, + 0x16, 0x2a, 0x43, 0x3a, 0xc0, 0x07, 0xcc, 0x42, 0x6a, 0xe4, 0x93, 0xf4, 0x64, 0x47, 0x83, 0xb3, + 0xcc, 0x5b, 0xd4, 0x7d, 0x58, 0x3e, 0xee, 0x86, 0x31, 0x83, 0x31, 0x00, 0x55, 0x20, 0x47, 0xbc, + 0xb0, 0xe5, 0xf4, 0x38, 0x5b, 0xa2, 0x49, 0xa4, 0x64, 0xf5, 0x3d, 0xdb, 0xea, 0x5a, 0x21, 0x55, + 0xae, 0xbc, 0x16, 0xb5, 0xd5, 0x2d, 0x58, 0x88, 0xf4, 0xfb, 0x39, 0x0c, 0xd1, 0xe7, 0x69, 0x40, + 0x32, 0x1d, 0x6e, 0x86, 0x46, 0x8e, 0xa0, 0x72, 0xea, 0x08, 0x5e, 0x81, 0x92, 0x8f, 0x09, 0x2b, + 0x06, 0xef, 0x92, 0xa2, 0x5d, 0x8a, 0x1c, 0xc8, 0x3a, 0x7d, 0x05, 0xc0, 0x71, 0x4d, 0x41, 0x84, + 0x09, 0xaa, 0x40, 0x20, 0x0c, 0xbd, 0x01, 0x19, 0xa2, 0x9f, 0x01, 0xf7, 0x17, 0x2f, 0xca, 0x07, + 0x87, 0x85, 0x25, 0x75, 0x11, 0x96, 0xd4, 0xef, 0x3e, 0xda, 0xdc, 0xa4, 0x2c, 0xf2, 0xc3, 0xc8, + 0x46, 0x22, 0x13, 0x4a, 0x7d, 0x2b, 0x08, 0x2c, 0xa7, 0xa7, 0x13, 0xba, 0x41, 0x25, 0x43, 0xed, + 0xd0, 0xef, 0x3c, 0xcb, 0x0e, 0x0d, 0xad, 0xb6, 0x7e, 0x97, 0x91, 0xb8, 0xe7, 0x9a, 0x98, 0x93, + 0x2f, 0xf6, 0x63, 0x50, 0x40, 0xec, 0x8d, 0xe1, 0x79, 0xbe, 0xfb, 0x91, 0xd5, 0x27, 0x46, 0xc7, + 0xb4, 0x82, 0x7d, 0xbd, 0x73, 0x14, 0xe2, 0x80, 0x6e, 0xd8, 0x8c, 0xb6, 0x28, 0x61, 0x5b, 0x56, + 0xb0, 0xdf, 0x24, 0xb8, 0xea, 0xfb, 0x30, 0x2b, 0x11, 0x46, 0x57, 0x20, 0x47, 0x85, 0x61, 0x99, + 0x6c, 0x6b, 0x9a, 0x70, 0x72, 0x5c, 0xcb, 0x12, 0x54, 0xbb, 0xa5, 0x65, 0x09, 0xaa, 0x6d, 0x12, + 0xb1, 0x62, 0xdf, 0x77, 0x7d, 0xbd, 0x8f, 0x83, 0xc0, 0xe8, 0x89, 0xcd, 0x2a, 0x52, 0xe0, 0x5d, + 0x06, 0x53, 0x97, 0x60, 0xf1, 0x9e, 0xeb, 0x9c, 0xda, 0x7d, 0xf5, 0xe7, 0x0a, 0x5c, 0x18, 0x41, + 0xf0, 0xed, 0xfc, 0x0e, 0x2c, 0x90, 0xd8, 0x47, 0x0f, 0xb0, 0x6f, 0xe1, 0x40, 0x67, 0x52, 0x57, + 0xa8, 0xd4, 0xbf, 0x31, 0x95, 0xa8, 0xb4, 0x79, 0x42, 0x67, 0x87, 0x92, 0xa1, 0x08, 0xf4, 0x01, + 0x20, 0xcb, 0x09, 0xb1, 0xef, 0x18, 0xb6, 0x3e, 0x08, 0x30, 0xa7, 0x9d, 0x3a, 0x0b, 0xed, 0xb2, + 0x20, 0xf4, 0x30, 0x60, 0x18, 0x75, 0x0e, 0x8a, 0x0f, 0x03, 0xec, 0x47, 0x2b, 0xfc, 0x01, 0x94, + 0x78, 0x9b, 0x2f, 0xac, 0x0d, 0x19, 0x62, 0xa2, 0x84, 0xb7, 0x4c, 0x9a, 0x70, 0x68, 0x20, 0x6d, + 0x09, 0x55, 0xa2, 0x14, 0xaa, 0x2a, 0xcc, 0x10, 0x20, 0x39, 0x43, 0x04, 0x20, 0x99, 0x81, 0xa8, + 0xad, 0xfe, 0x58, 0x81, 0xd2, 0xd6, 0x21, 0x76, 0xe2, 0x13, 0x27, 0x8c, 0x9b, 0x22, 0x19, 0xb7, + 0x4b, 0x50, 0x08, 0x0d, 0xbf, 0x87, 0x43, 0xb2, 0xd7, 0xec, 0x5c, 0xe4, 0x19, 0xa0, 0x6d, 0x92, + 0x63, 0x68, 0x5b, 0x7d, 0x8b, 0x1d, 0x87, 0x8c, 0xc6, 0x1a, 0xe8, 0x65, 0x58, 0x18, 0x38, 0x3e, + 0x36, 0x8d, 0x6e, 0x88, 0x4d, 0x1d, 0xd3, 0x29, 0xe8, 0xb1, 0xc8, 0x6b, 0xe5, 0x18, 0xc1, 0xa6, + 0x56, 0xff, 0x33, 0x05, 0x73, 0x82, 0x0b, 0x2e, 0x87, 0xbb, 0x90, 0xe5, 0x83, 0x98, 0x20, 0x1a, + 0x09, 0x82, 0x18, 0x1e, 0xca, 0x9a, 0x22, 0x60, 0x60, 0x44, 0xaa, 0x7f, 0x92, 0x82, 0x0c, 0x85, + 0xa3, 0x26, 0x14, 0xa2, 0xa8, 0x99, 0x6b, 0x4c, 0xb5, 0xce, 0xe2, 0xea, 0xba, 0x88, 0xab, 0xeb, + 0x0f, 0x44, 0x8f, 0x66, 0x9e, 0x90, 0xf9, 0xf4, 0x57, 0x35, 0x45, 0x8b, 0x87, 0x11, 0x33, 0x40, + 0xe9, 0xea, 0x92, 0x1b, 0x28, 0x50, 0xc8, 0x03, 0x22, 0xae, 0xaf, 0xcb, 0xe2, 0x62, 0xf1, 0x66, + 0xf1, 0xe4, 0xb8, 0x96, 0x7f, 0xc0, 0x44, 0xd6, 0x92, 0x84, 0xb7, 0x0e, 0xc4, 0xc0, 0xb8, 0x7e, + 0x48, 0x0e, 0xbc, 0x65, 0xb2, 0xd0, 0xa0, 0x39, 0x7f, 0x72, 0x5c, 0x9b, 0xd5, 0x04, 0xbc, 0xdd, + 0xd2, 0x66, 0xa3, 0x4e, 0x6d, 0x93, 0xec, 0x90, 0xe5, 0xec, 0xba, 0xdc, 0xee, 0xd2, 0x6f, 0x32, + 0x25, 0x33, 0xe2, 0x84, 0x08, 0x39, 0xc3, 0x45, 0x36, 0xe5, 0x43, 0x0a, 0x24, 0x53, 0x32, 0x74, + 0xdb, 0x54, 0xff, 0x46, 0x81, 0xf2, 0x0e, 0x0e, 0x1f, 0xb6, 0x49, 0x9c, 0x2d, 0x76, 0xfd, 0x3b, + 0x00, 0xfb, 0xf8, 0x88, 0xb9, 0x27, 0x21, 0xf2, 0x5b, 0x09, 0x22, 0x1f, 0x25, 0x50, 0x7f, 0x07, + 0x1f, 0x51, 0x3f, 0x16, 0x6c, 0x39, 0xa1, 0x7f, 0xa4, 0x15, 0xf6, 0x45, 0xbb, 0xfa, 0x16, 0xcc, + 0x0d, 0x23, 0x89, 0x9f, 0xd9, 0xc7, 0x47, 0x5c, 0xc3, 0xc8, 0x27, 0xd1, 0x21, 0xe6, 0x19, 0x89, + 0x2c, 0x8b, 0x1a, 0x6b, 0xdc, 0x4a, 0xbd, 0xa1, 0xa8, 0xe7, 0x61, 0x41, 0x9a, 0x8b, 0xed, 0xb0, + 0xfa, 0x35, 0x28, 0xdf, 0x1e, 0x5d, 0x01, 0x82, 0x99, 0x7d, 0x7c, 0x24, 0x92, 0x19, 0xfa, 0xad, + 0xfe, 0x3c, 0x05, 0x0b, 0xb7, 0x47, 0x47, 0xa3, 0xdf, 0x1f, 0xb3, 0xd6, 0x37, 0x13, 0xd6, 0x7a, + 0x8a, 0xc2, 0xc8, 0x62, 0xb9, 0xaa, 0x49, 0x4b, 0xde, 0x85, 0x0c, 0x73, 0xea, 0xd1, 0xba, 0x14, + 0x69, 0x5d, 0xe8, 0x36, 0x14, 0x6d, 0x23, 0x08, 0xf5, 0x81, 0x67, 0x1a, 0x21, 0x36, 0xb9, 0x6d, + 0x99, 0x4c, 0x0b, 0x67, 0xc9, 0xc8, 0x87, 0x6c, 0x60, 0xd5, 0x9b, 0x40, 0xb4, 0xdf, 0x96, 0x45, + 0x3b, 0xbb, 0xbe, 0x3e, 0xd5, 0x42, 0x29, 0x69, 0x79, 0x3b, 0xca, 0x30, 0xb7, 0x69, 0x0f, 0x82, + 0x10, 0xfb, 0xc2, 0x82, 0x7d, 0xa6, 0xc0, 0x7c, 0x04, 0xe2, 0x12, 0x7e, 0x05, 0xa0, 0xcb, 0x40, + 0xb1, 0x73, 0x28, 0x9d, 0x1c, 0xd7, 0x0a, 0xbc, 0x63, 0xbb, 0xa5, 0x15, 0x78, 0x87, 0xb6, 0x49, + 0x4c, 0x45, 0x7c, 0x06, 0xb0, 0x43, 0xcc, 0xa8, 0xc9, 0x23, 0x93, 0x72, 0x84, 0xd8, 0x62, 0x70, + 0xf4, 0x0d, 0x40, 0x98, 0x18, 0x55, 0xcf, 0xb7, 0x02, 0x1c, 0xf5, 0x66, 0x11, 0xd7, 0x42, 0x8c, + 0xe1, 0xdd, 0x55, 0x0f, 0x8a, 0x2d, 0xdf, 0xb0, 0x1c, 0xa1, 0x25, 0x57, 0xa0, 0xe4, 0xf9, 0x78, + 0xfd, 0xda, 0x75, 0xbd, 0x6f, 0xf8, 0xfb, 0x34, 0x2c, 0x4c, 0xaf, 0x65, 0xb4, 0x22, 0x03, 0xde, + 0xa5, 0x30, 0x62, 0x30, 0x83, 0xbd, 0x41, 0x68, 0xba, 0x4f, 0x1c, 0x11, 0xcb, 0x89, 0x36, 0x5a, + 0x86, 0xbc, 0xe9, 0xea, 0x26, 0xa1, 0xc9, 0xcd, 0x59, 0xce, 0x74, 0xe9, 0x14, 0x77, 0x66, 0xf2, + 0xa9, 0x72, 0x5a, 0xfd, 0x99, 0x02, 0x25, 0x3e, 0x65, 0x1c, 0x7a, 0x58, 0x01, 0x1b, 0x42, 0x42, + 0x22, 0xb6, 0x32, 0xb0, 0x82, 0x16, 0x87, 0xa0, 0x5b, 0xb0, 0x4c, 0xb1, 0xba, 0x8f, 0xfb, 0x0c, + 0xa4, 0x5b, 0x8e, 0x69, 0x75, 0x8d, 0xd0, 0xf5, 0x29, 0x03, 0x33, 0xda, 0x45, 0x93, 0x91, 0xe4, + 0xf8, 0xb6, 0x40, 0xa3, 0x6f, 0xc2, 0xa5, 0xd1, 0xb1, 0x22, 0x82, 0x26, 0xb1, 0x19, 0x0b, 0xdb, + 0x96, 0x87, 0x47, 0xb7, 0xe2, 0x0e, 0x77, 0x66, 0xf2, 0x4a, 0x39, 0xa5, 0x7e, 0x0c, 0xcb, 0x2d, + 0xdc, 0x75, 0xfb, 0x34, 0x48, 0x70, 0x9d, 0x1d, 0x5a, 0x51, 0x11, 0x32, 0xfb, 0x10, 0xf2, 0xdc, + 0xcf, 0xb3, 0xd3, 0x92, 0x69, 0x36, 0x4f, 0x8e, 0x6b, 0x39, 0xe6, 0xe8, 0x83, 0xff, 0x3e, 0xae, + 0xdd, 0xe8, 0x59, 0xe1, 0xde, 0xa0, 0x53, 0xef, 0xba, 0xfd, 0x46, 0xa4, 0x5d, 0x66, 0x27, 0xfe, + 0x6e, 0x78, 0xfb, 0xbd, 0x06, 0xfd, 0xf2, 0x3a, 0x75, 0x1e, 0x20, 0xe4, 0x58, 0x80, 0x10, 0xa8, + 0xc7, 0x0a, 0x9c, 0x97, 0x27, 0xff, 0xed, 0x4c, 0x8b, 0xf6, 0x60, 0x81, 0x1b, 0xe9, 0x3e, 0xee, + 0x77, 0xb0, 0x1f, 0xec, 0x59, 0x1e, 0xdd, 0x9b, 0xb9, 0x21, 0x63, 0xb0, 0x7f, 0x58, 0x17, 0x85, + 0x9f, 0x7a, 0x54, 0x1d, 0x8a, 0x2b, 0x46, 0xf5, 0xbb, 0xd1, 0x60, 0x2e, 0xb4, 0x32, 0xa3, 0x1a, + 0xc3, 0xd5, 0x3f, 0x4f, 0x43, 0x75, 0x9c, 0x74, 0xb9, 0x7a, 0x3c, 0x86, 0x2c, 0xab, 0x60, 0xf1, + 0x54, 0xf6, 0xad, 0xa4, 0xd2, 0xc8, 0x53, 0xc9, 0xd4, 0x59, 0x53, 0xb8, 0x3d, 0x46, 0xb1, 0xfa, + 0x59, 0x0a, 0xb2, 0x0c, 0x81, 0x1e, 0x0f, 0x47, 0x6b, 0x99, 0xe6, 0x46, 0x1c, 0xad, 0x9d, 0x55, + 0x98, 0x22, 0xc8, 0xbb, 0x08, 0x39, 0x2b, 0xd0, 0x89, 0x54, 0x44, 0x46, 0x61, 0x05, 0xdb, 0xd6, + 0x21, 0x3e, 0x1d, 0x54, 0xa7, 0xc7, 0x04, 0xd5, 0x1f, 0x00, 0x48, 0x5b, 0x30, 0xf3, 0xfc, 0x5b, + 0x20, 0x91, 0xa3, 0x09, 0x84, 0x38, 0x79, 0x19, 0x76, 0x96, 0x45, 0x5b, 0xd5, 0x60, 0x7e, 0x07, + 0x87, 0xc4, 0xba, 0x04, 0x09, 0x5e, 0x64, 0x24, 0x94, 0xe1, 0x6e, 0x23, 0x35, 0x1a, 0xca, 0x30, + 0x2b, 0xac, 0xfe, 0x53, 0x8a, 0x7a, 0x57, 0x4e, 0x94, 0x6f, 0xb1, 0x3e, 0xbd, 0x77, 0x1d, 0x22, + 0xf0, 0x4c, 0x87, 0xb3, 0x3f, 0xd6, 0xe1, 0x14, 0x84, 0xc3, 0x19, 0x97, 0xb0, 0xae, 0xc2, 0xac, + 0x6c, 0x28, 0xd2, 0x14, 0x25, 0x83, 0x48, 0xf2, 0xe7, 0x0d, 0x3a, 0xb6, 0xd5, 0xe5, 0x86, 0x8e, + 0xb7, 0xaa, 0xee, 0x04, 0x5e, 0xe7, 0xf6, 0xb0, 0xd7, 0xb9, 0x3e, 0xcd, 0x62, 0x4f, 0x39, 0x9d, + 0x17, 0xa1, 0xf4, 0x6d, 0x6c, 0xd8, 0xe1, 0x9e, 0xd8, 0xa5, 0x45, 0xc8, 0xf8, 0xd8, 0x30, 0xd9, + 0x8c, 0x79, 0x8d, 0x35, 0x88, 0x6f, 0x12, 0xdd, 0x78, 0x9c, 0xb0, 0x00, 0xf3, 0xdb, 0x5c, 0x4f, + 0x84, 0xbb, 0xfa, 0xdf, 0x14, 0x94, 0x63, 0x58, 0x94, 0x4d, 0x80, 0xd0, 0xa7, 0x68, 0x7f, 0x6e, + 0x4c, 0xa1, 0x81, 0x82, 0xa0, 0xa8, 0xab, 0xc4, 0xc4, 0xd0, 0x4f, 0x14, 0xc8, 0xb3, 0xc3, 0x88, + 0xc5, 0x01, 0x4f, 0xca, 0xe5, 0x46, 0x59, 0xe3, 0xc7, 0x5a, 0x6c, 0xfc, 0x9b, 0x84, 0xfe, 0x8f, + 0x7e, 0x75, 0xb6, 0xd3, 0x1a, 0xf1, 0x51, 0xfd, 0x18, 0x4a, 0x43, 0x74, 0xe5, 0x0d, 0xcc, 0xb0, + 0x0d, 0xdc, 0x91, 0x37, 0x70, 0x6e, 0xfd, 0xed, 0x29, 0xa4, 0x41, 0xa6, 0x13, 0xeb, 0xe0, 0x27, + 0x52, 0xda, 0x4c, 0x0f, 0x66, 0xef, 0xb8, 0x9d, 0x40, 0xda, 0x4a, 0x96, 0x3d, 0x28, 0x72, 0xf6, + 0xb0, 0x24, 0xd9, 0x44, 0x5a, 0x77, 0x60, 0x2d, 0x74, 0x83, 0x2b, 0x72, 0x9a, 0x32, 0x55, 0x93, + 0x05, 0x79, 0x60, 0xd7, 0xe9, 0x25, 0x01, 0xbb, 0x30, 0xa8, 0x93, 0x40, 0x9c, 0x69, 0xba, 0xfa, + 0x2f, 0x59, 0x28, 0xb2, 0x29, 0xf9, 0x76, 0x6f, 0xc1, 0x0c, 0xe9, 0xc5, 0x37, 0xfa, 0xe5, 0x84, + 0xed, 0x90, 0x87, 0x91, 0x06, 0xdf, 0x60, 0x3a, 0xbc, 0xfa, 0x3f, 0x19, 0x48, 0xdf, 0x71, 0x3b, + 0x68, 0x09, 0x52, 0xdc, 0xa8, 0xa6, 0x9b, 0xd9, 0x93, 0xe3, 0x5a, 0xaa, 0xdd, 0xd2, 0x52, 0x96, + 0x79, 0xc6, 0x53, 0x27, 0x67, 0x6b, 0x33, 0xc3, 0xd9, 0x1a, 0x3a, 0x84, 0xb9, 0xa1, 0x2a, 0x1a, + 0xab, 0x0e, 0x94, 0x9a, 0xef, 0x9e, 0x1c, 0xd7, 0x4a, 0x72, 0x19, 0x8d, 0xb8, 0xc7, 0x5b, 0x13, + 0xe9, 0x48, 0x70, 0x60, 0x47, 0x17, 0x18, 0x84, 0xb6, 0xd7, 0xa9, 0xb7, 0x5b, 0x5a, 0x49, 0xae, + 0xc2, 0x05, 0xd2, 0x76, 0x64, 0x87, 0xb6, 0xe3, 0x16, 0xe4, 0x58, 0x61, 0xd2, 0xa4, 0xd5, 0x9c, + 0xe4, 0x18, 0x76, 0x86, 0xc6, 0xaf, 0x62, 0x00, 0x19, 0x1b, 0x84, 0x86, 0x4f, 0xc6, 0xe6, 0x27, + 0x1d, 0xcb, 0x07, 0xa0, 0xb7, 0x20, 0xbf, 0x6b, 0x39, 0x56, 0xb0, 0x87, 0x4d, 0x5a, 0x06, 0x9c, + 0x64, 0x70, 0x34, 0x82, 0x8c, 0xee, 0xbb, 0xa6, 0xb5, 0x6b, 0x61, 0xb3, 0x02, 0x93, 0x8e, 0x16, + 0x23, 0x48, 0x00, 0xba, 0xeb, 0x1b, 0xb4, 0xd0, 0xa5, 0x77, 0xdd, 0xbe, 0x67, 0x63, 0xb2, 0x84, + 0xd9, 0x55, 0x65, 0x2d, 0xa5, 0x2d, 0x08, 0xcc, 0xa6, 0x40, 0x10, 0xfd, 0xa6, 0xa5, 0x8e, 0x4a, + 0x91, 0x19, 0x64, 0xda, 0x40, 0xf7, 0xe1, 0xfc, 0x9e, 0xd5, 0xdb, 0x7b, 0x62, 0x90, 0x10, 0x39, + 0x4e, 0x47, 0x4b, 0x13, 0x72, 0x83, 0xa2, 0xc1, 0x11, 0x86, 0x78, 0xa9, 0x98, 0xa4, 0x89, 0xbb, + 0x56, 0xdf, 0xb0, 0x2b, 0x73, 0x74, 0xd2, 0x72, 0x84, 0x68, 0x31, 0x38, 0x7a, 0x11, 0xe6, 0xfc, + 0x81, 0x43, 0xa3, 0x45, 0xbe, 0xb1, 0xf3, 0xb4, 0x67, 0x89, 0x43, 0x79, 0xcc, 0x70, 0x19, 0x0a, + 0x71, 0xbd, 0xb5, 0xcc, 0xd2, 0xdc, 0x08, 0xa0, 0x22, 0x28, 0x6f, 0xbb, 0x5d, 0x76, 0x41, 0x25, + 0xcc, 0xeb, 0xff, 0x29, 0xb0, 0x20, 0x01, 0x23, 0xfb, 0x5a, 0xb0, 0x05, 0x70, 0x82, 0x6b, 0x80, + 0x53, 0x04, 0x22, 0x88, 0xf0, 0x7c, 0x11, 0xb5, 0xea, 0x9f, 0x2a, 0x90, 0x17, 0x58, 0xf4, 0x02, + 0x14, 0x09, 0xc6, 0xb6, 0xc2, 0x23, 0x3d, 0x76, 0x48, 0xb3, 0x02, 0xf6, 0x0e, 0x3e, 0x22, 0x2b, + 0x8f, 0xba, 0xc4, 0x06, 0xae, 0xa0, 0x95, 0x04, 0x94, 0xf9, 0xd1, 0x2a, 0xe4, 0x6d, 0x23, 0xb4, + 0xc2, 0x81, 0xc9, 0x8c, 0x8d, 0xa2, 0x45, 0x6d, 0x22, 0x15, 0xdb, 0x75, 0x7a, 0x0c, 0x39, 0x43, + 0x91, 0x31, 0x40, 0x6d, 0xc2, 0xbc, 0x66, 0x38, 0x3d, 0xbc, 0xed, 0xf6, 0x84, 0x8d, 0x5b, 0x86, + 0x3c, 0xab, 0x3d, 0x0a, 0x33, 0xa1, 0xe5, 0x68, 0x5b, 0x2e, 0x9e, 0xa4, 0x24, 0xf3, 0xa7, 0xfe, + 0x57, 0x1a, 0xca, 0x31, 0x11, 0x2e, 0xc4, 0x77, 0xa3, 0x8a, 0x08, 0x73, 0x23, 0x49, 0x3e, 0x75, + 0x74, 0xf0, 0xd8, 0x9a, 0xc8, 0x3f, 0x2a, 0x00, 0xef, 0xf9, 0x38, 0x0c, 0x8f, 0xda, 0xce, 0xae, + 0x4b, 0x84, 0xc7, 0x13, 0x52, 0x9a, 0x41, 0x08, 0xe1, 0x71, 0x18, 0xb1, 0x2e, 0x64, 0x21, 0x0e, + 0x7e, 0xc2, 0xd0, 0x4c, 0x6c, 0x39, 0x07, 0x3f, 0xa1, 0xa8, 0x2b, 0x50, 0x32, 0x4c, 0x13, 0x9b, + 0x3a, 0x0f, 0xed, 0xb8, 0x69, 0x2b, 0x52, 0xa0, 0xc6, 0x60, 0xe8, 0x25, 0x98, 0xf7, 0x71, 0xdf, + 0x3d, 0x94, 0xba, 0x31, 0x13, 0x37, 0xc7, 0xc1, 0xa2, 0xe3, 0x12, 0x64, 0x7d, 0x6c, 0x04, 0x51, + 0x71, 0x99, 0xb7, 0x50, 0x05, 0x72, 0x26, 0xbb, 0x70, 0xe1, 0x96, 0x48, 0x34, 0xab, 0x3f, 0x53, + 0x44, 0x81, 0xe7, 0x0e, 0x64, 0xe8, 0x02, 0x79, 0x71, 0xa7, 0xfe, 0x14, 0xcf, 0xc5, 0x0b, 0xb2, + 0x92, 0xb0, 0x64, 0x19, 0x31, 0x12, 0xe8, 0x43, 0x98, 0xf5, 0xa8, 0x84, 0x74, 0x5a, 0x71, 0x61, + 0xc1, 0xcc, 0x6b, 0xd3, 0x08, 0x3e, 0x16, 0xb0, 0x08, 0x0e, 0xbc, 0x08, 0xc2, 0x93, 0xaf, 0x35, + 0x28, 0xdf, 0x1f, 0x60, 0xff, 0xe8, 0x3d, 0xdb, 0x70, 0x24, 0xb7, 0x78, 0x40, 0x60, 0x22, 0x8e, + 0xa3, 0x0d, 0xd5, 0x83, 0x05, 0xa9, 0x27, 0xd7, 0x8b, 0x0f, 0xe0, 0x92, 0x69, 0x05, 0x61, 0x70, + 0x60, 0xeb, 0xde, 0xde, 0x51, 0x60, 0x75, 0x0d, 0x5b, 0xa7, 0xdd, 0x75, 0xcf, 0x36, 0x1c, 0x9e, + 0x7d, 0x5f, 0x3e, 0x39, 0xae, 0x55, 0x5a, 0x56, 0x10, 0xee, 0xdc, 0xdf, 0x7e, 0x8f, 0xf7, 0x8a, + 0x49, 0x55, 0x38, 0x81, 0x53, 0x18, 0x75, 0x99, 0x5d, 0xe9, 0x92, 0x91, 0xbe, 0xd5, 0x19, 0x84, + 0x71, 0x7e, 0xa6, 0xfe, 0x2d, 0x40, 0xe5, 0x34, 0x8e, 0x33, 0xe5, 0x41, 0x49, 0xd4, 0xe9, 0x99, + 0xe8, 0xd8, 0xa9, 0xdf, 0x7a, 0xc6, 0xb5, 0xef, 0x38, 0x5a, 0xd1, 0x7d, 0x30, 0x11, 0x99, 0x1c, + 0xff, 0x16, 0x4d, 0x09, 0x81, 0xfa, 0x50, 0x94, 0x2e, 0xb0, 0xc4, 0xfd, 0x5d, 0xeb, 0x2c, 0x13, + 0xc6, 0x97, 0x5a, 0x43, 0xf1, 0xf6, 0x6c, 0x7c, 0xa9, 0x15, 0x54, 0x3f, 0x53, 0x00, 0xe2, 0x7e, + 0x44, 0x61, 0x59, 0xde, 0xc7, 0x37, 0x8c, 0xb7, 0xd0, 0x06, 0x64, 0xf9, 0xed, 0x5c, 0x6a, 0xda, + 0xdb, 0x39, 0x3e, 0x90, 0x16, 0x53, 0xd8, 0xa5, 0x5c, 0x70, 0x60, 0xb3, 0xf3, 0xc2, 0x8b, 0x29, + 0x14, 0xba, 0x73, 0x7f, 0x5b, 0x2b, 0xb0, 0x0e, 0x3b, 0x07, 0xf6, 0x9d, 0x99, 0x7c, 0xba, 0x3c, + 0x53, 0xfd, 0xe3, 0x34, 0x14, 0x68, 0x3d, 0x9a, 0x8a, 0xe6, 0x0b, 0x05, 0x2a, 0x43, 0x69, 0x98, + 0xde, 0x39, 0xd2, 0xe3, 0x64, 0x90, 0xc8, 0xe9, 0xfd, 0xb3, 0xc8, 0x29, 0x9a, 0xa1, 0xae, 0x49, + 0xf9, 0x5c, 0xf3, 0x88, 0xc6, 0x9b, 0x26, 0x13, 0xdd, 0xeb, 0x67, 0x8d, 0x56, 0x17, 0xfd, 0x31, + 0x34, 0xd1, 0x57, 0x61, 0x4e, 0xbe, 0xa6, 0x8c, 0xca, 0xd1, 0xc5, 0x78, 0x6f, 0xda, 0x26, 0xfa, + 0x16, 0x80, 0xe9, 0xbb, 0x9e, 0x87, 0x4d, 0xdd, 0x60, 0x39, 0xe7, 0x24, 0x5e, 0xb5, 0xc0, 0xc7, + 0x6c, 0x84, 0xd5, 0xdb, 0xb0, 0xfc, 0xd4, 0x25, 0x8d, 0x09, 0x96, 0x87, 0xca, 0x97, 0x69, 0x29, + 0xda, 0xad, 0xfe, 0x30, 0x05, 0x45, 0x59, 0x7f, 0x51, 0x08, 0xec, 0x75, 0x82, 0x7c, 0x2a, 0xde, + 0x7d, 0xde, 0x53, 0x11, 0xef, 0xc4, 0x50, 0x7e, 0x18, 0x0a, 0x68, 0xf5, 0x13, 0x98, 0x1b, 0xee, + 0x32, 0x26, 0x65, 0xdb, 0x19, 0x4e, 0xd9, 0xde, 0x7e, 0x2e, 0x8d, 0x18, 0x92, 0x81, 0x12, 0xbf, + 0x40, 0x49, 0x62, 0xe0, 0xd1, 0x30, 0x03, 0xbf, 0xfb, 0xbc, 0x52, 0x91, 0x79, 0xf8, 0x01, 0x94, + 0x47, 0x4f, 0xf5, 0x18, 0x0e, 0x1e, 0x0c, 0x73, 0xf0, 0xcd, 0xe7, 0x33, 0x1e, 0xd2, 0xfc, 0xbc, + 0x36, 0x78, 0x11, 0x2e, 0xdc, 0xa5, 0x6f, 0x80, 0xee, 0xe2, 0xd0, 0x30, 0xe3, 0xe2, 0xb5, 0xfa, + 0xaf, 0x0a, 0x2c, 0x8d, 0x62, 0xb8, 0x25, 0x35, 0x20, 0xdf, 0xe7, 0x30, 0xae, 0x2e, 0xdf, 0x4a, + 0x60, 0x6b, 0x3c, 0x91, 0xba, 0x00, 0xc8, 0xea, 0x11, 0x91, 0xad, 0xfe, 0x1e, 0x94, 0x86, 0x3a, + 0x8c, 0x91, 0xcc, 0xab, 0xc3, 0x92, 0x91, 0x33, 0xaf, 0x41, 0x68, 0xd9, 0x75, 0xfe, 0xb4, 0x29, + 0x9a, 0x58, 0x4a, 0xf8, 0x7e, 0x9a, 0x82, 0xf3, 0x5b, 0xce, 0xc1, 0x00, 0x0f, 0x30, 0xf5, 0x8e, + 0xc2, 0xc5, 0xfd, 0x26, 0x0b, 0x52, 0xcc, 0x7d, 0x46, 0xc1, 0x1d, 0x6b, 0xa0, 0xef, 0x4a, 0x71, + 0x18, 0xbd, 0xac, 0x6a, 0x6e, 0x9e, 0x1c, 0xd7, 0x72, 0x94, 0x2b, 0x3a, 0xe7, 0xcd, 0xa9, 0xe6, + 0xe4, 0xe3, 0xe2, 0x60, 0xee, 0x2a, 0x2c, 0x04, 0xfb, 0x96, 0xa7, 0x07, 0x7b, 0xee, 0xc0, 0x36, + 0x75, 0xc6, 0x01, 0xab, 0x9d, 0xcc, 0x13, 0xc4, 0x0e, 0x85, 0xdf, 0x27, 0x60, 0xf5, 0xaf, 0x52, + 0xb0, 0x38, 0x2c, 0x15, 0xbe, 0xdf, 0xf7, 0xe3, 0x10, 0x87, 0x6d, 0xf7, 0xeb, 0x49, 0x37, 0x5f, + 0x63, 0x28, 0xd4, 0xc5, 0x93, 0x94, 0x28, 0x36, 0xfa, 0x3b, 0x05, 0x72, 0x1c, 0xf8, 0x1b, 0x95, + 0xfa, 0xdb, 0x23, 0x11, 0xea, 0x8b, 0x49, 0xb7, 0xa5, 0xbe, 0xd1, 0xc5, 0x34, 0xe2, 0x12, 0xf1, + 0x68, 0x9c, 0x2a, 0xa5, 0xa5, 0x54, 0x49, 0xbd, 0x00, 0xe7, 0x37, 0xf7, 0x0c, 0x3f, 0xdc, 0x64, + 0x49, 0xaa, 0x38, 0x31, 0x8f, 0x61, 0x71, 0x18, 0xcc, 0xc5, 0xd7, 0x84, 0x1c, 0x4f, 0x67, 0xb9, + 0xf8, 0x54, 0x89, 0x89, 0x30, 0xa8, 0x8b, 0xe7, 0x79, 0x74, 0xf0, 0x0e, 0x7b, 0xb0, 0x10, 0xbd, + 0x02, 0x62, 0xb8, 0xab, 0xef, 0xc0, 0xd2, 0xf8, 0x77, 0x2d, 0x68, 0x16, 0x72, 0x0f, 0xef, 0xbd, + 0x73, 0xef, 0xdd, 0xf7, 0xef, 0x95, 0xcf, 0x91, 0xc6, 0xe6, 0xf6, 0xc3, 0x9d, 0x07, 0x5b, 0x5a, + 0x59, 0x41, 0x45, 0xc8, 0xb7, 0x36, 0x1e, 0x6c, 0x34, 0x37, 0x76, 0xb6, 0xca, 0x29, 0x54, 0x80, + 0xcc, 0x83, 0x8d, 0xe6, 0xf6, 0x56, 0x39, 0xbd, 0xfe, 0x93, 0x2a, 0x64, 0x36, 0xcc, 0xbe, 0xe5, + 0xa0, 0x10, 0x32, 0xf4, 0xca, 0x16, 0xbd, 0xf4, 0xec, 0x4b, 0x5d, 0xba, 0xc8, 0xea, 0xda, 0xa4, + 0xb7, 0xbf, 0x6a, 0xe5, 0x47, 0xff, 0xfc, 0x1f, 0x3f, 0x4d, 0x21, 0x54, 0x6e, 0xe8, 0xf4, 0xe1, + 0x65, 0xe3, 0xf0, 0x7a, 0x83, 0xde, 0x02, 0xa3, 0x3f, 0x52, 0xa0, 0x10, 0xbd, 0xff, 0x43, 0x2f, + 0x4f, 0xf0, 0xee, 0x2e, 0x9a, 0xfe, 0x95, 0xc9, 0x3a, 0x73, 0x16, 0x2e, 0x53, 0x16, 0x96, 0xd0, + 0xa2, 0xc4, 0x42, 0xf4, 0xa4, 0x10, 0xfd, 0x85, 0x02, 0xf3, 0x23, 0x0f, 0xfb, 0xd0, 0xf5, 0x69, + 0x1e, 0x01, 0x32, 0x96, 0xd6, 0xa7, 0x7f, 0x37, 0xa8, 0xbe, 0x44, 0x19, 0x7b, 0x01, 0xd5, 0xc6, + 0x31, 0xd6, 0xf8, 0x44, 0x7c, 0x7e, 0x1f, 0xfd, 0xb5, 0x02, 0x45, 0xf9, 0x51, 0x17, 0xaa, 0x4f, + 0xfc, 0xfa, 0x8b, 0x71, 0xd7, 0x98, 0xf2, 0xb5, 0x98, 0xfa, 0x1a, 0x65, 0xed, 0x1a, 0xaa, 0x3f, + 0x83, 0xb5, 0x06, 0x75, 0xe5, 0x41, 0xe3, 0x13, 0xfa, 0x4f, 0x39, 0x85, 0xf8, 0xbd, 0x01, 0x7a, + 0x65, 0xc2, 0x67, 0x09, 0x8c, 0xcb, 0xe9, 0x1e, 0x31, 0xa8, 0x6f, 0x51, 0x1e, 0x5f, 0x43, 0x37, + 0xa7, 0xe3, 0xb1, 0xc1, 0xde, 0xb3, 0xfc, 0x99, 0x02, 0xa5, 0xa1, 0x27, 0x1c, 0x28, 0x49, 0x48, + 0xe3, 0x5e, 0x81, 0x54, 0xaf, 0x4d, 0x3e, 0x80, 0xb3, 0xbc, 0x4a, 0x59, 0xae, 0xa2, 0x8a, 0xc4, + 0xb2, 0xe3, 0x3a, 0x8c, 0x41, 0xca, 0xc4, 0x47, 0x90, 0x65, 0xaf, 0x06, 0xd0, 0xda, 0x04, 0x0f, + 0x0b, 0x18, 0x1f, 0x5f, 0x9f, 0xf8, 0x09, 0x82, 0xba, 0x4c, 0x19, 0x38, 0x8f, 0x16, 0x24, 0x06, + 0xb8, 0x95, 0x23, 0xe7, 0x31, 0xba, 0xd1, 0x4e, 0x3c, 0x8f, 0xa3, 0x77, 0xec, 0x89, 0xe7, 0xf1, + 0xf4, 0x25, 0x39, 0x3f, 0x8f, 0xaa, 0xcc, 0xc3, 0xc0, 0x22, 0xdb, 0x75, 0x4b, 0xb9, 0x8a, 0x7e, + 0xa8, 0x40, 0xe1, 0xf6, 0x44, 0x6c, 0xdc, 0x9e, 0x86, 0x8d, 0x53, 0x57, 0xc8, 0x63, 0x45, 0xc1, + 0xd8, 0x40, 0x7f, 0x00, 0x39, 0x7e, 0x21, 0x8c, 0x92, 0x64, 0x3b, 0x7c, 0xe1, 0x5c, 0xbd, 0x3a, + 0x49, 0x57, 0x3e, 0x79, 0x95, 0x4e, 0xbe, 0x88, 0x90, 0x34, 0x39, 0xbf, 0x78, 0x46, 0x7f, 0xa8, + 0x40, 0x5e, 0xdc, 0x3d, 0xa0, 0xab, 0x13, 0x5d, 0x50, 0x30, 0x06, 0x5e, 0x9e, 0xe2, 0x32, 0x43, + 0xbd, 0x44, 0x39, 0xb8, 0x80, 0xce, 0x4b, 0x1c, 0x04, 0x62, 0xd6, 0x1f, 0x2b, 0x90, 0x65, 0x57, + 0x16, 0x89, 0x6a, 0x38, 0x74, 0xf9, 0x91, 0xa8, 0x86, 0x23, 0xf7, 0x1f, 0x5f, 0xa5, 0x93, 0xaf, + 0x0c, 0xc9, 0x7e, 0x8f, 0x76, 0x79, 0x5c, 0x40, 0x39, 0xfe, 0x49, 0x05, 0x21, 0xea, 0xf5, 0x89, + 0x82, 0x18, 0xb9, 0x4b, 0x49, 0x14, 0xc4, 0xe8, 0x45, 0xc6, 0x58, 0x41, 0x88, 0x4b, 0x03, 0xe4, + 0xc1, 0xcc, 0x1d, 0xb7, 0x13, 0xa0, 0xaf, 0x3d, 0xb3, 0x16, 0xcf, 0x66, 0x7e, 0x69, 0xc2, 0x9a, + 0xbd, 0x7a, 0x91, 0xce, 0xba, 0x80, 0xe6, 0xa5, 0x59, 0xbf, 0x47, 0x66, 0x22, 0xc7, 0x30, 0xaa, + 0x33, 0x26, 0xea, 0xff, 0x68, 0x8d, 0x33, 0x51, 0xff, 0x4f, 0x95, 0x2e, 0xc7, 0xba, 0xc5, 0xa8, + 0x7c, 0x49, 0xd9, 0x88, 0xaa, 0x2d, 0x89, 0x6c, 0x8c, 0x96, 0x88, 0x12, 0xd9, 0x38, 0x55, 0x25, + 0x1a, 0xcb, 0x06, 0xad, 0x12, 0x79, 0x64, 0xe2, 0xef, 0x42, 0x86, 0xbe, 0x46, 0x48, 0x0c, 0x4d, + 0xe4, 0x87, 0x14, 0x89, 0xa1, 0xc9, 0xd0, 0xf3, 0x07, 0xf5, 0xdc, 0x35, 0x05, 0x3d, 0x81, 0xa2, + 0x7c, 0x75, 0x9d, 0xe8, 0x58, 0xc7, 0xbc, 0x05, 0xa8, 0xbe, 0x7a, 0xa6, 0x3b, 0x71, 0xf5, 0x1c, + 0x39, 0x61, 0xe8, 0x74, 0x07, 0x74, 0x73, 0x4a, 0x7a, 0xcf, 0xc9, 0xc5, 0x5f, 0x2a, 0x90, 0x17, + 0x95, 0xc1, 0xc4, 0x13, 0x36, 0x52, 0x39, 0x4e, 0x3c, 0x61, 0xa3, 0xa5, 0x46, 0xf5, 0x6d, 0xba, + 0xc5, 0xaf, 0x0f, 0x9d, 0x30, 0x9a, 0x9a, 0xd8, 0x6e, 0xef, 0xf1, 0x2a, 0x5a, 0x19, 0x03, 0x6e, + 0x7c, 0x22, 0x12, 0xa2, 0xef, 0x93, 0x08, 0xad, 0x3c, 0x9a, 0xd5, 0xa2, 0xf5, 0xa9, 0x52, 0x60, + 0xc6, 0xf4, 0x8d, 0x33, 0xa4, 0xcd, 0x91, 0xa9, 0xba, 0x3c, 0x12, 0x65, 0xe8, 0xa6, 0xcc, 0xce, + 0x67, 0x0a, 0x2c, 0x6c, 0xd8, 0xf6, 0x70, 0x96, 0x8b, 0xae, 0x4d, 0x91, 0x10, 0x33, 0x16, 0xaf, + 0x4f, 0x9d, 0x42, 0xab, 0x2f, 0x50, 0x06, 0x2f, 0xa1, 0x65, 0x89, 0x41, 0x96, 0xf5, 0x8a, 0x3c, + 0x1a, 0x7d, 0xaa, 0x40, 0x51, 0x4e, 0x4a, 0x12, 0xd5, 0x7c, 0x4c, 0x52, 0x93, 0x18, 0x3f, 0x8e, + 0xcb, 0x76, 0xd4, 0x1a, 0x65, 0x6a, 0x19, 0x5d, 0x94, 0xfd, 0x1b, 0xe9, 0xc8, 0x53, 0x19, 0x12, + 0x7e, 0x15, 0xe5, 0x24, 0x31, 0x91, 0xa5, 0x31, 0x59, 0x7a, 0x22, 0x4b, 0xe3, 0xb2, 0x4f, 0xf5, + 0x0a, 0x65, 0xe9, 0x2b, 0xaa, 0x1c, 0x7b, 0x61, 0xd6, 0x51, 0xa7, 0xca, 0x76, 0x4b, 0xb9, 0xda, + 0xbc, 0xfa, 0xc5, 0xbf, 0xaf, 0x9c, 0xfb, 0xe2, 0x64, 0x45, 0xf9, 0xc5, 0xc9, 0x8a, 0xf2, 0xcb, + 0x93, 0x15, 0xe5, 0xdf, 0x4e, 0x56, 0x94, 0x4f, 0xbf, 0x5c, 0x39, 0xf7, 0x8b, 0x2f, 0x57, 0xce, + 0xfd, 0xf2, 0xcb, 0x95, 0x73, 0x8f, 0xf3, 0x62, 0x96, 0x4e, 0x96, 0x16, 0xec, 0x6e, 0xfc, 0x7f, + 0x00, 0x00, 0x00, 0xff, 0xff, 0xb6, 0xef, 0x33, 0x0d, 0x62, 0x36, 0x00, 0x00, } diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index 109c9e3740dd..efcb56bda25c 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -174,6 +174,10 @@ message TableDetailsResponse { // It can be used to find events pertaining to this table by filtering on // the 'target_id' field of events. int64 descriptor_id = 8 [(gogoproto.customname) = "DescriptorID"]; + // configure_zone_statement is the output of "SHOW ZONE CONFIGURATION FOR TABLE" + // for this table. It is a SQL statement that would re-configure the table's current + // zone if executed. + string configure_zone_statement = 9; } // TableStatsRequest is a request for detailed, computationally expensive diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index 52e72596e6c0..4448c7a094cd 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -1128,6 +1128,15 @@ func (tc *TestCluster) Restart() error { // RestartServer uses the cached ServerArgs to restart a Server specified by // the passed index. func (tc *TestCluster) RestartServer(idx int) error { + return tc.RestartServerWithInspect(idx, nil) +} + +// RestartServerWithInspect uses the cached ServerArgs to restart a Server +// specified by the passed index. We allow an optional inspect function to be +// passed in that can observe the server once its been re-created but before it's +// bean started. This is useful for tests that want to capture that the startup +// sequence performs the correct actions i.e. that on startup liveness is gossiped. +func (tc *TestCluster) RestartServerWithInspect(idx int, inspect func(s *server.TestServer)) error { if !tc.ServerStopped(idx) { return errors.Errorf("server %d must be stopped before attempting to restart", idx) } @@ -1168,6 +1177,10 @@ func (tc *TestCluster) RestartServer(idx int) error { tc.Servers[idx] = s tc.mu.serverStoppers[idx] = s.Stopper() + if inspect != nil { + inspect(s) + } + if err := srv.Start(); err != nil { return err } diff --git a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.fixtures.ts b/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.fixtures.ts index a3a084a98100..e2aca9af608b 100644 --- a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.fixtures.ts +++ b/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.fixtures.ts @@ -53,6 +53,8 @@ export const dbFullfilledProps: DatabaseSummaryProps = { rangeCount: 1, createStatement: 'CREATE TABLE comments (\n\ttype INT8 NOT NULL,\n\tobject_id INT8 NOT NULL,\n\tsub_id INT8 NOT NULL,\n\tcomment STRING NOT NULL,\n\tCONSTRAINT "primary" PRIMARY KEY (type ASC, object_id ASC, sub_id ASC),\n\tFAMILY "primary" (type, object_id, sub_id),\n\tFAMILY fam_4_comment (comment)\n)', + configureZoneStatement: + "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", grants: [ { user: "admin", privileges: ["DELETE"] }, { user: "admin", privileges: ["GRANT"] }, @@ -93,6 +95,8 @@ export const dbFullfilledProps: DatabaseSummaryProps = { rangeCount: 0, createStatement: 'CREATE TABLE descriptor (\n\tid INT8 NOT NULL,\n\tdescriptor BYTES NULL,\n\tCONSTRAINT "primary" PRIMARY KEY (id ASC),\n\tFAMILY "primary" (id),\n\tFAMILY fam_2_descriptor (descriptor)\n)', + configureZoneStatement: + "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", grants: [ { user: "admin", privileges: ["GRANT"] }, { user: "admin", privileges: ["SELECT"] }, @@ -126,6 +130,8 @@ export const dbFullfilledProps: DatabaseSummaryProps = { rangeCount: 1, createStatement: 'CREATE TABLE eventlog (\n\t"timestamp" TIMESTAMP NOT NULL,\n\t"eventType" STRING NOT NULL,\n\t"targetID" INT8 NOT NULL,\n\t"reportingID" INT8 NOT NULL,\n\tinfo STRING NULL,\n\t"uniqueID" BYTES NOT NULL DEFAULT uuid_v4(),\n\tCONSTRAINT "primary" PRIMARY KEY ("timestamp" ASC, "uniqueID" ASC),\n\tFAMILY "primary" ("timestamp", "uniqueID"),\n\tFAMILY "fam_2_eventType" ("eventType"),\n\tFAMILY "fam_3_targetID" ("targetID"),\n\tFAMILY "fam_4_reportingID" ("reportingID"),\n\tFAMILY fam_5_info (info)\n)', + configureZoneStatement: + "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", grants: [ { user: "admin", privileges: ["DELETE"] }, { user: "admin", privileges: ["GRANT"] }, @@ -165,6 +171,8 @@ export const dbFullfilledProps: DatabaseSummaryProps = { rangeCount: 1, createStatement: "CREATE TABLE jobs (\n\tid INT8 NOT NULL DEFAULT unique_rowid(),\n\tstatus STRING NOT NULL,\n\tcreated TIMESTAMP NOT NULL DEFAULT now():::TIMESTAMP,\n\tpayload BYTES NOT NULL,\n\tprogress BYTES NULL,\n\tCONSTRAINT \"primary\" PRIMARY KEY (id ASC),\n\tINDEX jobs_status_created_idx (status ASC, created ASC),\n\tFAMILY fam_0_id_status_created_payload (id, status, created, payload),\n\tFAMILY progress (progress)\n);\nALTER TABLE system.public.jobs CONFIGURE ZONE USING\n\trange_min_bytes = 16777216,\n\trange_max_bytes = 67108864,\n\tgc.ttlseconds = 600,\n\tnum_replicas = 5,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", + configureZoneStatement: + "ALTER TABLE system.public.jobs CONFIGURE ZONE USING\n\trange_min_bytes = 16777216,\n\trange_max_bytes = 67108864,\n\tgc.ttlseconds = 600,\n\tnum_replicas = 5,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", grants: [ { user: "admin", privileges: ["DELETE"] }, { user: "admin", privileges: ["GRANT"] }, @@ -204,6 +212,8 @@ export const dbFullfilledProps: DatabaseSummaryProps = { rangeCount: 1, createStatement: 'CREATE TABLE lease (\n\t"descID" INT8 NOT NULL,\n\tversion INT8 NOT NULL,\n\t"nodeID" INT8 NOT NULL,\n\texpiration TIMESTAMP NOT NULL,\n\tCONSTRAINT "primary" PRIMARY KEY ("descID" ASC, version ASC, expiration ASC, "nodeID" ASC),\n\tFAMILY "primary" ("descID", version, "nodeID", expiration)\n)', + configureZoneStatement: + "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", grants: [ { user: "admin", privileges: ["DELETE"] }, { user: "admin", privileges: ["GRANT"] }, @@ -243,6 +253,8 @@ export const dbFullfilledProps: DatabaseSummaryProps = { rangeCount: 1, createStatement: 'CREATE TABLE locations (\n\t"localityKey" STRING NOT NULL,\n\t"localityValue" STRING NOT NULL,\n\tlatitude DECIMAL(18,15) NOT NULL,\n\tlongitude DECIMAL(18,15) NOT NULL,\n\tCONSTRAINT "primary" PRIMARY KEY ("localityKey" ASC, "localityValue" ASC),\n\tFAMILY "fam_0_localityKey_localityValue_latitude_longitude" ("localityKey", "localityValue", latitude, longitude)\n)', + configureZoneStatement: + "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", grants: [ { user: "admin", privileges: ["DELETE"] }, { user: "admin", privileges: ["GRANT"] }, @@ -282,6 +294,8 @@ export const dbFullfilledProps: DatabaseSummaryProps = { rangeCount: 1, createStatement: 'CREATE TABLE namespace (\n\t"parentID" INT8 NOT NULL,\n\tname STRING NOT NULL,\n\tid INT8 NULL,\n\tCONSTRAINT "primary" PRIMARY KEY ("parentID" ASC, name ASC),\n\tFAMILY "primary" ("parentID", name),\n\tFAMILY fam_3_id (id)\n)', + configureZoneStatement: + "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", grants: [ { user: "admin", privileges: ["GRANT"] }, { user: "admin", privileges: ["SELECT"] }, diff --git a/pkg/ui/src/views/databases/containers/tableDetails/index.tsx b/pkg/ui/src/views/databases/containers/tableDetails/index.tsx index bf2e49ef6ede..6abb5dd20c06 100644 --- a/pkg/ui/src/views/databases/containers/tableDetails/index.tsx +++ b/pkg/ui/src/views/databases/containers/tableDetails/index.tsx @@ -30,7 +30,6 @@ import { SortSetting } from "src/views/shared/components/sortabletable"; import { SortedTable } from "src/views/shared/components/sortedtable"; const { TabPane } = Tabs; import { getMatchParamByName } from "src/util/query"; -import { databaseDetails } from "../databaseSummary"; import { Button } from "@cockroachlabs/cluster-ui"; import { ArrowLeft } from "@cockroachlabs/icons"; import SqlBox from "src/views/shared/components/sql/box"; @@ -61,7 +60,6 @@ interface TableMainActions { refreshTableStats: typeof refreshTableStats; refreshDatabaseDetails: typeof refreshDatabaseDetails; setSort: typeof databaseTableGrantsSortSetting.set; - dbResponse: protos.cockroach.server.serverpb.DatabaseDetailsResponse; } /** @@ -100,7 +98,7 @@ export class TableMain extends React.Component { prevPage = () => this.props.history.goBack(); render() { - const { tableInfo, grantsSortSetting, match, dbResponse } = this.props; + const { tableInfo, grantsSortSetting, match } = this.props; const database = getMatchParamByName(match, databaseNameAttr); const table = getMatchParamByName(match, tableNameAttr); @@ -130,7 +128,7 @@ export class TableMain extends React.Component { @@ -221,17 +219,12 @@ export function selectTableInfo( const mapStateToProps = ( state: AdminUIState, ownProps: RouteComponentProps, -) => ({ - tableInfo: selectTableInfo(state, ownProps), - grantsSortSetting: databaseTableGrantsSortSetting.selector(state), - dbResponse: - databaseDetails(state)[ - getMatchParamByName(ownProps.match, databaseNameAttr) - ] && - databaseDetails(state)[ - getMatchParamByName(ownProps.match, databaseNameAttr) - ].data, -}); +) => { + return { + tableInfo: selectTableInfo(state, ownProps), + grantsSortSetting: databaseTableGrantsSortSetting.selector(state), + }; +}; const mapDispatchToProps = { setSort: databaseTableGrantsSortSetting.set, diff --git a/pkg/ui/src/views/databases/data/tableInfo.tsx b/pkg/ui/src/views/databases/data/tableInfo.tsx index b57421610824..7dc11cd12e5a 100644 --- a/pkg/ui/src/views/databases/data/tableInfo.tsx +++ b/pkg/ui/src/views/databases/data/tableInfo.tsx @@ -27,6 +27,7 @@ export class TableInfo { public mvccSize: protos.cockroach.storage.enginepb.IMVCCStats; public rangeCount: number; public createStatement: string; + public configureZoneStatement: string; public grants: protos.cockroach.server.serverpb.TableDetailsResponse.IGrant[]; public numReplicas: number; constructor( @@ -42,6 +43,7 @@ export class TableInfo { this.rangeCount = stats && stats.range_count && stats.range_count.toNumber(); this.createStatement = details && details.create_table_statement; + this.configureZoneStatement = details && details.configure_zone_statement; this.grants = details && details.grants; this.numReplicas = details && details.zone_config && details.zone_config.num_replicas; diff --git a/pkg/ui/src/views/shared/components/sql/box.tsx b/pkg/ui/src/views/shared/components/sql/box.tsx index de660895da23..211fee2edc9e 100644 --- a/pkg/ui/src/views/shared/components/sql/box.tsx +++ b/pkg/ui/src/views/shared/components/sql/box.tsx @@ -13,17 +13,15 @@ import { Highlight } from "./highlight"; import classNames from "classnames/bind"; import styles from "./sqlhighlight.module.styl"; -import * as protos from "src/js/protos"; export interface SqlBoxProps { value: string; - zone?: protos.cockroach.server.serverpb.DatabaseDetailsResponse; + secondaryValue?: string; } const cx = classNames.bind(styles); class SqlBox extends React.Component { - preNode: React.RefObject = React.createRef(); render() { return (
diff --git a/pkg/ui/src/views/shared/components/sql/highlight.tsx b/pkg/ui/src/views/shared/components/sql/highlight.tsx index 59c67cd5036d..c5a3044b66d6 100644 --- a/pkg/ui/src/views/shared/components/sql/highlight.tsx +++ b/pkg/ui/src/views/shared/components/sql/highlight.tsx @@ -18,6 +18,7 @@ const cx = classNames.bind(styles); export class Highlight extends React.Component { preNode: React.RefObject = React.createRef(); + preNodeSecondary: React.RefObject = React.createRef(); shouldComponentUpdate(newProps: SqlBoxProps) { return newProps.value !== this.props.value; @@ -26,65 +27,34 @@ export class Highlight extends React.Component { componentDidMount() { hljs.configure({ tabReplace: " ", + languages: ["sql"], }); hljs.highlightBlock(this.preNode.current); + if (this.preNodeSecondary.current) { + hljs.highlightBlock(this.preNodeSecondary.current); + } } componentDidUpdate() { hljs.highlightBlock(this.preNode.current); + if (this.preNodeSecondary.current) { + hljs.highlightBlock(this.preNodeSecondary.current); + } } - renderZone = () => { - const { zone } = this.props; - const zoneConfig = zone.zone_config; - return ( - - CONFIGURE ZONE USING -
- range_min_bytes = - {`${String( - zoneConfig.range_min_bytes, - )},`} -
- range_max_bytes = - {`${String( - zoneConfig.range_max_bytes, - )},`} -
- {zoneConfig.gc?.ttl_seconds && ( - <> - gc.ttlseconds = - {`${zoneConfig.gc.ttl_seconds},`} -
- - )} - num_replicas = - {`${zoneConfig.num_replicas},`} -
- constraints = [' - {String(zoneConfig.constraints)} - '], -
- lease_preferences = [[' - - {String(zoneConfig.lease_preferences)} - - ']] -
- ); - }; - render() { - const { value, zone } = this.props; + const { value, secondaryValue } = this.props; return ( <> {value} - {zone && ( + {secondaryValue && ( <>
- {this.renderZone()} + + {secondaryValue} + )}