diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index dec8162a17fa..11f9beb9e864 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -90,6 +90,6 @@
trace.debug.enable | boolean | false | if set, traces for recent requests can be seen in the /debug page |
trace.lightstep.token | string |
| if set, traces go to Lightstep using this token |
trace.zipkin.collector | string |
| if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set |
-version | custom validation | 20.2-1 | set the active cluster version in the format '.' |
+version | custom validation | 20.2-2 | set the active cluster version in the format '.' |
diff --git a/pkg/clusterversion/clusterversion.go b/pkg/clusterversion/clusterversion.go
index 76add51de4ef..43f90e1fed64 100644
--- a/pkg/clusterversion/clusterversion.go
+++ b/pkg/clusterversion/clusterversion.go
@@ -41,9 +41,9 @@ package clusterversion
import (
"context"
-
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
+ "github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/redact"
)
@@ -76,6 +76,11 @@ func SetBeforeChange(
// Handle is a read-only view to the active cluster version and this binary's
// version details.
+//
+// XXX: We'll want this same handle to exist going forward. Except underneath
+// the hood the "setting" of things doesn't happen through this StateMachine guy
+// anymore. It happens through the one RPC.
+// XXX: I'm unsure what the migration story _there_ looks like.
type Handle interface {
// ActiveVersion returns the cluster's current active version: the minimum
// cluster version the caller may assume is in effect.
@@ -158,6 +163,32 @@ func MakeVersionHandle(sv *settings.Values) Handle {
return MakeVersionHandleWithOverride(sv, binaryVersion, binaryMinSupportedVersion)
}
+type MutableRef struct {
+ handleImpl *handleImpl
+}
+
+func (r MutableRef) Set(ctx context.Context, v *roachpb.Version) error {
+ sv := r.handleImpl.sv
+ newV := *v
+ if err := version.validateSupportedVersionInner(ctx, newV, sv); err != nil {
+ return err
+ }
+
+ value := ClusterVersion{Version: newV}
+ encoded, err := protoutil.Marshal(&value)
+ if err != nil {
+ return err
+ }
+ version.SetInternal(sv, encoded)
+ return nil
+}
+
+func MakeMutableRef(handle Handle) MutableRef {
+ return MutableRef{
+ handleImpl: handle.(*handleImpl),
+ }
+}
+
// MakeVersionHandleWithOverride returns a Handle that has its
// binary and minimum supported versions initialized to the provided versions.
//
diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go
index 732740f176a9..64e58be4dfcd 100644
--- a/pkg/clusterversion/cockroach_versions.go
+++ b/pkg/clusterversion/cockroach_versions.go
@@ -78,6 +78,7 @@ const (
VersionHBAForNonTLS
Version20_2
VersionStart21_1
+ VersionLRMTable
// Add new versions here (step one of two).
)
@@ -441,6 +442,11 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: VersionStart21_1,
Version: roachpb.Version{Major: 20, Minor: 2, Unstable: 1},
},
+ {
+ // XXX:
+ Key: VersionLRMTable,
+ Version: roachpb.Version{Major: 20, Minor: 2, Unstable: 2},
+ },
// Add new versions here (step two of two).
})
diff --git a/pkg/clusterversion/migration/migrations.go b/pkg/clusterversion/migration/migrations.go
new file mode 100644
index 000000000000..bfeb7f8cd233
--- /dev/null
+++ b/pkg/clusterversion/migration/migrations.go
@@ -0,0 +1,74 @@
+// Copyright 2020 The Cockroach Authors.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+package migration
+
+import (
+ "context"
+ "github.com/cockroachdb/cockroach/pkg/roachpb"
+)
+
+var _ Migration = UnreplicatedTruncatedStateMigration
+var _ Migration = GenerationComparableMigration
+var _ Migration = SeparateRaftLogMigration
+var _ Migration = ReplicatedLockTableMigration
+var _ Migration = ReplicasKnowReplicaIDMigration
+
+func UnreplicatedTruncatedStateMigration(ctx context.Context, h *Helper) error {
+ if err := h.IterRangeDescriptors(func(descs ...roachpb.RangeDescriptor) error {
+ // Really AdminMigrate but it doesn't exist yet. It takes an argument
+ // determining the range version to migrate into. Unsure if that's a
+ // different kind of version or just `h.Version()`. I think I lean
+ // towards the latter.
+ _, err := h.db.Scan(ctx, descs[0].StartKey, descs[len(descs)-1].EndKey, -1)
+ return err
+ }); err != nil {
+ return err
+ }
+ if err := h.EveryNode(ctx, "forced-replicagc"); err != nil {
+ return err
+ }
+ return nil
+}
+
+func GenerationComparableMigration(ctx context.Context, h *Helper) error {
+ // Hmm now that I look at this again I think we can remove it just like that.
+ // No snapshots are in flight any more that don't respect the generational
+ // rules. Replicas get replicaGC'ed eagerly. (Once round of forced-replicagc
+ // somewhere is enough). Need to think this through more.
+ return nil
+}
+
+func SeparateRaftLogMigration(ctx context.Context, h *Helper) error {
+ // Impl will loop through all repls on the node and migrate them one by one
+ // (get raftMu, check if already copied, if not copy stuff, delete old stuff,
+ // release raftMu).
+ if err := h.EveryNode(ctx, "move-raft-logs"); err != nil {
+ return err
+ }
+ // TODO: sanity check that all are really using separate raft log now?
+ return nil
+}
+
+func ReplicatedLockTableMigration(ctx context.Context, h *Helper) error {
+ // Pretty vanilla Raft migration, see
+ // https://github.com/cockroachdb/cockroach/issues/41720#issuecomment-590817261
+ // also run forced-replicagc.
+ // Should probably bake the forced replicaGC and migrate command into a
+ // helper, there's never a reason not to run forced-replicaGC after a Migrate.
+ return nil
+}
+
+func ReplicasKnowReplicaIDMigration(ctx context.Context, h *Helper) error {
+ // Is a migration necessary here or are we already nuking them at startup anyway?
+ // Oh seems like it, so nothing to do here:
+ // https://github.com/cockroachdb/cockroach/blob/f0e751f00b7ba41f39dd83ddc8238011fa5b9c19/pkg/storage/store.go#L1347-L1350
+ return nil
+}
diff --git a/pkg/clusterversion/migration/prototype.go b/pkg/clusterversion/migration/prototype.go
new file mode 100644
index 000000000000..472366d63430
--- /dev/null
+++ b/pkg/clusterversion/migration/prototype.go
@@ -0,0 +1,347 @@
+// Copyright 2020 The Cockroach Authors.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+// XXX: Skipping jobs for now per Andrew's advice.
+// XXX: Look at client_*_test.go files for client.DB usage/test ideas.
+// XXX: How do cluster migrations start machinery today, on change?
+// ANS: They just poll IsActive, and behave accordingly. No "migrations" happen.
+// We want something like VersionUpgradeHook to actually set up the thing, run
+// hook after every node will observe IsActive == true.
+// XXX: Need to define "node upgrade" RPC subsystem.
+package migration
+
+import (
+ "bytes"
+ "context"
+ "fmt"
+ "github.com/cockroachdb/cockroach/pkg/clusterversion"
+ "github.com/cockroachdb/cockroach/pkg/rpc"
+ "github.com/cockroachdb/cockroach/pkg/server/serverpb"
+ "github.com/cockroachdb/cockroach/pkg/sql"
+ "github.com/cockroachdb/cockroach/pkg/sqlmigrations/leasemanager"
+ "time"
+
+ "github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
+ "github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
+ "github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/errors"
+)
+
+type Migration func(context.Context, *Helper) error
+
+type Helper struct {
+ db *kv.DB
+ executor *sql.InternalExecutor
+ nodeDialer *nodedialer.Dialer
+}
+
+type Orchestrator struct {
+ db *kv.DB
+ executor *sql.InternalExecutor
+ nodeDialer *nodedialer.Dialer
+ leaseManager *leasemanager.LeaseManager
+
+ // XXX: Lots more stuff. Including facility to run SQL migrations
+ // (eventually).
+}
+
+func NewOrchestrator(leaseManager *leasemanager.LeaseManager, executor *sql.InternalExecutor, db *kv.DB, nodeDialer *nodedialer.Dialer) *Orchestrator {
+ return &Orchestrator{
+ db: db,
+ leaseManager: leaseManager,
+ executor: executor,
+ nodeDialer: nodeDialer,
+ }
+}
+
+// XXX: Use blockSize here as well. *adminServer.DecommissionStatus does
+// something similar.
+func (h *Helper) IterRangeDescriptors(
+ f func(...roachpb.RangeDescriptor) error,
+) error {
+ // Paginate through all ranges and invoke f with chunks of size ~blockSize.
+ // call h.Progress between invocations.
+ const pageSize = 5
+ descriptors := make([]roachpb.RangeDescriptor, pageSize)
+ err := h.db.Txn(context.TODO(), func(ctx context.Context, txn *kv.Txn) error {
+ return txn.Iterate(ctx, keys.MetaMin, keys.MetaMax, pageSize,
+ func(rows []kv.KeyValue) error {
+ for i, row := range rows {
+ if err := row.ValueProto(&descriptors[i]); err != nil {
+ return errors.Wrapf(err, "%s: unable to unmarshal range descriptor", row.Key)
+ }
+ }
+
+ if err := f(descriptors...); err != nil {
+ return err
+ }
+
+ time.Sleep(1 * time.Second)
+ return nil
+ })
+ })
+ if err != nil {
+ return err
+ }
+ return nil
+}
+
+func (h *Helper) Retry(f func() error) error {
+ for {
+ err := f()
+ if err != nil {
+ continue
+ }
+ return err
+ }
+}
+
+func (h *Helper) RequiredNodes(ctx context.Context) ([]roachpb.NodeID, error) {
+ kvs, err := h.db.Scan(ctx, keys.NodeLivenessPrefix, keys.NodeLivenessKeyMax, 0)
+ if err != nil {
+ return nil, errors.Wrap(err, "unable to get liveness")
+ }
+
+ var results []roachpb.NodeID
+ for _, kv := range kvs {
+ if kv.Value == nil {
+ return nil, errors.AssertionFailedf("missing liveness record")
+ }
+ var liveness kvserverpb.Liveness
+ if err := kv.Value.GetProto(&liveness); err != nil {
+ return nil, errors.Wrap(err, "invalid liveness record")
+ }
+
+ results = append(results, liveness.NodeID)
+ }
+
+ return results, nil
+}
+
+func (h *Helper) Progress(s string, num, denum int) {
+ // Set progress of the current step to num/denum. Denum can be zero if final
+ // steps not known (example: iterating through ranges)
+ // Free-form message can be attached (PII?)
+ //
+ // XXX: this API is crap. Also, where do we get the denominator from?
+}
+
+func (h *Helper) EveryNode(ctx context.Context, op string, args ...interface{}) error {
+ nodeIDs, err := h.RequiredNodes(ctx)
+ if err != nil {
+ return err
+ }
+ log.Infof(ctx, "xxx: performing %s on every node (nodeids=%s, args=%s)", op, nodeIDs, args)
+
+ for {
+ for _, nodeID := range nodeIDs {
+ // XXX: This can be parallelized. Is errgroup appropriate for prod use?
+ conn, err := h.nodeDialer.Dial(ctx, nodeID, rpc.DefaultClass)
+ if err != nil {
+ return err
+ }
+
+ var req *serverpb.EveryNodeRequest
+ if op == "ack-pending-version" {
+ // XXX: Need equivalent to batch_generated.go. We should change the
+ // signature of this method to accept EveryNodeRequest instead.
+ pendingV := args[0].(roachpb.Version)
+ req = &serverpb.EveryNodeRequest{
+ Request: serverpb.EveryNodeRequestUnion{
+ Value: &serverpb.EveryNodeRequestUnion_AckPendingVersion{
+ AckPendingVersion: &serverpb.AckPendingVersionRequest{
+ Version: &pendingV,
+ },
+ },
+ },
+ }
+ } else {
+ return errors.Newf("unsupported op %s", op)
+ }
+
+ client := serverpb.NewAdminClient(conn)
+ if _, err := client.EveryNode(ctx, req); err != nil {
+ return err
+ }
+ }
+
+ newNodeIDs, err := h.RequiredNodes(ctx)
+ if err != nil {
+ return err
+ }
+
+ if fmt.Sprintf("%v", newNodeIDs) == fmt.Sprintf("%v", nodeIDs) {
+ break
+ }
+ }
+ return nil
+}
+
+type V struct {
+ roachpb.Version
+ Migration // XXX: The hook, to be run post-gossip of cluster setting.
+}
+
+func (o *Orchestrator) RunMigrations(ctx context.Context, to roachpb.Version) error {
+ // XXX: Refresh this lease during execution of LRM.
+ _, err := o.leaseManager.AcquireLease(ctx, keys.LRMLease)
+ if err != nil {
+ return err
+ }
+
+ ml := MakeMigrationLogger(o.executor)
+
+ // XXX: This should called by something higher up that can own the job, and
+ // can set it up, etc. Probably sits on the orchestrator itself, and have a
+ // way to grab leases (through sql/../leasemanager). Use a system table for in
+ // progress state tracking.
+ // XXX: Do we have to get this key from KV? aren't we using the system table
+ // for it all? Can't we simply retrieve it from the store local key? Every
+ // migration pushes the gate out globally, so.
+ var ackedV roachpb.Version
+ if err := o.db.GetProto(ctx, keys.ClusterVersionKey, &ackedV); err != nil {
+ return err
+ }
+
+ if (ackedV == roachpb.Version{}) {
+ ackedV = clusterversion.TestingBinaryVersion
+ }
+
+ log.Infof(ctx, "xxx: existing version=%s", ackedV)
+ if ackedV == to {
+ log.Info(ctx, "xxx: nothing to do here")
+ return nil
+ }
+
+ // XXX: Hard-coded example data. These need to be assembled from a registry
+ // while taking into account ackedV.
+
+ // XXX: What should we do if no version is found in KV? We should write as
+ // part of initial cluster data anyway. For migrating clusters, we'll simply
+ // run all the migrations?
+
+ vs := []V{
+ {
+ Version: to,
+ Migration: func(ctx context.Context, h *Helper) error {
+ txn := h.db.NewTxn(ctx, "xxx: txn")
+ _, err := h.executor.Query(
+ ctx, "xxx: select", txn,
+ `SELECT 1`,
+ )
+ if err != nil {
+ return err
+ }
+
+ time.Sleep(time.Second)
+
+ log.Infof(ctx, "xxx: retrieved value=%s", ackedV)
+ log.Infof(ctx, "xxx: ran named migration for version=%s", to)
+
+ batch := 0
+ // XXX: Shouldn't our intended range look like {keys.MetaMin, keys.TenantTableDataMax}
+ _ = h.IterRangeDescriptors(func(descriptors ...roachpb.RangeDescriptor) error {
+ batch += 1
+ for _, desc := range descriptors {
+ if bytes.Compare(desc.StartKey, keys.LocalMax) < 0 {
+ // XXX: errors out out /Min, System/NodeLiveness
+ // otherwise. Is this what we want?
+ continue
+ }
+ if err := h.db.AdminMigrate(ctx, desc.StartKey, desc.EndKey); err != nil {
+ return err
+ }
+
+ }
+ if err := ml.InsertRecord(ctx, txn, fmt.Sprintf("finished batch %d", batch)); err != nil {
+ return err
+ }
+ return nil
+ })
+
+ if err := ml.InsertRecord(ctx, txn, fmt.Sprintf("completed %s ==> %s", ackedV, to)); err != nil {
+ return err
+ }
+ if err := txn.Commit(ctx); err != nil {
+ return err
+ }
+
+ return nil
+ },
+ },
+ }
+
+ for _, v := range vs {
+ h := &Helper{db: o.db, executor: o.executor, nodeDialer: o.nodeDialer}
+ // h.Progress should basically call j.Status(v.String()+": "+s, num, denum)
+
+ // Persist the beginning of this migration on all nodes. Basically they
+ // will persist the version, then move the cluster setting forward, then
+ // return.
+ //
+ // Should they store whether migration is ongoing? No.
+ // - Argument for yes: We might have functionality that is only safe
+ // when all nodes in the cluster have "done their part" (XXX: What does
+ // this mean) to migrate into it, and which we don't want to delay for
+ // one release.
+ // - Argument for no: We could just gate that functionality's activation
+ // on a later unstable version, and get the right behavior.
+ if err := h.EveryNode(ctx, "ack-pending-version", v.Version); err != nil {
+ return err
+ }
+ if err := v.Migration(ctx, h); err != nil {
+ return err
+ }
+ }
+
+ // XXX: Should this be a CPut instead?
+ if err := o.db.Put(ctx, keys.ClusterVersionKey, &to); err != nil {
+ return err
+ }
+
+ return nil
+}
+
+type MigrationLogger struct {
+ *sql.InternalExecutor
+}
+
+func MakeMigrationLogger(ie *sql.InternalExecutor) MigrationLogger {
+ return MigrationLogger{InternalExecutor: ie}
+}
+
+// InsertEventRecord inserts a single event into the event log as part of the
+// provided transaction.
+func (m MigrationLogger) InsertRecord(
+ ctx context.Context,
+ txn *kv.Txn,
+ arg interface{},
+) error {
+ // Record event record insertion in local log output.
+ txn.AddCommitTrigger(func(ctx context.Context) {
+ log.Infof(ctx, "xxx: inserted record %v", arg)
+ })
+
+ const insertMigrationTableStmt = `
+INSERT INTO system.lrm ("metadata") VALUES ($1)
+`
+ args := []interface{}{arg}
+ rows, err := m.Exec(ctx, "log-event", txn, insertMigrationTableStmt, args...)
+ if err != nil {
+ return err
+ }
+ if rows != 1 {
+ return errors.Errorf("%d rows affected by log insertion; expected exactly one row affected.", rows)
+ }
+ return nil
+}
diff --git a/pkg/clusterversion/setting.go b/pkg/clusterversion/setting.go
index f3714cbc4cbf..65f313a93c73 100644
--- a/pkg/clusterversion/setting.go
+++ b/pkg/clusterversion/setting.go
@@ -12,7 +12,6 @@ package clusterversion
import (
"context"
-
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -56,6 +55,11 @@ var version = registerClusterVersionSetting()
// clusterVersionSetting is the implementation of the 'version' setting. Like all
// setting structs, it is immutable, as Version is a global; all the state is
// maintained in a Handle instance.
+// XXX: We'll need another view into `version` that _also_ looks like cluster
+// setting to users, but underneath the hood uses a different machinery for
+// storage/mutation of data.
+// XXX: Who currently holds a reference to the mutable handle on this guy? It's
+// the setting that's stored in the registry.
type clusterVersionSetting struct {
settings.StateMachineSetting
}
@@ -248,6 +252,7 @@ func (cv *clusterVersionSetting) ValidateGossipUpdate(
if err := protoutil.Unmarshal(rawProto, &ver); err != nil {
return err
}
+ log.Infof(ctx, "xxx: validating gossip update for cluster version setting, v=%s", ver.String())
return cv.validateSupportedVersionInner(ctx, ver.Version, sv)
}
@@ -281,11 +286,12 @@ func (cv *clusterVersionSetting) validateSupportedVersionInner(
if vh.BinaryMinSupportedVersion() == (roachpb.Version{}) {
panic("BinaryMinSupportedVersion not set")
}
- if vh.BinaryVersion().Less(ver) {
- // TODO(tschottdorf): also ask gossip about other nodes.
- return errors.Errorf("cannot upgrade to %s: node running %s",
- ver, vh.BinaryVersion())
- }
+ // XXX: For testing purposes, letting versions ratchet up endlessly.
+ //if vh.BinaryVersion().Less(ver) {
+ // // TODO(tschottdorf): also ask gossip about other nodes.
+ // return errors.Errorf("cannot upgrade to %s: node running %s",
+ // ver, vh.BinaryVersion())
+ //}
if ver.Less(vh.BinaryMinSupportedVersion()) {
return errors.Errorf("node at %s cannot run %s (minimum version is %s)",
vh.BinaryVersion(), ver, vh.BinaryMinSupportedVersion())
diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go
index 528fc725217d..5eb51a8f7415 100644
--- a/pkg/keys/constants.go
+++ b/pkg/keys/constants.go
@@ -178,6 +178,9 @@ var (
// localStoreNodeTombstoneSuffix stores key value pairs that map
// nodeIDs to time of removal from cluster.
localStoreNodeTombstoneSuffix = []byte("ntmb")
+ // localStoreTargetVersionSuffix stores the target version.
+ // XXX: DOC:
+ localStoreTargetVersionSuffix = []byte("tver")
// localStoreLastUpSuffix stores the last timestamp that a store's node
// acknowledged that it was still running. This value will be regularly
// refreshed on all stores for a running node; the intention of this value
@@ -236,6 +239,8 @@ var (
// BootstrapVersion is the key at which clusters bootstrapped with a version
// > 1.0 persist the version at which they were bootstrapped.
BootstrapVersionKey = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("bootstrap-version")))
+ // ClusterVersionKey XXX: DOC:
+ ClusterVersionKey = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("cluster-version")))
//
// descIDGenerator is the global descriptor ID generator sequence used for
// table and namespace IDs for the system tenant. All other tenants use a
@@ -258,6 +263,8 @@ var (
// MigrationLease is the key that nodes must take a lease on in order to run
// system migrations on the cluster.
MigrationLease = roachpb.Key(makeKey(MigrationPrefix, roachpb.RKey("lease")))
+ // XXX
+ LRMLease = roachpb.Key(makeKey(MigrationPrefix, roachpb.RKey("lrm-lease")))
//
// TimeseriesPrefix is the key prefix for all timeseries data.
TimeseriesPrefix = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("tsd")))
@@ -383,6 +390,7 @@ const (
ScheduledJobsTableID = 37
TenantsRangesID = 38 // pseudo
SqllivenessID = 39
+ LRMID = 40 // XXX:
// CommentType is type for system.comments
DatabaseCommentType = 0
diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go
index 27b4efbc66a1..7afce6037fee 100644
--- a/pkg/keys/keys.go
+++ b/pkg/keys/keys.go
@@ -81,6 +81,11 @@ func StoreNodeTombstoneKey(nodeID roachpb.NodeID) roachpb.Key {
return MakeStoreKey(localStoreNodeTombstoneSuffix, encoding.EncodeUint32Ascending(nil, uint32(nodeID)))
}
+// StoreTargetVersionKey XXX: DOC:
+func StoreTargetVersionKey() roachpb.Key {
+ return MakeStoreKey(localStoreTargetVersionSuffix, nil)
+}
+
// DecodeNodeTombstoneKey returns the NodeID for the node tombstone.
func DecodeNodeTombstoneKey(key roachpb.Key) (roachpb.NodeID, error) {
suffix, detail, err := DecodeStoreKey(key)
diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go
index 098d9336d619..644fab16b9c6 100644
--- a/pkg/kv/batch.go
+++ b/pkg/kv/batch.go
@@ -244,6 +244,7 @@ func (b *Batch) fillResults(ctx context.Context) {
case *roachpb.EndTxnRequest:
case *roachpb.AdminMergeRequest:
case *roachpb.AdminSplitRequest:
+ case *roachpb.MigrateRequest:
case *roachpb.AdminUnsplitRequest:
case *roachpb.AdminTransferLeaseRequest:
case *roachpb.AdminChangeReplicasRequest:
@@ -629,6 +630,28 @@ func (b *Batch) adminSplit(splitKeyIn interface{}, expirationTime hlc.Timestamp)
b.initResult(1, 0, notRaw, nil)
}
+// XXX:
+func (b *Batch) adminMigrate(begin, end interface{}) {
+ beginKey, err := marshalKey(begin)
+ if err != nil {
+ b.initResult(0, 0, notRaw, err)
+ return
+ }
+ endKey, err := marshalKey(end)
+ if err != nil {
+ b.initResult(0, 0, notRaw, err)
+ return
+ }
+ req := &roachpb.MigrateRequest{
+ RequestHeader: roachpb.RequestHeader{
+ Key: beginKey,
+ EndKey: endKey,
+ },
+ }
+ b.appendReqs(req)
+ b.initResult(1, 0, notRaw, nil)
+}
+
func (b *Batch) adminUnsplit(splitKeyIn interface{}) {
splitKey, err := marshalKey(splitKeyIn)
if err != nil {
diff --git a/pkg/kv/db.go b/pkg/kv/db.go
index 47109435356f..207e06526bed 100644
--- a/pkg/kv/db.go
+++ b/pkg/kv/db.go
@@ -532,6 +532,13 @@ func (db *DB) AdminSplit(
return getOneErr(db.Run(ctx, b), b)
}
+// XXX: Migrates the range containing the key. Needs to take a target version.
+func (db *DB) AdminMigrate(ctx context.Context, begin, end interface{}) error {
+ b := &Batch{}
+ b.adminMigrate(begin, end)
+ return getOneErr(db.Run(ctx, b), b)
+}
+
// SplitAndScatter is a helper that wraps AdminSplit + AdminScatter.
func (db *DB) SplitAndScatter(
ctx context.Context, key roachpb.Key, expirationTime hlc.Timestamp,
diff --git a/pkg/kv/kvserver/batcheval/cmd_migrate.go b/pkg/kv/kvserver/batcheval/cmd_migrate.go
index b0b68f9c5053..c598b0d7ce4b 100644
--- a/pkg/kv/kvserver/batcheval/cmd_migrate.go
+++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go
@@ -12,6 +12,7 @@ package batcheval
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
@@ -31,7 +32,12 @@ func declareKeysMigrate(
// Migrate ensures that the range proactively carries out any outstanding
// below-Raft migrations.
func Migrate(
- _ context.Context, _ storage.ReadWriter, _ CommandArgs, _ roachpb.Response,
+ ctx context.Context, _ storage.ReadWriter, cArgs CommandArgs, _ roachpb.Response,
) (result.Result, error) {
+ args := cArgs.Args.(*roachpb.MigrateRequest)
+ h := cArgs.Header
+
+ log.Infof(ctx, "xxx: evaluating Migrate command for r%d [%s, %s)", h.RangeID, args.Key, args.EndKey)
+
return result.Result{}, nil
}
diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go
index e872afd58c41..79716d0a0b7e 100644
--- a/pkg/roachpb/api.pb.go
+++ b/pkg/roachpb/api.pb.go
@@ -72,7 +72,7 @@ func (x ReadConsistencyType) String() string {
return proto.EnumName(ReadConsistencyType_name, int32(x))
}
func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{0}
}
// ScanFormat is an enumeration of the available response formats for MVCCScan
@@ -100,7 +100,7 @@ func (x ScanFormat) String() string {
return proto.EnumName(ScanFormat_name, int32(x))
}
func (ScanFormat) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{1}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{1}
}
type ChecksumMode int32
@@ -147,7 +147,7 @@ func (x ChecksumMode) String() string {
return proto.EnumName(ChecksumMode_name, int32(x))
}
func (ChecksumMode) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{2}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{2}
}
// PushTxnType determines what action to take when pushing a transaction.
@@ -178,7 +178,7 @@ func (x PushTxnType) String() string {
return proto.EnumName(PushTxnType_name, int32(x))
}
func (PushTxnType) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{3}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{3}
}
type ExternalStorageProvider int32
@@ -219,7 +219,7 @@ func (x ExternalStorageProvider) String() string {
return proto.EnumName(ExternalStorageProvider_name, int32(x))
}
func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{4}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{4}
}
type MVCCFilter int32
@@ -242,7 +242,7 @@ func (x MVCCFilter) String() string {
return proto.EnumName(MVCCFilter_name, int32(x))
}
func (MVCCFilter) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{5}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{5}
}
type ResponseHeader_ResumeReason int32
@@ -268,7 +268,7 @@ func (x ResponseHeader_ResumeReason) String() string {
return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x))
}
func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{1, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{1, 0}
}
type CheckConsistencyResponse_Status int32
@@ -310,7 +310,7 @@ func (x CheckConsistencyResponse_Status) String() string {
return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x))
}
func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{25, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{25, 0}
}
// RequestHeader is supplied with every storage node request.
@@ -331,7 +331,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} }
func (m *RequestHeader) String() string { return proto.CompactTextString(m) }
func (*RequestHeader) ProtoMessage() {}
func (*RequestHeader) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{0}
}
func (m *RequestHeader) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -402,7 +402,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} }
func (m *ResponseHeader) String() string { return proto.CompactTextString(m) }
func (*ResponseHeader) ProtoMessage() {}
func (*ResponseHeader) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{1}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{1}
}
func (m *ResponseHeader) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -436,7 +436,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} }
func (m *GetRequest) String() string { return proto.CompactTextString(m) }
func (*GetRequest) ProtoMessage() {}
func (*GetRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{2}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{2}
}
func (m *GetRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -479,7 +479,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} }
func (m *GetResponse) String() string { return proto.CompactTextString(m) }
func (*GetResponse) ProtoMessage() {}
func (*GetResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{3}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{3}
}
func (m *GetResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -522,7 +522,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} }
func (m *PutRequest) String() string { return proto.CompactTextString(m) }
func (*PutRequest) ProtoMessage() {}
func (*PutRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{4}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{4}
}
func (m *PutRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -556,7 +556,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} }
func (m *PutResponse) String() string { return proto.CompactTextString(m) }
func (*PutResponse) ProtoMessage() {}
func (*PutResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{5}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{5}
}
func (m *PutResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -638,7 +638,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} }
func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) }
func (*ConditionalPutRequest) ProtoMessage() {}
func (*ConditionalPutRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{6}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{6}
}
func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -673,7 +673,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{}
func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) }
func (*ConditionalPutResponse) ProtoMessage() {}
func (*ConditionalPutResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{7}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{7}
}
func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -719,7 +719,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} }
func (m *InitPutRequest) String() string { return proto.CompactTextString(m) }
func (*InitPutRequest) ProtoMessage() {}
func (*InitPutRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{8}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{8}
}
func (m *InitPutRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -753,7 +753,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} }
func (m *InitPutResponse) String() string { return proto.CompactTextString(m) }
func (*InitPutResponse) ProtoMessage() {}
func (*InitPutResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{9}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{9}
}
func (m *InitPutResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -793,7 +793,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} }
func (m *IncrementRequest) String() string { return proto.CompactTextString(m) }
func (*IncrementRequest) ProtoMessage() {}
func (*IncrementRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{10}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{10}
}
func (m *IncrementRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -830,7 +830,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} }
func (m *IncrementResponse) String() string { return proto.CompactTextString(m) }
func (*IncrementResponse) ProtoMessage() {}
func (*IncrementResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{11}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{11}
}
func (m *IncrementResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -864,7 +864,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} }
func (m *DeleteRequest) String() string { return proto.CompactTextString(m) }
func (*DeleteRequest) ProtoMessage() {}
func (*DeleteRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{12}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{12}
}
func (m *DeleteRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -898,7 +898,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} }
func (m *DeleteResponse) String() string { return proto.CompactTextString(m) }
func (*DeleteResponse) ProtoMessage() {}
func (*DeleteResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{13}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{13}
}
func (m *DeleteResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -950,7 +950,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} }
func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) }
func (*DeleteRangeRequest) ProtoMessage() {}
func (*DeleteRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{14}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{14}
}
func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -987,7 +987,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} }
func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) }
func (*DeleteRangeResponse) ProtoMessage() {}
func (*DeleteRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{15}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{15}
}
func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1042,7 +1042,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} }
func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) }
func (*ClearRangeRequest) ProtoMessage() {}
func (*ClearRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{16}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{16}
}
func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1076,7 +1076,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} }
func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) }
func (*ClearRangeResponse) ProtoMessage() {}
func (*ClearRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{17}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{17}
}
func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1117,7 +1117,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} }
func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) }
func (*RevertRangeRequest) ProtoMessage() {}
func (*RevertRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{18}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{18}
}
func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1151,7 +1151,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} }
func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) }
func (*RevertRangeResponse) ProtoMessage() {}
func (*RevertRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{19}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{19}
}
func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1202,7 +1202,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} }
func (m *ScanRequest) String() string { return proto.CompactTextString(m) }
func (*ScanRequest) ProtoMessage() {}
func (*ScanRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{20}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{20}
}
func (m *ScanRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1254,7 +1254,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} }
func (m *ScanResponse) String() string { return proto.CompactTextString(m) }
func (*ScanResponse) ProtoMessage() {}
func (*ScanResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{21}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{21}
}
func (m *ScanResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1305,7 +1305,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} }
func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) }
func (*ReverseScanRequest) ProtoMessage() {}
func (*ReverseScanRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{22}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{22}
}
func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1357,7 +1357,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} }
func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) }
func (*ReverseScanResponse) ProtoMessage() {}
func (*ReverseScanResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{23}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{23}
}
func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1410,7 +1410,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest
func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) }
func (*CheckConsistencyRequest) ProtoMessage() {}
func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{24}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{24}
}
func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1447,7 +1447,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon
func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) }
func (*CheckConsistencyResponse) ProtoMessage() {}
func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{25}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{25}
}
func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1491,7 +1491,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc
func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) }
func (*CheckConsistencyResponse_Result) ProtoMessage() {}
func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{25, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{25, 0}
}
func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1539,7 +1539,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} }
func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) }
func (*RecomputeStatsRequest) ProtoMessage() {}
func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{26}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{26}
}
func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1575,7 +1575,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{}
func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) }
func (*RecomputeStatsResponse) ProtoMessage() {}
func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{27}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{27}
}
func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1686,7 +1686,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} }
func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) }
func (*EndTxnRequest) ProtoMessage() {}
func (*EndTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{28}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{28}
}
func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1732,7 +1732,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} }
func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) }
func (*EndTxnResponse) ProtoMessage() {}
func (*EndTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{29}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{29}
}
func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1793,7 +1793,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} }
func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) }
func (*AdminSplitRequest) ProtoMessage() {}
func (*AdminSplitRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{30}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{30}
}
func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1828,7 +1828,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} }
func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) }
func (*AdminSplitResponse) ProtoMessage() {}
func (*AdminSplitResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{31}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{31}
}
func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1867,7 +1867,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} }
func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) }
func (*AdminUnsplitRequest) ProtoMessage() {}
func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{32}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{32}
}
func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1902,7 +1902,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} }
func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) }
func (*AdminUnsplitResponse) ProtoMessage() {}
func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{33}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{33}
}
func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1945,7 +1945,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} }
func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) }
func (*AdminMergeRequest) ProtoMessage() {}
func (*AdminMergeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{34}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{34}
}
func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1980,7 +1980,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} }
func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) }
func (*AdminMergeResponse) ProtoMessage() {}
func (*AdminMergeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{35}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{35}
}
func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2018,7 +2018,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq
func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) }
func (*AdminTransferLeaseRequest) ProtoMessage() {}
func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{36}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{36}
}
func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2051,7 +2051,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe
func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) }
func (*AdminTransferLeaseResponse) ProtoMessage() {}
func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{37}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{37}
}
func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2086,7 +2086,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} }
func (m *ReplicationChange) String() string { return proto.CompactTextString(m) }
func (*ReplicationChange) ProtoMessage() {}
func (*ReplicationChange) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{38}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{38}
}
func (m *ReplicationChange) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2144,7 +2144,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR
func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) }
func (*AdminChangeReplicasRequest) ProtoMessage() {}
func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{39}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{39}
}
func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2179,7 +2179,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas
func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) }
func (*AdminChangeReplicasResponse) ProtoMessage() {}
func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{40}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{40}
}
func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2216,7 +2216,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq
func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) }
func (*AdminRelocateRangeRequest) ProtoMessage() {}
func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{41}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{41}
}
func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2249,7 +2249,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe
func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) }
func (*AdminRelocateRangeResponse) ProtoMessage() {}
func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{42}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{42}
}
func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2288,7 +2288,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} }
func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) }
func (*HeartbeatTxnRequest) ProtoMessage() {}
func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{43}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{43}
}
func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2325,7 +2325,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} }
func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) }
func (*HeartbeatTxnResponse) ProtoMessage() {}
func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{44}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{44}
}
func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2363,7 +2363,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} }
func (m *GCRequest) String() string { return proto.CompactTextString(m) }
func (*GCRequest) ProtoMessage() {}
func (*GCRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{45}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{45}
}
func (m *GCRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2397,7 +2397,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} }
func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) }
func (*GCRequest_GCKey) ProtoMessage() {}
func (*GCRequest_GCKey) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{45, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{45, 0}
}
func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2431,7 +2431,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} }
func (m *GCResponse) String() string { return proto.CompactTextString(m) }
func (*GCResponse) ProtoMessage() {}
func (*GCResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{46}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{46}
}
func (m *GCResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2500,7 +2500,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} }
func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) }
func (*PushTxnRequest) ProtoMessage() {}
func (*PushTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{47}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{47}
}
func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2543,7 +2543,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} }
func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) }
func (*PushTxnResponse) ProtoMessage() {}
func (*PushTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{48}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{48}
}
func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2590,7 +2590,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} }
func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) }
func (*RecoverTxnRequest) ProtoMessage() {}
func (*RecoverTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{49}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{49}
}
func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2626,7 +2626,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} }
func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) }
func (*RecoverTxnResponse) ProtoMessage() {}
func (*RecoverTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{50}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{50}
}
func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2670,7 +2670,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} }
func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) }
func (*QueryTxnRequest) ProtoMessage() {}
func (*QueryTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{51}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{51}
}
func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2714,7 +2714,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} }
func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) }
func (*QueryTxnResponse) ProtoMessage() {}
func (*QueryTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{52}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{52}
}
func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2774,7 +2774,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} }
func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) }
func (*QueryIntentRequest) ProtoMessage() {}
func (*QueryIntentRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{53}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{53}
}
func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2810,7 +2810,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} }
func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) }
func (*QueryIntentResponse) ProtoMessage() {}
func (*QueryIntentResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{54}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{54}
}
func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2857,7 +2857,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} }
func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) }
func (*ResolveIntentRequest) ProtoMessage() {}
func (*ResolveIntentRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{55}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{55}
}
func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2892,7 +2892,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} }
func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) }
func (*ResolveIntentResponse) ProtoMessage() {}
func (*ResolveIntentResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{56}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{56}
}
func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2943,7 +2943,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq
func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) }
func (*ResolveIntentRangeRequest) ProtoMessage() {}
func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{57}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{57}
}
func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2978,7 +2978,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe
func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) }
func (*ResolveIntentRangeResponse) ProtoMessage() {}
func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{58}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{58}
}
func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3015,7 +3015,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} }
func (m *MergeRequest) String() string { return proto.CompactTextString(m) }
func (*MergeRequest) ProtoMessage() {}
func (*MergeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{59}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{59}
}
func (m *MergeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3049,7 +3049,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} }
func (m *MergeResponse) String() string { return proto.CompactTextString(m) }
func (*MergeResponse) ProtoMessage() {}
func (*MergeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{60}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{60}
}
func (m *MergeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3094,7 +3094,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} }
func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) }
func (*TruncateLogRequest) ProtoMessage() {}
func (*TruncateLogRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{61}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{61}
}
func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3128,7 +3128,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} }
func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) }
func (*TruncateLogResponse) ProtoMessage() {}
func (*TruncateLogResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{62}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{62}
}
func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3172,7 +3172,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} }
func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) }
func (*RequestLeaseRequest) ProtoMessage() {}
func (*RequestLeaseRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{63}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{63}
}
func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3221,7 +3221,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} }
func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) }
func (*TransferLeaseRequest) ProtoMessage() {}
func (*TransferLeaseRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{64}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{64}
}
func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3258,7 +3258,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} }
func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) }
func (*LeaseInfoRequest) ProtoMessage() {}
func (*LeaseInfoRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{65}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{65}
}
func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3295,7 +3295,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} }
func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) }
func (*LeaseInfoResponse) ProtoMessage() {}
func (*LeaseInfoResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{66}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{66}
}
func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3330,7 +3330,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} }
func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) }
func (*RequestLeaseResponse) ProtoMessage() {}
func (*RequestLeaseResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{67}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{67}
}
func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3385,7 +3385,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{}
func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) }
func (*ComputeChecksumRequest) ProtoMessage() {}
func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{68}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{68}
}
func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3422,7 +3422,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse
func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) }
func (*ComputeChecksumResponse) ProtoMessage() {}
func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{69}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{69}
}
func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3462,7 +3462,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} }
func (m *ExternalStorage) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage) ProtoMessage() {}
func (*ExternalStorage) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{70}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{70}
}
func (m *ExternalStorage) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3496,7 +3496,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L
func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_LocalFilePath) ProtoMessage() {}
func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{70, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{70, 0}
}
func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3529,7 +3529,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} }
func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_Http) ProtoMessage() {}
func (*ExternalStorage_Http) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{70, 1}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{70, 1}
}
func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3569,7 +3569,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} }
func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_S3) ProtoMessage() {}
func (*ExternalStorage_S3) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{70, 2}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{70, 2}
}
func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3608,7 +3608,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} }
func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_GCS) ProtoMessage() {}
func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{70, 3}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{70, 3}
}
func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3644,7 +3644,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} }
func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_Azure) ProtoMessage() {}
func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{70, 4}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{70, 4}
}
func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3683,7 +3683,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo
func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_Workload) ProtoMessage() {}
func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{70, 5}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{70, 5}
}
func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3723,7 +3723,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT
func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_FileTable) ProtoMessage() {}
func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{70, 6}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{70, 6}
}
func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3763,7 +3763,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} }
func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) }
func (*WriteBatchRequest) ProtoMessage() {}
func (*WriteBatchRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{71}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{71}
}
func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3797,7 +3797,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} }
func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) }
func (*WriteBatchResponse) ProtoMessage() {}
func (*WriteBatchResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{72}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{72}
}
func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3833,7 +3833,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} }
func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) }
func (*FileEncryptionOptions) ProtoMessage() {}
func (*FileEncryptionOptions) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{73}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{73}
}
func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3903,7 +3903,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} }
func (m *ExportRequest) String() string { return proto.CompactTextString(m) }
func (*ExportRequest) ProtoMessage() {}
func (*ExportRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{74}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{74}
}
func (m *ExportRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3955,7 +3955,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} }
func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) }
func (*BulkOpSummary) ProtoMessage() {}
func (*BulkOpSummary) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{75}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{75}
}
func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3991,7 +3991,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} }
func (m *ExportResponse) String() string { return proto.CompactTextString(m) }
func (*ExportResponse) ProtoMessage() {}
func (*ExportResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{76}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{76}
}
func (m *ExportResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4031,7 +4031,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} }
func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) }
func (*ExportResponse_File) ProtoMessage() {}
func (*ExportResponse_File) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{76, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{76, 0}
}
func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4082,7 +4082,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} }
func (m *ImportRequest) String() string { return proto.CompactTextString(m) }
func (*ImportRequest) ProtoMessage() {}
func (*ImportRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{77}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{77}
}
func (m *ImportRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4117,7 +4117,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} }
func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) }
func (*ImportRequest_File) ProtoMessage() {}
func (*ImportRequest_File) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{77, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{77, 0}
}
func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4153,7 +4153,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek
func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) }
func (*ImportRequest_TableRekey) ProtoMessage() {}
func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{77, 1}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{77, 1}
}
func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4188,7 +4188,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} }
func (m *ImportResponse) String() string { return proto.CompactTextString(m) }
func (*ImportResponse) ProtoMessage() {}
func (*ImportResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{78}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{78}
}
func (m *ImportResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4226,7 +4226,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} }
func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) }
func (*AdminScatterRequest) ProtoMessage() {}
func (*AdminScatterRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{79}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{79}
}
func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4263,7 +4263,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} }
func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) }
func (*AdminScatterResponse) ProtoMessage() {}
func (*AdminScatterResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{80}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{80}
}
func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4296,7 +4296,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse
func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) }
func (*AdminScatterResponse_Range) ProtoMessage() {}
func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{80, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{80, 0}
}
func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4325,6 +4325,11 @@ var xxx_messageInfo_AdminScatterResponse_Range proto.InternalMessageInfo
// any legacy modes that they are currently in (as defined by the delta between
// the current version and the TargetVersion). When this command returns, the
// ranges are ready to run with the TargetVersion.
+//
+// XXX: Should this be an admin request instead? There's no good way for us to
+// get the applied index of admin requests which don't go through the usual
+// eval+propose codepaths, and we do need to know the MLAI for the request to
+// wait for it to apply before returning.
type MigrateRequest struct {
RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"`
// The new version that will become active next.
@@ -4335,7 +4340,7 @@ func (m *MigrateRequest) Reset() { *m = MigrateRequest{} }
func (m *MigrateRequest) String() string { return proto.CompactTextString(m) }
func (*MigrateRequest) ProtoMessage() {}
func (*MigrateRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{81}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{81}
}
func (m *MigrateRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4369,7 +4374,7 @@ func (m *MigrateResponse) Reset() { *m = MigrateResponse{} }
func (m *MigrateResponse) String() string { return proto.CompactTextString(m) }
func (*MigrateResponse) ProtoMessage() {}
func (*MigrateResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{82}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{82}
}
func (m *MigrateResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4414,7 +4419,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif
func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) }
func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {}
func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{83}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{83}
}
func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4452,7 +4457,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri
func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) }
func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {}
func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{84}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{84}
}
func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4505,7 +4510,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} }
func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) }
func (*AddSSTableRequest) ProtoMessage() {}
func (*AddSSTableRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{85}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{85}
}
func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4539,7 +4544,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} }
func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) }
func (*AddSSTableResponse) ProtoMessage() {}
func (*AddSSTableResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{86}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{86}
}
func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4583,7 +4588,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} }
func (m *RefreshRequest) String() string { return proto.CompactTextString(m) }
func (*RefreshRequest) ProtoMessage() {}
func (*RefreshRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{87}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{87}
}
func (m *RefreshRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4617,7 +4622,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} }
func (m *RefreshResponse) String() string { return proto.CompactTextString(m) }
func (*RefreshResponse) ProtoMessage() {}
func (*RefreshResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{88}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{88}
}
func (m *RefreshResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4656,7 +4661,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} }
func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) }
func (*RefreshRangeRequest) ProtoMessage() {}
func (*RefreshRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{89}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{89}
}
func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4690,7 +4695,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} }
func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) }
func (*RefreshRangeResponse) ProtoMessage() {}
func (*RefreshRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{90}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{90}
}
func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4739,7 +4744,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} }
func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) }
func (*SubsumeRequest) ProtoMessage() {}
func (*SubsumeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{91}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{91}
}
func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4788,7 +4793,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} }
func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) }
func (*SubsumeResponse) ProtoMessage() {}
func (*SubsumeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{92}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{92}
}
func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4823,7 +4828,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} }
func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) }
func (*RangeStatsRequest) ProtoMessage() {}
func (*RangeStatsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{93}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{93}
}
func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4865,7 +4870,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} }
func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) }
func (*RangeStatsResponse) ProtoMessage() {}
func (*RangeStatsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{94}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{94}
}
func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4949,7 +4954,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} }
func (m *RequestUnion) String() string { return proto.CompactTextString(m) }
func (*RequestUnion) ProtoMessage() {}
func (*RequestUnion) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{95}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{95}
}
func (m *RequestUnion) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -6430,7 +6435,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} }
func (m *ResponseUnion) String() string { return proto.CompactTextString(m) }
func (*ResponseUnion) ProtoMessage() {}
func (*ResponseUnion) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{96}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{96}
}
func (m *ResponseUnion) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -7995,7 +8000,7 @@ func (m *Header) Reset() { *m = Header{} }
func (m *Header) String() string { return proto.CompactTextString(m) }
func (*Header) ProtoMessage() {}
func (*Header) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{97}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{97}
}
func (m *Header) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8033,7 +8038,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} }
func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) }
func (*ClientRangeInfo) ProtoMessage() {}
func (*ClientRangeInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{98}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{98}
}
func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8069,7 +8074,7 @@ type BatchRequest struct {
func (m *BatchRequest) Reset() { *m = BatchRequest{} }
func (*BatchRequest) ProtoMessage() {}
func (*BatchRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{99}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{99}
}
func (m *BatchRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8106,7 +8111,7 @@ type BatchResponse struct {
func (m *BatchResponse) Reset() { *m = BatchResponse{} }
func (*BatchResponse) ProtoMessage() {}
func (*BatchResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{100}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{100}
}
func (m *BatchResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8180,7 +8185,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} }
func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) }
func (*BatchResponse_Header) ProtoMessage() {}
func (*BatchResponse_Header) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{100, 0}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{100, 0}
}
func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8218,7 +8223,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} }
func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) }
func (*RangeLookupRequest) ProtoMessage() {}
func (*RangeLookupRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{101}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{101}
}
func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8256,7 +8261,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} }
func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) }
func (*RangeLookupResponse) ProtoMessage() {}
func (*RangeLookupResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{102}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{102}
}
func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8295,7 +8300,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} }
func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) }
func (*RangeFeedRequest) ProtoMessage() {}
func (*RangeFeedRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{103}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{103}
}
func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8336,7 +8341,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} }
func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) }
func (*RangeFeedValue) ProtoMessage() {}
func (*RangeFeedValue) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{104}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{104}
}
func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8377,7 +8382,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} }
func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) }
func (*RangeFeedCheckpoint) ProtoMessage() {}
func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{105}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{105}
}
func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8414,7 +8419,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} }
func (m *RangeFeedError) String() string { return proto.CompactTextString(m) }
func (*RangeFeedError) ProtoMessage() {}
func (*RangeFeedError) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{106}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{106}
}
func (m *RangeFeedError) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8451,7 +8456,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} }
func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) }
func (*RangeFeedEvent) ProtoMessage() {}
func (*RangeFeedEvent) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{107}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{107}
}
func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8492,7 +8497,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq
func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) }
func (*GossipSubscriptionRequest) ProtoMessage() {}
func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{108}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{108}
}
func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8532,7 +8537,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent
func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) }
func (*GossipSubscriptionEvent) ProtoMessage() {}
func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{109}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{109}
}
func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8568,7 +8573,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} }
func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) }
func (*JoinNodeRequest) ProtoMessage() {}
func (*JoinNodeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{110}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{110}
}
func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8607,7 +8612,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} }
func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) }
func (*JoinNodeResponse) ProtoMessage() {}
func (*JoinNodeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_198bc6c44786058a, []int{111}
+ return fileDescriptor_api_8fd5dd2504ba807c, []int{111}
}
func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -40340,9 +40345,9 @@ var (
ErrIntOverflowApi = fmt.Errorf("proto: integer overflow")
)
-func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_198bc6c44786058a) }
+func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_8fd5dd2504ba807c) }
-var fileDescriptor_api_198bc6c44786058a = []byte{
+var fileDescriptor_api_8fd5dd2504ba807c = []byte{
// 8073 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x6b, 0x6c, 0x23, 0x59,
0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0xf0, 0xa1, 0xd2, 0x95, 0xba, 0x9b, 0xa3, 0x99, 0x69, 0xa9,
diff --git a/pkg/rpc/heartbeat.go b/pkg/rpc/heartbeat.go
index f0540bc31fb3..6bc8b4d850c2 100644
--- a/pkg/rpc/heartbeat.go
+++ b/pkg/rpc/heartbeat.go
@@ -91,8 +91,10 @@ func checkVersion(ctx context.Context, st *cluster.Settings, peerVersion roachpb
"cluster requires at least version %s, but peer did not provide a version", activeVersion)
}
if peerVersion.Less(activeVersion.Version) {
- return errors.Errorf(
- "cluster requires at least version %s, but peer has version %s", activeVersion, peerVersion)
+ // XXX: Hacking this in to let us ratchet things willy nilly.
+ //return errors.Errorf(
+ // "cluster requires at least version %s, but peer has version %s", activeVersion, peerVersion)
+ return nil
}
return nil
}
diff --git a/pkg/server/admin.go b/pkg/server/admin.go
index 8a7f2716dc99..c3b5d1f9a7e6 100644
--- a/pkg/server/admin.go
+++ b/pkg/server/admin.go
@@ -1747,6 +1747,43 @@ func (s *adminServer) Decommission(
return s.DecommissionStatus(ctx, &serverpb.DecommissionStatusRequest{NodeIDs: nodeIDs})
}
+// EveryNode XXX: DOC:
+func (s *adminServer) EveryNode(
+ ctx context.Context, req *serverpb.EveryNodeRequest,
+) (*serverpb.EveryNodeResponse, error) {
+ {
+ got, err := s.server.GetPendingVersion(ctx)
+ if err != nil {
+ return nil, err
+ }
+ log.Infof(ctx, "xxx: found existing-version=%s", got)
+ }
+ {
+ // XXX: We need something similar to batch_generated.go. Even for the
+ // response.
+ ackReq := req.Request.GetAckPendingVersion()
+ if err := s.server.PersistPendingVersion(ctx, ackReq.Version); err != nil {
+ return nil, err
+ }
+
+ // XXX: TODO: Bump the local version gate here. On 21.1 nodes we'll no
+ // longer use gossip to propagate version bumps. We'll still have
+ // disseminate it, but we won't listen to it or anything.
+ _ = s.server.ClusterSettings().MutableVersionRef.Set(ctx, ackReq.Version)
+
+ ackResp := &serverpb.AckPendingVersionResponse{}
+ ackRespU := &serverpb.EveryNodeResponseUnion_AckPendingVersion{
+ AckPendingVersion: ackResp,
+ }
+ resp := &serverpb.EveryNodeResponse{
+ Response: serverpb.EveryNodeResponseUnion{
+ Value: ackRespU,
+ },
+ }
+ return resp, nil
+ }
+}
+
// DataDistribution returns a count of replicas on each node for each table.
func (s *adminServer) DataDistribution(
ctx context.Context, req *serverpb.DataDistributionRequest,
diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go
index 5452a2e59447..43d06a3ca95c 100644
--- a/pkg/server/node_test.go
+++ b/pkg/server/node_test.go
@@ -640,3 +640,113 @@ func TestNodeSendUnknownBatchRequest(t *testing.T) {
t.Fatalf("expected unsupported request, not %v", br.Error)
}
}
+
+// TestAddNewStoresToExistingNodes tests database behavior with
+// multiple stores per node, in particular when new stores are
+// added while nodes are shut down. This test starts a cluster with
+// three nodes, shuts down all nodes and adds a store to each node,
+// and ensures nodes start back up successfully. See #39415.
+func TestAddNewStoresToExistingNodes(t *testing.T) {
+ defer leaktest.AfterTest(t)()
+ defer log.Scope(t).Close(t)
+
+ ctx := context.Background()
+
+ n1s1, n1cleanup1 := testutils.TempDir(t)
+ defer n1cleanup1()
+ n2s1, n2cleanup1 := testutils.TempDir(t)
+ defer n2cleanup1()
+ n3s1, n3cleanup1 := testutils.TempDir(t)
+ defer n3cleanup1()
+
+ numNodes := 3
+ tcArgs := base.TestClusterArgs{
+ ServerArgsPerNode: map[int]base.TestServerArgs{
+ // NB: on my local (beefy) machine, upreplication
+ // takes ~6s. This is pretty hefty compared to ~1s
+ // with ephemeral stores. But - we need the real
+ // stores here. At the time of writing, we perform
+ // ~100 change replicas txns, all in all, and
+ // 0.06s for a replication change does seem ok.
+ 0: {StoreSpecs: []base.StoreSpec{{Path: n1s1}}},
+ 1: {StoreSpecs: []base.StoreSpec{{Path: n2s1}}},
+ 2: {StoreSpecs: []base.StoreSpec{{Path: n3s1}}},
+ },
+ }
+
+ tc := testcluster.StartTestCluster(t, numNodes, tcArgs)
+ // NB: it's important that this test wait for full replication. Otherwise,
+ // with only a single voter on the range that allocates store IDs, it can
+ // pass erroneously. StartTestCluster already calls it, but we call it
+ // again explicitly.
+ tc.WaitForFullReplication()
+ clusterID := tc.Server(0).ClusterID()
+ tc.Stopper().Stop(ctx)
+
+ // Add an additional store to each node.
+ n1s2, n1cleanup2 := testutils.TempDir(t)
+ defer n1cleanup2()
+ n2s2, n2cleanup2 := testutils.TempDir(t)
+ defer n2cleanup2()
+ n3s2, n3cleanup2 := testutils.TempDir(t)
+ defer n3cleanup2()
+
+ tcArgs = base.TestClusterArgs{
+ // We need ParallelStart since this is an existing cluster. If
+ // we started sequentially, then the first node would hang forever
+ // waiting for the KV layer to become available, but that only
+ // happens when the second node also starts.
+ ParallelStart: true,
+ ReplicationMode: base.ReplicationManual, // saves time
+ ServerArgsPerNode: map[int]base.TestServerArgs{
+ 0: {
+ StoreSpecs: []base.StoreSpec{
+ {Path: n1s1}, {Path: n1s2},
+ },
+ },
+ 1: {
+ StoreSpecs: []base.StoreSpec{
+ {Path: n2s1}, {Path: n2s2},
+ },
+ },
+ 2: {
+ StoreSpecs: []base.StoreSpec{
+ {Path: n3s1}, {Path: n3s2},
+ },
+ },
+ },
+ }
+
+ // Start all nodes with additional stores.
+ tc = testcluster.StartTestCluster(t, numNodes, tcArgs)
+ defer tc.Stopper().Stop(ctx)
+
+ // Sanity check that we're testing what we wanted to test and didn't accidentally
+ // bootstrap three single-node clusters (who knows).
+ for _, srv := range tc.Servers {
+ require.Equal(t, clusterID, srv.ClusterID())
+ }
+
+ // Ensure all nodes have 2 stores available.
+ testutils.SucceedsSoon(t, func() error {
+ for _, server := range tc.Servers {
+ var storeCount = 0
+
+ err := server.GetStores().(*kvserver.Stores).VisitStores(
+ func(s *kvserver.Store) error {
+ storeCount++
+ return nil
+ },
+ )
+ if err != nil {
+ return errors.Errorf("failed to visit all nodes, got %v", err)
+ }
+
+ if storeCount != 2 {
+ return errors.Errorf("expected two stores to be available on node %v, got %v stores instead", server.NodeID(), storeCount)
+ }
+ }
+
+ return nil
+ })
+}
diff --git a/pkg/server/server.go b/pkg/server/server.go
index cdf55a145e7c..f7a95793dec9 100644
--- a/pkg/server/server.go
+++ b/pkg/server/server.go
@@ -35,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts"
+ "github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
@@ -2392,3 +2393,46 @@ func (s *Server) RunLocalSQL(
func (s *Server) Insecure() bool {
return s.cfg.Insecure
}
+
+// PersistPendingVersion XXX: TODO: DOC:
+func (s *Server) PersistPendingVersion(ctx context.Context, version *roachpb.Version) error {
+ log.Infof(ctx, "xxx: received op=ack-pending-version, args=%s", version)
+ var v roachpb.Value
+ if err := v.SetProto(version); err != nil {
+ return err
+ }
+ for _, eng := range s.engines {
+ if err := storage.MVCCPut(
+ ctx, eng, nil /* MVCCStats */, keys.StoreTargetVersionKey(), hlc.Timestamp{}, v, nil, /* txn */
+ ); err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+// GetPendingVersion XXX: TODO: DOC:
+func (s *Server) GetPendingVersion(ctx context.Context) (roachpb.Version, error) {
+ for _, eng := range s.engines {
+ v, _, err := storage.MVCCGet(ctx, eng, keys.StoreTargetVersionKey(), hlc.Timestamp{}, storage.MVCCGetOptions{})
+ if err != nil {
+ return roachpb.Version{}, err
+ }
+ if v == nil {
+ // Not found. // XXX: Should this be possible?
+ continue
+ }
+ var ver roachpb.Version
+ if err := v.GetProto(&ver); err != nil {
+ return roachpb.Version{}, err
+ }
+
+ // XXX: We're just returning from the first one we find. What about the
+ // others?
+ return ver, nil
+ }
+
+ // XXX: Is this what we want? When no version is found on disk the very
+ // first time?
+ return clusterversion.TestingBinaryVersion, nil
+}
diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go
index 49cfa0c4b119..452943fe3735 100644
--- a/pkg/server/server_sql.go
+++ b/pkg/server/server_sql.go
@@ -13,6 +13,7 @@ package server
import (
"context"
"fmt"
+ "github.com/cockroachdb/cockroach/pkg/sqlmigrations/leasemanager"
"math"
"net"
"os"
@@ -22,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/blobs"
"github.com/cockroachdb/cockroach/pkg/blobs/blobspb"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
+ "github.com/cockroachdb/cockroach/pkg/clusterversion/migration"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/jobs"
@@ -84,6 +86,7 @@ type sqlServer struct {
sessionRegistry *sql.SessionRegistry
jobRegistry *jobs.Registry
migMgr *sqlmigrations.Manager
+ lrmOrchestrator *migration.Orchestrator
statsRefresher *stats.Refresher
temporaryObjectCleaner *sql.TemporaryObjectCleaner
internalMemMetrics sql.MemoryMetrics
@@ -459,14 +462,6 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
RangeDescriptorCache: cfg.distSender.RangeDescriptorCache(),
RoleMemberCache: &sql.MembershipCache{},
TestingKnobs: sqlExecutorTestingKnobs,
- VersionUpgradeHook: func(ctx context.Context, to roachpb.Version) error {
- // TODO(irfansharif): Do something real here. We want to be able to
- // send out a Migrate request spanning the entire keyspace. We'll
- // need to make sure all stores have synced once to persist any raft
- // command applications.
- log.Infof(ctx, "version upgrade callback called with target version=%s", to.String())
- return nil
- },
DistSQLPlanner: sql.NewDistSQLPlanner(
ctx,
@@ -646,6 +641,25 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
sqlExecutorTestingKnobs,
)
+ // XXX: Could use the same pkg/sqlmigrations/leasemanager/lease instance
+ // between this and sqlmigrations. Will need to get pulled up above or
+ // something.
+ lrmLeaseMgr := leasemanager.New(cfg.db, execCfg.Clock, leasemanager.Options{})
+ lrmMgr := migration.NewOrchestrator(
+ lrmLeaseMgr,
+ cfg.circularInternalExecutor,
+ cfg.db,
+ cfg.nodeDialer,
+ )
+ execCfg.VersionUpgradeHook = func(ctx context.Context, to roachpb.Version) error {
+ // TODO(irfansharif): Do something real here. We want to be able to
+ // send out a Migrate request spanning the entire keyspace. We'll
+ // need to make sure all stores have synced once to persist any raft
+ // command applications.
+ log.Infof(ctx, "xxx: running migrations for target version=%s", to.String())
+ return lrmMgr.RunMigrations(ctx, to)
+ }
+
return &sqlServer{
pgServer: pgServer,
distSQLServer: distSQLServer,
@@ -663,6 +677,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
stmtDiagnosticsRegistry: stmtDiagnosticsRegistry,
sqlLivenessProvider: cfg.sqlLivenessProvider,
metricsRegistry: cfg.registry,
+ lrmOrchestrator: lrmMgr,
}, nil
}
diff --git a/pkg/server/serverpb/admin.pb.go b/pkg/server/serverpb/admin.pb.go
index 2f08d8bf7509..6f06cbae3400 100644
--- a/pkg/server/serverpb/admin.pb.go
+++ b/pkg/server/serverpb/admin.pb.go
@@ -9,6 +9,7 @@ import math "math"
import zonepb "github.com/cockroachdb/cockroach/pkg/config/zonepb"
import jobspb "github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
import kvserverpb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
+import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb"
import enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb"
import catalog "github.com/cockroachdb/cockroach/pkg/ts/catalog"
import metric "github.com/cockroachdb/cockroach/pkg/util/metric"
@@ -73,7 +74,7 @@ func (x ZoneConfigurationLevel) String() string {
return proto.EnumName(ZoneConfigurationLevel_name, int32(x))
}
func (ZoneConfigurationLevel) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_admin_d35ae12d295a7e1d, []int{0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{0}
}
// DatabasesRequest requests a list of databases.
@@ -84,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_d35ae12d295a7e1d, []int{0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{0}
}
func (m *DatabasesRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -118,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_d35ae12d295a7e1d, []int{1}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{1}
}
func (m *DatabasesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -154,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_d35ae12d295a7e1d, []int{2}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{2}
}
func (m *DatabaseDetailsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -203,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_d35ae12d295a7e1d, []int{3}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{3}
}
func (m *DatabaseDetailsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -239,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_d35ae12d295a7e1d, []int{3, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{3, 0}
}
func (m *DatabaseDetailsResponse_Grant) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -279,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_d35ae12d295a7e1d, []int{4}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{4}
}
func (m *TableDetailsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -336,7 +337,7 @@ func (m *TableDetailsResponse) Reset() { *m = TableDetailsResponse{} }
func (m *TableDetailsResponse) String() string { return proto.CompactTextString(m) }
func (*TableDetailsResponse) ProtoMessage() {}
func (*TableDetailsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_admin_d35ae12d295a7e1d, []int{5}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{5}
}
func (m *TableDetailsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -373,7 +374,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_d35ae12d295a7e1d, []int{5, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{5, 0}
}
func (m *TableDetailsResponse_Grant) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -417,7 +418,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_d35ae12d295a7e1d, []int{5, 1}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{5, 1}
}
func (m *TableDetailsResponse_Column) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -463,7 +464,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_d35ae12d295a7e1d, []int{5, 2}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{5, 2}
}
func (m *TableDetailsResponse_Index) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -504,7 +505,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_d35ae12d295a7e1d, []int{6}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{6}
}
func (m *TableStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -558,7 +559,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_d35ae12d295a7e1d, []int{7}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{7}
}
func (m *TableStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -596,7 +597,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_d35ae12d295a7e1d, []int{7, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{7, 0}
}
func (m *TableStatsResponse_MissingNode) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -630,7 +631,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_d35ae12d295a7e1d, []int{8}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{8}
}
func (m *NonTableStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -669,7 +670,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_d35ae12d295a7e1d, []int{9}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{9}
}
func (m *NonTableStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -702,7 +703,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_d35ae12d295a7e1d, []int{10}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{10}
}
func (m *UsersRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -737,7 +738,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_d35ae12d295a7e1d, []int{11}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{11}
}
func (m *UsersResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -771,7 +772,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_d35ae12d295a7e1d, []int{11, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{11, 0}
}
func (m *UsersResponse_User) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -818,7 +819,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_d35ae12d295a7e1d, []int{12}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{12}
}
func (m *EventsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -853,7 +854,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_d35ae12d295a7e1d, []int{13}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{13}
}
func (m *EventsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -898,7 +899,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_d35ae12d295a7e1d, []int{13, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{13, 0}
}
func (m *EventsResponse_Event) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -934,7 +935,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_d35ae12d295a7e1d, []int{14}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{14}
}
func (m *SetUIDataRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -967,7 +968,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_d35ae12d295a7e1d, []int{15}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{15}
}
func (m *SetUIDataResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1002,7 +1003,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_d35ae12d295a7e1d, []int{16}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{16}
}
func (m *GetUIDataRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1039,7 +1040,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_d35ae12d295a7e1d, []int{17}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{17}
}
func (m *GetUIDataResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1075,7 +1076,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_d35ae12d295a7e1d, []int{17, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{17, 0}
}
func (m *GetUIDataResponse_Value) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1108,7 +1109,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_d35ae12d295a7e1d, []int{18}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{18}
}
func (m *ClusterRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1147,7 +1148,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_d35ae12d295a7e1d, []int{19}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{19}
}
func (m *ClusterResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1195,7 +1196,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_d35ae12d295a7e1d, []int{20}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{20}
}
func (m *DrainRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1280,7 +1281,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_d35ae12d295a7e1d, []int{21}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{21}
}
func (m *DrainResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1315,7 +1316,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_d35ae12d295a7e1d, []int{22}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{22}
}
func (m *DecommissionStatusRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1353,7 +1354,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_d35ae12d295a7e1d, []int{23}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{23}
}
func (m *DecommissionRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1388,7 +1389,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_d35ae12d295a7e1d, []int{24}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{24}
}
func (m *DecommissionStatusResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1427,7 +1428,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_d35ae12d295a7e1d, []int{24, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{24, 0}
}
func (m *DecommissionStatusResponse_Status) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1452,6 +1453,374 @@ func (m *DecommissionStatusResponse_Status) XXX_DiscardUnknown() {
var xxx_messageInfo_DecommissionStatusResponse_Status proto.InternalMessageInfo
+// AckPendingVersionRequest XXX:
+type AckPendingVersionRequest struct {
+ Version *roachpb.Version `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"`
+}
+
+func (m *AckPendingVersionRequest) Reset() { *m = AckPendingVersionRequest{} }
+func (m *AckPendingVersionRequest) String() string { return proto.CompactTextString(m) }
+func (*AckPendingVersionRequest) ProtoMessage() {}
+func (*AckPendingVersionRequest) Descriptor() ([]byte, []int) {
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{25}
+}
+func (m *AckPendingVersionRequest) XXX_Unmarshal(b []byte) error {
+ return m.Unmarshal(b)
+}
+func (m *AckPendingVersionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ b = b[:cap(b)]
+ n, err := m.MarshalTo(b)
+ if err != nil {
+ return nil, err
+ }
+ return b[:n], nil
+}
+func (dst *AckPendingVersionRequest) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_AckPendingVersionRequest.Merge(dst, src)
+}
+func (m *AckPendingVersionRequest) XXX_Size() int {
+ return m.Size()
+}
+func (m *AckPendingVersionRequest) XXX_DiscardUnknown() {
+ xxx_messageInfo_AckPendingVersionRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AckPendingVersionRequest proto.InternalMessageInfo
+
+// AckPendingVersionResponse XXX:
+type AckPendingVersionResponse struct {
+}
+
+func (m *AckPendingVersionResponse) Reset() { *m = AckPendingVersionResponse{} }
+func (m *AckPendingVersionResponse) String() string { return proto.CompactTextString(m) }
+func (*AckPendingVersionResponse) ProtoMessage() {}
+func (*AckPendingVersionResponse) Descriptor() ([]byte, []int) {
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{26}
+}
+func (m *AckPendingVersionResponse) XXX_Unmarshal(b []byte) error {
+ return m.Unmarshal(b)
+}
+func (m *AckPendingVersionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ b = b[:cap(b)]
+ n, err := m.MarshalTo(b)
+ if err != nil {
+ return nil, err
+ }
+ return b[:n], nil
+}
+func (dst *AckPendingVersionResponse) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_AckPendingVersionResponse.Merge(dst, src)
+}
+func (m *AckPendingVersionResponse) XXX_Size() int {
+ return m.Size()
+}
+func (m *AckPendingVersionResponse) XXX_DiscardUnknown() {
+ xxx_messageInfo_AckPendingVersionResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AckPendingVersionResponse proto.InternalMessageInfo
+
+// EveryNodeRequestUnion XXX:
+type EveryNodeRequestUnion struct {
+ // Types that are valid to be assigned to Value:
+ // *EveryNodeRequestUnion_AckPendingVersion
+ Value isEveryNodeRequestUnion_Value `protobuf_oneof:"value"`
+}
+
+func (m *EveryNodeRequestUnion) Reset() { *m = EveryNodeRequestUnion{} }
+func (m *EveryNodeRequestUnion) String() string { return proto.CompactTextString(m) }
+func (*EveryNodeRequestUnion) ProtoMessage() {}
+func (*EveryNodeRequestUnion) Descriptor() ([]byte, []int) {
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{27}
+}
+func (m *EveryNodeRequestUnion) XXX_Unmarshal(b []byte) error {
+ return m.Unmarshal(b)
+}
+func (m *EveryNodeRequestUnion) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ b = b[:cap(b)]
+ n, err := m.MarshalTo(b)
+ if err != nil {
+ return nil, err
+ }
+ return b[:n], nil
+}
+func (dst *EveryNodeRequestUnion) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_EveryNodeRequestUnion.Merge(dst, src)
+}
+func (m *EveryNodeRequestUnion) XXX_Size() int {
+ return m.Size()
+}
+func (m *EveryNodeRequestUnion) XXX_DiscardUnknown() {
+ xxx_messageInfo_EveryNodeRequestUnion.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EveryNodeRequestUnion proto.InternalMessageInfo
+
+type isEveryNodeRequestUnion_Value interface {
+ isEveryNodeRequestUnion_Value()
+ MarshalTo([]byte) (int, error)
+ Size() int
+}
+
+type EveryNodeRequestUnion_AckPendingVersion struct {
+ AckPendingVersion *AckPendingVersionRequest `protobuf:"bytes,1,opt,name=ack_pending_version,json=ackPendingVersion,proto3,oneof"`
+}
+
+func (*EveryNodeRequestUnion_AckPendingVersion) isEveryNodeRequestUnion_Value() {}
+
+func (m *EveryNodeRequestUnion) GetValue() isEveryNodeRequestUnion_Value {
+ if m != nil {
+ return m.Value
+ }
+ return nil
+}
+
+func (m *EveryNodeRequestUnion) GetAckPendingVersion() *AckPendingVersionRequest {
+ if x, ok := m.GetValue().(*EveryNodeRequestUnion_AckPendingVersion); ok {
+ return x.AckPendingVersion
+ }
+ return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*EveryNodeRequestUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+ return _EveryNodeRequestUnion_OneofMarshaler, _EveryNodeRequestUnion_OneofUnmarshaler, _EveryNodeRequestUnion_OneofSizer, []interface{}{
+ (*EveryNodeRequestUnion_AckPendingVersion)(nil),
+ }
+}
+
+func _EveryNodeRequestUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+ m := msg.(*EveryNodeRequestUnion)
+ // value
+ switch x := m.Value.(type) {
+ case *EveryNodeRequestUnion_AckPendingVersion:
+ _ = b.EncodeVarint(1<<3 | proto.WireBytes)
+ if err := b.EncodeMessage(x.AckPendingVersion); err != nil {
+ return err
+ }
+ case nil:
+ default:
+ return fmt.Errorf("EveryNodeRequestUnion.Value has unexpected type %T", x)
+ }
+ return nil
+}
+
+func _EveryNodeRequestUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+ m := msg.(*EveryNodeRequestUnion)
+ switch tag {
+ case 1: // value.ack_pending_version
+ if wire != proto.WireBytes {
+ return true, proto.ErrInternalBadWireType
+ }
+ msg := new(AckPendingVersionRequest)
+ err := b.DecodeMessage(msg)
+ m.Value = &EveryNodeRequestUnion_AckPendingVersion{msg}
+ return true, err
+ default:
+ return false, nil
+ }
+}
+
+func _EveryNodeRequestUnion_OneofSizer(msg proto.Message) (n int) {
+ m := msg.(*EveryNodeRequestUnion)
+ // value
+ switch x := m.Value.(type) {
+ case *EveryNodeRequestUnion_AckPendingVersion:
+ s := proto.Size(x.AckPendingVersion)
+ n += 1 // tag and wire
+ n += proto.SizeVarint(uint64(s))
+ n += s
+ case nil:
+ default:
+ panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+ }
+ return n
+}
+
+type EveryNodeResponseUnion struct {
+ // Types that are valid to be assigned to Value:
+ // *EveryNodeResponseUnion_AckPendingVersion
+ Value isEveryNodeResponseUnion_Value `protobuf_oneof:"value"`
+}
+
+func (m *EveryNodeResponseUnion) Reset() { *m = EveryNodeResponseUnion{} }
+func (m *EveryNodeResponseUnion) String() string { return proto.CompactTextString(m) }
+func (*EveryNodeResponseUnion) ProtoMessage() {}
+func (*EveryNodeResponseUnion) Descriptor() ([]byte, []int) {
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{28}
+}
+func (m *EveryNodeResponseUnion) XXX_Unmarshal(b []byte) error {
+ return m.Unmarshal(b)
+}
+func (m *EveryNodeResponseUnion) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ b = b[:cap(b)]
+ n, err := m.MarshalTo(b)
+ if err != nil {
+ return nil, err
+ }
+ return b[:n], nil
+}
+func (dst *EveryNodeResponseUnion) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_EveryNodeResponseUnion.Merge(dst, src)
+}
+func (m *EveryNodeResponseUnion) XXX_Size() int {
+ return m.Size()
+}
+func (m *EveryNodeResponseUnion) XXX_DiscardUnknown() {
+ xxx_messageInfo_EveryNodeResponseUnion.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EveryNodeResponseUnion proto.InternalMessageInfo
+
+type isEveryNodeResponseUnion_Value interface {
+ isEveryNodeResponseUnion_Value()
+ MarshalTo([]byte) (int, error)
+ Size() int
+}
+
+type EveryNodeResponseUnion_AckPendingVersion struct {
+ AckPendingVersion *AckPendingVersionResponse `protobuf:"bytes,1,opt,name=ack_pending_version,json=ackPendingVersion,proto3,oneof"`
+}
+
+func (*EveryNodeResponseUnion_AckPendingVersion) isEveryNodeResponseUnion_Value() {}
+
+func (m *EveryNodeResponseUnion) GetValue() isEveryNodeResponseUnion_Value {
+ if m != nil {
+ return m.Value
+ }
+ return nil
+}
+
+func (m *EveryNodeResponseUnion) GetAckPendingVersion() *AckPendingVersionResponse {
+ if x, ok := m.GetValue().(*EveryNodeResponseUnion_AckPendingVersion); ok {
+ return x.AckPendingVersion
+ }
+ return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*EveryNodeResponseUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+ return _EveryNodeResponseUnion_OneofMarshaler, _EveryNodeResponseUnion_OneofUnmarshaler, _EveryNodeResponseUnion_OneofSizer, []interface{}{
+ (*EveryNodeResponseUnion_AckPendingVersion)(nil),
+ }
+}
+
+func _EveryNodeResponseUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+ m := msg.(*EveryNodeResponseUnion)
+ // value
+ switch x := m.Value.(type) {
+ case *EveryNodeResponseUnion_AckPendingVersion:
+ _ = b.EncodeVarint(1<<3 | proto.WireBytes)
+ if err := b.EncodeMessage(x.AckPendingVersion); err != nil {
+ return err
+ }
+ case nil:
+ default:
+ return fmt.Errorf("EveryNodeResponseUnion.Value has unexpected type %T", x)
+ }
+ return nil
+}
+
+func _EveryNodeResponseUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+ m := msg.(*EveryNodeResponseUnion)
+ switch tag {
+ case 1: // value.ack_pending_version
+ if wire != proto.WireBytes {
+ return true, proto.ErrInternalBadWireType
+ }
+ msg := new(AckPendingVersionResponse)
+ err := b.DecodeMessage(msg)
+ m.Value = &EveryNodeResponseUnion_AckPendingVersion{msg}
+ return true, err
+ default:
+ return false, nil
+ }
+}
+
+func _EveryNodeResponseUnion_OneofSizer(msg proto.Message) (n int) {
+ m := msg.(*EveryNodeResponseUnion)
+ // value
+ switch x := m.Value.(type) {
+ case *EveryNodeResponseUnion_AckPendingVersion:
+ s := proto.Size(x.AckPendingVersion)
+ n += 1 // tag and wire
+ n += proto.SizeVarint(uint64(s))
+ n += s
+ case nil:
+ default:
+ panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+ }
+ return n
+}
+
+// EveryNodeRequest XXX:
+type EveryNodeRequest struct {
+ Request EveryNodeRequestUnion `protobuf:"bytes,1,opt,name=request,proto3" json:"request"`
+}
+
+func (m *EveryNodeRequest) Reset() { *m = EveryNodeRequest{} }
+func (m *EveryNodeRequest) String() string { return proto.CompactTextString(m) }
+func (*EveryNodeRequest) ProtoMessage() {}
+func (*EveryNodeRequest) Descriptor() ([]byte, []int) {
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{29}
+}
+func (m *EveryNodeRequest) XXX_Unmarshal(b []byte) error {
+ return m.Unmarshal(b)
+}
+func (m *EveryNodeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ b = b[:cap(b)]
+ n, err := m.MarshalTo(b)
+ if err != nil {
+ return nil, err
+ }
+ return b[:n], nil
+}
+func (dst *EveryNodeRequest) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_EveryNodeRequest.Merge(dst, src)
+}
+func (m *EveryNodeRequest) XXX_Size() int {
+ return m.Size()
+}
+func (m *EveryNodeRequest) XXX_DiscardUnknown() {
+ xxx_messageInfo_EveryNodeRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EveryNodeRequest proto.InternalMessageInfo
+
+// EveryNodeResponse XXX:
+type EveryNodeResponse struct {
+ Response EveryNodeResponseUnion `protobuf:"bytes,1,opt,name=response,proto3" json:"response"`
+}
+
+func (m *EveryNodeResponse) Reset() { *m = EveryNodeResponse{} }
+func (m *EveryNodeResponse) String() string { return proto.CompactTextString(m) }
+func (*EveryNodeResponse) ProtoMessage() {}
+func (*EveryNodeResponse) Descriptor() ([]byte, []int) {
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{30}
+}
+func (m *EveryNodeResponse) XXX_Unmarshal(b []byte) error {
+ return m.Unmarshal(b)
+}
+func (m *EveryNodeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ b = b[:cap(b)]
+ n, err := m.MarshalTo(b)
+ if err != nil {
+ return nil, err
+ }
+ return b[:n], nil
+}
+func (dst *EveryNodeResponse) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_EveryNodeResponse.Merge(dst, src)
+}
+func (m *EveryNodeResponse) XXX_Size() int {
+ return m.Size()
+}
+func (m *EveryNodeResponse) XXX_DiscardUnknown() {
+ xxx_messageInfo_EveryNodeResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EveryNodeResponse proto.InternalMessageInfo
+
// SettingsRequest inquires what are the current settings in the cluster.
type SettingsRequest struct {
// The array of setting names to retrieve.
@@ -1469,7 +1838,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_d35ae12d295a7e1d, []int{25}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{31}
}
func (m *SettingsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1503,7 +1872,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_d35ae12d295a7e1d, []int{26}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{32}
}
func (m *SettingsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1539,7 +1908,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_d35ae12d295a7e1d, []int{26, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{32, 0}
}
func (m *SettingsResponse_Value) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1592,7 +1961,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_d35ae12d295a7e1d, []int{27}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{33}
}
func (m *HealthRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1626,7 +1995,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_d35ae12d295a7e1d, []int{28}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{34}
}
func (m *HealthResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1659,7 +2028,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_d35ae12d295a7e1d, []int{29}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{35}
}
func (m *LivenessRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1694,7 +2063,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_d35ae12d295a7e1d, []int{30}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{36}
}
func (m *LivenessResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1730,7 +2099,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_d35ae12d295a7e1d, []int{31}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{37}
}
func (m *JobsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1764,7 +2133,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_d35ae12d295a7e1d, []int{32}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{38}
}
func (m *JobsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1817,7 +2186,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_d35ae12d295a7e1d, []int{32, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{38, 0}
}
func (m *JobsResponse_Job) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1850,7 +2219,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_d35ae12d295a7e1d, []int{33}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{39}
}
func (m *LocationsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1884,7 +2253,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_d35ae12d295a7e1d, []int{34}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{40}
}
func (m *LocationsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1920,7 +2289,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_d35ae12d295a7e1d, []int{34, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{40, 0}
}
func (m *LocationsResponse_Location) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1962,7 +2331,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_d35ae12d295a7e1d, []int{35}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{41}
}
func (m *RangeLogRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1996,7 +2365,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_d35ae12d295a7e1d, []int{36}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{42}
}
func (m *RangeLogResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2036,7 +2405,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_d35ae12d295a7e1d, []int{36, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{42, 0}
}
func (m *RangeLogResponse_PrettyInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2070,7 +2439,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_d35ae12d295a7e1d, []int{36, 1}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{42, 1}
}
func (m *RangeLogResponse_Event) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2105,7 +2474,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_d35ae12d295a7e1d, []int{37}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{43}
}
func (m *QueryPlanRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2140,7 +2509,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_d35ae12d295a7e1d, []int{38}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{44}
}
func (m *QueryPlanResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2172,7 +2541,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_d35ae12d295a7e1d, []int{39}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{45}
}
func (m *DataDistributionRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2208,7 +2577,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_d35ae12d295a7e1d, []int{40}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{46}
}
func (m *DataDistributionResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2246,7 +2615,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_d35ae12d295a7e1d, []int{40, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{46, 0}
}
func (m *DataDistributionResponse_ZoneConfig) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2281,7 +2650,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_d35ae12d295a7e1d, []int{40, 1}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{46, 1}
}
func (m *DataDistributionResponse_TableInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2315,7 +2684,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_d35ae12d295a7e1d, []int{40, 2}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{46, 2}
}
func (m *DataDistributionResponse_DatabaseInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2348,7 +2717,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_d35ae12d295a7e1d, []int{41}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{47}
}
func (m *MetricMetadataRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2382,7 +2751,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_d35ae12d295a7e1d, []int{42}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{48}
}
func (m *MetricMetadataResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2425,7 +2794,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_d35ae12d295a7e1d, []int{43}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{49}
}
func (m *EnqueueRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2458,7 +2827,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_d35ae12d295a7e1d, []int{44}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{50}
}
func (m *EnqueueRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2495,7 +2864,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_d35ae12d295a7e1d, []int{44, 0}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{50, 0}
}
func (m *EnqueueRangeResponse_Details) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2528,7 +2897,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_d35ae12d295a7e1d, []int{45}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{51}
}
func (m *ChartCatalogRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2562,7 +2931,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_d35ae12d295a7e1d, []int{46}
+ return fileDescriptor_admin_a5fd0c6a410748d8, []int{52}
}
func (m *ChartCatalogResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2624,6 +2993,12 @@ func init() {
proto.RegisterType((*DecommissionRequest)(nil), "cockroach.server.serverpb.DecommissionRequest")
proto.RegisterType((*DecommissionStatusResponse)(nil), "cockroach.server.serverpb.DecommissionStatusResponse")
proto.RegisterType((*DecommissionStatusResponse_Status)(nil), "cockroach.server.serverpb.DecommissionStatusResponse.Status")
+ proto.RegisterType((*AckPendingVersionRequest)(nil), "cockroach.server.serverpb.AckPendingVersionRequest")
+ proto.RegisterType((*AckPendingVersionResponse)(nil), "cockroach.server.serverpb.AckPendingVersionResponse")
+ proto.RegisterType((*EveryNodeRequestUnion)(nil), "cockroach.server.serverpb.EveryNodeRequestUnion")
+ proto.RegisterType((*EveryNodeResponseUnion)(nil), "cockroach.server.serverpb.EveryNodeResponseUnion")
+ proto.RegisterType((*EveryNodeRequest)(nil), "cockroach.server.serverpb.EveryNodeRequest")
+ proto.RegisterType((*EveryNodeResponse)(nil), "cockroach.server.serverpb.EveryNodeResponse")
proto.RegisterType((*SettingsRequest)(nil), "cockroach.server.serverpb.SettingsRequest")
proto.RegisterType((*SettingsResponse)(nil), "cockroach.server.serverpb.SettingsResponse")
proto.RegisterMapType((map[string]SettingsResponse_Value)(nil), "cockroach.server.serverpb.SettingsResponse.KeyValuesEntry")
@@ -2746,6 +3121,8 @@ type AdminClient interface {
// If this ever becomes exposed via HTTP, ensure that it performs
// authorization. See #42567.
Decommission(ctx context.Context, in *DecommissionRequest, opts ...grpc.CallOption) (*DecommissionStatusResponse, error)
+ // EveryNode XXX:
+ EveryNode(ctx context.Context, in *EveryNodeRequest, opts ...grpc.CallOption) (*EveryNodeResponse, error)
// DecommissionStatus retrieves the decommissioning status of the specified nodes.
// If this ever becomes exposed via HTTP, ensure that it performs
// authorization. See #42567.
@@ -2966,6 +3343,15 @@ func (c *adminClient) Decommission(ctx context.Context, in *DecommissionRequest,
return out, nil
}
+func (c *adminClient) EveryNode(ctx context.Context, in *EveryNodeRequest, opts ...grpc.CallOption) (*EveryNodeResponse, error) {
+ out := new(EveryNodeResponse)
+ err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/EveryNode", in, out, opts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
func (c *adminClient) DecommissionStatus(ctx context.Context, in *DecommissionStatusRequest, opts ...grpc.CallOption) (*DecommissionStatusResponse, error) {
out := new(DecommissionStatusResponse)
err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/DecommissionStatus", in, out, opts...)
@@ -3091,6 +3477,8 @@ type AdminServer interface {
// If this ever becomes exposed via HTTP, ensure that it performs
// authorization. See #42567.
Decommission(context.Context, *DecommissionRequest) (*DecommissionStatusResponse, error)
+ // EveryNode XXX:
+ EveryNode(context.Context, *EveryNodeRequest) (*EveryNodeResponse, error)
// DecommissionStatus retrieves the decommissioning status of the specified nodes.
// If this ever becomes exposed via HTTP, ensure that it performs
// authorization. See #42567.
@@ -3449,6 +3837,24 @@ func _Admin_Decommission_Handler(srv interface{}, ctx context.Context, dec func(
return interceptor(ctx, in, info, handler)
}
+func _Admin_EveryNode_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(EveryNodeRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(AdminServer).EveryNode(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: "/cockroach.server.serverpb.Admin/EveryNode",
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(AdminServer).EveryNode(ctx, req.(*EveryNodeRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
func _Admin_DecommissionStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(DecommissionStatusRequest)
if err := dec(in); err != nil {
@@ -3629,6 +4035,10 @@ var _Admin_serviceDesc = grpc.ServiceDesc{
MethodName: "Decommission",
Handler: _Admin_Decommission_Handler,
},
+ {
+ MethodName: "EveryNode",
+ Handler: _Admin_EveryNode_Handler,
+ },
{
MethodName: "DecommissionStatus",
Handler: _Admin_DecommissionStatus_Handler,
@@ -4995,7 +5405,7 @@ func (m *DecommissionStatusResponse_Status) MarshalTo(dAtA []byte) (int, error)
return i, nil
}
-func (m *SettingsRequest) Marshal() (dAtA []byte, err error) {
+func (m *AckPendingVersionRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
@@ -5005,40 +5415,25 @@ func (m *SettingsRequest) Marshal() (dAtA []byte, err error) {
return dAtA[:n], nil
}
-func (m *SettingsRequest) MarshalTo(dAtA []byte) (int, error) {
+func (m *AckPendingVersionRequest) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
- if len(m.Keys) > 0 {
- for _, s := range m.Keys {
- dAtA[i] = 0xa
- i++
- l = len(s)
- for l >= 1<<7 {
- dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
- l >>= 7
- i++
- }
- dAtA[i] = uint8(l)
- i++
- i += copy(dAtA[i:], s)
- }
- }
- if m.UnredactedValues {
- dAtA[i] = 0x10
+ if m.Version != nil {
+ dAtA[i] = 0xa
i++
- if m.UnredactedValues {
- dAtA[i] = 1
- } else {
- dAtA[i] = 0
+ i = encodeVarintAdmin(dAtA, i, uint64(m.Version.Size()))
+ n15, err := m.Version.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
}
- i++
+ i += n15
}
return i, nil
}
-func (m *SettingsResponse) Marshal() (dAtA []byte, err error) {
+func (m *AckPendingVersionResponse) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
@@ -5048,46 +5443,15 @@ func (m *SettingsResponse) Marshal() (dAtA []byte, err error) {
return dAtA[:n], nil
}
-func (m *SettingsResponse) MarshalTo(dAtA []byte) (int, error) {
+func (m *AckPendingVersionResponse) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
- if len(m.KeyValues) > 0 {
- keysForKeyValues := make([]string, 0, len(m.KeyValues))
- for k := range m.KeyValues {
- keysForKeyValues = append(keysForKeyValues, string(k))
- }
- github_com_gogo_protobuf_sortkeys.Strings(keysForKeyValues)
- for _, k := range keysForKeyValues {
- dAtA[i] = 0xa
- i++
- v := m.KeyValues[string(k)]
- msgSize := 0
- if (&v) != nil {
- msgSize = (&v).Size()
- msgSize += 1 + sovAdmin(uint64(msgSize))
- }
- mapSize := 1 + len(k) + sovAdmin(uint64(len(k))) + msgSize
- i = encodeVarintAdmin(dAtA, i, uint64(mapSize))
- dAtA[i] = 0xa
- i++
- i = encodeVarintAdmin(dAtA, i, uint64(len(k)))
- i += copy(dAtA[i:], k)
- dAtA[i] = 0x12
- i++
- i = encodeVarintAdmin(dAtA, i, uint64((&v).Size()))
- n15, err := (&v).MarshalTo(dAtA[i:])
- if err != nil {
- return 0, err
- }
- i += n15
- }
- }
return i, nil
}
-func (m *SettingsResponse_Value) Marshal() (dAtA []byte, err error) {
+func (m *EveryNodeRequestUnion) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
@@ -5097,15 +5461,237 @@ func (m *SettingsResponse_Value) Marshal() (dAtA []byte, err error) {
return dAtA[:n], nil
}
-func (m *SettingsResponse_Value) MarshalTo(dAtA []byte) (int, error) {
+func (m *EveryNodeRequestUnion) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
- if len(m.Value) > 0 {
+ if m.Value != nil {
+ nn16, err := m.Value.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += nn16
+ }
+ return i, nil
+}
+
+func (m *EveryNodeRequestUnion_AckPendingVersion) MarshalTo(dAtA []byte) (int, error) {
+ i := 0
+ if m.AckPendingVersion != nil {
dAtA[i] = 0xa
i++
- i = encodeVarintAdmin(dAtA, i, uint64(len(m.Value)))
+ i = encodeVarintAdmin(dAtA, i, uint64(m.AckPendingVersion.Size()))
+ n17, err := m.AckPendingVersion.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += n17
+ }
+ return i, nil
+}
+func (m *EveryNodeResponseUnion) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalTo(dAtA)
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *EveryNodeResponseUnion) MarshalTo(dAtA []byte) (int, error) {
+ var i int
+ _ = i
+ var l int
+ _ = l
+ if m.Value != nil {
+ nn18, err := m.Value.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += nn18
+ }
+ return i, nil
+}
+
+func (m *EveryNodeResponseUnion_AckPendingVersion) MarshalTo(dAtA []byte) (int, error) {
+ i := 0
+ if m.AckPendingVersion != nil {
+ dAtA[i] = 0xa
+ i++
+ i = encodeVarintAdmin(dAtA, i, uint64(m.AckPendingVersion.Size()))
+ n19, err := m.AckPendingVersion.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += n19
+ }
+ return i, nil
+}
+func (m *EveryNodeRequest) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalTo(dAtA)
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *EveryNodeRequest) MarshalTo(dAtA []byte) (int, error) {
+ var i int
+ _ = i
+ var l int
+ _ = l
+ dAtA[i] = 0xa
+ i++
+ i = encodeVarintAdmin(dAtA, i, uint64(m.Request.Size()))
+ n20, err := m.Request.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += n20
+ return i, nil
+}
+
+func (m *EveryNodeResponse) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalTo(dAtA)
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *EveryNodeResponse) MarshalTo(dAtA []byte) (int, error) {
+ var i int
+ _ = i
+ var l int
+ _ = l
+ dAtA[i] = 0xa
+ i++
+ i = encodeVarintAdmin(dAtA, i, uint64(m.Response.Size()))
+ n21, err := m.Response.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += n21
+ return i, nil
+}
+
+func (m *SettingsRequest) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalTo(dAtA)
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *SettingsRequest) MarshalTo(dAtA []byte) (int, error) {
+ var i int
+ _ = i
+ var l int
+ _ = l
+ if len(m.Keys) > 0 {
+ for _, s := range m.Keys {
+ dAtA[i] = 0xa
+ i++
+ l = len(s)
+ for l >= 1<<7 {
+ dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
+ l >>= 7
+ i++
+ }
+ dAtA[i] = uint8(l)
+ i++
+ i += copy(dAtA[i:], s)
+ }
+ }
+ if m.UnredactedValues {
+ dAtA[i] = 0x10
+ i++
+ if m.UnredactedValues {
+ dAtA[i] = 1
+ } else {
+ dAtA[i] = 0
+ }
+ i++
+ }
+ return i, nil
+}
+
+func (m *SettingsResponse) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalTo(dAtA)
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *SettingsResponse) MarshalTo(dAtA []byte) (int, error) {
+ var i int
+ _ = i
+ var l int
+ _ = l
+ if len(m.KeyValues) > 0 {
+ keysForKeyValues := make([]string, 0, len(m.KeyValues))
+ for k := range m.KeyValues {
+ keysForKeyValues = append(keysForKeyValues, string(k))
+ }
+ github_com_gogo_protobuf_sortkeys.Strings(keysForKeyValues)
+ for _, k := range keysForKeyValues {
+ dAtA[i] = 0xa
+ i++
+ v := m.KeyValues[string(k)]
+ msgSize := 0
+ if (&v) != nil {
+ msgSize = (&v).Size()
+ msgSize += 1 + sovAdmin(uint64(msgSize))
+ }
+ mapSize := 1 + len(k) + sovAdmin(uint64(len(k))) + msgSize
+ i = encodeVarintAdmin(dAtA, i, uint64(mapSize))
+ dAtA[i] = 0xa
+ i++
+ i = encodeVarintAdmin(dAtA, i, uint64(len(k)))
+ i += copy(dAtA[i:], k)
+ dAtA[i] = 0x12
+ i++
+ i = encodeVarintAdmin(dAtA, i, uint64((&v).Size()))
+ n22, err := (&v).MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += n22
+ }
+ }
+ return i, nil
+}
+
+func (m *SettingsResponse_Value) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalTo(dAtA)
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *SettingsResponse_Value) MarshalTo(dAtA []byte) (int, error) {
+ var i int
+ _ = i
+ var l int
+ _ = l
+ if len(m.Value) > 0 {
+ dAtA[i] = 0xa
+ i++
+ i = encodeVarintAdmin(dAtA, i, uint64(len(m.Value)))
i += copy(dAtA[i:], m.Value)
}
if len(m.Type) > 0 {
@@ -5350,21 +5936,21 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) {
i += copy(dAtA[i:], m.Username)
}
if len(m.DescriptorIDs) > 0 {
- dAtA17 := make([]byte, len(m.DescriptorIDs)*10)
- var j16 int
+ dAtA24 := make([]byte, len(m.DescriptorIDs)*10)
+ var j23 int
for _, num := range m.DescriptorIDs {
for num >= 1<<7 {
- dAtA17[j16] = uint8(uint64(num)&0x7f | 0x80)
+ dAtA24[j23] = uint8(uint64(num)&0x7f | 0x80)
num >>= 7
- j16++
+ j23++
}
- dAtA17[j16] = uint8(num)
- j16++
+ dAtA24[j23] = uint8(num)
+ j23++
}
dAtA[i] = 0x2a
i++
- i = encodeVarintAdmin(dAtA, i, uint64(j16))
- i += copy(dAtA[i:], dAtA17[:j16])
+ i = encodeVarintAdmin(dAtA, i, uint64(j23))
+ i += copy(dAtA[i:], dAtA24[:j23])
}
if len(m.Status) > 0 {
dAtA[i] = 0x32
@@ -5376,41 +5962,41 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x3a
i++
i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Created)))
- n18, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Created, dAtA[i:])
+ n25, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Created, dAtA[i:])
if err != nil {
return 0, err
}
- i += n18
+ i += n25
}
if m.Started != nil {
dAtA[i] = 0x42
i++
i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Started)))
- n19, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Started, dAtA[i:])
+ n26, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Started, dAtA[i:])
if err != nil {
return 0, err
}
- i += n19
+ i += n26
}
if m.Finished != nil {
dAtA[i] = 0x4a
i++
i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Finished)))
- n20, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Finished, dAtA[i:])
+ n27, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Finished, dAtA[i:])
if err != nil {
return 0, err
}
- i += n20
+ i += n27
}
if m.Modified != nil {
dAtA[i] = 0x52
i++
i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Modified)))
- n21, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Modified, dAtA[i:])
+ n28, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Modified, dAtA[i:])
if err != nil {
return 0, err
}
- i += n21
+ i += n28
}
if m.FractionCompleted != 0 {
dAtA[i] = 0x5d
@@ -5428,11 +6014,11 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x6a
i++
i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.HighwaterTimestamp)))
- n22, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.HighwaterTimestamp, dAtA[i:])
+ n29, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.HighwaterTimestamp, dAtA[i:])
if err != nil {
return 0, err
}
- i += n22
+ i += n29
}
if len(m.HighwaterDecimal) > 0 {
dAtA[i] = 0x72
@@ -5677,19 +6263,19 @@ func (m *RangeLogResponse_Event) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintAdmin(dAtA, i, uint64(m.Event.Size()))
- n23, err := m.Event.MarshalTo(dAtA[i:])
+ n30, err := m.Event.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n23
+ i += n30
dAtA[i] = 0x12
i++
i = encodeVarintAdmin(dAtA, i, uint64(m.PrettyInfo.Size()))
- n24, err := m.PrettyInfo.MarshalTo(dAtA[i:])
+ n31, err := m.PrettyInfo.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n24
+ i += n31
return i, nil
}
@@ -5798,11 +6384,11 @@ func (m *DataDistributionResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintAdmin(dAtA, i, uint64((&v).Size()))
- n25, err := (&v).MarshalTo(dAtA[i:])
+ n32, err := (&v).MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n25
+ i += n32
}
}
if len(m.ZoneConfigs) > 0 {
@@ -5829,11 +6415,11 @@ func (m *DataDistributionResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintAdmin(dAtA, i, uint64((&v).Size()))
- n26, err := (&v).MarshalTo(dAtA[i:])
+ n33, err := (&v).MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n26
+ i += n33
}
}
return i, nil
@@ -5863,11 +6449,11 @@ func (m *DataDistributionResponse_ZoneConfig) MarshalTo(dAtA []byte) (int, error
dAtA[i] = 0x12
i++
i = encodeVarintAdmin(dAtA, i, uint64(m.Config.Size()))
- n27, err := m.Config.MarshalTo(dAtA[i:])
+ n34, err := m.Config.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n27
+ i += n34
if len(m.ConfigSQL) > 0 {
dAtA[i] = 0x22
i++
@@ -5921,11 +6507,11 @@ func (m *DataDistributionResponse_TableInfo) MarshalTo(dAtA []byte) (int, error)
dAtA[i] = 0x1a
i++
i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.DroppedAt)))
- n28, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.DroppedAt, dAtA[i:])
+ n35, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.DroppedAt, dAtA[i:])
if err != nil {
return 0, err
}
- i += n28
+ i += n35
}
return i, nil
}
@@ -5969,11 +6555,11 @@ func (m *DataDistributionResponse_DatabaseInfo) MarshalTo(dAtA []byte) (int, err
dAtA[i] = 0x12
i++
i = encodeVarintAdmin(dAtA, i, uint64((&v).Size()))
- n29, err := (&v).MarshalTo(dAtA[i:])
+ n36, err := (&v).MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n29
+ i += n36
}
}
return i, nil
@@ -6036,11 +6622,11 @@ func (m *MetricMetadataResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintAdmin(dAtA, i, uint64((&v).Size()))
- n30, err := (&v).MarshalTo(dAtA[i:])
+ n37, err := (&v).MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n30
+ i += n37
}
}
return i, nil
@@ -6858,120 +7444,212 @@ func (m *DecommissionStatusResponse_Status) Size() (n int) {
return n
}
-func (m *SettingsRequest) Size() (n int) {
+func (m *AckPendingVersionRequest) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
- if len(m.Keys) > 0 {
- for _, s := range m.Keys {
- l = len(s)
- n += 1 + l + sovAdmin(uint64(l))
- }
- }
- if m.UnredactedValues {
- n += 2
+ if m.Version != nil {
+ l = m.Version.Size()
+ n += 1 + l + sovAdmin(uint64(l))
}
return n
}
-func (m *SettingsResponse) Size() (n int) {
+func (m *AckPendingVersionResponse) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
- if len(m.KeyValues) > 0 {
- for k, v := range m.KeyValues {
- _ = k
- _ = v
- l = v.Size()
- mapEntrySize := 1 + len(k) + sovAdmin(uint64(len(k))) + 1 + l + sovAdmin(uint64(l))
- n += mapEntrySize + 1 + sovAdmin(uint64(mapEntrySize))
- }
- }
return n
}
-func (m *SettingsResponse_Value) Size() (n int) {
+func (m *EveryNodeRequestUnion) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
- l = len(m.Value)
- if l > 0 {
- n += 1 + l + sovAdmin(uint64(l))
+ if m.Value != nil {
+ n += m.Value.Size()
}
- l = len(m.Type)
- if l > 0 {
- n += 1 + l + sovAdmin(uint64(l))
+ return n
+}
+
+func (m *EveryNodeRequestUnion_AckPendingVersion) Size() (n int) {
+ if m == nil {
+ return 0
}
- l = len(m.Description)
- if l > 0 {
+ var l int
+ _ = l
+ if m.AckPendingVersion != nil {
+ l = m.AckPendingVersion.Size()
n += 1 + l + sovAdmin(uint64(l))
}
- if m.Public {
- n += 2
- }
return n
}
-
-func (m *HealthRequest) Size() (n int) {
+func (m *EveryNodeResponseUnion) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
- if m.Ready {
- n += 2
+ if m.Value != nil {
+ n += m.Value.Size()
}
return n
}
-func (m *HealthResponse) Size() (n int) {
+func (m *EveryNodeResponseUnion_AckPendingVersion) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
+ if m.AckPendingVersion != nil {
+ l = m.AckPendingVersion.Size()
+ n += 1 + l + sovAdmin(uint64(l))
+ }
return n
}
-
-func (m *LivenessRequest) Size() (n int) {
+func (m *EveryNodeRequest) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
+ l = m.Request.Size()
+ n += 1 + l + sovAdmin(uint64(l))
return n
}
-func (m *LivenessResponse) Size() (n int) {
+func (m *EveryNodeResponse) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
- if len(m.Livenesses) > 0 {
- for _, e := range m.Livenesses {
- l = e.Size()
- n += 1 + l + sovAdmin(uint64(l))
- }
- }
- if len(m.Statuses) > 0 {
- for k, v := range m.Statuses {
- _ = k
- _ = v
- mapEntrySize := 1 + sovAdmin(uint64(k)) + 1 + sovAdmin(uint64(v))
- n += mapEntrySize + 1 + sovAdmin(uint64(mapEntrySize))
- }
- }
+ l = m.Response.Size()
+ n += 1 + l + sovAdmin(uint64(l))
return n
}
-func (m *JobsRequest) Size() (n int) {
+func (m *SettingsRequest) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ if len(m.Keys) > 0 {
+ for _, s := range m.Keys {
+ l = len(s)
+ n += 1 + l + sovAdmin(uint64(l))
+ }
+ }
+ if m.UnredactedValues {
+ n += 2
+ }
+ return n
+}
+
+func (m *SettingsResponse) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ if len(m.KeyValues) > 0 {
+ for k, v := range m.KeyValues {
+ _ = k
+ _ = v
+ l = v.Size()
+ mapEntrySize := 1 + len(k) + sovAdmin(uint64(len(k))) + 1 + l + sovAdmin(uint64(l))
+ n += mapEntrySize + 1 + sovAdmin(uint64(mapEntrySize))
+ }
+ }
+ return n
+}
+
+func (m *SettingsResponse_Value) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ l = len(m.Value)
+ if l > 0 {
+ n += 1 + l + sovAdmin(uint64(l))
+ }
+ l = len(m.Type)
+ if l > 0 {
+ n += 1 + l + sovAdmin(uint64(l))
+ }
+ l = len(m.Description)
+ if l > 0 {
+ n += 1 + l + sovAdmin(uint64(l))
+ }
+ if m.Public {
+ n += 2
+ }
+ return n
+}
+
+func (m *HealthRequest) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ if m.Ready {
+ n += 2
+ }
+ return n
+}
+
+func (m *HealthResponse) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ return n
+}
+
+func (m *LivenessRequest) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ return n
+}
+
+func (m *LivenessResponse) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ if len(m.Livenesses) > 0 {
+ for _, e := range m.Livenesses {
+ l = e.Size()
+ n += 1 + l + sovAdmin(uint64(l))
+ }
+ }
+ if len(m.Statuses) > 0 {
+ for k, v := range m.Statuses {
+ _ = k
+ _ = v
+ mapEntrySize := 1 + sovAdmin(uint64(k)) + 1 + sovAdmin(uint64(v))
+ n += mapEntrySize + 1 + sovAdmin(uint64(mapEntrySize))
+ }
+ }
+ return n
+}
+
+func (m *JobsRequest) Size() (n int) {
if m == nil {
return 0
}
@@ -11568,6 +12246,463 @@ func (m *DecommissionStatusResponse_Status) Unmarshal(dAtA []byte) error {
}
return nil
}
+func (m *AckPendingVersionRequest) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= (uint64(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: AckPendingVersionRequest: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: AckPendingVersionRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ case 1:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= (int(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ postIndex := iNdEx + msglen
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ if m.Version == nil {
+ m.Version = &roachpb.Version{}
+ }
+ if err := m.Version.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ iNdEx = postIndex
+ default:
+ iNdEx = preIndex
+ skippy, err := skipAdmin(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if skippy < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
+func (m *AckPendingVersionResponse) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= (uint64(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: AckPendingVersionResponse: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: AckPendingVersionResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ default:
+ iNdEx = preIndex
+ skippy, err := skipAdmin(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if skippy < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
+func (m *EveryNodeRequestUnion) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= (uint64(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: EveryNodeRequestUnion: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: EveryNodeRequestUnion: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ case 1:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field AckPendingVersion", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= (int(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ postIndex := iNdEx + msglen
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ v := &AckPendingVersionRequest{}
+ if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ m.Value = &EveryNodeRequestUnion_AckPendingVersion{v}
+ iNdEx = postIndex
+ default:
+ iNdEx = preIndex
+ skippy, err := skipAdmin(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if skippy < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
+func (m *EveryNodeResponseUnion) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= (uint64(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: EveryNodeResponseUnion: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: EveryNodeResponseUnion: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ case 1:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field AckPendingVersion", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= (int(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ postIndex := iNdEx + msglen
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ v := &AckPendingVersionResponse{}
+ if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ m.Value = &EveryNodeResponseUnion_AckPendingVersion{v}
+ iNdEx = postIndex
+ default:
+ iNdEx = preIndex
+ skippy, err := skipAdmin(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if skippy < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
+func (m *EveryNodeRequest) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= (uint64(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: EveryNodeRequest: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: EveryNodeRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ case 1:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field Request", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= (int(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ postIndex := iNdEx + msglen
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ if err := m.Request.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ iNdEx = postIndex
+ default:
+ iNdEx = preIndex
+ skippy, err := skipAdmin(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if skippy < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
+func (m *EveryNodeResponse) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= (uint64(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: EveryNodeResponse: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: EveryNodeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ case 1:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field Response", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowAdmin
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= (int(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ postIndex := iNdEx + msglen
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ if err := m.Response.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ iNdEx = postIndex
+ default:
+ iNdEx = preIndex
+ skippy, err := skipAdmin(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if skippy < 0 {
+ return ErrInvalidLengthAdmin
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
func (m *SettingsRequest) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
@@ -15675,268 +16810,280 @@ var (
ErrIntOverflowAdmin = fmt.Errorf("proto: integer overflow")
)
-func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_d35ae12d295a7e1d) }
-
-var fileDescriptor_admin_d35ae12d295a7e1d = []byte{
- // 4157 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0xcf, 0x73, 0x1b, 0xd7,
- 0x79, 0x5a, 0x80, 0xf8, 0xf5, 0x11, 0x20, 0xc1, 0x27, 0x8a, 0x02, 0x21, 0x85, 0xa0, 0x57, 0x51,
- 0x4c, 0xcb, 0x0e, 0x20, 0x51, 0xb2, 0x9d, 0xca, 0x76, 0x52, 0x82, 0xe0, 0x28, 0x90, 0x25, 0xd9,
- 0x5a, 0x4a, 0xf2, 0x44, 0xa9, 0xbd, 0x5d, 0x60, 0x9f, 0xc0, 0x0d, 0x17, 0xbb, 0xcb, 0xdd, 0x05,
- 0x65, 0xda, 0x4d, 0xa6, 0x49, 0xda, 0x4e, 0x0f, 0x9d, 0x8e, 0x9b, 0xf4, 0xe6, 0x53, 0x7b, 0x68,
- 0x0f, 0xbd, 0xd4, 0x39, 0xf6, 0xd0, 0xb3, 0xa7, 0x97, 0xa4, 0xd3, 0x76, 0x26, 0xbd, 0x30, 0x2d,
- 0xdd, 0x53, 0xfa, 0x1f, 0x74, 0xa6, 0xd3, 0xce, 0xfb, 0xb5, 0xfb, 0x00, 0x42, 0x4b, 0x80, 0x8a,
- 0x73, 0x01, 0xf6, 0x7d, 0xdf, 0x7b, 0xdf, 0xfb, 0xde, 0xf7, 0xbe, 0xf7, 0xfd, 0x7a, 0x0f, 0x2e,
- 0x04, 0xd8, 0xdf, 0xc7, 0x7e, 0x83, 0xfd, 0x79, 0x9d, 0x86, 0x61, 0xf6, 0x2d, 0xa7, 0xee, 0xf9,
- 0x6e, 0xe8, 0xa2, 0xe5, 0xae, 0xdb, 0xdd, 0xf5, 0x5d, 0xa3, 0xbb, 0x53, 0x67, 0xf8, 0xba, 0xe8,
- 0x56, 0xad, 0x74, 0x5d, 0xe7, 0x89, 0xd5, 0x6b, 0x7c, 0xe4, 0x3a, 0xd8, 0xeb, 0xd0, 0x3f, 0x36,
- 0xa8, 0xba, 0xf4, 0x3d, 0xb7, 0x13, 0x34, 0xc8, 0x8f, 0xd7, 0xa1, 0x7f, 0x1c, 0x7e, 0x71, 0x74,
- 0xa6, 0x20, 0x34, 0xc2, 0x81, 0xc0, 0x5e, 0x08, 0x42, 0xd7, 0x37, 0x7a, 0xb8, 0x81, 0x9d, 0x9e,
- 0x45, 0x49, 0xf6, 0xf7, 0xbb, 0x5d, 0x8e, 0xbc, 0xbc, 0xbb, 0xdf, 0xd8, 0xdd, 0xe7, 0xe3, 0xc5,
- 0x87, 0xd7, 0x69, 0xd8, 0xd6, 0x3e, 0x76, 0x70, 0x20, 0x68, 0xac, 0x3e, 0xab, 0x9b, 0xdb, 0xe3,
- 0x3d, 0x56, 0xc2, 0xa0, 0xd1, 0x35, 0x42, 0xc3, 0x76, 0x7b, 0x8d, 0xee, 0x8e, 0xe1, 0x87, 0x3a,
- 0x6f, 0x71, 0x7c, 0x65, 0x10, 0x5a, 0x76, 0xa3, 0x8f, 0x43, 0xdf, 0xea, 0xf2, 0x3f, 0x8e, 0x59,
- 0xec, 0xb9, 0x3d, 0x97, 0x7e, 0x36, 0xc8, 0x97, 0x58, 0x53, 0xcf, 0x75, 0x7b, 0x36, 0x6e, 0x18,
- 0x9e, 0xd5, 0x30, 0x1c, 0xc7, 0x0d, 0x8d, 0xd0, 0x72, 0x1d, 0xc1, 0x4f, 0x8d, 0x63, 0x69, 0xab,
- 0x33, 0x78, 0xd2, 0x08, 0xad, 0x3e, 0x0e, 0x42, 0xa3, 0xef, 0xb1, 0x0e, 0x2a, 0x82, 0x72, 0xcb,
- 0x08, 0x8d, 0x8e, 0x11, 0xe0, 0x40, 0xc3, 0x7b, 0x03, 0x1c, 0x84, 0xea, 0x35, 0x58, 0x90, 0x60,
- 0x81, 0xe7, 0x3a, 0x01, 0x46, 0x17, 0xa1, 0x60, 0x0a, 0x60, 0x45, 0x59, 0x4d, 0xaf, 0x15, 0xb4,
- 0x18, 0xa0, 0xde, 0x80, 0x25, 0x31, 0xa4, 0x85, 0x43, 0xc3, 0xb2, 0x05, 0x31, 0x54, 0x85, 0xbc,
- 0xe8, 0x56, 0x51, 0x56, 0x95, 0xb5, 0x82, 0x16, 0xb5, 0xd5, 0xcf, 0xd2, 0x70, 0xfe, 0xd8, 0x30,
- 0x3e, 0xdf, 0x23, 0xc8, 0xf6, 0x7c, 0xc3, 0x09, 0xd9, 0x64, 0xb3, 0xeb, 0xdf, 0xa8, 0x3f, 0x53,
- 0x13, 0xea, 0xcf, 0xa0, 0x51, 0xbf, 0x45, 0x08, 0x34, 0x67, 0x3e, 0x3f, 0xac, 0x9d, 0xd1, 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, 0xe8, 0xb0, 0x56, 0x6c, 0x45, 0x88, 0x76,
- 0x4b, 0x2b, 0xc6, 0xdd, 0xda, 0x26, 0xba, 0x0d, 0xb3, 0x44, 0x03, 0x75, 0xa6, 0x94, 0x95, 0x99,
- 0x55, 0x65, 0x6d, 0x76, 0xfd, 0x92, 0xc4, 0x34, 0x43, 0xd4, 0x99, 0xb6, 0xd6, 0x1f, 0xbb, 0x0e,
- 0xde, 0xa4, 0x10, 0xce, 0x1f, 0x7c, 0x14, 0x41, 0xd0, 0xfb, 0xb0, 0x20, 0xd1, 0xd2, 0x6d, 0xbc,
- 0x8f, 0xed, 0x4a, 0x66, 0x55, 0x59, 0x9b, 0x5b, 0xbf, 0x96, 0x20, 0x86, 0x98, 0xe6, 0xc0, 0xa7,
- 0x5a, 0x70, 0x87, 0x0c, 0xd4, 0xe6, 0x63, 0xca, 0x14, 0x50, 0x7d, 0x03, 0x32, 0x54, 0x32, 0x08,
- 0xc1, 0xcc, 0x20, 0xc0, 0x3e, 0xdf, 0x17, 0xfa, 0x8d, 0x56, 0x00, 0x3c, 0xdf, 0xda, 0xb7, 0x6c,
- 0xdc, 0x8b, 0xc5, 0x13, 0x43, 0xd4, 0x5b, 0x70, 0xf6, 0x01, 0x11, 0xd6, 0xe4, 0xdb, 0x8c, 0x16,
- 0x21, 0x43, 0xe5, 0x5b, 0x49, 0x51, 0x04, 0x6b, 0xa8, 0x7f, 0x9c, 0x87, 0xc5, 0x61, 0x4a, 0x7c,
- 0xe7, 0xb7, 0x47, 0x76, 0xfe, 0xd5, 0x84, 0x25, 0x8f, 0x23, 0x30, 0x76, 0xdb, 0x1f, 0x41, 0xae,
- 0xeb, 0xda, 0x83, 0xbe, 0xc3, 0xd6, 0x34, 0xbb, 0xfe, 0xda, 0xb4, 0x54, 0x37, 0xe9, 0x70, 0x4e,
- 0x56, 0x10, 0x43, 0x0f, 0x21, 0x67, 0x39, 0x26, 0xfe, 0x10, 0x07, 0x95, 0xf4, 0xe9, 0xb8, 0x6d,
- 0x93, 0xe1, 0x82, 0x2c, 0xa7, 0x45, 0xb4, 0xd4, 0x37, 0x9c, 0x1e, 0x51, 0x81, 0x81, 0x13, 0x52,
- 0x6d, 0x4a, 0x6b, 0x40, 0x41, 0x9b, 0x04, 0x82, 0x6e, 0xc0, 0x52, 0xd7, 0xc7, 0x46, 0x88, 0x75,
- 0xa6, 0xcd, 0xc4, 0x92, 0xe1, 0x3e, 0x76, 0x42, 0xaa, 0x27, 0x05, 0x6d, 0x91, 0x61, 0xe9, 0x8c,
- 0xdb, 0x02, 0x37, 0xaa, 0xa4, 0xd9, 0xdf, 0xb8, 0x92, 0xe6, 0x7e, 0x53, 0x4a, 0x7a, 0xfc, 0x18,
- 0xe6, 0x27, 0x39, 0x86, 0xcf, 0xa5, 0xdb, 0xd5, 0x7f, 0x54, 0x20, 0xcb, 0xb6, 0x99, 0x0c, 0x27,
- 0x06, 0x42, 0x0c, 0x27, 0xdf, 0x04, 0x16, 0x1e, 0x78, 0x42, 0x8d, 0xe9, 0x37, 0xd1, 0x7b, 0x67,
- 0x60, 0xdb, 0x54, 0xbd, 0x89, 0xa1, 0xc8, 0x6b, 0x51, 0x1b, 0x5d, 0x22, 0x4b, 0x78, 0x62, 0x0c,
- 0xec, 0x50, 0xdf, 0x37, 0xec, 0x01, 0xa6, 0xdb, 0x58, 0x20, 0x0c, 0x53, 0xe0, 0x23, 0x02, 0x43,
- 0xd7, 0xe1, 0x5c, 0x0f, 0x3b, 0x98, 0xc9, 0x42, 0xc7, 0x1f, 0x7a, 0x3e, 0x0e, 0x02, 0xcb, 0x75,
- 0xc4, 0x3e, 0xc6, 0xc8, 0xad, 0x08, 0x87, 0x96, 0x20, 0xbb, 0x63, 0x99, 0x26, 0x76, 0xe8, 0x16,
- 0xe6, 0x35, 0xde, 0xaa, 0x7e, 0xa6, 0x40, 0x86, 0xea, 0xd3, 0x58, 0xfe, 0x97, 0x20, 0x3b, 0x70,
- 0xac, 0xbd, 0x01, 0x5b, 0x41, 0x5e, 0xe3, 0x2d, 0x54, 0x86, 0x74, 0x80, 0xf7, 0x98, 0x9d, 0xd3,
- 0xc8, 0x27, 0xe9, 0xc9, 0x14, 0x9c, 0xb3, 0xcc, 0x5b, 0xd4, 0x09, 0x58, 0x3e, 0xee, 0x86, 0x31,
- 0x83, 0x31, 0x00, 0x55, 0x20, 0x47, 0x5c, 0xa8, 0xe5, 0xf4, 0x38, 0x5b, 0xa2, 0x49, 0xa4, 0x64,
- 0xf5, 0x3d, 0xdb, 0xea, 0x5a, 0x21, 0x55, 0x91, 0xbc, 0x16, 0xb5, 0xd5, 0x2d, 0x58, 0x88, 0xb4,
- 0xf4, 0x39, 0xcc, 0xc9, 0x67, 0x69, 0x40, 0x32, 0x1d, 0x6e, 0x4c, 0x46, 0x0e, 0x92, 0x72, 0xec,
- 0x20, 0x5d, 0x82, 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, 0x72,
- 0xfe, 0x02, 0x6e, 0xf5, 0x2f, 0xcb, 0xea, 0xcf, 0x62, 0x8a, 0xba, 0x88, 0x29, 0xea, 0x77, 0x1f,
- 0x6d, 0x6e, 0x52, 0x16, 0xf9, 0x91, 0x62, 0x23, 0x91, 0x09, 0xa5, 0xbe, 0x15, 0x04, 0x96, 0xd3,
- 0xd3, 0x09, 0xdd, 0xa0, 0x92, 0xa1, 0xd6, 0xe4, 0x77, 0x4e, 0xb2, 0x26, 0x43, 0xab, 0xad, 0xdf,
- 0x65, 0x24, 0xee, 0xb9, 0x26, 0xe6, 0xe4, 0x8b, 0xfd, 0x18, 0x14, 0x10, 0xab, 0x61, 0x78, 0x9e,
- 0xef, 0x7e, 0x68, 0xf5, 0x89, 0xe9, 0x30, 0xad, 0x60, 0x57, 0xef, 0x1c, 0x84, 0x38, 0xa0, 0x1b,
- 0x36, 0xa3, 0x2d, 0x4a, 0xd8, 0x96, 0x15, 0xec, 0x36, 0x09, 0xae, 0xfa, 0x1e, 0xcc, 0x4a, 0x84,
- 0xd1, 0x25, 0xc8, 0x51, 0x61, 0x58, 0x26, 0xdb, 0x9a, 0x26, 0x1c, 0x1d, 0xd6, 0xb2, 0x04, 0xd5,
- 0x6e, 0x69, 0x59, 0x82, 0x6a, 0x9b, 0x44, 0xac, 0xd8, 0xf7, 0x5d, 0x5f, 0xef, 0xe3, 0x20, 0x30,
- 0x7a, 0x62, 0xb3, 0x8a, 0x14, 0x78, 0x97, 0xc1, 0xd4, 0x25, 0x58, 0xbc, 0xe7, 0x3a, 0xc7, 0x76,
- 0x5f, 0xfd, 0xb9, 0x02, 0xe7, 0x46, 0x10, 0x7c, 0x3b, 0xbf, 0x03, 0x0b, 0x24, 0x82, 0xd1, 0x03,
- 0xec, 0x5b, 0x38, 0xd0, 0x99, 0xd4, 0x15, 0x2a, 0xf5, 0xaf, 0x4f, 0x25, 0x2a, 0x6d, 0x9e, 0xd0,
- 0xd9, 0xa6, 0x64, 0x28, 0x02, 0x7d, 0x17, 0x90, 0xe5, 0x84, 0xd8, 0x77, 0x0c, 0x5b, 0x1f, 0x04,
- 0x98, 0xd3, 0x4e, 0x9d, 0x86, 0x76, 0x59, 0x10, 0x7a, 0x18, 0x30, 0x8c, 0x3a, 0x07, 0xc5, 0x87,
- 0x01, 0xf6, 0xa3, 0x15, 0xfe, 0x00, 0x4a, 0xbc, 0xcd, 0x17, 0xd6, 0x86, 0x0c, 0x31, 0x51, 0xc2,
- 0xe7, 0x25, 0x4d, 0x38, 0x34, 0x90, 0xb6, 0x84, 0x2a, 0x51, 0x0a, 0x55, 0x15, 0x66, 0x08, 0x90,
- 0x9c, 0x21, 0x02, 0x90, 0xcc, 0x40, 0xd4, 0x56, 0x7f, 0xac, 0x40, 0x69, 0x6b, 0x1f, 0x3b, 0xf1,
- 0x89, 0x13, 0xc6, 0x4d, 0x91, 0x8c, 0xdb, 0x05, 0x28, 0x84, 0x86, 0xdf, 0xc3, 0x21, 0xd9, 0x6b,
- 0x76, 0x2e, 0xf2, 0x0c, 0xd0, 0x36, 0xc9, 0x31, 0xb4, 0xad, 0xbe, 0xc5, 0x8e, 0x43, 0x46, 0x63,
- 0x0d, 0xf4, 0x32, 0x2c, 0x0c, 0x1c, 0x1f, 0x9b, 0x46, 0x37, 0xc4, 0xa6, 0x8e, 0xe9, 0x14, 0xf4,
- 0x58, 0xe4, 0xb5, 0x72, 0x8c, 0x60, 0x53, 0xab, 0xbf, 0x4e, 0xc1, 0x9c, 0xe0, 0x82, 0xcb, 0xe1,
- 0x2e, 0x64, 0xf9, 0x20, 0x26, 0x88, 0x46, 0x82, 0x20, 0x86, 0x87, 0xb2, 0xa6, 0x70, 0xfb, 0x8c,
- 0x48, 0xf5, 0x4f, 0x53, 0x90, 0xa1, 0x70, 0xd4, 0x84, 0x42, 0x14, 0xfb, 0x72, 0x8d, 0xa9, 0xd6,
- 0x59, 0x74, 0x5c, 0x17, 0xd1, 0x71, 0xfd, 0x81, 0xe8, 0xd1, 0xcc, 0x13, 0x32, 0x9f, 0xfc, 0xaa,
- 0xa6, 0x68, 0xf1, 0x30, 0x62, 0x06, 0x28, 0x5d, 0x5d, 0x72, 0x03, 0x05, 0x0a, 0x79, 0x40, 0xc4,
- 0xf5, 0x92, 0x2c, 0x2e, 0x16, 0x35, 0x16, 0x8f, 0x0e, 0x6b, 0xf9, 0x07, 0x4c, 0x64, 0x2d, 0x49,
- 0x78, 0xeb, 0x40, 0x0c, 0x8c, 0xeb, 0x87, 0xe4, 0xc0, 0x5b, 0x26, 0x73, 0xf0, 0xcd, 0xf9, 0xa3,
- 0xc3, 0xda, 0xac, 0x26, 0xe0, 0xed, 0x96, 0x36, 0x1b, 0x75, 0x6a, 0x9b, 0x64, 0x87, 0x2c, 0xe7,
- 0x89, 0xcb, 0xed, 0x2e, 0xfd, 0x26, 0x53, 0x32, 0x23, 0x4e, 0x88, 0x90, 0x33, 0x5c, 0x64, 0x53,
- 0x3e, 0xa4, 0x40, 0x32, 0x25, 0x43, 0xb7, 0x4d, 0xf5, 0xef, 0x14, 0x28, 0x6f, 0xe3, 0xf0, 0x61,
- 0x9b, 0x44, 0xcb, 0x62, 0xd7, 0xbf, 0x03, 0xb0, 0x8b, 0x0f, 0x98, 0x7b, 0x12, 0x22, 0xbf, 0x99,
- 0x20, 0xf2, 0x51, 0x02, 0xf5, 0xb7, 0xf1, 0x01, 0xf5, 0x63, 0xc1, 0x96, 0x13, 0xfa, 0x07, 0x5a,
- 0x61, 0x57, 0xb4, 0xab, 0x6f, 0xc2, 0xdc, 0x30, 0x92, 0xf8, 0x99, 0x5d, 0x7c, 0xc0, 0x35, 0x8c,
- 0x7c, 0x12, 0x1d, 0x62, 0x9e, 0x91, 0xc8, 0xb2, 0xa8, 0xb1, 0xc6, 0xcd, 0xd4, 0x37, 0x14, 0xf5,
- 0x2c, 0x2c, 0x48, 0x73, 0xb1, 0x1d, 0x56, 0xbf, 0x06, 0xe5, 0x5b, 0xa3, 0x2b, 0x40, 0x30, 0xb3,
- 0x8b, 0x0f, 0x44, 0x4a, 0x42, 0xbf, 0xd5, 0x9f, 0xa7, 0x60, 0xe1, 0xd6, 0xe8, 0x68, 0xf4, 0xfb,
- 0x63, 0xd6, 0xfa, 0x46, 0xc2, 0x5a, 0x8f, 0x51, 0x18, 0x59, 0x2c, 0x57, 0x35, 0x69, 0xc9, 0x4f,
- 0x20, 0xc3, 0x9c, 0x7a, 0xb4, 0x2e, 0x45, 0x5a, 0x17, 0xba, 0x05, 0x45, 0xdb, 0x08, 0x42, 0x7d,
- 0xe0, 0x99, 0x46, 0x88, 0x4d, 0x6e, 0x5b, 0x26, 0xd3, 0xc2, 0x59, 0x32, 0xf2, 0x21, 0x1b, 0x58,
- 0xf5, 0x26, 0x10, 0xed, 0xb7, 0x65, 0xd1, 0xce, 0xae, 0xaf, 0x4f, 0xb5, 0x50, 0x4a, 0x5a, 0xde,
- 0x8e, 0x32, 0xcc, 0x6d, 0xda, 0x83, 0x20, 0xc4, 0xbe, 0xb0, 0x60, 0x9f, 0x2a, 0x30, 0x1f, 0x81,
- 0xb8, 0x84, 0x5f, 0x01, 0xe8, 0x32, 0x50, 0xec, 0x1c, 0x4a, 0x47, 0x87, 0xb5, 0x02, 0xef, 0xd8,
- 0x6e, 0x69, 0x05, 0xde, 0xa1, 0x6d, 0x12, 0x53, 0x11, 0x9f, 0x01, 0xec, 0x10, 0x33, 0x6a, 0xf2,
- 0xc8, 0xa4, 0x1c, 0x21, 0xb6, 0x18, 0x1c, 0x7d, 0x1d, 0x10, 0x26, 0x46, 0xd5, 0xf3, 0xad, 0x00,
- 0x47, 0xbd, 0x59, 0xc4, 0xb5, 0x10, 0x63, 0x78, 0x77, 0xd5, 0x83, 0x62, 0xcb, 0x37, 0x2c, 0x47,
- 0x68, 0xc9, 0x25, 0x28, 0x79, 0x3e, 0x5e, 0xbf, 0x7a, 0x4d, 0xef, 0x1b, 0xfe, 0x2e, 0x0d, 0x0b,
- 0xd3, 0x6b, 0x19, 0xad, 0xc8, 0x80, 0x77, 0x29, 0x8c, 0x18, 0xcc, 0x60, 0x67, 0x10, 0x9a, 0xee,
- 0x53, 0x47, 0xc4, 0x72, 0xa2, 0x8d, 0x96, 0x21, 0x6f, 0xba, 0xba, 0x49, 0x68, 0x72, 0x73, 0x96,
- 0x33, 0x5d, 0x3a, 0xc5, 0xed, 0x99, 0x7c, 0xaa, 0x9c, 0x56, 0x7f, 0xa6, 0x40, 0x89, 0x4f, 0x19,
- 0x87, 0x1e, 0x56, 0xc0, 0x86, 0x90, 0x90, 0x88, 0xad, 0x0c, 0xac, 0xa0, 0xc5, 0x21, 0xe8, 0x26,
- 0x2c, 0x53, 0xac, 0xee, 0xe3, 0x3e, 0x03, 0xe9, 0x96, 0x63, 0x5a, 0x5d, 0x23, 0x74, 0x7d, 0xca,
- 0xc0, 0x8c, 0x76, 0xde, 0x64, 0x24, 0x39, 0xbe, 0x2d, 0xd0, 0xe8, 0x9b, 0x70, 0x61, 0x74, 0xac,
- 0x88, 0x83, 0x49, 0x6c, 0xc6, 0xc2, 0xb6, 0xe5, 0xe1, 0xd1, 0xad, 0xb8, 0xc3, 0xed, 0x99, 0xbc,
- 0x52, 0x4e, 0xa9, 0x1f, 0xc1, 0x72, 0x0b, 0x77, 0xdd, 0x3e, 0x0d, 0x12, 0x5c, 0x67, 0x9b, 0x96,
- 0x43, 0x84, 0xcc, 0xde, 0x87, 0x3c, 0xf7, 0xf3, 0xec, 0xb4, 0x64, 0x9a, 0xcd, 0xa3, 0xc3, 0x5a,
- 0x8e, 0x39, 0xfa, 0xe0, 0x7f, 0x0e, 0x6b, 0xd7, 0x7b, 0x56, 0xb8, 0x33, 0xe8, 0xd4, 0xbb, 0x6e,
- 0xbf, 0x11, 0x69, 0x97, 0xd9, 0x89, 0xbf, 0x1b, 0xde, 0x6e, 0xaf, 0x41, 0xbf, 0xbc, 0x4e, 0x9d,
- 0x07, 0x08, 0x39, 0x16, 0x20, 0x04, 0xea, 0xbf, 0x2a, 0x70, 0x56, 0x9e, 0xfc, 0xb7, 0x33, 0x2d,
- 0xfa, 0x00, 0x16, 0xb8, 0x91, 0xee, 0xe3, 0x7e, 0x07, 0xfb, 0xc1, 0x8e, 0xe5, 0xd1, 0xbd, 0x19,
- 0x4e, 0x5b, 0x76, 0xf7, 0xeb, 0xa2, 0x7c, 0x23, 0x62, 0x38, 0x12, 0xbc, 0x45, 0x43, 0xb8, 0xa8,
- 0xca, 0x8c, 0x56, 0x0c, 0x57, 0xff, 0x2c, 0x0d, 0xd5, 0x71, 0x32, 0xe5, 0x4a, 0xf1, 0x18, 0xb2,
- 0xac, 0xe8, 0xc4, 0xd3, 0xd0, 0x37, 0x93, 0xca, 0x1a, 0xcf, 0x24, 0x53, 0x67, 0x4d, 0xe1, 0xec,
- 0x18, 0xc5, 0xea, 0x5f, 0xa4, 0x20, 0xcb, 0x10, 0xe8, 0xf1, 0x70, 0x8c, 0x96, 0x69, 0x6e, 0xc4,
- 0x31, 0xda, 0x69, 0x45, 0x28, 0x42, 0xbb, 0xf3, 0x90, 0xb3, 0x02, 0xdd, 0xb6, 0xf6, 0xa3, 0x3c,
- 0xc2, 0x0a, 0xee, 0x58, 0xfb, 0xf8, 0x78, 0x28, 0x9d, 0x1e, 0x13, 0x4a, 0xdf, 0x07, 0x90, 0x04,
- 0x3f, 0x73, 0x5a, 0xc1, 0x4b, 0x44, 0x68, 0xb2, 0x20, 0x4e, 0x59, 0x86, 0x9d, 0x5b, 0xd1, 0x56,
- 0x35, 0x98, 0xdf, 0xc6, 0x21, 0xb1, 0x24, 0x41, 0x82, 0xc7, 0x18, 0x09, 0x5b, 0xb8, 0x8b, 0x48,
- 0x8d, 0x86, 0x2d, 0xcc, 0xe2, 0xaa, 0xff, 0x9c, 0xa2, 0x9e, 0x94, 0x13, 0xe5, 0x1b, 0xab, 0x4f,
- 0xef, 0x49, 0x87, 0x08, 0x9c, 0xe8, 0x5c, 0x76, 0xc7, 0x3a, 0x97, 0x82, 0x70, 0x2e, 0xe3, 0x92,
- 0xd3, 0x55, 0x98, 0x95, 0x8d, 0x42, 0x9a, 0xa2, 0x64, 0x10, 0x49, 0xf4, 0xbc, 0x41, 0xc7, 0xb6,
- 0xba, 0xdc, 0xa8, 0xf1, 0x56, 0xd5, 0x9d, 0xc0, 0xc3, 0xdc, 0x1a, 0xf6, 0x30, 0xd7, 0xa6, 0x59,
- 0xec, 0x31, 0x07, 0x73, 0x19, 0x4a, 0xdf, 0xc6, 0x86, 0x1d, 0xee, 0x88, 0x5d, 0x5a, 0x84, 0x8c,
- 0x8f, 0x0d, 0x93, 0xcd, 0x98, 0xd7, 0x58, 0x83, 0xf8, 0x21, 0xd1, 0x8d, 0xc7, 0x04, 0x0b, 0x30,
- 0x7f, 0x87, 0xd7, 0x60, 0x85, 0x6b, 0xfa, 0x75, 0x0a, 0xca, 0x31, 0x8c, 0xef, 0xcf, 0x3b, 0x00,
- 0xa2, 0x56, 0x1b, 0xed, 0xcf, 0x4b, 0x27, 0xea, 0x9d, 0x20, 0x23, 0xea, 0x1f, 0x31, 0x09, 0xf4,
- 0x13, 0x05, 0xf2, 0xec, 0xe0, 0x61, 0x71, 0x98, 0x93, 0xb2, 0xb5, 0x51, 0x86, 0xf8, 0x11, 0x16,
- 0xdb, 0xfd, 0x06, 0xa1, 0xff, 0xa3, 0x5f, 0x9d, 0xee, 0x64, 0x46, 0x7c, 0x54, 0x3d, 0x28, 0x0d,
- 0xd1, 0x95, 0xb7, 0x2d, 0xc3, 0xb6, 0xad, 0x2d, 0x6f, 0xdb, 0xdc, 0xfa, 0xf5, 0x13, 0x65, 0x40,
- 0x26, 0x11, 0xdc, 0xf3, 0xd3, 0x27, 0x6d, 0x9c, 0x07, 0xb3, 0xb7, 0xdd, 0x4e, 0x20, 0x6d, 0x1b,
- 0xcb, 0x0a, 0x14, 0x39, 0x2b, 0x58, 0x92, 0xac, 0x1e, 0xad, 0x27, 0xb0, 0x16, 0xba, 0xce, 0x95,
- 0x36, 0x4d, 0x59, 0xa9, 0xc9, 0xe2, 0xdb, 0xb3, 0xeb, 0xb4, 0x72, 0xcf, 0xaa, 0xf8, 0x75, 0x12,
- 0x60, 0x33, 0xad, 0x56, 0xff, 0x2d, 0x0b, 0x45, 0x36, 0x25, 0xdf, 0xda, 0x2d, 0x98, 0x21, 0xbd,
- 0xf8, 0xa6, 0xbe, 0x9c, 0xb0, 0x09, 0xf2, 0x30, 0xd2, 0xe0, 0xdb, 0x4a, 0x87, 0x57, 0xff, 0x37,
- 0x03, 0xe9, 0xdb, 0x6e, 0x07, 0x2d, 0x41, 0x8a, 0x9b, 0xcd, 0x74, 0x33, 0x7b, 0x74, 0x58, 0x4b,
- 0xb5, 0x5b, 0x5a, 0xca, 0x32, 0x4f, 0x79, 0xc2, 0xe4, 0x2c, 0x6c, 0x66, 0x38, 0x0b, 0x43, 0xfb,
- 0x30, 0x37, 0x54, 0xe3, 0x62, 0x59, 0x7f, 0xa9, 0xf9, 0xce, 0xd1, 0x61, 0xad, 0x24, 0x17, 0xb9,
- 0x88, 0xdb, 0xbb, 0x39, 0x91, 0x66, 0x04, 0x7b, 0x76, 0x74, 0xbd, 0x40, 0x68, 0x7b, 0x9d, 0x7a,
- 0xbb, 0xa5, 0x95, 0xe4, 0x1a, 0x59, 0x20, 0x6d, 0x47, 0x76, 0x68, 0x3b, 0x6e, 0x42, 0x8e, 0x95,
- 0x0d, 0x4d, 0x5a, 0xa5, 0x49, 0x8e, 0x4d, 0x67, 0x68, 0x5c, 0x2a, 0x06, 0x90, 0xb1, 0x41, 0x68,
- 0xf8, 0x64, 0x6c, 0x7e, 0xd2, 0xb1, 0x7c, 0x00, 0x7a, 0x13, 0xf2, 0x4f, 0x2c, 0xc7, 0x0a, 0x76,
- 0xb0, 0x59, 0x29, 0x4c, 0x38, 0x38, 0x1a, 0x41, 0x46, 0xf7, 0x5d, 0xd3, 0x7a, 0x62, 0x61, 0xb3,
- 0x02, 0x93, 0x8e, 0x16, 0x23, 0x48, 0x60, 0xf9, 0xc4, 0x37, 0x68, 0x01, 0x4b, 0xef, 0xba, 0x7d,
- 0xcf, 0xc6, 0x64, 0x09, 0xb3, 0xab, 0xca, 0x5a, 0x4a, 0x5b, 0x10, 0x98, 0x4d, 0x81, 0x20, 0xfa,
- 0x4d, 0x4b, 0x18, 0x95, 0x22, 0x33, 0xbe, 0xb4, 0x81, 0xee, 0xc3, 0xd9, 0x1d, 0xab, 0xb7, 0xf3,
- 0xd4, 0x20, 0xa1, 0x6f, 0x9c, 0x66, 0x96, 0x26, 0xe4, 0x06, 0x45, 0x83, 0x23, 0x0c, 0xf1, 0x48,
- 0x31, 0x49, 0x13, 0x77, 0xad, 0xbe, 0x61, 0x57, 0xe6, 0xe8, 0xa4, 0xe5, 0x08, 0xd1, 0x62, 0x70,
- 0x74, 0x19, 0xe6, 0xfc, 0x81, 0x43, 0xa3, 0x40, 0xbe, 0xb1, 0xf3, 0xb4, 0x67, 0x89, 0x43, 0x79,
- 0x54, 0x70, 0x11, 0x0a, 0x71, 0x9d, 0xb8, 0xcc, 0xd2, 0xd7, 0x08, 0xa0, 0x22, 0x28, 0xdf, 0x71,
- 0xbb, 0xec, 0xfa, 0x48, 0x98, 0xd2, 0xff, 0x53, 0x60, 0x41, 0x02, 0x46, 0x55, 0x98, 0x82, 0x2d,
- 0x80, 0x13, 0x14, 0xe9, 0x8f, 0x11, 0x88, 0x20, 0xc2, 0xcb, 0x45, 0xd4, 0xaa, 0x7f, 0xae, 0x40,
- 0x5e, 0x60, 0xd1, 0x0b, 0x50, 0x24, 0x18, 0xdb, 0x0a, 0x0f, 0xf4, 0xd8, 0xf9, 0xcc, 0x0a, 0xd8,
- 0xdb, 0xf8, 0x80, 0xac, 0x3c, 0xea, 0x12, 0x9b, 0xb5, 0x82, 0x56, 0x12, 0x50, 0xe6, 0x33, 0xab,
- 0x90, 0xb7, 0x8d, 0xd0, 0x0a, 0x07, 0x26, 0x33, 0x36, 0x8a, 0x16, 0xb5, 0x89, 0x54, 0x6c, 0xd7,
- 0xe9, 0x31, 0xe4, 0x0c, 0x45, 0xc6, 0x00, 0xb5, 0x09, 0xf3, 0x9a, 0xe1, 0xf4, 0xf0, 0x1d, 0xb7,
- 0x27, 0x6c, 0xdc, 0x32, 0xe4, 0x59, 0x4d, 0x51, 0x98, 0x09, 0x2d, 0x47, 0xdb, 0x72, 0x51, 0x24,
- 0x25, 0x99, 0x3f, 0xf5, 0xbf, 0xd3, 0x50, 0x8e, 0x89, 0x44, 0x0e, 0x49, 0x54, 0x3a, 0x98, 0xf3,
- 0x48, 0xf2, 0x9f, 0xa3, 0x83, 0xc7, 0xd6, 0x3a, 0xfe, 0x49, 0x01, 0x78, 0xd7, 0xc7, 0x61, 0x78,
- 0xd0, 0x76, 0x9e, 0xb8, 0x44, 0x78, 0x3c, 0xd1, 0xa4, 0x99, 0x81, 0x10, 0x1e, 0x87, 0x11, 0xeb,
- 0x42, 0x16, 0xe2, 0xe0, 0xa7, 0x0c, 0xcd, 0xc4, 0x96, 0x73, 0xf0, 0x53, 0x8a, 0xba, 0x04, 0x25,
- 0xc3, 0x34, 0xb1, 0xa9, 0xf3, 0xe0, 0x8d, 0x9b, 0xb6, 0x22, 0x05, 0x6a, 0x0c, 0x86, 0x5e, 0x84,
- 0x79, 0x1f, 0xf7, 0xdd, 0x7d, 0xa9, 0x1b, 0x33, 0x71, 0x73, 0x1c, 0x2c, 0x3a, 0x2e, 0x41, 0xd6,
- 0xc7, 0x46, 0x10, 0x15, 0x8d, 0x79, 0x0b, 0x55, 0x20, 0x67, 0xb2, 0xeb, 0x10, 0x6e, 0x89, 0x44,
- 0xb3, 0xfa, 0x33, 0x45, 0x14, 0x6e, 0x6e, 0x43, 0x86, 0x2e, 0x90, 0x17, 0x6d, 0xea, 0x27, 0xfa,
- 0x2b, 0x21, 0x2c, 0x59, 0x46, 0x8c, 0x04, 0x7a, 0x1f, 0x66, 0x3d, 0x2a, 0x21, 0x9d, 0x56, 0x52,
- 0x58, 0xe0, 0xf2, 0xda, 0x34, 0x82, 0x8f, 0x05, 0x2c, 0x42, 0x02, 0x2f, 0x82, 0xf0, 0xa4, 0x6a,
- 0x0d, 0xca, 0xf7, 0x07, 0xd8, 0x3f, 0x78, 0xd7, 0x36, 0x1c, 0xc9, 0x2d, 0xee, 0x11, 0x98, 0x88,
- 0xd9, 0x68, 0x43, 0xf5, 0x60, 0x41, 0xea, 0xc9, 0xf5, 0xe2, 0xbb, 0x70, 0xc1, 0xb4, 0x82, 0x30,
- 0xd8, 0xb3, 0x75, 0x6f, 0xe7, 0x20, 0xb0, 0xba, 0x86, 0xad, 0xd3, 0xee, 0xba, 0x67, 0x1b, 0x0e,
- 0xcf, 0xaa, 0x2f, 0x1e, 0x1d, 0xd6, 0x2a, 0x2d, 0x2b, 0x08, 0xb7, 0xef, 0xdf, 0x79, 0x97, 0xf7,
- 0x8a, 0x49, 0x55, 0x38, 0x81, 0x63, 0x18, 0x75, 0x99, 0x5d, 0xb8, 0x92, 0x91, 0xbe, 0xd5, 0x19,
- 0x84, 0x71, 0xde, 0xa5, 0xfe, 0x3d, 0x40, 0xe5, 0x38, 0x8e, 0x33, 0xe5, 0x41, 0x49, 0xd4, 0xdf,
- 0x99, 0xe8, 0xd8, 0xa9, 0xdf, 0x3a, 0xe1, 0x52, 0x76, 0x1c, 0xad, 0xe8, 0xb6, 0x96, 0x88, 0x4c,
- 0x8e, 0x75, 0x8b, 0xa6, 0x84, 0x40, 0x7d, 0x28, 0x4a, 0xd7, 0x4b, 0xe2, 0x76, 0xad, 0x75, 0x9a,
- 0x09, 0xe3, 0x2b, 0xa7, 0xa1, 0xd8, 0x7a, 0x36, 0xbe, 0x72, 0x0a, 0xaa, 0x9f, 0x2a, 0x00, 0x71,
- 0x3f, 0xa2, 0xb0, 0x2c, 0xb3, 0xe3, 0x1b, 0xc6, 0x5b, 0x68, 0x03, 0xb2, 0xfc, 0xee, 0x2c, 0x35,
- 0xed, 0xdd, 0x19, 0x1f, 0x48, 0x8b, 0x24, 0xec, 0xca, 0x2c, 0xd8, 0xb3, 0xd9, 0x79, 0xe1, 0x45,
- 0x12, 0x0a, 0xdd, 0xbe, 0x7f, 0x47, 0x2b, 0xb0, 0x0e, 0xdb, 0x7b, 0xf6, 0xed, 0x99, 0x7c, 0xba,
- 0x3c, 0x53, 0xfd, 0x93, 0x34, 0x14, 0x68, 0x9d, 0x99, 0x8a, 0xe6, 0x73, 0x05, 0x2a, 0x43, 0x89,
- 0x96, 0xde, 0x39, 0xd0, 0xe3, 0x74, 0x8f, 0xc8, 0xe9, 0xbd, 0xd3, 0xc8, 0x29, 0x9a, 0xa1, 0xae,
- 0x49, 0x19, 0x5b, 0xf3, 0x80, 0x46, 0x99, 0x26, 0x13, 0xdd, 0xeb, 0xa7, 0x8d, 0x51, 0x17, 0xfd,
- 0x31, 0x34, 0xd1, 0x57, 0x61, 0x4e, 0xbe, 0x44, 0x8c, 0xca, 0xcc, 0xc5, 0x78, 0x6f, 0xda, 0x26,
- 0xfa, 0x16, 0x80, 0xe9, 0xbb, 0x9e, 0x87, 0x4d, 0xdd, 0x60, 0x59, 0xe5, 0x24, 0x5e, 0xb5, 0xc0,
- 0xc7, 0x6c, 0x84, 0xd5, 0x5b, 0xb0, 0xfc, 0xcc, 0x25, 0x8d, 0x09, 0x91, 0x87, 0xca, 0x92, 0x69,
- 0x29, 0xda, 0xad, 0xfe, 0x30, 0x05, 0x45, 0x59, 0x7f, 0x51, 0x08, 0xec, 0xed, 0x80, 0x7c, 0x2a,
- 0xde, 0x79, 0xde, 0x53, 0x11, 0xef, 0xc4, 0x50, 0x2e, 0x18, 0x0a, 0x68, 0xf5, 0x63, 0x98, 0x1b,
- 0xee, 0x32, 0x26, 0x3d, 0xdb, 0x1e, 0x4e, 0xcf, 0xde, 0x7a, 0x2e, 0x8d, 0x18, 0x92, 0x81, 0x12,
- 0xbf, 0x0f, 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, 0xe6, 0x77, 0x1e, 0xce, 0xdd, 0xa5, 0x2f, 0x74, 0xee, 0xe2, 0xd0, 0x30, 0xe3, 0xa2, 0xb4,
- 0xfa, 0xef, 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, 0xf0,
- 0x28, 0x9a, 0x58, 0x4a, 0xf8, 0x7e, 0x9a, 0x82, 0xb3, 0x5b, 0xce, 0xde, 0x00, 0x0f, 0x30, 0xf5,
- 0x8e, 0xc2, 0xc5, 0x7d, 0x99, 0x25, 0x27, 0xe6, 0x3e, 0xa3, 0xe0, 0x8e, 0x35, 0xd0, 0x07, 0x52,
- 0x1c, 0x46, 0x2f, 0xa1, 0x9a, 0x9b, 0x47, 0x87, 0xb5, 0x1c, 0xe5, 0x8a, 0xce, 0x79, 0x63, 0xaa,
- 0x39, 0xf9, 0xb8, 0x38, 0x98, 0xbb, 0x02, 0x0b, 0xc1, 0xae, 0xe5, 0xe9, 0xc1, 0x8e, 0x3b, 0xb0,
- 0x4d, 0x9d, 0x71, 0xc0, 0xea, 0x24, 0xf3, 0x04, 0xb1, 0x4d, 0xe1, 0xf7, 0x09, 0x58, 0xfd, 0x9b,
- 0x14, 0x2c, 0x0e, 0x4b, 0x85, 0xef, 0xf7, 0xfd, 0x38, 0xc4, 0x61, 0xdb, 0xfd, 0x7a, 0xd2, 0x8d,
- 0xd6, 0x18, 0x0a, 0x75, 0xf1, 0x60, 0x24, 0x8a, 0x8d, 0xfe, 0x41, 0x81, 0x1c, 0x07, 0x7e, 0xa9,
- 0x52, 0x7f, 0x6b, 0x24, 0x42, 0xbd, 0x9c, 0x74, 0x0b, 0xea, 0x1b, 0x5d, 0x4c, 0x23, 0x2e, 0x11,
- 0x8f, 0xc6, 0xa9, 0x52, 0x5a, 0x4a, 0x95, 0xd4, 0x73, 0x70, 0x76, 0x73, 0xc7, 0xf0, 0xc3, 0x4d,
- 0x96, 0xa4, 0x8a, 0x13, 0xf3, 0x18, 0x16, 0x87, 0xc1, 0x5c, 0x7c, 0x4d, 0xc8, 0xf1, 0x74, 0x96,
- 0x8b, 0x4f, 0x95, 0x98, 0x08, 0x83, 0xba, 0x78, 0x3c, 0x47, 0x07, 0x6f, 0xb3, 0x87, 0x08, 0xd1,
- 0x1b, 0x1d, 0x86, 0xbb, 0xf2, 0x36, 0x2c, 0x8d, 0x7f, 0x75, 0x82, 0x66, 0x21, 0xf7, 0xf0, 0xde,
- 0xdb, 0xf7, 0xde, 0x79, 0xef, 0x5e, 0xf9, 0x0c, 0x69, 0x6c, 0xde, 0x79, 0xb8, 0xfd, 0x60, 0x4b,
- 0x2b, 0x2b, 0xa8, 0x08, 0xf9, 0xd6, 0xc6, 0x83, 0x8d, 0xe6, 0xc6, 0xf6, 0x56, 0x39, 0x85, 0x0a,
- 0x90, 0x79, 0xb0, 0xd1, 0xbc, 0xb3, 0x55, 0x4e, 0xaf, 0xff, 0xa4, 0x0a, 0x99, 0x0d, 0xb3, 0x6f,
- 0x39, 0x28, 0x84, 0x0c, 0xbd, 0x8a, 0x45, 0x2f, 0x9e, 0x7c, 0x59, 0x4b, 0x17, 0x59, 0x5d, 0x9b,
- 0xf4, 0x56, 0x57, 0xad, 0xfc, 0xe8, 0x5f, 0xfe, 0xeb, 0xa7, 0x29, 0x84, 0xca, 0x0d, 0x9d, 0xbe,
- 0x86, 0x6c, 0xec, 0x5f, 0x6b, 0xd0, 0xdb, 0x5d, 0xf4, 0x47, 0x0a, 0x14, 0xa2, 0xd7, 0x79, 0xe8,
- 0xe5, 0x09, 0x5e, 0xc5, 0x45, 0xd3, 0xbf, 0x32, 0x59, 0x67, 0xce, 0xc2, 0x45, 0xca, 0xc2, 0x12,
- 0x5a, 0x94, 0x58, 0x88, 0x1e, 0xfc, 0xa1, 0xbf, 0x52, 0x60, 0x7e, 0xe4, 0xd9, 0x1d, 0xba, 0x36,
- 0xcd, 0x13, 0x3d, 0xc6, 0xd2, 0xfa, 0xf4, 0xaf, 0xfa, 0xd4, 0x17, 0x29, 0x63, 0x2f, 0xa0, 0xda,
- 0x38, 0xc6, 0x1a, 0x1f, 0x8b, 0xcf, 0xef, 0xa3, 0xbf, 0x55, 0xa0, 0x28, 0x3f, 0xb9, 0x42, 0xf5,
- 0x89, 0xdf, 0x66, 0x31, 0xee, 0x1a, 0x53, 0xbe, 0xe5, 0x52, 0x5f, 0xa3, 0xac, 0x5d, 0x45, 0xf5,
- 0x13, 0x58, 0x6b, 0x50, 0x57, 0x1e, 0x34, 0x3e, 0xa6, 0xff, 0x94, 0x53, 0x88, 0xdf, 0x11, 0xa0,
- 0x57, 0x26, 0x7c, 0x6e, 0xc0, 0xb8, 0x9c, 0xee, 0x71, 0x82, 0xfa, 0x26, 0xe5, 0xf1, 0x35, 0x74,
- 0x63, 0x3a, 0x1e, 0x1b, 0xec, 0x9d, 0xca, 0x5f, 0x2a, 0x50, 0x1a, 0x7a, 0x9a, 0x81, 0x92, 0x84,
- 0x34, 0xee, 0x75, 0x47, 0xf5, 0xea, 0xe4, 0x03, 0x38, 0xcb, 0xab, 0x94, 0xe5, 0x2a, 0xaa, 0x48,
- 0x2c, 0x3b, 0xae, 0xc3, 0x18, 0xa4, 0x4c, 0x7c, 0x08, 0x59, 0xf6, 0x1a, 0x00, 0xad, 0x4d, 0xf0,
- 0x60, 0x80, 0xf1, 0xf1, 0xd2, 0xc4, 0x4f, 0x0b, 0xd4, 0x65, 0xca, 0xc0, 0x59, 0xb4, 0x20, 0x31,
- 0xc0, 0xad, 0x1c, 0x39, 0x8f, 0xd1, 0x4d, 0x75, 0xe2, 0x79, 0x1c, 0xbd, 0x3b, 0x4f, 0x3c, 0x8f,
- 0xc7, 0x2f, 0xbf, 0xf9, 0x79, 0x54, 0x65, 0x1e, 0x06, 0x16, 0xd9, 0xae, 0x9b, 0xca, 0x15, 0xf4,
- 0x43, 0x05, 0x0a, 0xb7, 0x26, 0x62, 0xe3, 0xd6, 0x34, 0x6c, 0x1c, 0xbb, 0x1a, 0x1e, 0x2b, 0x0a,
- 0xc6, 0x06, 0xfa, 0x03, 0xc8, 0xf1, 0x8b, 0x5e, 0x94, 0x24, 0xdb, 0xe1, 0x8b, 0xe4, 0xea, 0x95,
- 0x49, 0xba, 0xf2, 0xc9, 0xab, 0x74, 0xf2, 0x45, 0x84, 0xa4, 0xc9, 0xf9, 0x85, 0x32, 0xfa, 0x43,
- 0x05, 0xf2, 0xe2, 0x9e, 0x01, 0x5d, 0x99, 0xe8, 0x32, 0x82, 0x31, 0xf0, 0xf2, 0x14, 0x17, 0x17,
- 0xea, 0x05, 0xca, 0xc1, 0x39, 0x74, 0x56, 0xe2, 0x20, 0x10, 0xb3, 0xfe, 0x58, 0x81, 0x2c, 0xbb,
- 0x9e, 0x48, 0x54, 0xc3, 0xa1, 0x8b, 0x8e, 0x44, 0x35, 0x1c, 0xb9, 0xeb, 0xf8, 0x2a, 0x9d, 0x7c,
- 0x65, 0x48, 0xf6, 0x3b, 0xb4, 0xcb, 0xe3, 0x02, 0xca, 0xf1, 0x4f, 0x2a, 0x08, 0x51, 0xaf, 0x4f,
- 0x14, 0xc4, 0xc8, 0xbd, 0x49, 0xa2, 0x20, 0x46, 0xaf, 0x2f, 0xc6, 0x0a, 0x42, 0xdc, 0x8e, 0x20,
- 0x0f, 0x66, 0x6e, 0xbb, 0x9d, 0x00, 0x7d, 0xed, 0xc4, 0x5a, 0x3c, 0x9b, 0xf9, 0xc5, 0x09, 0x6b,
- 0xf6, 0xea, 0x79, 0x3a, 0xeb, 0x02, 0x9a, 0x97, 0x66, 0xfd, 0x1e, 0x99, 0x89, 0x1c, 0xc3, 0xa8,
- 0xce, 0x98, 0xa8, 0xff, 0xa3, 0x35, 0xce, 0x44, 0xfd, 0x3f, 0x56, 0xba, 0x1c, 0xeb, 0x16, 0xa3,
- 0xf2, 0x25, 0x65, 0x23, 0xaa, 0xb6, 0x24, 0xb2, 0x31, 0x5a, 0x22, 0x4a, 0x64, 0xe3, 0x58, 0x95,
- 0x68, 0x2c, 0x1b, 0xb4, 0x4a, 0xe4, 0x91, 0x89, 0x3f, 0x80, 0x0c, 0x7d, 0x65, 0x90, 0x18, 0x9a,
- 0xc8, 0x0f, 0x24, 0x12, 0x43, 0x93, 0xa1, 0x67, 0x0d, 0xea, 0x99, 0xab, 0x0a, 0x7a, 0x0a, 0x45,
- 0xf9, 0x72, 0x3a, 0xd1, 0xb1, 0x8e, 0xb9, 0xe3, 0xaf, 0xbe, 0x7a, 0xaa, 0x5b, 0x6f, 0xf5, 0x0c,
- 0x39, 0x61, 0xe8, 0x78, 0x07, 0x74, 0x63, 0x4a, 0x7a, 0xcf, 0xc9, 0xc5, 0x5f, 0x2b, 0x90, 0x17,
- 0x95, 0xc1, 0xc4, 0x13, 0x36, 0x52, 0x39, 0x4e, 0x3c, 0x61, 0xa3, 0xa5, 0x46, 0xf5, 0x2d, 0xba,
- 0xc5, 0xaf, 0x0f, 0x9d, 0x30, 0x9a, 0x9a, 0xd8, 0x6e, 0xef, 0xf1, 0x2a, 0x5a, 0x19, 0x03, 0x6e,
- 0x7c, 0x2c, 0x12, 0xa2, 0xef, 0x93, 0x08, 0xad, 0x3c, 0x9a, 0xd5, 0xa2, 0xf5, 0xa9, 0x52, 0x60,
- 0xc6, 0xf4, 0xf5, 0x53, 0xa4, 0xcd, 0x91, 0xa9, 0xba, 0x38, 0x12, 0x65, 0xe8, 0xa6, 0xcc, 0xce,
- 0xa7, 0x0a, 0x2c, 0x6c, 0xd8, 0xf6, 0x70, 0x96, 0x8b, 0xae, 0x4e, 0x91, 0x10, 0x33, 0x16, 0xaf,
- 0x4d, 0x9d, 0x42, 0xab, 0x2f, 0x50, 0x06, 0x2f, 0xa0, 0x65, 0x89, 0x41, 0x96, 0xf5, 0x8a, 0x3c,
- 0x1a, 0x7d, 0xa2, 0x40, 0x51, 0x4e, 0x4a, 0x12, 0xd5, 0x7c, 0x4c, 0x52, 0x93, 0x18, 0x3f, 0x8e,
- 0xcb, 0x76, 0xd4, 0x1a, 0x65, 0x6a, 0x19, 0x9d, 0x97, 0xfd, 0x1b, 0xe9, 0xc8, 0x53, 0x19, 0x12,
- 0x7e, 0x15, 0xe5, 0x24, 0x31, 0x91, 0xa5, 0x31, 0x59, 0x7a, 0x22, 0x4b, 0xe3, 0xb2, 0x4f, 0xf5,
- 0x12, 0x65, 0xe9, 0x2b, 0xaa, 0x1c, 0x7b, 0x61, 0xd6, 0x51, 0xa7, 0xca, 0x76, 0x53, 0xb9, 0xd2,
- 0xbc, 0xf2, 0xf9, 0x7f, 0xae, 0x9c, 0xf9, 0xfc, 0x68, 0x45, 0xf9, 0xc5, 0xd1, 0x8a, 0xf2, 0xcb,
- 0xa3, 0x15, 0xe5, 0x3f, 0x8e, 0x56, 0x94, 0x4f, 0xbe, 0x58, 0x39, 0xf3, 0x8b, 0x2f, 0x56, 0xce,
- 0xfc, 0xf2, 0x8b, 0x95, 0x33, 0x8f, 0xf3, 0x62, 0x96, 0x4e, 0x96, 0x16, 0xec, 0xae, 0xff, 0x7f,
- 0x00, 0x00, 0x00, 0xff, 0xff, 0x81, 0x85, 0xbd, 0x83, 0xf7, 0x35, 0x00, 0x00,
+func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_a5fd0c6a410748d8) }
+
+var fileDescriptor_admin_a5fd0c6a410748d8 = []byte{
+ // 4342 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x4d, 0x73, 0x1b, 0x47,
+ 0x76, 0x1c, 0x80, 0x20, 0x80, 0x47, 0x80, 0x04, 0x5a, 0x14, 0x05, 0x42, 0x5a, 0x92, 0x1e, 0xad,
+ 0xd6, 0xb4, 0xec, 0x05, 0x24, 0x4a, 0xb6, 0x37, 0xb2, 0xbd, 0x1b, 0x82, 0x60, 0xc9, 0x90, 0x25,
+ 0x59, 0x1a, 0x4a, 0x72, 0xad, 0x36, 0xf6, 0x64, 0x80, 0x69, 0x82, 0xb3, 0x1c, 0xcc, 0x0c, 0x67,
+ 0x06, 0x94, 0x69, 0x67, 0xb7, 0xb2, 0xbb, 0xf9, 0xac, 0x4a, 0xa5, 0x9c, 0x6c, 0x6e, 0x3e, 0x25,
+ 0x87, 0xe4, 0x90, 0x4b, 0xbc, 0xc7, 0x1c, 0x72, 0x76, 0xe5, 0xb2, 0x9b, 0x4a, 0x52, 0xb5, 0xb9,
+ 0x70, 0x13, 0x3a, 0xa7, 0xcd, 0x3f, 0x48, 0x25, 0x95, 0x54, 0x7f, 0xcd, 0x34, 0x40, 0x68, 0x04,
+ 0x50, 0xeb, 0xbd, 0x00, 0xd3, 0xef, 0x75, 0xbf, 0x7e, 0xfd, 0xfa, 0xf5, 0xeb, 0xf7, 0x5e, 0x3f,
+ 0x38, 0x1f, 0x60, 0xff, 0x00, 0xfb, 0x75, 0xf6, 0xe7, 0xb5, 0xeb, 0x86, 0xd9, 0xb3, 0x9c, 0x9a,
+ 0xe7, 0xbb, 0xa1, 0x8b, 0x96, 0x3a, 0x6e, 0x67, 0xcf, 0x77, 0x8d, 0xce, 0x6e, 0x8d, 0xe1, 0x6b,
+ 0xa2, 0x5b, 0xb5, 0xd2, 0x71, 0x9d, 0x1d, 0xab, 0x5b, 0xff, 0xc8, 0x75, 0xb0, 0xd7, 0xa6, 0x7f,
+ 0x6c, 0x50, 0x75, 0xf1, 0xbb, 0x6e, 0x3b, 0xa8, 0x93, 0x1f, 0xaf, 0x4d, 0xff, 0x04, 0x9c, 0x12,
+ 0xf2, 0xda, 0xf5, 0x1e, 0x0e, 0x0d, 0xd3, 0x08, 0x0d, 0x0e, 0xbf, 0x30, 0xcc, 0x41, 0x10, 0x1a,
+ 0x61, 0x5f, 0x8c, 0x3a, 0x1f, 0x84, 0xae, 0x6f, 0x74, 0x71, 0x1d, 0x3b, 0x5d, 0x8b, 0x4e, 0xd5,
+ 0x3b, 0xe8, 0x74, 0x38, 0xf2, 0xd2, 0xde, 0x41, 0x7d, 0xef, 0x80, 0x8f, 0x17, 0x1f, 0x5e, 0xbb,
+ 0x6e, 0x5b, 0x07, 0xd8, 0xc1, 0x81, 0xa0, 0xb1, 0xfa, 0xb4, 0x6e, 0x6e, 0x97, 0xf7, 0x58, 0x0e,
+ 0x83, 0x7a, 0xc7, 0x08, 0x0d, 0xdb, 0xed, 0xd6, 0x3b, 0xbb, 0x86, 0x1f, 0xea, 0xbc, 0xc5, 0xf1,
+ 0x95, 0x7e, 0x68, 0xd9, 0x84, 0x71, 0xdf, 0xea, 0xf0, 0x3f, 0x8e, 0x59, 0xe8, 0xba, 0x5d, 0x97,
+ 0x7e, 0xd6, 0xc9, 0x97, 0x58, 0x53, 0xd7, 0x75, 0xbb, 0x36, 0xae, 0x1b, 0x9e, 0x55, 0x37, 0x1c,
+ 0xc7, 0x0d, 0x8d, 0xd0, 0x72, 0x1d, 0xc1, 0xcf, 0x0a, 0xc7, 0xd2, 0x56, 0xbb, 0xbf, 0x53, 0x0f,
+ 0xad, 0x1e, 0x0e, 0x42, 0xa3, 0xe7, 0xb1, 0x0e, 0x2a, 0x82, 0x52, 0xd3, 0x08, 0x8d, 0xb6, 0x11,
+ 0xe0, 0x40, 0xc3, 0xfb, 0x7d, 0x1c, 0x84, 0xea, 0x55, 0x28, 0x4b, 0xb0, 0xc0, 0x73, 0x9d, 0x00,
+ 0xa3, 0x0b, 0x90, 0x37, 0x05, 0xb0, 0xa2, 0xac, 0xa6, 0xd7, 0xf2, 0x5a, 0x0c, 0x50, 0xaf, 0xc3,
+ 0xa2, 0x18, 0xd2, 0xc4, 0xa1, 0x61, 0xd9, 0x82, 0x18, 0xaa, 0x42, 0x4e, 0x74, 0xab, 0x28, 0xab,
+ 0xca, 0x5a, 0x5e, 0x8b, 0xda, 0xea, 0x67, 0x69, 0x38, 0x77, 0x62, 0x18, 0x9f, 0xef, 0x11, 0xcc,
+ 0x74, 0x7d, 0xc3, 0x09, 0xd9, 0x64, 0xb3, 0xeb, 0xdf, 0xa8, 0x3d, 0x55, 0x43, 0x6a, 0x4f, 0xa1,
+ 0x51, 0xbb, 0x49, 0x08, 0x34, 0xa6, 0x3f, 0x3f, 0x5a, 0x99, 0xd2, 0x38, 0x35, 0xb4, 0x02, 0xb3,
+ 0xa1, 0xd1, 0xb6, 0xb1, 0xee, 0x18, 0x3d, 0x1c, 0x54, 0x52, 0x74, 0x25, 0x40, 0x41, 0x77, 0x09,
+ 0x04, 0xbd, 0x0a, 0x45, 0x13, 0x07, 0x1d, 0xdf, 0xf2, 0x42, 0xd7, 0xd7, 0x2d, 0xb3, 0x92, 0x5e,
+ 0x55, 0xd6, 0xd2, 0x8d, 0xd2, 0xf1, 0xd1, 0x4a, 0xa1, 0x19, 0x21, 0x5a, 0x4d, 0xad, 0x10, 0x77,
+ 0x6b, 0x99, 0xe8, 0x16, 0xcc, 0x12, 0xcd, 0xd4, 0x99, 0xb2, 0x56, 0xa6, 0x57, 0x95, 0xb5, 0xd9,
+ 0xf5, 0x8b, 0x12, 0xd3, 0x0c, 0x51, 0x63, 0x5a, 0x5c, 0x7b, 0xec, 0x3a, 0x78, 0x93, 0x42, 0x38,
+ 0x7f, 0xf0, 0x51, 0x04, 0x41, 0xef, 0x43, 0x59, 0xa2, 0xa5, 0xdb, 0xf8, 0x00, 0xdb, 0x95, 0xcc,
+ 0xaa, 0xb2, 0x36, 0xb7, 0x7e, 0x35, 0x41, 0x0c, 0x31, 0xcd, 0xbe, 0x4f, 0xb5, 0xe0, 0x36, 0x19,
+ 0xa8, 0xcd, 0xc7, 0x94, 0x29, 0xa0, 0xfa, 0x06, 0x64, 0xa8, 0x64, 0x10, 0x82, 0xe9, 0x7e, 0x80,
+ 0x7d, 0xbe, 0x2f, 0xf4, 0x1b, 0x2d, 0x03, 0x78, 0xbe, 0x75, 0x60, 0xd9, 0xb8, 0x1b, 0x8b, 0x27,
+ 0x86, 0xa8, 0x37, 0xe1, 0xcc, 0x03, 0x22, 0xac, 0xf1, 0xb7, 0x19, 0x2d, 0x40, 0x86, 0xca, 0xb7,
+ 0x92, 0xa2, 0x08, 0xd6, 0x50, 0x7f, 0x3f, 0x07, 0x0b, 0x83, 0x94, 0xf8, 0xce, 0x6f, 0x0f, 0xed,
+ 0xfc, 0xab, 0x09, 0x4b, 0x1e, 0x45, 0x60, 0xe4, 0xb6, 0x3f, 0x82, 0x6c, 0xc7, 0xb5, 0xfb, 0x3d,
+ 0x87, 0xad, 0x69, 0x76, 0xfd, 0xb5, 0x49, 0xa9, 0x6e, 0xd2, 0xe1, 0x9c, 0xac, 0x20, 0x86, 0x1e,
+ 0x42, 0xd6, 0x72, 0x4c, 0xfc, 0x21, 0x0e, 0x2a, 0xe9, 0xd3, 0x71, 0xdb, 0x22, 0xc3, 0x05, 0x59,
+ 0x4e, 0x8b, 0x68, 0xa9, 0x6f, 0x38, 0x5d, 0xa2, 0x02, 0x7d, 0x27, 0xa4, 0xda, 0x94, 0xd6, 0x80,
+ 0x82, 0x36, 0x09, 0x04, 0x5d, 0x87, 0xc5, 0x8e, 0x8f, 0x8d, 0x10, 0xeb, 0x4c, 0x9b, 0x89, 0x25,
+ 0xc3, 0x3d, 0xec, 0x84, 0x54, 0x4f, 0xf2, 0xda, 0x02, 0xc3, 0xd2, 0x19, 0xb7, 0x05, 0x6e, 0x58,
+ 0x49, 0x67, 0x7e, 0xe5, 0x4a, 0x9a, 0xfd, 0x55, 0x29, 0xe9, 0xc9, 0x63, 0x98, 0x1b, 0xe7, 0x18,
+ 0x3e, 0x97, 0x6e, 0x57, 0xff, 0x41, 0x81, 0x19, 0xb6, 0xcd, 0x64, 0x38, 0x31, 0x10, 0x62, 0x38,
+ 0xf9, 0x26, 0xb0, 0xf0, 0xd0, 0x13, 0x6a, 0x4c, 0xbf, 0x89, 0xde, 0x3b, 0x7d, 0xdb, 0xa6, 0xea,
+ 0x4d, 0x0c, 0x45, 0x4e, 0x8b, 0xda, 0xe8, 0x22, 0x59, 0xc2, 0x8e, 0xd1, 0xb7, 0x43, 0xfd, 0xc0,
+ 0xb0, 0xfb, 0x98, 0x6e, 0x63, 0x9e, 0x30, 0x4c, 0x81, 0x8f, 0x08, 0x0c, 0x5d, 0x83, 0xb3, 0x5d,
+ 0xec, 0x60, 0x26, 0x0b, 0x1d, 0x7f, 0xe8, 0xf9, 0x38, 0x08, 0x2c, 0xd7, 0x11, 0xfb, 0x18, 0x23,
+ 0xb7, 0x22, 0x1c, 0x5a, 0x84, 0x99, 0x5d, 0xcb, 0x34, 0xb1, 0x43, 0xb7, 0x30, 0xa7, 0xf1, 0x56,
+ 0xf5, 0x33, 0x05, 0x32, 0x54, 0x9f, 0x46, 0xf2, 0xbf, 0x08, 0x33, 0x7d, 0xc7, 0xda, 0xef, 0xb3,
+ 0x15, 0xe4, 0x34, 0xde, 0x42, 0x25, 0x48, 0x07, 0x78, 0x9f, 0xd9, 0x39, 0x8d, 0x7c, 0x92, 0x9e,
+ 0x4c, 0xc1, 0x39, 0xcb, 0xbc, 0x45, 0x2f, 0x01, 0xcb, 0xc7, 0x9d, 0x30, 0x66, 0x30, 0x06, 0xa0,
+ 0x0a, 0x64, 0xc9, 0x15, 0x6a, 0x39, 0x5d, 0xce, 0x96, 0x68, 0x12, 0x29, 0x59, 0x3d, 0xcf, 0xb6,
+ 0x3a, 0x56, 0x48, 0x55, 0x24, 0xa7, 0x45, 0x6d, 0x75, 0x0b, 0xca, 0x91, 0x96, 0x3e, 0x87, 0x39,
+ 0xf9, 0x2c, 0x0d, 0x48, 0xa6, 0xc3, 0x8d, 0xc9, 0xd0, 0x41, 0x52, 0x4e, 0x1c, 0xa4, 0x8b, 0x50,
+ 0xf4, 0x31, 0x61, 0xc5, 0xe0, 0x5d, 0x52, 0xb4, 0x4b, 0x81, 0x03, 0x59, 0xa7, 0xaf, 0x00, 0x38,
+ 0xae, 0x29, 0x88, 0x30, 0x41, 0xe5, 0x09, 0x84, 0xa1, 0x37, 0x20, 0x43, 0xce, 0x5f, 0xc0, 0xad,
+ 0xfe, 0x25, 0x59, 0xfd, 0x99, 0x4f, 0x51, 0x13, 0x3e, 0x45, 0xed, 0xce, 0xa3, 0xcd, 0x4d, 0xca,
+ 0x22, 0x3f, 0x52, 0x6c, 0x24, 0x32, 0xa1, 0xd8, 0xb3, 0x82, 0xc0, 0x72, 0xba, 0x3a, 0xa1, 0x1b,
+ 0x54, 0x32, 0xd4, 0x9a, 0xfc, 0xc6, 0xb3, 0xac, 0xc9, 0xc0, 0x6a, 0x6b, 0x77, 0x18, 0x89, 0xbb,
+ 0xae, 0x89, 0x39, 0xf9, 0x42, 0x2f, 0x06, 0x05, 0xc4, 0x6a, 0x18, 0x9e, 0xe7, 0xbb, 0x1f, 0x5a,
+ 0x3d, 0x62, 0x3a, 0x4c, 0x2b, 0xd8, 0xd3, 0xdb, 0x87, 0x21, 0x0e, 0xe8, 0x86, 0x4d, 0x6b, 0x0b,
+ 0x12, 0xb6, 0x69, 0x05, 0x7b, 0x0d, 0x82, 0xab, 0xbe, 0x07, 0xb3, 0x12, 0x61, 0x74, 0x11, 0xb2,
+ 0x54, 0x18, 0x96, 0xc9, 0xb6, 0xa6, 0x01, 0xc7, 0x47, 0x2b, 0x33, 0x04, 0xd5, 0x6a, 0x6a, 0x33,
+ 0x04, 0xd5, 0x32, 0x89, 0x58, 0xb1, 0xef, 0xbb, 0xbe, 0xde, 0xc3, 0x41, 0x60, 0x74, 0xc5, 0x66,
+ 0x15, 0x28, 0xf0, 0x0e, 0x83, 0xa9, 0x8b, 0xb0, 0x70, 0xd7, 0x75, 0x4e, 0xec, 0xbe, 0xfa, 0x53,
+ 0x05, 0xce, 0x0e, 0x21, 0xf8, 0x76, 0x7e, 0x1b, 0xca, 0xc4, 0x83, 0xd1, 0x03, 0xec, 0x5b, 0x38,
+ 0xd0, 0x99, 0xd4, 0x15, 0x2a, 0xf5, 0xaf, 0x4f, 0x24, 0x2a, 0x6d, 0x9e, 0xd0, 0xd9, 0xa6, 0x64,
+ 0x28, 0x02, 0x7d, 0x07, 0x90, 0xe5, 0x84, 0xd8, 0x77, 0x0c, 0x5b, 0xef, 0x07, 0x98, 0xd3, 0x4e,
+ 0x9d, 0x86, 0x76, 0x49, 0x10, 0x7a, 0x18, 0x30, 0x8c, 0x3a, 0x07, 0x85, 0x87, 0x01, 0xf6, 0xa3,
+ 0x15, 0x7e, 0x1f, 0x8a, 0xbc, 0xcd, 0x17, 0xd6, 0x82, 0x0c, 0x31, 0x51, 0xe2, 0xce, 0x4b, 0x9a,
+ 0x70, 0x60, 0x20, 0x6d, 0x09, 0x55, 0xa2, 0x14, 0xaa, 0x2a, 0x4c, 0x13, 0x20, 0x39, 0x43, 0x04,
+ 0x20, 0x99, 0x81, 0xa8, 0xad, 0xfe, 0x48, 0x81, 0xe2, 0xd6, 0x01, 0x76, 0xe2, 0x13, 0x27, 0x8c,
+ 0x9b, 0x22, 0x19, 0xb7, 0xf3, 0x90, 0x0f, 0x0d, 0xbf, 0x8b, 0x43, 0xb2, 0xd7, 0xec, 0x5c, 0xe4,
+ 0x18, 0xa0, 0x65, 0x92, 0x63, 0x68, 0x5b, 0x3d, 0x8b, 0x1d, 0x87, 0x8c, 0xc6, 0x1a, 0xe8, 0x65,
+ 0x28, 0xf7, 0x1d, 0x1f, 0x9b, 0x46, 0x27, 0xc4, 0xa6, 0x8e, 0xe9, 0x14, 0xf4, 0x58, 0xe4, 0xb4,
+ 0x52, 0x8c, 0x60, 0x53, 0xab, 0xbf, 0x4c, 0xc1, 0x9c, 0xe0, 0x82, 0xcb, 0xe1, 0x0e, 0xcc, 0xf0,
+ 0x41, 0x4c, 0x10, 0xf5, 0x04, 0x41, 0x0c, 0x0e, 0x65, 0x4d, 0x71, 0xed, 0x33, 0x22, 0xd5, 0x3f,
+ 0x4a, 0x41, 0x86, 0xc2, 0x51, 0x03, 0xf2, 0x91, 0xef, 0xcb, 0x35, 0xa6, 0x5a, 0x63, 0xde, 0x71,
+ 0x4d, 0x78, 0xc7, 0xb5, 0x07, 0xa2, 0x47, 0x23, 0x47, 0xc8, 0x7c, 0xf2, 0x8b, 0x15, 0x45, 0x8b,
+ 0x87, 0x11, 0x33, 0x40, 0xe9, 0xea, 0xd2, 0x35, 0x90, 0xa7, 0x90, 0x07, 0x44, 0x5c, 0x2f, 0xc9,
+ 0xe2, 0x62, 0x5e, 0x63, 0xe1, 0xf8, 0x68, 0x25, 0xf7, 0x80, 0x89, 0xac, 0x29, 0x09, 0x6f, 0x1d,
+ 0x88, 0x81, 0x71, 0xfd, 0x90, 0x1c, 0x78, 0xcb, 0x64, 0x17, 0x7c, 0x63, 0xfe, 0xf8, 0x68, 0x65,
+ 0x56, 0x13, 0xf0, 0x56, 0x53, 0x9b, 0x8d, 0x3a, 0xb5, 0x4c, 0xb2, 0x43, 0x96, 0xb3, 0xe3, 0x72,
+ 0xbb, 0x4b, 0xbf, 0xc9, 0x94, 0xcc, 0x88, 0x13, 0x22, 0xe4, 0x0c, 0x17, 0xd8, 0x94, 0x0f, 0x29,
+ 0x90, 0x4c, 0xc9, 0xd0, 0x2d, 0x53, 0xfd, 0x5b, 0x05, 0x4a, 0xdb, 0x38, 0x7c, 0xd8, 0x22, 0xde,
+ 0xb2, 0xd8, 0xf5, 0x6f, 0x03, 0xec, 0xe1, 0x43, 0x76, 0x3d, 0x09, 0x91, 0xdf, 0x48, 0x10, 0xf9,
+ 0x30, 0x81, 0xda, 0x3b, 0xf8, 0x90, 0xde, 0x63, 0xc1, 0x96, 0x13, 0xfa, 0x87, 0x5a, 0x7e, 0x4f,
+ 0xb4, 0xab, 0x6f, 0xc2, 0xdc, 0x20, 0x92, 0xdc, 0x33, 0x7b, 0xf8, 0x90, 0x6b, 0x18, 0xf9, 0x24,
+ 0x3a, 0xc4, 0x6e, 0x46, 0x22, 0xcb, 0x82, 0xc6, 0x1a, 0x37, 0x52, 0xdf, 0x50, 0xd4, 0x33, 0x50,
+ 0x96, 0xe6, 0x62, 0x3b, 0xac, 0x7e, 0x0d, 0x4a, 0x37, 0x87, 0x57, 0x80, 0x60, 0x7a, 0x0f, 0x1f,
+ 0x8a, 0x90, 0x84, 0x7e, 0xab, 0x3f, 0x4d, 0x41, 0xf9, 0xe6, 0xf0, 0x68, 0xf4, 0xdb, 0x23, 0xd6,
+ 0xfa, 0x46, 0xc2, 0x5a, 0x4f, 0x50, 0x18, 0x5a, 0x2c, 0x57, 0x35, 0x69, 0xc9, 0x3b, 0x90, 0x61,
+ 0x97, 0x7a, 0xb4, 0x2e, 0x45, 0x5a, 0x17, 0xba, 0x09, 0x05, 0xdb, 0x08, 0x42, 0xbd, 0xef, 0x99,
+ 0x46, 0x88, 0x4d, 0x6e, 0x5b, 0xc6, 0xd3, 0xc2, 0x59, 0x32, 0xf2, 0x21, 0x1b, 0x58, 0xf5, 0xc6,
+ 0x10, 0xed, 0xdb, 0xb2, 0x68, 0x67, 0xd7, 0xd7, 0x27, 0x5a, 0x28, 0x25, 0x2d, 0x6f, 0x47, 0x09,
+ 0xe6, 0x36, 0xed, 0x7e, 0x10, 0x62, 0x5f, 0x58, 0xb0, 0x4f, 0x15, 0x98, 0x8f, 0x40, 0x5c, 0xc2,
+ 0xaf, 0x00, 0x74, 0x18, 0x28, 0xbe, 0x1c, 0x8a, 0xc7, 0x47, 0x2b, 0x79, 0xde, 0xb1, 0xd5, 0xd4,
+ 0xf2, 0xbc, 0x43, 0xcb, 0x24, 0xa6, 0x22, 0x3e, 0x03, 0xd8, 0x21, 0x66, 0xd4, 0xe4, 0x9e, 0x49,
+ 0x29, 0x42, 0x6c, 0x31, 0x38, 0xfa, 0x3a, 0x20, 0x4c, 0x8c, 0xaa, 0xe7, 0x5b, 0x01, 0x8e, 0x7a,
+ 0x33, 0x8f, 0xab, 0x1c, 0x63, 0x78, 0x77, 0xd5, 0x83, 0x42, 0xd3, 0x37, 0x2c, 0x47, 0x68, 0xc9,
+ 0x45, 0x28, 0x7a, 0x3e, 0x5e, 0xbf, 0x72, 0x55, 0xef, 0x19, 0xfe, 0x1e, 0x75, 0x0b, 0xd3, 0x6b,
+ 0x19, 0xad, 0xc0, 0x80, 0x77, 0x28, 0x8c, 0x18, 0xcc, 0x60, 0xb7, 0x1f, 0x9a, 0xee, 0x13, 0x47,
+ 0xf8, 0x72, 0xa2, 0x8d, 0x96, 0x20, 0x67, 0xba, 0xba, 0x49, 0x68, 0x72, 0x73, 0x96, 0x35, 0x5d,
+ 0x3a, 0xc5, 0xad, 0xe9, 0x5c, 0xaa, 0x94, 0x56, 0x7f, 0xa2, 0x40, 0x91, 0x4f, 0x19, 0xbb, 0x1e,
+ 0x56, 0xc0, 0x86, 0x10, 0x97, 0x88, 0xad, 0x0c, 0xac, 0xa0, 0xc9, 0x21, 0xe8, 0x06, 0x2c, 0x51,
+ 0xac, 0xee, 0xe3, 0x1e, 0x03, 0xe9, 0x96, 0x63, 0x5a, 0x1d, 0x23, 0x74, 0x7d, 0xca, 0xc0, 0xb4,
+ 0x76, 0xce, 0x64, 0x24, 0x39, 0xbe, 0x25, 0xd0, 0xe8, 0x9b, 0x70, 0x7e, 0x78, 0xac, 0xf0, 0x83,
+ 0x89, 0x6f, 0xc6, 0xdc, 0xb6, 0xa5, 0xc1, 0xd1, 0xcd, 0xb8, 0xc3, 0xad, 0xe9, 0x9c, 0x52, 0x4a,
+ 0xa9, 0x1f, 0xc1, 0x52, 0x13, 0x77, 0xdc, 0x1e, 0x75, 0x12, 0x5c, 0x67, 0x9b, 0xa6, 0x43, 0x84,
+ 0xcc, 0xde, 0x87, 0x1c, 0xbf, 0xe7, 0xd9, 0x69, 0xc9, 0x34, 0x1a, 0xc7, 0x47, 0x2b, 0x59, 0x76,
+ 0xd1, 0x07, 0xff, 0x7d, 0xb4, 0x72, 0xad, 0x6b, 0x85, 0xbb, 0xfd, 0x76, 0xad, 0xe3, 0xf6, 0xea,
+ 0x91, 0x76, 0x99, 0xed, 0xf8, 0xbb, 0xee, 0xed, 0x75, 0xeb, 0x3c, 0x17, 0x53, 0xe3, 0x0e, 0x42,
+ 0x96, 0x39, 0x08, 0x81, 0xfa, 0x2f, 0x0a, 0x9c, 0x91, 0x27, 0xff, 0xf5, 0x4c, 0x8b, 0x3e, 0x80,
+ 0x32, 0x37, 0xd2, 0x3d, 0xdc, 0x6b, 0x63, 0x3f, 0xd8, 0xb5, 0x3c, 0xba, 0x37, 0x83, 0x61, 0xcb,
+ 0xde, 0x41, 0x4d, 0xa4, 0x6f, 0x84, 0x0f, 0x47, 0x9c, 0xb7, 0x68, 0x08, 0x17, 0x55, 0x89, 0xd1,
+ 0x8a, 0xe1, 0xea, 0x9f, 0xa4, 0xa1, 0x3a, 0x4a, 0xa6, 0x5c, 0x29, 0x1e, 0xc3, 0x0c, 0x4b, 0x3a,
+ 0xf1, 0x30, 0xf4, 0xcd, 0xa4, 0xb4, 0xc6, 0x53, 0xc9, 0xd4, 0x58, 0x53, 0x5c, 0x76, 0x8c, 0x62,
+ 0xf5, 0xcf, 0x52, 0x30, 0xc3, 0x10, 0xe8, 0xf1, 0xa0, 0x8f, 0x96, 0x69, 0x6c, 0xc4, 0x3e, 0xda,
+ 0x69, 0x45, 0x28, 0x5c, 0xbb, 0x73, 0x90, 0xb5, 0x02, 0xdd, 0xb6, 0x0e, 0xa2, 0x38, 0xc2, 0x0a,
+ 0x6e, 0x5b, 0x07, 0xf8, 0xa4, 0x2b, 0x9d, 0x1e, 0xe1, 0x4a, 0xdf, 0x07, 0x90, 0x04, 0x3f, 0x7d,
+ 0x5a, 0xc1, 0x4b, 0x44, 0x68, 0xb0, 0x20, 0x4e, 0x59, 0x86, 0x9d, 0x5b, 0xd1, 0x56, 0xef, 0x41,
+ 0x65, 0xa3, 0xb3, 0x77, 0x0f, 0x3b, 0xa6, 0xe5, 0x74, 0x1f, 0x61, 0x5f, 0xd6, 0xb4, 0xeb, 0x90,
+ 0x3d, 0x60, 0x90, 0xc8, 0x21, 0x88, 0xf9, 0x10, 0x8b, 0x17, 0x63, 0x44, 0x57, 0xf5, 0x3c, 0x2c,
+ 0x8d, 0xa0, 0xc8, 0x6f, 0xa8, 0x3f, 0x54, 0xe0, 0xec, 0xd6, 0x01, 0xf6, 0x0f, 0x89, 0xcc, 0xf8,
+ 0x3c, 0x0f, 0x1d, 0x12, 0x1c, 0x61, 0x38, 0x63, 0x74, 0xf6, 0x74, 0x8f, 0x8d, 0xd3, 0x07, 0x27,
+ 0xbe, 0x96, 0xa0, 0x05, 0x4f, 0x63, 0xff, 0xed, 0x29, 0xad, 0x6c, 0x0c, 0xe3, 0x1a, 0x59, 0x6e,
+ 0xf6, 0xd5, 0x3f, 0x56, 0x60, 0x51, 0xe2, 0x84, 0xf1, 0xc7, 0x58, 0xd9, 0x49, 0x62, 0xe5, 0xfa,
+ 0x64, 0xac, 0x30, 0xba, 0xcf, 0xe0, 0xc5, 0x84, 0xd2, 0xb0, 0x50, 0xd0, 0x3d, 0xc8, 0xfa, 0xec,
+ 0x93, 0x4f, 0x7c, 0x25, 0xd9, 0xd3, 0x3b, 0x29, 0x52, 0x91, 0x33, 0xe1, 0x64, 0xd4, 0x5d, 0x28,
+ 0x9f, 0x58, 0x30, 0xda, 0x86, 0x9c, 0xcf, 0xbf, 0xf9, 0x3c, 0x57, 0xc7, 0x9b, 0x47, 0x12, 0x18,
+ 0x9f, 0x28, 0x22, 0xa4, 0x6a, 0x30, 0xbf, 0x8d, 0x43, 0x72, 0x3d, 0x05, 0x09, 0x6e, 0xc8, 0x90,
+ 0x2f, 0xcc, 0xfd, 0x8e, 0xd4, 0xb0, 0x2f, 0xcc, 0xae, 0x71, 0xf5, 0x9f, 0x52, 0xd4, 0x3d, 0xe3,
+ 0x44, 0x39, 0xf7, 0xfa, 0xe4, 0xee, 0xd9, 0x00, 0x81, 0x67, 0x7a, 0x2c, 0x7b, 0x23, 0x3d, 0x96,
+ 0xbc, 0xf0, 0x58, 0x46, 0x65, 0x3c, 0x56, 0x61, 0x56, 0xbe, 0x69, 0xd2, 0x14, 0x25, 0x83, 0xd0,
+ 0x22, 0xcc, 0x78, 0xfd, 0xb6, 0x6d, 0x75, 0xf8, 0x4d, 0xc9, 0x5b, 0x55, 0x77, 0x0c, 0xb7, 0xe5,
+ 0xe6, 0xa0, 0xdb, 0x72, 0x75, 0x92, 0xc5, 0x9e, 0xf0, 0x5a, 0x2e, 0x41, 0xf1, 0x6d, 0x6c, 0xd8,
+ 0xe1, 0xae, 0xd8, 0xa5, 0x05, 0xc8, 0xf8, 0xd8, 0x30, 0xd9, 0x8c, 0x39, 0x8d, 0x35, 0x88, 0x73,
+ 0x23, 0xba, 0xf1, 0x0d, 0x2e, 0xc3, 0xfc, 0x6d, 0x9e, 0xd8, 0x17, 0xfe, 0xce, 0x2f, 0x53, 0x50,
+ 0x8a, 0x61, 0x7c, 0x7f, 0xde, 0x05, 0x10, 0x0f, 0x00, 0xd1, 0xfe, 0xbc, 0xf4, 0x4c, 0x63, 0x26,
+ 0xc8, 0x88, 0xa4, 0x5a, 0x4c, 0x02, 0xfd, 0xb9, 0x02, 0x39, 0x66, 0xcd, 0xb1, 0xb8, 0x21, 0x92,
+ 0x52, 0x00, 0xc3, 0x0c, 0xf1, 0x7b, 0x41, 0x6c, 0xf7, 0x1b, 0x84, 0xfe, 0x0f, 0x7f, 0x71, 0x3a,
+ 0x73, 0x1f, 0xf1, 0x51, 0xf5, 0xa0, 0x38, 0x40, 0x57, 0xde, 0xb6, 0x0c, 0xdb, 0xb6, 0x96, 0xbc,
+ 0x6d, 0x73, 0x03, 0xf6, 0x6c, 0xb4, 0x0c, 0xc8, 0x24, 0x82, 0x7b, 0x6e, 0xd2, 0xa5, 0x8d, 0xf3,
+ 0x60, 0xf6, 0x96, 0xdb, 0x0e, 0xa4, 0x6d, 0x63, 0xa1, 0xa6, 0x22, 0x87, 0x9a, 0x8b, 0xd2, 0x55,
+ 0x4a, 0x93, 0x54, 0xac, 0x85, 0xae, 0x71, 0xa5, 0x4d, 0x53, 0x56, 0x56, 0x64, 0xf1, 0xed, 0xdb,
+ 0x35, 0xfa, 0x4c, 0xc4, 0x9e, 0x8c, 0x6a, 0x24, 0x6a, 0x63, 0x5a, 0xad, 0xfe, 0xeb, 0x0c, 0x14,
+ 0xd8, 0x94, 0x7c, 0x6b, 0xb7, 0x60, 0x9a, 0xf4, 0xe2, 0x9b, 0xfa, 0x72, 0xc2, 0x26, 0xc8, 0xc3,
+ 0x48, 0x83, 0x6f, 0x2b, 0x1d, 0x5e, 0xfd, 0xdf, 0x0c, 0xa4, 0x6f, 0xb9, 0x6d, 0xb4, 0x08, 0x29,
+ 0x7e, 0x17, 0xa7, 0x1b, 0x33, 0xc7, 0x47, 0x2b, 0xa9, 0x56, 0x53, 0x4b, 0x59, 0xe6, 0x29, 0x4f,
+ 0x98, 0x1c, 0xda, 0x4f, 0x0f, 0x86, 0xf6, 0xe8, 0x00, 0xe6, 0x06, 0x12, 0xa7, 0x2c, 0x95, 0x54,
+ 0x6c, 0xbc, 0x7b, 0x7c, 0xb4, 0x52, 0x94, 0x33, 0xa7, 0xc4, 0x97, 0xba, 0x31, 0x96, 0x66, 0x04,
+ 0xfb, 0x76, 0xf4, 0x66, 0x45, 0x68, 0x7b, 0xed, 0x5a, 0xab, 0xa9, 0x15, 0xe5, 0xc4, 0x6b, 0x20,
+ 0x6d, 0xc7, 0xcc, 0xc0, 0x76, 0xdc, 0x80, 0x2c, 0xcb, 0x45, 0x9b, 0x34, 0xf5, 0x97, 0x1c, 0xf0,
+ 0x4c, 0xd3, 0x60, 0x47, 0x0c, 0x20, 0x63, 0x83, 0xd0, 0xf0, 0xc9, 0xd8, 0xdc, 0xb8, 0x63, 0xf9,
+ 0x00, 0xf4, 0x26, 0xe4, 0x76, 0x2c, 0xc7, 0x0a, 0x76, 0xb1, 0x59, 0xc9, 0x8f, 0x39, 0x38, 0x1a,
+ 0x41, 0x46, 0xf7, 0x5c, 0xd3, 0xda, 0xb1, 0xb0, 0x59, 0x81, 0x71, 0x47, 0x8b, 0x11, 0x24, 0x5a,
+ 0xd9, 0xf1, 0x0d, 0x9a, 0x15, 0xd5, 0x3b, 0x6e, 0xcf, 0xb3, 0x31, 0x59, 0xc2, 0xec, 0xaa, 0xb2,
+ 0x96, 0xd2, 0xca, 0x02, 0xb3, 0x29, 0x10, 0x44, 0xbf, 0x69, 0x5e, 0xac, 0x52, 0x60, 0xc6, 0x97,
+ 0x36, 0xd0, 0x7d, 0x38, 0xb3, 0x6b, 0x75, 0x77, 0x9f, 0x18, 0x24, 0x9e, 0x8a, 0x73, 0x17, 0xc5,
+ 0x31, 0xb9, 0x41, 0xd1, 0xe0, 0x08, 0x43, 0x6e, 0xa4, 0x98, 0xa4, 0x89, 0x3b, 0x56, 0xcf, 0xb0,
+ 0x2b, 0x73, 0x74, 0xd2, 0x52, 0x84, 0x68, 0x32, 0x38, 0xba, 0x04, 0x73, 0x7e, 0xdf, 0xa1, 0xa1,
+ 0x05, 0xdf, 0xd8, 0x79, 0xda, 0xb3, 0xc8, 0xa1, 0xdc, 0xd5, 0xbc, 0x00, 0xf9, 0xf8, 0xf1, 0xa1,
+ 0xc4, 0x72, 0x22, 0x11, 0x40, 0x45, 0x50, 0xba, 0xed, 0x76, 0xd8, 0x9b, 0xa4, 0x30, 0xa5, 0xff,
+ 0xa7, 0x40, 0x59, 0x02, 0x46, 0xa9, 0xbd, 0xbc, 0x2d, 0x80, 0x63, 0xbc, 0xfc, 0x9c, 0x20, 0x10,
+ 0x41, 0xc4, 0x2d, 0x17, 0x51, 0xab, 0xfe, 0xa9, 0x02, 0x39, 0x81, 0x45, 0x2f, 0x40, 0x81, 0x60,
+ 0x6c, 0x2b, 0x3c, 0xd4, 0xe3, 0xcb, 0x67, 0x56, 0xc0, 0xde, 0xc1, 0x87, 0x64, 0xe5, 0x51, 0x97,
+ 0xd8, 0xac, 0xe5, 0xb5, 0xa2, 0x80, 0xb2, 0x3b, 0xb3, 0x0a, 0x39, 0xdb, 0x08, 0xad, 0xb0, 0x6f,
+ 0x32, 0x63, 0xa3, 0x68, 0x51, 0x9b, 0x48, 0xc5, 0x76, 0x9d, 0x2e, 0x43, 0x4e, 0x53, 0x64, 0x0c,
+ 0x50, 0x1b, 0x30, 0xaf, 0x19, 0x4e, 0x17, 0xdf, 0x76, 0xbb, 0xc2, 0xc6, 0x2d, 0x41, 0x8e, 0x25,
+ 0xaa, 0x85, 0x99, 0xd0, 0xb2, 0xb4, 0x2d, 0x67, 0xda, 0x52, 0x92, 0xf9, 0x53, 0xff, 0x2b, 0x0d,
+ 0xa5, 0x98, 0x48, 0x74, 0x21, 0x89, 0xf4, 0x19, 0xbb, 0x3c, 0x92, 0xee, 0xcf, 0xe1, 0xc1, 0x23,
+ 0x13, 0x68, 0xff, 0xa8, 0x00, 0xdc, 0xf3, 0x71, 0x18, 0x1e, 0xb6, 0x9c, 0x1d, 0x97, 0x08, 0x8f,
+ 0x67, 0x2f, 0x68, 0xb8, 0x29, 0x84, 0xc7, 0x61, 0xc4, 0xba, 0x90, 0x85, 0x38, 0xf8, 0x09, 0x43,
+ 0x33, 0xb1, 0x65, 0x1d, 0xfc, 0x84, 0xa2, 0x2e, 0x42, 0xd1, 0x30, 0x4d, 0x6c, 0xea, 0x3c, 0x22,
+ 0xe0, 0xa6, 0xad, 0x40, 0x81, 0x1a, 0x83, 0xa1, 0x17, 0x61, 0xde, 0xc7, 0x3d, 0xf7, 0x40, 0xea,
+ 0xc6, 0x4c, 0xdc, 0x1c, 0x07, 0x8b, 0x8e, 0x8b, 0x30, 0xe3, 0x63, 0x23, 0x88, 0x5e, 0x22, 0x78,
+ 0x0b, 0x55, 0x20, 0x6b, 0xb2, 0x37, 0x36, 0x6e, 0x89, 0x44, 0xb3, 0xfa, 0x13, 0x45, 0x64, 0x03,
+ 0x6f, 0x41, 0x86, 0x2e, 0x90, 0xfb, 0x84, 0xb5, 0x67, 0xde, 0x57, 0x42, 0x58, 0xb2, 0x8c, 0x18,
+ 0x09, 0xf4, 0x3e, 0xcc, 0x7a, 0x54, 0x42, 0x3a, 0x4d, 0xcf, 0x31, 0xc7, 0xe5, 0xb5, 0x49, 0x04,
+ 0x1f, 0x0b, 0x58, 0xb8, 0x04, 0x5e, 0x04, 0xe1, 0x91, 0xfa, 0x1a, 0x94, 0xee, 0xf7, 0xb1, 0x7f,
+ 0x78, 0xcf, 0x36, 0x1c, 0xe9, 0x5a, 0xdc, 0x27, 0x30, 0xe1, 0xb3, 0xd1, 0x86, 0xea, 0x41, 0x59,
+ 0xea, 0xc9, 0xf5, 0xe2, 0x3b, 0x70, 0xde, 0xb4, 0x82, 0x30, 0xd8, 0xb7, 0x75, 0x6f, 0xf7, 0x30,
+ 0xb0, 0x3a, 0x86, 0xad, 0xd3, 0xee, 0xba, 0x67, 0x1b, 0x0e, 0x4f, 0xd5, 0x5c, 0x38, 0x3e, 0x5a,
+ 0xa9, 0x34, 0xad, 0x20, 0xdc, 0xbe, 0x7f, 0xfb, 0x1e, 0xef, 0x15, 0x93, 0xaa, 0x70, 0x02, 0x27,
+ 0x30, 0xea, 0x12, 0x7b, 0xc5, 0x27, 0x23, 0x7d, 0xab, 0xdd, 0x0f, 0xe3, 0x18, 0x45, 0xfd, 0x3b,
+ 0x80, 0xca, 0x49, 0x1c, 0x67, 0xca, 0x83, 0xa2, 0x78, 0xd4, 0x61, 0xa2, 0x63, 0xa7, 0x7e, 0xeb,
+ 0x19, 0x2f, 0xfd, 0xa3, 0x68, 0x45, 0x25, 0x00, 0x44, 0x64, 0xb2, 0xaf, 0x5b, 0x30, 0x25, 0x04,
+ 0xea, 0x41, 0x41, 0x7a, 0xb3, 0x14, 0x4f, 0xb6, 0xcd, 0xd3, 0x4c, 0x18, 0xbf, 0x63, 0x0e, 0xf8,
+ 0xd6, 0xb3, 0xf1, 0x3b, 0x66, 0x50, 0xfd, 0x54, 0x01, 0x88, 0xfb, 0x11, 0x85, 0x65, 0xe9, 0x02,
+ 0xbe, 0x61, 0xbc, 0x85, 0x36, 0x60, 0x86, 0x3f, 0xc8, 0xa6, 0x26, 0x7d, 0x90, 0xe5, 0x03, 0x69,
+ 0xe6, 0x8d, 0xbd, 0xc3, 0x06, 0xfb, 0x36, 0x3b, 0x2f, 0x3c, 0xf3, 0x46, 0xa1, 0xdb, 0xf7, 0x6f,
+ 0x6b, 0x79, 0xd6, 0x61, 0x7b, 0xdf, 0xbe, 0x35, 0x9d, 0x4b, 0x97, 0xa6, 0xab, 0x7f, 0x90, 0x86,
+ 0x3c, 0x7d, 0xbc, 0xa0, 0xa2, 0xf9, 0x5c, 0x81, 0xca, 0x40, 0xf4, 0xae, 0xb7, 0x0f, 0xf5, 0x38,
+ 0x87, 0x40, 0xe4, 0xf4, 0xde, 0x69, 0xe4, 0x14, 0xcd, 0x50, 0xd3, 0xa4, 0x34, 0x40, 0x83, 0x46,
+ 0x59, 0x2d, 0x93, 0x89, 0xee, 0xf5, 0xd3, 0xfa, 0xa8, 0x0b, 0xfe, 0x08, 0x9a, 0xe8, 0xab, 0x30,
+ 0x27, 0xbf, 0x4c, 0x47, 0x6f, 0x17, 0x85, 0x78, 0x6f, 0x5a, 0x26, 0xfa, 0x16, 0x80, 0xe9, 0xbb,
+ 0x9e, 0x87, 0x4d, 0xdd, 0x60, 0xa9, 0x8a, 0x71, 0x6e, 0xd5, 0x3c, 0x1f, 0xb3, 0x11, 0x56, 0x6f,
+ 0xc2, 0xd2, 0x53, 0x97, 0x34, 0xc2, 0x45, 0x1e, 0xc8, 0x75, 0xa7, 0x25, 0x6f, 0xb7, 0xfa, 0x83,
+ 0x14, 0x14, 0x64, 0xfd, 0x45, 0x21, 0xb0, 0x82, 0x14, 0xf9, 0x54, 0xbc, 0xfb, 0xbc, 0xa7, 0x22,
+ 0xde, 0x89, 0x81, 0x58, 0x30, 0x14, 0xd0, 0xea, 0xc7, 0x30, 0x37, 0xd8, 0x65, 0x44, 0x78, 0xb6,
+ 0x3d, 0x18, 0x9e, 0xbd, 0xf5, 0x5c, 0x1a, 0x31, 0x20, 0x03, 0x25, 0x2e, 0x3a, 0x4a, 0x62, 0xe0,
+ 0xd1, 0x20, 0x03, 0xbf, 0xf9, 0xbc, 0x52, 0x91, 0x79, 0xf8, 0x3e, 0x94, 0x86, 0x4f, 0xf5, 0x08,
+ 0x0e, 0x1e, 0x0c, 0x72, 0xf0, 0xcd, 0xe7, 0x33, 0x1e, 0xd2, 0xfc, 0x3c, 0x91, 0x7c, 0x0e, 0xce,
+ 0xde, 0xa1, 0x65, 0x5f, 0x77, 0x78, 0xf1, 0x9a, 0xb0, 0xa5, 0xff, 0xa6, 0xc0, 0xe2, 0x30, 0x86,
+ 0x5b, 0x52, 0x03, 0x72, 0xa2, 0xd4, 0x8d, 0xab, 0xcb, 0xb7, 0x12, 0xd8, 0x1a, 0x4d, 0xa4, 0x26,
+ 0x00, 0xb2, 0x7a, 0x44, 0x64, 0xab, 0xbf, 0x05, 0xc5, 0x81, 0x0e, 0x23, 0x24, 0xf3, 0xea, 0xa0,
+ 0x64, 0xe4, 0xc8, 0xab, 0x1f, 0x5a, 0x76, 0x8d, 0x57, 0xb3, 0x45, 0x13, 0x4b, 0x01, 0xdf, 0x8f,
+ 0x53, 0x70, 0x66, 0xcb, 0xd9, 0xef, 0xe3, 0x3e, 0xa6, 0xb7, 0xa3, 0xb8, 0xe2, 0xbe, 0xcc, 0x3c,
+ 0x26, 0xbb, 0x3e, 0x23, 0xe7, 0x8e, 0x35, 0xd0, 0x07, 0x92, 0x1f, 0x46, 0x5f, 0x36, 0x1b, 0x9b,
+ 0xc7, 0x47, 0x2b, 0x59, 0xca, 0x15, 0x9d, 0xf3, 0xfa, 0x44, 0x73, 0xf2, 0x71, 0xb1, 0x33, 0x77,
+ 0x19, 0xca, 0xc1, 0x9e, 0xe5, 0xe9, 0xc1, 0xae, 0xdb, 0xb7, 0x4d, 0x9d, 0x71, 0xc0, 0xf2, 0x24,
+ 0xf3, 0x04, 0xb1, 0x4d, 0xe1, 0xf7, 0x09, 0x58, 0xfd, 0xeb, 0x14, 0x2c, 0x0c, 0x4a, 0x85, 0xef,
+ 0xf7, 0xfd, 0xd8, 0xc5, 0x61, 0xdb, 0xfd, 0x7a, 0x52, 0x52, 0x6b, 0x04, 0x85, 0x9a, 0xa8, 0x42,
+ 0x8a, 0x7c, 0xa3, 0xbf, 0x57, 0x20, 0xcb, 0x81, 0x5f, 0xaa, 0xd4, 0xdf, 0x1a, 0xf2, 0x50, 0x2f,
+ 0x25, 0x3d, 0xad, 0xfb, 0x46, 0x07, 0x53, 0x8f, 0x4b, 0xf8, 0xa3, 0x71, 0xa8, 0x94, 0x96, 0x42,
+ 0x25, 0xf5, 0x2c, 0x9c, 0xd9, 0xdc, 0x35, 0xfc, 0x70, 0x93, 0x05, 0xa9, 0xe2, 0xc4, 0x3c, 0x86,
+ 0x85, 0x41, 0x30, 0x17, 0x5f, 0x03, 0xb2, 0x3c, 0x9c, 0xe5, 0xe2, 0x53, 0x25, 0x26, 0xc2, 0xa0,
+ 0x26, 0x2a, 0x32, 0xe9, 0xe0, 0x6d, 0x56, 0xdd, 0x12, 0x15, 0x7e, 0x31, 0xdc, 0xe5, 0x77, 0x60,
+ 0x71, 0x74, 0x29, 0x13, 0x9a, 0x85, 0xec, 0xc3, 0xbb, 0xef, 0xdc, 0x7d, 0xf7, 0xbd, 0xbb, 0xa5,
+ 0x29, 0xd2, 0xd8, 0xbc, 0xfd, 0x70, 0xfb, 0xc1, 0x96, 0x56, 0x52, 0x50, 0x01, 0x72, 0xcd, 0x8d,
+ 0x07, 0x1b, 0x8d, 0x8d, 0xed, 0xad, 0x52, 0x0a, 0xe5, 0x21, 0xf3, 0x60, 0xa3, 0x71, 0x7b, 0xab,
+ 0x94, 0x5e, 0xff, 0x9f, 0x2a, 0x64, 0x36, 0xcc, 0x9e, 0xe5, 0xa0, 0x10, 0x32, 0xf4, 0x7d, 0x1f,
+ 0xbd, 0xf8, 0xec, 0x0a, 0x00, 0xba, 0xc8, 0xea, 0xda, 0xb8, 0xa5, 0x02, 0x6a, 0xe5, 0x87, 0xff,
+ 0xfc, 0x9f, 0x3f, 0x4e, 0x21, 0x54, 0xaa, 0xeb, 0xb4, 0xf4, 0xb6, 0x7e, 0x70, 0xb5, 0x4e, 0x4b,
+ 0x06, 0xd0, 0xef, 0x29, 0x90, 0x8f, 0x4a, 0x3e, 0xd1, 0xcb, 0x63, 0x94, 0x5a, 0x46, 0xd3, 0xbf,
+ 0x32, 0x5e, 0x67, 0xce, 0xc2, 0x05, 0xca, 0xc2, 0x22, 0x5a, 0x90, 0x58, 0x88, 0xaa, 0x48, 0xd1,
+ 0x5f, 0x2a, 0x30, 0x3f, 0x54, 0xcb, 0x89, 0xae, 0x4e, 0x52, 0xf7, 0xc9, 0x58, 0x5a, 0x9f, 0xbc,
+ 0x54, 0x54, 0x7d, 0x91, 0x32, 0xf6, 0x02, 0x5a, 0x19, 0xc5, 0x58, 0xfd, 0x63, 0xf1, 0xf9, 0x3d,
+ 0xf4, 0x37, 0x0a, 0x14, 0xe4, 0x3a, 0x3e, 0x54, 0x1b, 0xbb, 0xe0, 0x8f, 0x71, 0x57, 0x9f, 0xb0,
+ 0x40, 0x50, 0x7d, 0x8d, 0xb2, 0x76, 0x05, 0xd5, 0x9e, 0xc1, 0x5a, 0x9d, 0x5e, 0xe5, 0x41, 0xfd,
+ 0x63, 0xfa, 0x4f, 0x39, 0x85, 0xb8, 0x38, 0x05, 0xbd, 0x32, 0x66, 0x0d, 0x0b, 0xe3, 0x72, 0xb2,
+ 0x8a, 0x17, 0xf5, 0x4d, 0xca, 0xe3, 0x6b, 0xe8, 0xfa, 0x64, 0x3c, 0xd6, 0x59, 0xf1, 0xd3, 0x5f,
+ 0x28, 0x50, 0x1c, 0xa8, 0xf7, 0x41, 0x49, 0x42, 0x1a, 0x55, 0x32, 0x54, 0xbd, 0x32, 0xfe, 0x00,
+ 0xce, 0xf2, 0x2a, 0x65, 0xb9, 0x8a, 0x2a, 0x12, 0xcb, 0x8e, 0xeb, 0x30, 0x06, 0x29, 0x13, 0x1f,
+ 0xc2, 0x0c, 0x2b, 0x31, 0x41, 0x6b, 0x63, 0x54, 0xa1, 0x30, 0x3e, 0x5e, 0x1a, 0xbb, 0x5e, 0x45,
+ 0x5d, 0xa2, 0x0c, 0x9c, 0x41, 0x65, 0x89, 0x01, 0x6e, 0xe5, 0xc8, 0x79, 0x8c, 0xca, 0x1f, 0x12,
+ 0xcf, 0xe3, 0x70, 0x41, 0x46, 0xe2, 0x79, 0x3c, 0x59, 0x51, 0xc1, 0xcf, 0xa3, 0x2a, 0xf3, 0xd0,
+ 0xb7, 0xc8, 0x76, 0xdd, 0x50, 0x2e, 0xa3, 0x1f, 0x28, 0x90, 0xbf, 0x39, 0x16, 0x1b, 0x37, 0x27,
+ 0x61, 0xe3, 0x44, 0xbd, 0xc1, 0x48, 0x51, 0x30, 0x36, 0xd0, 0xef, 0x40, 0x96, 0x57, 0x0f, 0xa0,
+ 0x24, 0xd9, 0x0e, 0x56, 0x27, 0x54, 0x2f, 0x8f, 0xd3, 0x95, 0x4f, 0x5e, 0xa5, 0x93, 0x2f, 0x20,
+ 0x24, 0x4d, 0xce, 0xab, 0x14, 0xd0, 0xef, 0x2a, 0x90, 0x13, 0xef, 0x0c, 0xe8, 0xf2, 0x58, 0x8f,
+ 0x11, 0x8c, 0x81, 0x97, 0x27, 0x78, 0xb8, 0x50, 0xcf, 0x53, 0x0e, 0xce, 0xa2, 0x33, 0x12, 0x07,
+ 0x81, 0x98, 0xf5, 0x47, 0x0a, 0xcc, 0xb0, 0xe7, 0x89, 0x44, 0x35, 0x1c, 0x78, 0xe8, 0x48, 0x54,
+ 0xc3, 0xa1, 0xb7, 0x8e, 0xaf, 0xd2, 0xc9, 0x97, 0x07, 0x64, 0xbf, 0x4b, 0xbb, 0x3c, 0xce, 0xa3,
+ 0x2c, 0xff, 0xa4, 0x82, 0x10, 0xf9, 0xfa, 0x44, 0x41, 0x0c, 0xbd, 0x9b, 0x24, 0x0a, 0x62, 0xf8,
+ 0xf9, 0x62, 0xa4, 0x20, 0xc4, 0xeb, 0x08, 0xf2, 0x60, 0xfa, 0x96, 0xdb, 0x0e, 0xd0, 0xd7, 0x9e,
+ 0x99, 0x8b, 0x67, 0x33, 0xbf, 0x38, 0x66, 0xce, 0x5e, 0x3d, 0x47, 0x67, 0x2d, 0xa3, 0x79, 0x69,
+ 0xd6, 0xef, 0x92, 0x99, 0xc8, 0x31, 0x8c, 0xf2, 0x8c, 0x89, 0xfa, 0x3f, 0x9c, 0xe3, 0x4c, 0xd4,
+ 0xff, 0x13, 0xa9, 0xcb, 0x91, 0xd7, 0x62, 0x94, 0xbe, 0xa4, 0x6c, 0x44, 0xd9, 0x96, 0x44, 0x36,
+ 0x86, 0x53, 0x44, 0x89, 0x6c, 0x9c, 0xc8, 0x12, 0x8d, 0x64, 0x83, 0x66, 0x89, 0x3c, 0x32, 0xf1,
+ 0x07, 0x90, 0xa1, 0xa5, 0x2b, 0x89, 0xae, 0x89, 0x5c, 0x75, 0x93, 0xe8, 0x9a, 0x0c, 0xd4, 0xca,
+ 0xa8, 0x53, 0x57, 0x14, 0xf4, 0x04, 0x0a, 0x72, 0xc5, 0x43, 0xe2, 0xc5, 0x3a, 0xa2, 0x70, 0xa4,
+ 0xfa, 0xea, 0xa9, 0x4a, 0x29, 0xd4, 0x29, 0xb4, 0x0b, 0xf9, 0xe8, 0xe1, 0x37, 0x51, 0xbc, 0xc3,
+ 0xcf, 0xd0, 0x89, 0xe2, 0x3d, 0xf1, 0x96, 0xac, 0x4e, 0x91, 0xb3, 0x8c, 0x4e, 0xb2, 0x82, 0xae,
+ 0x4f, 0xc8, 0xf9, 0x73, 0xae, 0xf7, 0xaf, 0x14, 0xc8, 0x89, 0x1c, 0x64, 0xe2, 0x59, 0x1e, 0xca,
+ 0x51, 0x27, 0x9e, 0xe5, 0xe1, 0xa4, 0xa6, 0xfa, 0x16, 0x55, 0xa6, 0xd7, 0x07, 0xce, 0x32, 0x0d,
+ 0x82, 0x6c, 0xb7, 0xfb, 0x78, 0x15, 0x2d, 0x8f, 0x00, 0xd7, 0x3f, 0x16, 0xa1, 0xd7, 0xf7, 0x88,
+ 0x2f, 0x58, 0x1a, 0x8e, 0x9f, 0xd1, 0xfa, 0x44, 0xc1, 0x36, 0x63, 0xfa, 0xda, 0x29, 0x02, 0xf4,
+ 0xc8, 0x28, 0x5e, 0x18, 0xf2, 0x67, 0x74, 0x53, 0x66, 0xe7, 0x53, 0x05, 0xca, 0x1b, 0xb6, 0x3d,
+ 0x18, 0x4f, 0xa3, 0x2b, 0x13, 0x84, 0xde, 0x8c, 0xc5, 0xab, 0x13, 0x07, 0xeb, 0xea, 0x0b, 0x94,
+ 0xc1, 0xf3, 0x68, 0x49, 0x62, 0x90, 0xc5, 0xd7, 0x22, 0x62, 0x47, 0x9f, 0x28, 0x50, 0x90, 0xc3,
+ 0x9f, 0xc4, 0x03, 0x35, 0x22, 0x7c, 0x4a, 0xf4, 0x54, 0x47, 0xc5, 0x55, 0xea, 0x0a, 0x65, 0x6a,
+ 0x09, 0x9d, 0x93, 0x6f, 0x52, 0xd2, 0x91, 0x07, 0x4d, 0xc4, 0xd1, 0x2b, 0xc8, 0xe1, 0x68, 0x22,
+ 0x4b, 0x23, 0xf2, 0x01, 0x89, 0x2c, 0x8d, 0x8a, 0x73, 0xd5, 0x8b, 0x94, 0xa5, 0xaf, 0xa8, 0xb2,
+ 0x97, 0x87, 0x59, 0x47, 0x9d, 0x2a, 0xdb, 0x0d, 0xe5, 0x72, 0xe3, 0xf2, 0xe7, 0xff, 0xb1, 0x3c,
+ 0xf5, 0xf9, 0xf1, 0xb2, 0xf2, 0xb3, 0xe3, 0x65, 0xe5, 0xe7, 0xc7, 0xcb, 0xca, 0xbf, 0x1f, 0x2f,
+ 0x2b, 0x9f, 0x7c, 0xb1, 0x3c, 0xf5, 0xb3, 0x2f, 0x96, 0xa7, 0x7e, 0xfe, 0xc5, 0xf2, 0xd4, 0xe3,
+ 0x9c, 0x98, 0xa5, 0x3d, 0x43, 0x53, 0x83, 0xd7, 0xfe, 0x3f, 0x00, 0x00, 0xff, 0xff, 0x6d, 0x50,
+ 0x29, 0xed, 0xce, 0x38, 0x00, 0x00,
}
diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto
index c74026c14bf0..f21e44119fec 100644
--- a/pkg/server/serverpb/admin.proto
+++ b/pkg/server/serverpb/admin.proto
@@ -14,6 +14,7 @@ option go_package = "serverpb";
import "config/zonepb/zone.proto";
import "jobs/jobspb/jobs.proto";
+import "roachpb/metadata.proto";
import "server/serverpb/status.proto";
import "storage/enginepb/mvcc.proto";
import "kv/kvserver/kvserverpb/liveness.proto";
@@ -455,6 +456,40 @@ message DecommissionStatusResponse {
repeated Status status = 2 [(gogoproto.nullable) = false];
}
+// XXX: Should all this stuff sit in a new proto entirely? It's pretty self
+// contained.
+
+// AckPendingVersionRequest XXX:
+message AckPendingVersionRequest {
+ roachpb.Version version = 1;
+}
+
+// AckPendingVersionResponse XXX:
+message AckPendingVersionResponse { }
+
+// EveryNodeRequestUnion XXX:
+message EveryNodeRequestUnion {
+ oneof value {
+ AckPendingVersionRequest ack_pending_version = 1;
+ }
+}
+
+message EveryNodeResponseUnion {
+ oneof value {
+ AckPendingVersionResponse ack_pending_version = 1;
+ }
+}
+
+// EveryNodeRequest XXX:
+message EveryNodeRequest {
+ EveryNodeRequestUnion request = 1 [(gogoproto.nullable) = false];
+}
+
+// EveryNodeResponse XXX:
+message EveryNodeResponse {
+ EveryNodeResponseUnion response = 1 [(gogoproto.nullable) = false];
+}
+
// SettingsRequest inquires what are the current settings in the cluster.
message SettingsRequest {
// The array of setting names to retrieve.
@@ -864,6 +899,10 @@ service Admin {
rpc Decommission(DecommissionRequest) returns (DecommissionStatusResponse) {
}
+ // EveryNode XXX:
+ rpc EveryNode(EveryNodeRequest) returns (EveryNodeResponse) {
+ }
+
// DecommissionStatus retrieves the decommissioning status of the specified nodes.
// If this ever becomes exposed via HTTP, ensure that it performs
// authorization. See #42567.
diff --git a/pkg/settings/cluster/cluster_settings.go b/pkg/settings/cluster/cluster_settings.go
index ac9239196f37..3fe5dfa6f53c 100644
--- a/pkg/settings/cluster/cluster_settings.go
+++ b/pkg/settings/cluster/cluster_settings.go
@@ -48,7 +48,8 @@ type Settings struct {
// Version provides a read-only view to the active cluster version and this
// binary's version details.
- Version clusterversion.Handle
+ Version clusterversion.Handle
+ MutableVersionRef clusterversion.MutableRef
}
// TelemetryOptOut is a place for controlling whether to opt out of telemetry or not.
@@ -116,6 +117,8 @@ func MakeClusterSettings() *Settings {
sv := &s.SV
s.Version = clusterversion.MakeVersionHandle(&s.SV)
+ // XXX: Should the init happen through this mutable ref instead?
+ s.MutableVersionRef = clusterversion.MakeMutableRef(s.Version)
sv.Init(s.Version)
s.Tracer = tracing.NewTracer()
diff --git a/pkg/settings/statemachine.go b/pkg/settings/statemachine.go
index 7056337f48d1..1b1e1c3335d1 100644
--- a/pkg/settings/statemachine.go
+++ b/pkg/settings/statemachine.go
@@ -63,6 +63,7 @@ type StateMachineSettingImpl interface {
// The state machine as well as its encoding are represented by the
// StateMachineSettingImpl backing this StateMachineSetting; it is in charge to
// converting to/from strings and performing validations.
+// XXX: Need the equivalent of this guy to be placed under pkg/clusterversion.
type StateMachineSetting struct {
impl StateMachineSettingImpl
common
diff --git a/pkg/settings/updater.go b/pkg/settings/updater.go
index dbb11359c6ed..e7a83c9ce449 100644
--- a/pkg/settings/updater.go
+++ b/pkg/settings/updater.go
@@ -121,10 +121,12 @@ func (u updater) Set(key, rawValue string, vt string) error {
return err
}
return setting.set(u.sv, d)
- case *StateMachineSetting:
- return setting.set(u.sv, []byte(rawValue))
+ //case *StateMachineSetting:
+ // return setting.set(u.sv, []byte(rawValue))
+ default:
+ // XXX: Disconnecting the state machine setting here.
+ return nil
}
- return nil
}
// ResetRemaining sets all settings not updated by the updater to their default values.
diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go
index 6ecf2c285127..40caa5b21b01 100644
--- a/pkg/sql/catalog/bootstrap/metadata.go
+++ b/pkg/sql/catalog/bootstrap/metadata.go
@@ -342,6 +342,10 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) {
target.AddDescriptor(keys.SystemDatabaseID, systemschema.ScheduledJobsTable)
target.AddDescriptor(keys.SystemDatabaseID, systemschema.SqllivenessTable)
+
+ // Tables introduced in 21.1.
+ // XXX:
+ target.AddDescriptor(keys.SystemDatabaseID, systemschema.LRMTable)
}
// addSplitIDs adds a split point for each of the PseudoTableIDs to the supplied
diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go
index 1cfd8d6f1970..0178db1c7bcc 100644
--- a/pkg/sql/catalog/descpb/privilege.go
+++ b/pkg/sql/catalog/descpb/privilege.go
@@ -418,6 +418,7 @@ var SystemAllowedPrivileges = map[ID]privilege.List{
keys.StatementDiagnosticsTableID: privilege.ReadWriteData,
keys.ScheduledJobsTableID: privilege.ReadWriteData,
keys.SqllivenessID: privilege.ReadWriteData,
+ keys.LRMID: privilege.ReadWriteData,
}
// SetOwner sets the owner of the privilege descriptor to the provided string.
diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go
index b922458e9443..3dfe4e9c750b 100644
--- a/pkg/sql/catalog/systemschema/system.go
+++ b/pkg/sql/catalog/systemschema/system.go
@@ -1684,6 +1684,30 @@ var (
FormatVersion: descpb.InterleavedFormatVersion,
NextMutationID: 1,
})
+
+ // XXX:
+ LRMTable = tabledesc.NewImmutable(descpb.TableDescriptor{
+ Name: "lrm",
+ ID: keys.LRMID,
+ ParentID: keys.SystemDatabaseID,
+ UnexposedParentSchemaID: keys.PublicSchemaID,
+ Version: 1,
+ Columns: []descpb.ColumnDescriptor{
+ {Name: "job_id", ID: 1, Type: types.Int, DefaultExpr: &uniqueRowIDString, Nullable: false},
+ {Name: "metadata", ID: 2, Type: types.String, Nullable: false},
+ },
+ NextColumnID: 3,
+ Families: []descpb.ColumnFamilyDescriptor{
+ {Name: "primary", ID: 0, ColumnNames: []string{"job_id", "metadata"}, ColumnIDs: []descpb.ColumnID{1, 2}},
+ },
+ NextFamilyID: 1,
+ PrimaryIndex: pk("job_id"),
+ NextIndexID: 2,
+ Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor(
+ descpb.SystemAllowedPrivileges[keys.LRMID], security.NodeUser),
+ FormatVersion: descpb.InterleavedFormatVersion,
+ NextMutationID: 1,
+ })
)
// newCommentPrivilegeDescriptor returns a privilege descriptor for comment table
diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go
index 4ca4d8e92970..d4273bf76244 100644
--- a/pkg/sqlmigrations/migrations.go
+++ b/pkg/sqlmigrations/migrations.go
@@ -373,6 +373,13 @@ var backwardCompatibleMigrations = []migrationDescriptor{
workFn: markDeprecatedSchemaChangeJobsFailed,
includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionLeasedDatabaseDescriptors),
},
+ {
+ // Introduced in v21.1.
+ name: "xxx: introduce new jobs table",
+ workFn: createLRMTable,
+ includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionLRMTable),
+ newDescriptorIDs: staticIDs(keys.LRMID),
+ },
}
func staticIDs(
@@ -1062,6 +1069,16 @@ func createRoleOptionsTable(ctx context.Context, r runner) error {
return nil
}
+// XXX:
+func createLRMTable(ctx context.Context, r runner) error {
+ err := createSystemTable(ctx, r, systemschema.LRMTable)
+ if err != nil {
+ return errors.Wrap(err, "failed to create system.role_options")
+ }
+
+ return nil
+}
+
func extendCreateRoleWithCreateLogin(ctx context.Context, r runner) error {
// Add the CREATELOGIN option to roles that already have CREATEROLE.
const upsertCreateRoleStmt = `
@@ -1217,6 +1234,8 @@ func populateVersionSetting(ctx context.Context, r runner) error {
return err
}
+ // XXX: this can sometimes fail with "no version found". I think because we
+ // disconnected the gossip prop.
if err := r.execAsRoot(
ctx, "set-setting", "SET CLUSTER SETTING version = $1", v.String(),
); err != nil {