diff --git a/pkg/acceptance/cluster/dockercluster.go b/pkg/acceptance/cluster/dockercluster.go index 4239ce86cc81..58b7883109c5 100644 --- a/pkg/acceptance/cluster/dockercluster.go +++ b/pkg/acceptance/cluster/dockercluster.go @@ -31,7 +31,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logflags" @@ -66,7 +66,7 @@ const CockroachBinaryInContainer = "/cockroach/cockroach" var cockroachImage = flag.String("i", defaultImage, "the docker image to run") var cockroachEntry = flag.String("e", "", "the entry point for the image") var waitOnStop = flag.Bool("w", false, "wait for the user to interrupt before tearing down the cluster") -var maxRangeBytes = *zonepb.DefaultZoneConfig().RangeMaxBytes +var maxRangeBytes = *config.DefaultZoneConfig().RangeMaxBytes // CockroachBinary is the path to the host-side binary to use. var CockroachBinary = flag.String("b", func() string { diff --git a/pkg/acceptance/localcluster/cluster.go b/pkg/acceptance/localcluster/cluster.go index ffe80e5ca9ab..5483edf29b05 100644 --- a/pkg/acceptance/localcluster/cluster.go +++ b/pkg/acceptance/localcluster/cluster.go @@ -31,7 +31,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" @@ -370,7 +370,7 @@ func (c *Cluster) isReplicated() (bool, string) { // UpdateZoneConfig updates the default zone config for the cluster. func (c *Cluster) UpdateZoneConfig(rangeMinBytes, rangeMaxBytes int64) { - zone := zonepb.DefaultZoneConfig() + zone := config.DefaultZoneConfig() zone.RangeMinBytes = proto.Int64(rangeMinBytes) zone.RangeMaxBytes = proto.Int64(rangeMaxBytes) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 1c59f557220c..a66d87f669eb 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -33,7 +33,6 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl/sampledataccl" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -1000,7 +999,7 @@ func TestBackupRestoreControlJob(t *testing.T) { t.Fatal(err) } last := uint32(v.ValueInt()) - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(5000) config.TestingSetZoneConfig(last+1, zoneConfig) } diff --git a/pkg/ccl/changefeedccl/helpers_test.go b/pkg/ccl/changefeedccl/helpers_test.go index e713369dc82a..4864121d8594 100644 --- a/pkg/ccl/changefeedccl/helpers_test.go +++ b/pkg/ccl/changefeedccl/helpers_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -329,7 +330,6 @@ func forceTableGC( database, table string, ) { t.Helper() - if err := tsi.ForceTableGC(context.TODO(), database, table, tsi.Clock().Now()); err != nil { - t.Fatal(err) - } + sqlutils.ForceTableGC( + t, tsi.DistSenderI().(*kv.DistSender), sqlDB.DB, database, table, tsi.Clock().Now()) } diff --git a/pkg/ccl/importccl/exportcsv_test.go b/pkg/ccl/importccl/exportcsv_test.go index 84b8a33e7556..3ca5f90e3ea2 100644 --- a/pkg/ccl/importccl/exportcsv_test.go +++ b/pkg/ccl/importccl/exportcsv_test.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -53,7 +52,7 @@ func setupExportableBank(t *testing.T, nodes, rows int) (*sqlutils.SQLRunner, st t.Fatal(err) } last := uint32(v.ValueInt()) - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(5000) config.TestingSetZoneConfig(last+1, zoneConfig) db.Exec(t, "ALTER TABLE bank SCATTER") diff --git a/pkg/ccl/importccl/load.go b/pkg/ccl/importccl/load.go index 29a121663da2..8348116af142 100644 --- a/pkg/ccl/importccl/load.go +++ b/pkg/ccl/importccl/load.go @@ -18,7 +18,7 @@ import ( "math/rand" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -54,7 +54,7 @@ func Load( tempPrefix string, ) (backupccl.BackupDescriptor, error) { if loadChunkBytes == 0 { - loadChunkBytes = *zonepb.DefaultZoneConfig().RangeMaxBytes / 2 + loadChunkBytes = *config.DefaultZoneConfig().RangeMaxBytes / 2 } var txCtx transform.ExprTransformContext diff --git a/pkg/ccl/partitionccl/drop_test.go b/pkg/ccl/partitionccl/drop_test.go index 2b5b377b78b2..4ecac55391c3 100644 --- a/pkg/ccl/partitionccl/drop_test.go +++ b/pkg/ccl/partitionccl/drop_test.go @@ -14,7 +14,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -26,7 +26,7 @@ import ( "github.com/pkg/errors" ) -func subzoneExists(cfg *zonepb.ZoneConfig, index uint32, partition string) bool { +func subzoneExists(cfg *config.ZoneConfig, index uint32, partition string) bool { for _, s := range cfg.Subzones { if s.IndexID == index && s.PartitionName == partition { return true @@ -85,7 +85,7 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) { sqlDB.Exec(t, `DROP INDEX t.kv@i`) // All zone configs should still exist. var buf []byte - cfg := &zonepb.ZoneConfig{} + cfg := &config.ZoneConfig{} sqlDB.QueryRow(t, "SELECT config FROM system.zones WHERE id = $1", tableDesc.ID).Scan(&buf) if err := protoutil.Unmarshal(buf, cfg); err != nil { t.Fatal(err) diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index 6a2445f849bf..3931d86c8d45 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/importccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -98,7 +98,7 @@ type partitioningTest struct { zoneConfigStmts string // subzones are the `configs` shorthand parsed into Subzones. - subzones []zonepb.Subzone + subzones []config.Subzone } } @@ -154,7 +154,7 @@ func (pt *partitioningTest) parse() error { } var indexName string - var subzone zonepb.Subzone + var subzone config.Subzone subzoneParts := strings.Split(subzoneShort, ".") switch len(subzoneParts) { case 1: @@ -191,7 +191,7 @@ func (pt *partitioningTest) parse() error { } } - var parsedConstraints zonepb.ConstraintsList + var parsedConstraints config.ConstraintsList if err := yaml.UnmarshalStrict([]byte("["+constraints+"]"), &parsedConstraints); err != nil { return errors.Wrapf(err, "parsing constraints: %s", constraints) } @@ -1122,7 +1122,7 @@ func verifyScansOnNode( func setupPartitioningTestCluster( ctx context.Context, t testing.TB, ) (*gosql.DB, *sqlutils.SQLRunner, func()) { - cfg := zonepb.DefaultZoneConfig() + cfg := config.DefaultZoneConfig() cfg.NumReplicas = proto.Int32(1) tsArgs := func(attr string) base.TestServerArgs { diff --git a/pkg/ccl/partitionccl/zone_test.go b/pkg/ccl/partitionccl/zone_test.go index da85a1d3455e..047df424f8cb 100644 --- a/pkg/ccl/partitionccl/zone_test.go +++ b/pkg/ccl/partitionccl/zone_test.go @@ -15,7 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -48,9 +48,9 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) { yamlDefault := fmt.Sprintf("gc: {ttlseconds: %d}", s.(*server.TestServer).Cfg.DefaultZoneConfig.GC.TTLSeconds) yamlOverride := "gc: {ttlseconds: 42}" zoneOverride := s.(*server.TestServer).Cfg.DefaultZoneConfig - zoneOverride.GC = &zonepb.GCPolicy{TTLSeconds: 42} - partialZoneOverride := *zonepb.NewZoneConfig() - partialZoneOverride.GC = &zonepb.GCPolicy{TTLSeconds: 42} + zoneOverride.GC = &config.GCPolicy{TTLSeconds: 42} + partialZoneOverride := *config.NewZoneConfig() + partialZoneOverride.GC = &config.GCPolicy{TTLSeconds: 42} dbID := sqlutils.QueryDatabaseID(t, db, "d") tableID := sqlutils.QueryTableID(t, db, "d", "t") diff --git a/pkg/cli/context.go b/pkg/cli/context.go index fec92531554f..d3803443ea96 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -17,7 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -106,8 +106,8 @@ func initCLIDefaults() { serverCfg.DelayedBootstrapFn = nil serverCfg.SocketFile = "" serverCfg.JoinList = nil - serverCfg.DefaultZoneConfig = zonepb.DefaultZoneConfig() - serverCfg.DefaultSystemZoneConfig = zonepb.DefaultSystemZoneConfig() + serverCfg.DefaultZoneConfig = config.DefaultZoneConfig() + serverCfg.DefaultSystemZoneConfig = config.DefaultSystemZoneConfig() startCtx.serverInsecure = baseCfg.Insecure startCtx.serverSSLCertsDir = base.DefaultCertsDirectory diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 2350cbb8c9b5..f2dd9a6d3cfa 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cli/syncbench" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -539,7 +538,7 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error { &desc, snap, hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}, - zonepb.GCPolicy{TTLSeconds: int32(gcTTLInSeconds)}, + config.GCPolicy{TTLSeconds: int32(gcTTLInSeconds)}, storage.NoopGCer{}, func(_ context.Context, _ []roachpb.Intent) error { return nil }, func(_ context.Context, _ *roachpb.Transaction, _ []roachpb.Intent) error { return nil }, diff --git a/pkg/cmd/github-pull-request-make/testdata/10305.diff b/pkg/cmd/github-pull-request-make/testdata/10305.diff index f67a633a5a7a..de024424ae8d 100644 --- a/pkg/cmd/github-pull-request-make/testdata/10305.diff +++ b/pkg/cmd/github-pull-request-make/testdata/10305.diff @@ -4533,7 +4533,7 @@ index 3c820233240..245a210d19b 100644 repl *Replica, + status *LeaseStatus, desc *roachpb.RangeDescriptor, - zone zonepb.ZoneConfig, + zone config.ZoneConfig, checkTransferLeaseSource bool, @@ -398,7 +399,7 @@ func (rq *replicateQueue) transferLease( ); target != (roachpb.ReplicaDescriptor{}) { diff --git a/pkg/cmd/gossipsim/main.go b/pkg/cmd/gossipsim/main.go index 69b7dd240b7d..8a48245a57fe 100644 --- a/pkg/cmd/gossipsim/main.go +++ b/pkg/cmd/gossipsim/main.go @@ -63,7 +63,7 @@ import ( "strconv" "strings" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/gossip/simulation" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -303,7 +303,7 @@ func main() { stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) - n := simulation.NewNetwork(stopper, nodeCount, true, zonepb.DefaultZoneConfigRef()) + n := simulation.NewNetwork(stopper, nodeCount, true, config.DefaultZoneConfigRef()) n.SimulateNetwork( func(cycle int, network *simulation.Network) bool { // Output dot graph. diff --git a/pkg/config/keys.go b/pkg/config/keys.go index 347f10b4e61f..9d4d626f287f 100644 --- a/pkg/config/keys.go +++ b/pkg/config/keys.go @@ -19,12 +19,15 @@ import ( // MakeZoneKeyPrefix returns the key prefix for id's row in the system.zones // table. func MakeZoneKeyPrefix(id uint32) roachpb.Key { - return keys.ZoneKeyPrefix(id) + k := keys.MakeTablePrefix(uint32(keys.ZonesTableID)) + k = encoding.EncodeUvarintAscending(k, uint64(keys.ZonesTablePrimaryIndexID)) + return encoding.EncodeUvarintAscending(k, uint64(id)) } // MakeZoneKey returns the key for id's entry in the system.zones table. func MakeZoneKey(id uint32) roachpb.Key { - return keys.ZoneKey(id) + k := MakeZoneKeyPrefix(id) + return keys.MakeFamilyKey(k, uint32(keys.ZonesTableConfigColFamID)) } // DecodeObjectID decodes the object ID from the front of key. It returns the diff --git a/pkg/config/system.go b/pkg/config/system.go index 3486d306c29c..cf6b51dd1824 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -16,17 +16,15 @@ import ( "fmt" "sort" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) type zoneConfigHook func( sysCfg *SystemConfig, objectID uint32, -) (zone *zonepb.ZoneConfig, placeholder *zonepb.ZoneConfig, cache bool, err error) +) (zone *ZoneConfig, placeholder *ZoneConfig, cache bool, err error) var ( // ZoneConfigHook is a function used to lookup a zone config given a table @@ -37,11 +35,15 @@ var ( // testingLargestIDHook is a function used to bypass GetLargestObjectID // in tests. testingLargestIDHook func(uint32) uint32 + + // SplitAtIDHook is a function that is used to check if a given + // descriptor comes from a database or a table view. + SplitAtIDHook func(uint32, *SystemConfig) bool ) type zoneEntry struct { - zone *zonepb.ZoneConfig - placeholder *zonepb.ZoneConfig + zone *ZoneConfig + placeholder *ZoneConfig // combined merges the zone and placeholder configs into a combined config. // If both have subzone information, the placeholder information is preferred. @@ -52,7 +54,7 @@ type zoneEntry struct { // enough bake time to ensure this is OK to do. Until then, only use the // combined value in GetZoneConfigForObject, which is only used by the // optimizer. - combined *zonepb.ZoneConfig + combined *ZoneConfig } // SystemConfig embeds a SystemConfigEntries message which contains an @@ -65,22 +67,20 @@ type zoneEntry struct { // that should not be considered for splits. type SystemConfig struct { SystemConfigEntries - DefaultZoneConfig *zonepb.ZoneConfig + DefaultZoneConfig *ZoneConfig mu struct { syncutil.RWMutex - zoneCache map[uint32]zoneEntry - // A cache keeping track of descriptor ids that are known to not need - // range splits. - shouldntSplitCache map[uint32]struct{} + zoneCache map[uint32]zoneEntry + shouldSplitCache map[uint32]bool } } // NewSystemConfig returns an initialized instance of SystemConfig. -func NewSystemConfig(defaultZoneConfig *zonepb.ZoneConfig) *SystemConfig { +func NewSystemConfig(defaultZoneConfig *ZoneConfig) *SystemConfig { sc := &SystemConfig{} sc.DefaultZoneConfig = defaultZoneConfig sc.mu.zoneCache = map[uint32]zoneEntry{} - sc.mu.shouldntSplitCache = map[uint32]struct{}{} + sc.mu.shouldSplitCache = map[uint32]bool{} return sc } @@ -107,15 +107,36 @@ func (s *SystemConfig) Equal(other *SystemConfigEntries) bool { return true } -// idRequiresTestingSplit is an unfortunate method that deals with low-level -// tests that install zone configs (in the unfortunate testingZoneConfig global -// map) without actually creating a corresponding table. They expect a split -// point around that zone, but were it not for this, they wouldn't get it. -func (s *SystemConfig) idRequiresTestingSplit(id uint32) bool { +// GetDesc looks for the descriptor value given a key, if a zone is created in +// a test without creating a Descriptor, a dummy descriptor is returned. +// If the key is invalid in decoding an ID, GetDesc panics. +func (s *SystemConfig) GetDesc(key roachpb.Key) *roachpb.Value { + if getVal := s.GetValue(key); getVal != nil { + return getVal + } + + id, err := keys.DecodeDescMetadataID(key) + if err != nil { + // No ID found for key. No roachpb.Value corresponds to this key. + panic(err) + } + testingLock.Lock() - _, ok := testingZoneConfig[id] + _, ok := testingZoneConfig[uint32(id)] testingLock.Unlock() - return ok + + if ok { + // A test installed a zone config for this ID, but no descriptor. + // Synthesize an empty descriptor to force split to occur, or else the + // zone config won't apply to any ranges. Most tests that use + // TestingSetZoneConfig are too low-level to create tables and zone + // configs through proper channels. + // + // Getting here outside tests is impossible. + val := roachpb.MakeValueFromBytes(nil) + return &val + } + return nil } // GetValue searches the kv list for 'key' and returns its @@ -229,7 +250,7 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { // GetZoneConfigForKey looks up the zone config for the object (table // or database, specified by key.id). It is the caller's // responsibility to ensure that the range does not need to be split. -func (s *SystemConfig) GetZoneConfigForKey(key roachpb.RKey) (*zonepb.ZoneConfig, error) { +func (s *SystemConfig) GetZoneConfigForKey(key roachpb.RKey) (*ZoneConfig, error) { return s.getZoneConfigForKey(DecodeKeyIntoZoneIDAndSuffix(key)) } @@ -279,7 +300,7 @@ func isPseudoTableID(id uint32) bool { // identifier. // NOTE: any subzones from the zone placeholder will be automatically merged // into the cached zone so the caller doesn't need special-case handling code. -func (s *SystemConfig) GetZoneConfigForObject(id uint32) (*zonepb.ZoneConfig, error) { +func (s *SystemConfig) GetZoneConfigForObject(id uint32) (*ZoneConfig, error) { entry, err := s.getZoneEntry(id) if err != nil { return nil, err @@ -289,7 +310,7 @@ func (s *SystemConfig) GetZoneConfigForObject(id uint32) (*zonepb.ZoneConfig, er // getZoneEntry returns the zone entry for the given object ID. In the fast // path, the zone is already in the cache, and is directly returned. Otherwise, -// getZoneEntry will hydrate new zonepb.ZoneConfig(s) from the SystemConfig and install +// getZoneEntry will hydrate new ZoneConfig(s) from the SystemConfig and install // them as an entry in the cache. func (s *SystemConfig) getZoneEntry(id uint32) (zoneEntry, error) { s.mu.RLock() @@ -326,9 +347,7 @@ func (s *SystemConfig) getZoneEntry(id uint32) (zoneEntry, error) { return zoneEntry{}, nil } -func (s *SystemConfig) getZoneConfigForKey( - id uint32, keySuffix []byte, -) (*zonepb.ZoneConfig, error) { +func (s *SystemConfig) getZoneConfigForKey(id uint32, keySuffix []byte) (*ZoneConfig, error) { entry, err := s.getZoneEntry(id) if err != nil { return nil, err @@ -359,7 +378,7 @@ var staticSplits = []roachpb.RKey{ roachpb.RKey(keys.NodeLivenessKeyMax), // end of node liveness span roachpb.RKey(keys.TimeseriesPrefix), // start of timeseries span roachpb.RKey(keys.TimeseriesPrefix.PrefixEnd()), // end of timeseries span - roachpb.RKey(keys.SystemConfigSplitKey), // end of system ranges / start of system config tables + roachpb.RKey(keys.TableDataMin), // end of system ranges / start of system config tables } // StaticSplits are predefined split points in the system keyspace. @@ -381,188 +400,135 @@ func StaticSplits() []roachpb.RKey { // ComputeSplitKey takes a start and end key and returns the first key at which // to split the span [start, end). Returns nil if no splits are required. // -// Splits are required between user tables (i.e. /table/), between -// indexes/partitions that have zone configs applied to them, at the start of -// the system-config tables (i.e. /table/0), and at certain points within the +// Splits are required between user tables (i.e. /table/), at the start +// of the system-config tables (i.e. /table/0), and at certain points within the // system ranges that come before the system tables. The system-config range is // somewhat special in that it can contain multiple SQL tables // (/table/0-/table/) within a single range. -func (s *SystemConfig) ComputeSplitKey( - ctx context.Context, startKey, endKey roachpb.RKey, -) roachpb.RKey { - // If the span we want to split starts in the static splits region, deal with - // it. - // Since v19.1, these split points are created at startup for new clusters, so - // this function is not expected to be called for these spans. - // In theory, stores created by an older version benefit from this code if - // they had never gone through the split queue. - maxStaticSplit := staticSplits[len(staticSplits)-1] - if startKey.Less(maxStaticSplit) { - for _, split := range staticSplits { - if startKey.Less(split) { - if split.Less(endKey) { - // The split point is contained within [startKey, endKey), so we need to - // create the split. - return split - } - // [startKey, endKey) is contained between the previous split point and - // this split point. - return nil +func (s *SystemConfig) ComputeSplitKey(startKey, endKey roachpb.RKey) (rr roachpb.RKey) { + // Before dealing with splits necessitated by SQL tables, handle all of the + // static splits earlier in the keyspace. Note that this list must be kept in + // the proper order (ascending in the keyspace) for the logic below to work. + // + // For new clusters, the static splits correspond to ranges created at + // bootstrap time. Older stores might be used with a version with more + // staticSplits though, in which case this code is useful. + for _, split := range staticSplits { + if startKey.Less(split) { + if split.Less(endKey) { + // The split point is contained within [startKey, endKey), so we need to + // create the split. + return split } + // [startKey, endKey) is contained between the previous split point and + // this split point. + return nil } + // [startKey, endKey) is somewhere greater than this split point. Continue. } - // If the span starts below user space, deal with it. - // Here we split at every table or pseudo-table boundary. Same as above, most - // of these split points are created at cluster bootstrap, but there's - // migrations (new system tables) that benefit from this code. - userTableMin := roachpb.RKey(keys.UserTableDataMin) - if startKey.Less(userTableMin) { - startID, _, ok := DecodeObjectID(startKey) - if !ok || startID <= keys.MaxSystemConfigDescID { - // We don't split in the system config range. - startID = keys.MaxSystemConfigDescID + 1 - } - // We're going to split at every table boundary between - // MaxSystemConfigDescID+1 and the largest system table id (inclusive). - endID, err := s.GetLargestObjectID(keys.MaxReservedDescID) - if err != nil { - log.Fatalf(ctx, "unable to determine largest reserved object ID from system config: %s", err) - } + // If the above iteration over the static split points didn't decide anything, + // the key range must be somewhere in the SQL table part of the keyspace. + startID, _, ok := DecodeObjectID(startKey) + if !ok || startID <= keys.MaxSystemConfigDescID { + // The start key is either: + // - not part of the structured data span + // - part of the system span + // In either case, start looking for splits at the first ID usable + // by the user data span. + startID = keys.MaxSystemConfigDescID + 1 + } + + // Build key prefixes for sequential table IDs until we reach endKey. Note + // that there are two disjoint sets of sequential keys: non-system reserved + // tables have sequential IDs, as do user tables, but the two ranges contain a + // gap. + + // findSplitKey returns the first possible split key between the given range + // of IDs. + findSplitKey := func(startID, endID uint32) roachpb.RKey { + // endID could be smaller than startID if we don't have user tables. for id := startID; id <= endID; id++ { - // This code is not equipped to deal with subzones (which would require - // splits), so assume that the system zones (to the extent that they - // exist) don't have them. - s.assertNoSubzones(ctx, id) tableKey := roachpb.RKey(keys.MakeTablePrefix(id)) - if startKey.Less(tableKey) { + // This logic is analogous to the well-commented static split logic above. + if startKey.Less(tableKey) && s.shouldSplit(id) { if tableKey.Less(endKey) { - // The split point is contained within [startKey, endKey), so we need to - // create the split. return tableKey } - // [startKey, endKey) is contained between the previous split point and - // this split point. return nil } - } - } - - return s.findUserspaceSplitKey(ctx, startKey, endKey) -} -// findUserspaceSplitKey returns the first possible split point for the -// [startKey-endKey) by only considering user-space splits (i.e. between tables -// and possibly indexes and partitions). Ultimately the decision of where to -// split is delegated to sqlbase. -func (s *SystemConfig) findUserspaceSplitKey( - ctx context.Context, startKey, endKey roachpb.RKey, -) roachpb.RKey { - startID, _, ok := DecodeObjectID(startKey) - if !ok { - log.Fatalf(ctx, "failed to decode user-space span: %s-%s", startKey, endKey) - } - // We might have been given a key that's technically below user space - // (e.g. the start of the last system table). We're going to start searching for - // splits in the user space key range. - if startID < keys.MinUserDescID { - startID = keys.MinUserDescID - } - - endID, err := s.GetLargestObjectID(0 /* maxID */) - if err != nil { - log.Fatalf(ctx, "unable to determine largest object ID from system config: %s", err) - } - - // Iterate over all the tables in between startKey and endKey. - // We'll return early as soon as we get to the first split point. - // endID could be smaller than startID if we don't have user tables. - for id := startID; id <= endID; id++ { - if s.idRequiresTestingSplit(id) { - tableKey := roachpb.RKey(keys.MakeTablePrefix(id)) - if startKey.Less(tableKey) && tableKey.Less(endKey) { - return tableKey + zoneVal := s.GetValue(MakeZoneKey(id)) + if zoneVal == nil { + continue } - } - if s.shouldntSplit(id) { - continue - } - - // Read the zone config for the table. - zoneVal := s.GetValue(MakeZoneKey(id)) - var zone *zonepb.ZoneConfig - if zoneVal != nil { - zone = new(zonepb.ZoneConfig) - if err := zoneVal.GetProto(zone); err != nil { + var zone ZoneConfig + if err := zoneVal.GetProto(&zone); err != nil { // An error while decoding the zone proto is unfortunate, but logging a // message here would be excessively spammy. Just move on, which // effectively assumes there are no subzones for this table. - zone = nil + continue + } + // This logic is analogous to the well-commented static split logic above. + for _, s := range zone.subzoneSplits() { + subzoneKey := append(tableKey, s...) + if startKey.Less(subzoneKey) { + if subzoneKey.Less(endKey) { + return subzoneKey + } + return nil + } } } + return nil + } - descVal := s.GetValue(keys.DescMetadataKey(id)) - if descVal == nil { - continue - } - - // Here we delegate to SQL to give us the split points for this table. - tableSplits, err := sqlbase.SplitKeysForTable(descVal, zone) + // If the startKey falls within the non-system reserved range, compute those + // keys first. + if startID <= keys.MaxReservedDescID { + endID, err := s.GetLargestObjectID(keys.MaxReservedDescID) if err != nil { - log.Fatalf(ctx, "unexpected failure to compute split keys: %s", err) + log.Errorf(context.TODO(), "unable to determine largest reserved object ID from system config: %s", err) + return nil } - if tableSplits == nil { - // SQL just told us that this descriptor is not a table. - // Remember to not attempt it again. - s.cacheNonSplittableDesc(id) - continue - } - // Return the smallest splitKey above the startKey. - for _, k := range tableSplits { - if startKey.Less(k) && k.Less(endKey) { - return k - } + if splitKey := findSplitKey(startID, endID); splitKey != nil { + return splitKey } + startID = keys.MaxReservedDescID + 1 } - return nil + + // Find the split key in the user space. + endID, err := s.GetLargestObjectID(0) + if err != nil { + log.Errorf(context.TODO(), "unable to determine largest object ID from system config: %s", err) + return nil + } + return findSplitKey(startID, endID) } // NeedsSplit returns whether the range [startKey, endKey) needs a split due -// to table boundaries or zone configs. -func (s *SystemConfig) NeedsSplit(ctx context.Context, startKey, endKey roachpb.RKey) bool { - return len(s.ComputeSplitKey(ctx, startKey, endKey)) > 0 +// to zone configs. +func (s *SystemConfig) NeedsSplit(startKey, endKey roachpb.RKey) bool { + return len(s.ComputeSplitKey(startKey, endKey)) > 0 } // shouldSplit checks if the ID is eligible for a split at all. // It uses the internal cache to find a value, and tries to find -// it using the hook if keeping ID isn't found in the cache. -func (s *SystemConfig) shouldntSplit(ID uint32) bool { +// it using the hook if ID isn't found in the cache. +func (s *SystemConfig) shouldSplit(ID uint32) bool { s.mu.RLock() - _, ok := s.mu.shouldntSplitCache[ID] + shouldSplit, ok := s.mu.shouldSplitCache[ID] s.mu.RUnlock() - return ok -} - -// cacheNonSplittableDesc remembers that id corresponds to a descriptor that -// doesn't need any splits. -func (s *SystemConfig) cacheNonSplittableDesc(id uint32) { - s.mu.Lock() - s.mu.shouldntSplitCache[id] = struct{}{} - s.mu.Unlock() -} - -// assertNoSubzones fatals if the config for zone id contains any subzones. -func (s *SystemConfig) assertNoSubzones(ctx context.Context, id uint32) { - zoneVal := s.GetValue(MakeZoneKey(id)) - var zone *zonepb.ZoneConfig - if zoneVal == nil { - return - } - zone = new(zonepb.ZoneConfig) - if err := zoneVal.GetProto(zone); err != nil { - return - } - if len(zone.Subzones) != 0 { - log.Fatalf(ctx, "unexpected subzones for table: %d", id) + if !ok { + // Check if the descriptor ID is not one of the reserved + // IDs that refer to ranges but not any actual descriptors. + shouldSplit = true + if ID >= keys.MinUserDescID { + shouldSplit = SplitAtIDHook(ID, s) + } + s.mu.Lock() + s.mu.shouldSplitCache[ID] = shouldSplit + s.mu.Unlock() } + return shouldSplit } diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index 88685c2afeb2..0cbde1a404c2 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -11,18 +11,15 @@ package config_test import ( - "context" "sort" "testing" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/gogo/protobuf/proto" ) @@ -49,40 +46,22 @@ func sqlKV(tableID uint32, indexID, descriptorID uint64) roachpb.KeyValue { return kv(k, nil) } -// descriptor returns a KeyValue, with the keys being a descriptor's key, and -// the value being an empty TableDescriptor proto (only with the ID set). func descriptor(descriptorID uint64) roachpb.KeyValue { - kv := roachpb.KeyValue{ - Key: sqlbase.MakeDescMetadataKey(sqlbase.ID(descriptorID)), - } - if err := kv.Value.SetProto(&sqlbase.Descriptor{ - Union: &sqlbase.Descriptor_Table{ - Table: &sqlbase.TableDescriptor{ - // Fill in the descriptor just enough for the test to work. - ID: sqlbase.ID(descriptorID), - }, - }, - }); err != nil { - panic(err) - } - // We need to set some timestamp on this proto, otherwise unwrapping the - // descriptor fatals. - kv.Value.Timestamp = hlc.Timestamp{WallTime: 123} - return kv + return kv(sqlbase.MakeDescMetadataKey(sqlbase.ID(descriptorID)), nil) } -func zoneConfig(descriptorID uint32, spans ...zonepb.SubzoneSpan) roachpb.KeyValue { +func zoneConfig(descriptorID uint32, spans ...config.SubzoneSpan) roachpb.KeyValue { kv := roachpb.KeyValue{ Key: config.MakeZoneKey(descriptorID), } - if err := kv.Value.SetProto(&zonepb.ZoneConfig{SubzoneSpans: spans}); err != nil { + if err := kv.Value.SetProto(&config.ZoneConfig{SubzoneSpans: spans}); err != nil { panic(err) } return kv } -func subzone(start, end string) zonepb.SubzoneSpan { - return zonepb.SubzoneSpan{Key: []byte(start), EndKey: []byte(end)} +func subzone(start, end string) config.SubzoneSpan { + return config.SubzoneSpan{Key: []byte(start), EndKey: []byte(end)} } func kv(k, v []byte) roachpb.KeyValue { @@ -127,7 +106,7 @@ func TestGet(t *testing.T) { {someKeys, "d", &cVal}, } - cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) + cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) for tcNum, tc := range testCases { cfg.Values = tc.values if val := cfg.GetValue([]byte(tc.key)); !proto.Equal(val, tc.value) { @@ -181,7 +160,7 @@ func TestGetLargestID(t *testing.T) { // Real SQL layout. func() testCase { - ms := sqlbase.MakeMetadataSchema(zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + ms := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) descIDs := ms.DescriptorIDs() maxDescID := descIDs[len(descIDs)-1] kvs, _ /* splits */ := ms.GetInitialValues() @@ -205,7 +184,7 @@ func TestGetLargestID(t *testing.T) { }, 5, 7, ""}, } - cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) + cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) for tcNum, tc := range testCases { cfg.Values = tc.values ret, err := cfg.GetLargestObjectID(tc.maxID) @@ -276,13 +255,13 @@ func TestComputeSplitKeySystemRanges(t *testing.T) { {roachpb.RKey(keys.TimeseriesPrefix.PrefixEnd()), roachpb.RKeyMax, keys.SystemConfigSplitKey}, } - cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) - kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef()).GetInitialValues() + cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) + kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues() cfg.SystemConfigEntries = config.SystemConfigEntries{ Values: kvs, } for tcNum, tc := range testCases { - splitKey := cfg.ComputeSplitKey(context.Background(), tc.start, tc.end) + splitKey := cfg.ComputeSplitKey(tc.start, tc.end) expected := roachpb.RKey(tc.split) if !splitKey.Equal(expected) { t.Errorf("#%d: bad split:\ngot: %v\nexpected: %v", tcNum, splitKey, expected) @@ -307,7 +286,7 @@ func TestComputeSplitKeyTableIDs(t *testing.T) { // separately above. minKey := roachpb.RKey(keys.TimeseriesPrefix.PrefixEnd()) - schema := sqlbase.MakeMetadataSchema(zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + schema := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) // Real system tables only. baseSql, _ /* splits */ := schema.GetInitialValues() // Real system tables plus some user stuff. @@ -326,6 +305,12 @@ func TestComputeSplitKeyTableIDs(t *testing.T) { start, end roachpb.RKey split roachpb.RKey // nil to indicate no split is expected }{ + // No data. + {nil, minKey, roachpb.RKeyMax, tkey(0)}, + {nil, tkey(start), roachpb.RKeyMax, nil}, + {nil, tkey(start), tkey(start + 10), nil}, + {nil, minKey, tkey(start + 10), tkey(0)}, + // Reserved descriptors. {baseSql, minKey, roachpb.RKeyMax, tkey(0)}, {baseSql, tkey(start), roachpb.RKeyMax, nil}, @@ -380,10 +365,10 @@ func TestComputeSplitKeyTableIDs(t *testing.T) { {userSQL, tkey(start + 1), tkey(start + 5), nil}, } - cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) + cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) for tcNum, tc := range testCases { cfg.Values = tc.values - splitKey := cfg.ComputeSplitKey(context.Background(), tc.start, tc.end) + splitKey := cfg.ComputeSplitKey(tc.start, tc.end) if !splitKey.Equal(tc.split) { t.Errorf("#%d: bad split:\ngot: %v\nexpected: %v", tcNum, splitKey, tc.split) } @@ -443,9 +428,9 @@ func TestGetZoneConfigForKey(t *testing.T) { defer func() { config.ZoneConfigHook = originalZoneConfigHook }() - cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) + cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) - kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, zonepb.DefaultSystemZoneConfigRef()).GetInitialValues() + kvs, _ /* splits */ := sqlbase.MakeMetadataSchema(cfg.DefaultZoneConfig, config.DefaultSystemZoneConfigRef()).GetInitialValues() cfg.SystemConfigEntries = config.SystemConfigEntries{ Values: kvs, } @@ -453,9 +438,9 @@ func TestGetZoneConfigForKey(t *testing.T) { var objectID uint32 config.ZoneConfigHook = func( _ *config.SystemConfig, id uint32, - ) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) { + ) (*config.ZoneConfig, *config.ZoneConfig, bool, error) { objectID = id - return &zonepb.ZoneConfig{}, nil, false, nil + return &config.ZoneConfig{}, nil, false, nil } _, err := cfg.GetZoneConfigForKey(tc.key) if err != nil { diff --git a/pkg/config/testutil.go b/pkg/config/testutil.go index 8c0ee7782c51..1fb4352222e1 100644 --- a/pkg/config/testutil.go +++ b/pkg/config/testutil.go @@ -11,12 +11,11 @@ package config import ( - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) -type zoneConfigMap map[uint32]zonepb.ZoneConfig +type zoneConfigMap map[uint32]ZoneConfig var ( testingZoneConfig zoneConfigMap @@ -70,15 +69,13 @@ func testingResetZoneConfigHook() { // TestingSetZoneConfig sets the zone config entry for object 'id' // in the testing map. -func TestingSetZoneConfig(id uint32, zone zonepb.ZoneConfig) { +func TestingSetZoneConfig(id uint32, zone ZoneConfig) { testingLock.Lock() defer testingLock.Unlock() testingZoneConfig[id] = zone } -func testingZoneConfigHook( - _ *SystemConfig, id uint32, -) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) { +func testingZoneConfigHook(_ *SystemConfig, id uint32) (*ZoneConfig, *ZoneConfig, bool, error) { testingLock.Lock() defer testingLock.Unlock() if zone, ok := testingZoneConfig[id]; ok { diff --git a/pkg/config/zonepb/zone.go b/pkg/config/zone.go similarity index 99% rename from pkg/config/zonepb/zone.go rename to pkg/config/zone.go index eccf0045a0ea..5ca7b432705c 100644 --- a/pkg/config/zonepb/zone.go +++ b/pkg/config/zone.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package zonepb +package config import ( "bytes" @@ -583,8 +583,7 @@ func (z *ZoneConfig) DeleteIndexSubzones(indexID uint32) { z.Subzones = subzones } -// SubzoneSplits returns the split points determined by a ZoneConfig's subzones. -func (z ZoneConfig) SubzoneSplits() []roachpb.RKey { +func (z ZoneConfig) subzoneSplits() []roachpb.RKey { var out []roachpb.RKey for _, span := range z.SubzoneSpans { // TODO(benesch): avoid a split at the first partition's start key when it diff --git a/pkg/config/zonepb/zone.pb.go b/pkg/config/zone.pb.go similarity index 89% rename from pkg/config/zonepb/zone.pb.go rename to pkg/config/zone.pb.go index 80a3ea607366..21763b715b3c 100644 --- a/pkg/config/zonepb/zone.pb.go +++ b/pkg/config/zone.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: config/zonepb/zone.proto +// source: config/zone.proto -package zonepb +package config import proto "github.com/gogo/protobuf/proto" import fmt "fmt" @@ -64,7 +64,7 @@ func (x *Constraint_Type) UnmarshalJSON(data []byte) error { return nil } func (Constraint_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_zone_aa6a4d1d521e856b, []int{1, 0} + return fileDescriptor_zone_db9e8f274931475b, []int{1, 0} } // GCPolicy defines garbage collection policies which apply to MVCC @@ -84,7 +84,7 @@ func (m *GCPolicy) Reset() { *m = GCPolicy{} } func (m *GCPolicy) String() string { return proto.CompactTextString(m) } func (*GCPolicy) ProtoMessage() {} func (*GCPolicy) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_aa6a4d1d521e856b, []int{0} + return fileDescriptor_zone_db9e8f274931475b, []int{0} } func (m *GCPolicy) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -111,7 +111,7 @@ var xxx_messageInfo_GCPolicy proto.InternalMessageInfo // Constraint constrains the stores a replica can be stored on. type Constraint struct { - Type Constraint_Type `protobuf:"varint,1,opt,name=type,enum=cockroach.config.zonepb.Constraint_Type" json:"type"` + Type Constraint_Type `protobuf:"varint,1,opt,name=type,enum=cockroach.config.Constraint_Type" json:"type"` // Key is only set if this is a constraint on locality. Key string `protobuf:"bytes,2,opt,name=key" json:"key"` // Value to constrain to. @@ -121,7 +121,7 @@ type Constraint struct { func (m *Constraint) Reset() { *m = Constraint{} } func (*Constraint) ProtoMessage() {} func (*Constraint) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_aa6a4d1d521e856b, []int{1} + return fileDescriptor_zone_db9e8f274931475b, []int{1} } func (m *Constraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -161,7 +161,7 @@ func (m *Constraints) Reset() { *m = Constraints{} } func (m *Constraints) String() string { return proto.CompactTextString(m) } func (*Constraints) ProtoMessage() {} func (*Constraints) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_aa6a4d1d521e856b, []int{2} + return fileDescriptor_zone_db9e8f274931475b, []int{2} } func (m *Constraints) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -196,7 +196,7 @@ func (m *LeasePreference) Reset() { *m = LeasePreference{} } func (m *LeasePreference) String() string { return proto.CompactTextString(m) } func (*LeasePreference) ProtoMessage() {} func (*LeasePreference) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_aa6a4d1d521e856b, []int{3} + return fileDescriptor_zone_db9e8f274931475b, []int{3} } func (m *LeasePreference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -274,7 +274,7 @@ func (m *ZoneConfig) Reset() { *m = ZoneConfig{} } func (m *ZoneConfig) String() string { return proto.CompactTextString(m) } func (*ZoneConfig) ProtoMessage() {} func (*ZoneConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_aa6a4d1d521e856b, []int{4} + return fileDescriptor_zone_db9e8f274931475b, []int{4} } func (m *ZoneConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -315,7 +315,7 @@ func (m *Subzone) Reset() { *m = Subzone{} } func (m *Subzone) String() string { return proto.CompactTextString(m) } func (*Subzone) ProtoMessage() {} func (*Subzone) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_aa6a4d1d521e856b, []int{5} + return fileDescriptor_zone_db9e8f274931475b, []int{5} } func (m *Subzone) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -361,7 +361,7 @@ func (m *SubzoneSpan) Reset() { *m = SubzoneSpan{} } func (m *SubzoneSpan) String() string { return proto.CompactTextString(m) } func (*SubzoneSpan) ProtoMessage() {} func (*SubzoneSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_aa6a4d1d521e856b, []int{6} + return fileDescriptor_zone_db9e8f274931475b, []int{6} } func (m *SubzoneSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -387,14 +387,14 @@ func (m *SubzoneSpan) XXX_DiscardUnknown() { var xxx_messageInfo_SubzoneSpan proto.InternalMessageInfo func init() { - proto.RegisterType((*GCPolicy)(nil), "cockroach.config.zonepb.GCPolicy") - proto.RegisterType((*Constraint)(nil), "cockroach.config.zonepb.Constraint") - proto.RegisterType((*Constraints)(nil), "cockroach.config.zonepb.Constraints") - proto.RegisterType((*LeasePreference)(nil), "cockroach.config.zonepb.LeasePreference") - proto.RegisterType((*ZoneConfig)(nil), "cockroach.config.zonepb.ZoneConfig") - proto.RegisterType((*Subzone)(nil), "cockroach.config.zonepb.Subzone") - proto.RegisterType((*SubzoneSpan)(nil), "cockroach.config.zonepb.SubzoneSpan") - proto.RegisterEnum("cockroach.config.zonepb.Constraint_Type", Constraint_Type_name, Constraint_Type_value) + proto.RegisterType((*GCPolicy)(nil), "cockroach.config.GCPolicy") + proto.RegisterType((*Constraint)(nil), "cockroach.config.Constraint") + proto.RegisterType((*Constraints)(nil), "cockroach.config.Constraints") + proto.RegisterType((*LeasePreference)(nil), "cockroach.config.LeasePreference") + proto.RegisterType((*ZoneConfig)(nil), "cockroach.config.ZoneConfig") + proto.RegisterType((*Subzone)(nil), "cockroach.config.Subzone") + proto.RegisterType((*SubzoneSpan)(nil), "cockroach.config.SubzoneSpan") + proto.RegisterEnum("cockroach.config.Constraint_Type", Constraint_Type_name, Constraint_Type_value) } func (this *GCPolicy) Equal(that interface{}) bool { if that == nil { @@ -2379,62 +2379,62 @@ var ( ErrIntOverflowZone = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("config/zonepb/zone.proto", fileDescriptor_zone_aa6a4d1d521e856b) } - -var fileDescriptor_zone_aa6a4d1d521e856b = []byte{ - // 858 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0x3b, 0x73, 0xe3, 0x54, - 0x14, 0xf6, 0xb5, 0x1d, 0x5b, 0x7b, 0xec, 0x24, 0xe6, 0xee, 0xb2, 0x11, 0x61, 0xb0, 0x8c, 0x80, - 0xc1, 0xbc, 0x64, 0x26, 0xd0, 0x90, 0x19, 0x8a, 0x95, 0x6d, 0x16, 0x91, 0x7d, 0x18, 0xc5, 0x50, - 0x2c, 0x85, 0x46, 0x96, 0x6e, 0x14, 0x4d, 0xe4, 0x2b, 0x8d, 0x24, 0x43, 0xc4, 0xd0, 0x51, 0x33, - 0x43, 0x49, 0x99, 0x7f, 0x00, 0x25, 0x3f, 0x21, 0xe5, 0xd2, 0x2d, 0x8d, 0x07, 0x9c, 0x86, 0x3a, - 0x25, 0x15, 0xa3, 0x2b, 0x59, 0x92, 0xb3, 0xeb, 0x25, 0x50, 0xe9, 0xea, 0x9c, 0xef, 0x7c, 0xe7, - 0x7d, 0x80, 0x37, 0x5c, 0x7a, 0x64, 0x5b, 0xbd, 0x6f, 0x5d, 0x4a, 0xbc, 0x09, 0xfb, 0x48, 0x9e, - 0xef, 0x86, 0x2e, 0xde, 0x31, 0x5c, 0xe3, 0xc4, 0x77, 0x75, 0xe3, 0x58, 0x4a, 0x30, 0x52, 0x82, - 0xd9, 0xbd, 0x65, 0xb9, 0x96, 0xcb, 0x30, 0xbd, 0xf8, 0x95, 0xc0, 0x45, 0x05, 0xb8, 0xbb, 0xfd, - 0x91, 0xeb, 0xd8, 0x46, 0x84, 0x3f, 0x80, 0x46, 0x18, 0x3a, 0x5a, 0x40, 0x0c, 0x97, 0x9a, 0x01, - 0x8f, 0x3a, 0xa8, 0xbb, 0x21, 0xe3, 0xf3, 0xb9, 0x50, 0x5a, 0xcc, 0x05, 0x18, 0x8f, 0xef, 0x1d, - 0x26, 0x1a, 0x15, 0xc2, 0xd0, 0x49, 0xdf, 0xfb, 0xdc, 0xaf, 0x67, 0x02, 0xfa, 0xeb, 0x4c, 0x40, - 0xe2, 0x6f, 0x08, 0xa0, 0xef, 0xd2, 0x20, 0xf4, 0x75, 0x9b, 0x86, 0x58, 0x86, 0x6a, 0x18, 0x79, - 0x84, 0xd1, 0x6c, 0xed, 0x75, 0xa5, 0x35, 0x71, 0x49, 0xb9, 0x89, 0x34, 0x8e, 0x3c, 0x22, 0x57, - 0x63, 0x87, 0x2a, 0xb3, 0xc5, 0xb7, 0xa1, 0x72, 0x42, 0x22, 0xbe, 0xdc, 0x41, 0xdd, 0x1b, 0xa9, - 0x22, 0x16, 0xe0, 0x5d, 0xd8, 0xf8, 0x5a, 0x77, 0x66, 0x84, 0xaf, 0x14, 0x34, 0x89, 0x48, 0xfc, - 0x18, 0xaa, 0x31, 0x0f, 0xde, 0x81, 0x9b, 0x83, 0xe1, 0x48, 0x1d, 0xf6, 0xef, 0x8c, 0x87, 0x03, - 0x6d, 0xf4, 0xf0, 0x50, 0x19, 0x2b, 0x5f, 0x0e, 0x5b, 0x25, 0xdc, 0x04, 0x4e, 0x1d, 0x7e, 0xfe, - 0x85, 0xa2, 0x0e, 0x07, 0x2d, 0x84, 0xb7, 0x00, 0x46, 0xea, 0xc3, 0x4f, 0x15, 0x59, 0x19, 0x0f, - 0x07, 0xad, 0xf2, 0x7e, 0xf3, 0xa7, 0x33, 0xa1, 0x94, 0xe5, 0xf4, 0x03, 0x82, 0x46, 0x1e, 0x60, - 0x80, 0x0f, 0xa0, 0x61, 0xe4, 0xbf, 0x7c, 0xad, 0x53, 0xe9, 0x36, 0xf6, 0x5e, 0xbb, 0x46, 0x6e, - 0x69, 0x8c, 0x45, 0x6b, 0xfc, 0x26, 0x34, 0xe9, 0x6c, 0xaa, 0xf9, 0xc4, 0x73, 0x6c, 0x43, 0x0f, - 0xf8, 0x3a, 0x2b, 0x78, 0x0a, 0xa4, 0xb3, 0xa9, 0x9a, 0x2a, 0x0a, 0x35, 0xfe, 0x1e, 0xc1, 0xf6, - 0x3d, 0xa2, 0x07, 0x64, 0xe4, 0x93, 0x23, 0xe2, 0x13, 0x6a, 0x10, 0x7c, 0xb4, 0x1a, 0x13, 0xba, - 0x7e, 0x4c, 0x42, 0xec, 0xea, 0x72, 0x2e, 0xec, 0x44, 0xfa, 0xd4, 0xd9, 0x17, 0x0b, 0x2c, 0xef, - 0x1e, 0x39, 0xee, 0x37, 0xe2, 0x4a, 0xb8, 0x85, 0x28, 0x7e, 0xa9, 0x01, 0x3c, 0x72, 0x29, 0xe9, - 0x33, 0x62, 0x2c, 0xc3, 0xb6, 0xaf, 0x53, 0x8b, 0x68, 0x53, 0x9b, 0x6a, 0x93, 0x28, 0x24, 0x01, - 0xeb, 0x58, 0x45, 0xde, 0xbd, 0x9c, 0x0b, 0xb7, 0x13, 0xee, 0x2b, 0x00, 0x51, 0xdd, 0x64, 0x92, - 0xfb, 0x36, 0x95, 0xe3, 0xff, 0x02, 0x87, 0x7e, 0x9a, 0x72, 0x54, 0xd6, 0x70, 0x2c, 0x01, 0x19, - 0x87, 0x7e, 0x9a, 0x70, 0x7c, 0x04, 0x65, 0xcb, 0xe0, 0xab, 0x1d, 0xd4, 0x6d, 0xec, 0xbd, 0xba, - 0x36, 0xff, 0xe5, 0xb8, 0xcb, 0xb5, 0xc5, 0x5c, 0x28, 0xdf, 0xed, 0xab, 0x65, 0xcb, 0xc0, 0xfb, - 0x57, 0x5a, 0xb1, 0xc1, 0x5a, 0xb1, 0x73, 0x39, 0x17, 0x6e, 0x26, 0xbe, 0x8b, 0x5a, 0x71, 0xa5, - 0x3b, 0xd8, 0x7a, 0xd6, 0x4c, 0xbc, 0x7e, 0x8d, 0xfa, 0x07, 0xff, 0xad, 0x01, 0xf8, 0x2b, 0xd8, - 0x0c, 0x66, 0x93, 0x98, 0x46, 0x0b, 0x3c, 0x9d, 0xc6, 0x03, 0xf3, 0x7c, 0x57, 0x87, 0x09, 0xfa, - 0xd0, 0xd3, 0xa9, 0xdc, 0x4a, 0x5d, 0x71, 0x89, 0xab, 0xf7, 0x44, 0xb5, 0x19, 0xe4, 0xea, 0x00, - 0x3f, 0x00, 0x2e, 0xfd, 0x0f, 0x78, 0x8e, 0xf1, 0x76, 0xfe, 0x8d, 0xf7, 0x19, 0x9c, 0x19, 0x07, - 0xfe, 0x0e, 0x5e, 0x70, 0xe2, 0x41, 0xd5, 0xbc, 0x6c, 0x52, 0x03, 0xfe, 0x06, 0x23, 0x5e, 0x7f, - 0x0b, 0xae, 0x8c, 0xb6, 0xfc, 0x46, 0xea, 0xe0, 0x95, 0xc4, 0xc1, 0x53, 0x84, 0x69, 0x95, 0x5a, - 0xce, 0xaa, 0x5d, 0x3c, 0x0a, 0x2f, 0xda, 0xf4, 0x98, 0xf8, 0x76, 0x48, 0x4c, 0xad, 0xd8, 0x1d, - 0xe8, 0xa0, 0x2e, 0x97, 0xee, 0xd8, 0xad, 0x0c, 0x52, 0x5c, 0xf1, 0x01, 0xbc, 0x9c, 0x9b, 0x3e, - 0x9d, 0x42, 0xa3, 0x40, 0xf0, 0x52, 0x06, 0xbc, 0x12, 0x78, 0x61, 0x59, 0x3e, 0xab, 0x72, 0xa8, - 0x55, 0x16, 0x7f, 0x46, 0x50, 0x4f, 0xcb, 0x86, 0xdf, 0x06, 0xce, 0xa6, 0x26, 0x39, 0xd5, 0x6c, - 0x93, 0x5d, 0xc7, 0x4d, 0x79, 0x3b, 0x3d, 0xb2, 0x75, 0x25, 0x96, 0x2b, 0x03, 0xb5, 0xce, 0x00, - 0x8a, 0x89, 0xdf, 0x81, 0x2d, 0x4f, 0xf7, 0x43, 0x3b, 0xb4, 0x5d, 0xaa, 0x51, 0x7d, 0x4a, 0x56, - 0x8e, 0xe1, 0x66, 0xa6, 0x7b, 0xa0, 0x4f, 0x09, 0xbe, 0x03, 0xb5, 0xa4, 0x9e, 0x6c, 0x77, 0x9e, - 0x77, 0x04, 0xf2, 0xed, 0x4d, 0x99, 0x52, 0xc3, 0xc2, 0x92, 0xff, 0x8e, 0xa0, 0x51, 0x18, 0x20, - 0xfc, 0x49, 0x72, 0x8b, 0xe3, 0x80, 0x9b, 0xf2, 0x87, 0x7f, 0xcf, 0x85, 0xf7, 0x2d, 0x3b, 0x3c, - 0x9e, 0x4d, 0x24, 0xc3, 0x9d, 0xf6, 0x32, 0x3f, 0xe6, 0x24, 0x7f, 0xf7, 0xbc, 0x13, 0xab, 0xc7, - 0x5e, 0xde, 0x44, 0x3a, 0x20, 0x51, 0x72, 0xbb, 0xef, 0x43, 0x9d, 0x50, 0x53, 0x5b, 0xde, 0xf5, - 0xff, 0xcb, 0x55, 0x23, 0xd4, 0x3c, 0x20, 0x11, 0x7e, 0x2b, 0x5f, 0x0a, 0x56, 0x33, 0x96, 0xfa, - 0xf2, 0x8a, 0x2e, 0x47, 0x9c, 0x95, 0x35, 0xcf, 0x4d, 0xee, 0x9e, 0xff, 0xd9, 0x2e, 0x9d, 0x2f, - 0xda, 0xe8, 0xf1, 0xa2, 0x8d, 0x9e, 0x2c, 0xda, 0xe8, 0x8f, 0x45, 0x1b, 0xfd, 0x78, 0xd1, 0x2e, - 0x3d, 0xbe, 0x68, 0x97, 0x9e, 0x5c, 0xb4, 0x4b, 0x8f, 0x6a, 0x49, 0xa1, 0xfe, 0x09, 0x00, 0x00, - 0xff, 0xff, 0x56, 0x2b, 0xf9, 0xe6, 0x69, 0x07, 0x00, 0x00, +func init() { proto.RegisterFile("config/zone.proto", fileDescriptor_zone_db9e8f274931475b) } + +var fileDescriptor_zone_db9e8f274931475b = []byte{ + // 851 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0xbf, 0x73, 0xe3, 0x44, + 0x14, 0xf6, 0xda, 0x8e, 0xad, 0x7b, 0x76, 0x12, 0xdf, 0xde, 0x71, 0xf1, 0x05, 0xce, 0x32, 0x9a, + 0x61, 0x30, 0xbf, 0x6c, 0x26, 0xd0, 0x60, 0x86, 0x02, 0xd9, 0x26, 0x88, 0xbb, 0xe3, 0x8c, 0x62, + 0x28, 0xae, 0x11, 0xb2, 0xbc, 0x51, 0x34, 0x91, 0x57, 0x1a, 0x49, 0x86, 0x88, 0xa1, 0xa6, 0x63, + 0x86, 0x92, 0x32, 0x25, 0x7f, 0x02, 0x7f, 0x42, 0x86, 0xea, 0xca, 0xa3, 0xf1, 0x80, 0xd3, 0x50, + 0xa7, 0xa4, 0x62, 0xb4, 0x5a, 0x4b, 0xb2, 0xe3, 0xdc, 0x00, 0xdd, 0xee, 0xbe, 0xf7, 0xbe, 0xf7, + 0xde, 0xb7, 0xef, 0x7b, 0x70, 0xdb, 0x70, 0xe8, 0xb1, 0x65, 0x76, 0xbe, 0x73, 0x28, 0x69, 0xbb, + 0x9e, 0x13, 0x38, 0xb8, 0x66, 0x38, 0xc6, 0xa9, 0xe7, 0xe8, 0xc6, 0x49, 0x3b, 0x36, 0xee, 0xdf, + 0x35, 0x1d, 0xd3, 0x61, 0xc6, 0x4e, 0x74, 0x8a, 0xfd, 0x24, 0x05, 0x84, 0xc3, 0xde, 0xd0, 0xb1, + 0x2d, 0x23, 0xc4, 0xef, 0x41, 0x25, 0x08, 0x6c, 0xcd, 0x27, 0x86, 0x43, 0x27, 0x7e, 0x1d, 0x35, + 0x51, 0x6b, 0x4b, 0xc6, 0x17, 0x73, 0x31, 0xb7, 0x98, 0x8b, 0x30, 0x1a, 0x3d, 0x3a, 0x8a, 0x2d, + 0x2a, 0x04, 0x81, 0xcd, 0xcf, 0x5d, 0xe1, 0xd7, 0x73, 0x11, 0xfd, 0x75, 0x2e, 0x22, 0xe9, 0x37, + 0x04, 0xd0, 0x73, 0xa8, 0x1f, 0x78, 0xba, 0x45, 0x03, 0xfc, 0x21, 0x14, 0x83, 0xd0, 0x25, 0x0c, + 0x66, 0xe7, 0xe0, 0xd5, 0xf6, 0x7a, 0x41, 0xed, 0xd4, 0xb7, 0x3d, 0x0a, 0x5d, 0x22, 0x17, 0xa3, + 0x4c, 0x2a, 0x0b, 0xc2, 0xf7, 0xa0, 0x70, 0x4a, 0xc2, 0x7a, 0xbe, 0x89, 0x5a, 0xb7, 0xb8, 0x21, + 0x7a, 0xc0, 0xfb, 0xb0, 0xf5, 0x8d, 0x6e, 0xcf, 0x48, 0xbd, 0x90, 0xb1, 0xc4, 0x4f, 0xd2, 0x47, + 0x50, 0x8c, 0x70, 0xf0, 0x1e, 0xdc, 0xe9, 0x0f, 0x86, 0xea, 0xa0, 0xf7, 0xf1, 0x68, 0xd0, 0xd7, + 0x86, 0x4f, 0x8e, 0x94, 0x91, 0xf2, 0xd5, 0xa0, 0x96, 0xc3, 0x55, 0x10, 0xd4, 0xc1, 0x17, 0x5f, + 0x2a, 0xea, 0xa0, 0x5f, 0x43, 0x78, 0x07, 0x60, 0xa8, 0x3e, 0xf9, 0x54, 0x91, 0x95, 0xd1, 0xa0, + 0x5f, 0xcb, 0x77, 0xab, 0x3f, 0x9f, 0x8b, 0xb9, 0xa4, 0x99, 0x1f, 0x10, 0x54, 0xd2, 0x02, 0x7d, + 0xdc, 0x87, 0x8a, 0x91, 0x5e, 0xeb, 0xa5, 0x66, 0xa1, 0x55, 0x39, 0x78, 0xe5, 0x45, 0x4d, 0xf1, + 0xe2, 0xb2, 0x61, 0xf8, 0x75, 0xa8, 0xd2, 0xd9, 0x54, 0xf3, 0x88, 0x6b, 0x5b, 0x86, 0xee, 0xd7, + 0xcb, 0x8c, 0x62, 0xee, 0x48, 0x67, 0x53, 0x95, 0x1b, 0x32, 0xac, 0x7e, 0x0f, 0xbb, 0x8f, 0x88, + 0xee, 0x93, 0xa1, 0x47, 0x8e, 0x89, 0x47, 0xa8, 0x41, 0xb0, 0xb6, 0x5a, 0x0b, 0xfa, 0x17, 0xb5, + 0x88, 0x51, 0x8a, 0xab, 0xb9, 0xb8, 0x17, 0xea, 0x53, 0xbb, 0x2b, 0x65, 0xc2, 0xdf, 0x3e, 0xb6, + 0x9d, 0x6f, 0xa5, 0x95, 0x32, 0x33, 0xd9, 0x7f, 0x2c, 0x01, 0x3c, 0x75, 0x28, 0xe9, 0x31, 0x44, + 0x2c, 0xc3, 0xae, 0xa7, 0x53, 0x93, 0x68, 0x53, 0x8b, 0x6a, 0xe3, 0x30, 0x20, 0x3e, 0xfb, 0xa2, + 0x82, 0xbc, 0x7f, 0x35, 0x17, 0xef, 0xc5, 0xd8, 0x6b, 0x0e, 0x92, 0xba, 0xcd, 0x5e, 0x1e, 0x5b, + 0x54, 0x8e, 0xee, 0x19, 0x0c, 0xfd, 0x8c, 0x63, 0x14, 0x6e, 0xc0, 0x58, 0x3a, 0x24, 0x18, 0xfa, + 0x59, 0x8c, 0x71, 0x00, 0x79, 0xd3, 0xa8, 0x17, 0x9b, 0xa8, 0x55, 0x39, 0xd8, 0xbf, 0xde, 0xf8, + 0x72, 0xa2, 0xe5, 0xd2, 0x62, 0x2e, 0xe6, 0x0f, 0x7b, 0x6a, 0xde, 0x34, 0x70, 0x77, 0x8d, 0xfb, + 0x2d, 0xc6, 0xfd, 0xde, 0xd5, 0x5c, 0xbc, 0x13, 0x27, 0xcd, 0x5a, 0xa5, 0x95, 0xef, 0xc0, 0x5f, + 0x6f, 0xfa, 0xfd, 0x07, 0x2f, 0x62, 0xdc, 0xff, 0x6f, 0x94, 0xe3, 0x11, 0x6c, 0xfb, 0xb3, 0x71, + 0x24, 0x60, 0xcd, 0x77, 0x75, 0x1a, 0x8d, 0xc6, 0x0d, 0x39, 0x8e, 0x62, 0xb7, 0x23, 0x57, 0xa7, + 0x72, 0x8d, 0xe7, 0x10, 0xe2, 0x1c, 0xef, 0x48, 0x6a, 0xd5, 0x4f, 0xcd, 0x3e, 0x3e, 0x04, 0x81, + 0xdf, 0xfd, 0xba, 0xc0, 0x00, 0xef, 0xdf, 0x08, 0xb8, 0x01, 0x2c, 0x09, 0xc6, 0x33, 0xb8, 0x6d, + 0x47, 0x53, 0xa8, 0xb9, 0xc9, 0x18, 0xfa, 0xf5, 0x5b, 0x0c, 0x71, 0x83, 0xb2, 0xd7, 0x06, 0x56, + 0x7e, 0x8d, 0x23, 0x3f, 0x88, 0x91, 0xaf, 0x21, 0x71, 0x42, 0x6a, 0xf6, 0x6a, 0x9c, 0x8f, 0x3f, + 0x80, 0x97, 0x2c, 0x7a, 0x42, 0x3c, 0x2b, 0x20, 0x13, 0x2d, 0xfb, 0x03, 0xd0, 0x44, 0x2d, 0x81, + 0x0b, 0xe7, 0x6e, 0xe2, 0xb2, 0x2a, 0xd8, 0x97, 0xd3, 0xd0, 0xeb, 0xb5, 0x57, 0x32, 0x00, 0xf7, + 0x13, 0xc7, 0xb5, 0xc2, 0x33, 0x4a, 0xf8, 0xac, 0x28, 0xa0, 0x5a, 0x5e, 0xfa, 0x05, 0x41, 0x99, + 0xf3, 0x85, 0xdf, 0x04, 0xc1, 0xa2, 0x13, 0x72, 0xa6, 0x59, 0x13, 0xb6, 0xe4, 0xb6, 0xe5, 0x5d, + 0xbe, 0x2b, 0xcb, 0x4a, 0xf4, 0xae, 0xf4, 0xd5, 0x32, 0x73, 0x50, 0x26, 0xf8, 0x2d, 0xd8, 0x71, + 0x75, 0x2f, 0xb0, 0x02, 0xcb, 0xa1, 0x1a, 0xd5, 0xa7, 0x64, 0x65, 0xb5, 0x6d, 0x27, 0xb6, 0xcf, + 0xf5, 0x29, 0xc1, 0x5d, 0x28, 0xc5, 0x44, 0x32, 0x61, 0x6c, 0x94, 0x76, 0xaa, 0x49, 0x0e, 0xc1, + 0x23, 0x32, 0xd2, 0xfd, 0x1d, 0x41, 0x25, 0x33, 0x2b, 0xf8, 0x93, 0x78, 0xa5, 0x46, 0x95, 0x56, + 0xe5, 0xf7, 0xff, 0x9e, 0x8b, 0xef, 0x9a, 0x56, 0x70, 0x32, 0x1b, 0xb7, 0x0d, 0x67, 0xda, 0x49, + 0x12, 0x4c, 0xc6, 0xe9, 0xb9, 0xe3, 0x9e, 0x9a, 0x1d, 0x76, 0x72, 0xc7, 0xed, 0x87, 0x24, 0x8c, + 0x57, 0xf0, 0x63, 0x28, 0x13, 0x3a, 0xd1, 0x96, 0xeb, 0xf9, 0xff, 0x62, 0x95, 0x08, 0x9d, 0x3c, + 0x24, 0x21, 0x7e, 0x23, 0x1d, 0x7c, 0x46, 0x16, 0xeb, 0x79, 0xb9, 0x13, 0x97, 0xd3, 0xcc, 0xf8, + 0x4c, 0x7b, 0x93, 0x5b, 0x17, 0x7f, 0x36, 0x72, 0x17, 0x8b, 0x06, 0x7a, 0xb6, 0x68, 0xa0, 0xe7, + 0x8b, 0x06, 0xfa, 0x63, 0xd1, 0x40, 0x3f, 0x5d, 0x36, 0x72, 0xcf, 0x2e, 0x1b, 0xb9, 0xe7, 0x97, + 0x8d, 0xdc, 0x53, 0xce, 0xc7, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x17, 0xde, 0xa3, 0x98, 0x1b, + 0x07, 0x00, 0x00, } diff --git a/pkg/config/zonepb/zone.proto b/pkg/config/zone.proto similarity index 99% rename from pkg/config/zonepb/zone.proto rename to pkg/config/zone.proto index 6e0c294b5d41..1a91a861c499 100644 --- a/pkg/config/zonepb/zone.proto +++ b/pkg/config/zone.proto @@ -9,8 +9,8 @@ // licenses/APL.txt. syntax = "proto2"; -package cockroach.config.zonepb; -option go_package = "zonepb"; +package cockroach.config; +option go_package = "config"; import "gogoproto/gogo.proto"; diff --git a/pkg/config/zonepb/zone_test.go b/pkg/config/zone_test.go similarity index 99% rename from pkg/config/zonepb/zone_test.go rename to pkg/config/zone_test.go index 325791eeb672..6899ac50b3c4 100644 --- a/pkg/config/zonepb/zone_test.go +++ b/pkg/config/zone_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package zonepb +package config import ( "fmt" diff --git a/pkg/config/zonepb/zone_yaml.go b/pkg/config/zone_yaml.go similarity index 99% rename from pkg/config/zonepb/zone_yaml.go rename to pkg/config/zone_yaml.go index d57382d3d865..fddeb24f786a 100644 --- a/pkg/config/zonepb/zone_yaml.go +++ b/pkg/config/zone_yaml.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package zonepb +package config import ( "fmt" diff --git a/pkg/gossip/client_test.go b/pkg/gossip/client_test.go index 628d10ec1abf..51e5df876b19 100644 --- a/pkg/gossip/client_test.go +++ b/pkg/gossip/client_test.go @@ -18,7 +18,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -61,7 +61,7 @@ func startGossipAtAddr( rpcContext.NodeID.Set(context.TODO(), nodeID) server := rpc.NewServer(rpcContext) - g := NewTest(nodeID, rpcContext, server, stopper, registry, zonepb.DefaultZoneConfigRef()) + g := NewTest(nodeID, rpcContext, server, stopper, registry, config.DefaultZoneConfigRef()) ln, err := netutil.ListenAndServeGRPC(stopper, server, addr) if err != nil { t.Fatal(err) @@ -122,7 +122,7 @@ func startFakeServerGossips( lRPCContext := rpc.NewInsecureTestingContextWithClusterID(clock, stopper, clusterID) lserver := rpc.NewServer(lRPCContext) - local := NewTest(localNodeID, lRPCContext, lserver, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + local := NewTest(localNodeID, lRPCContext, lserver, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) lln, err := netutil.ListenAndServeGRPC(stopper, lserver, util.IsolatedTestAddr) if err != nil { t.Fatal(err) @@ -476,7 +476,7 @@ func TestClientRegisterWithInitNodeID(t *testing.T) { server := rpc.NewServer(rpcContext) // node ID must be non-zero gnode := NewTest( - nodeID, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef(), + nodeID, rpcContext, server, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef(), ) g = append(g, gnode) diff --git a/pkg/gossip/convergence_test.go b/pkg/gossip/convergence_test.go index 34e922ea16f9..6443c7f6f519 100644 --- a/pkg/gossip/convergence_test.go +++ b/pkg/gossip/convergence_test.go @@ -14,7 +14,7 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip/simulation" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -60,7 +60,7 @@ func TestConvergence(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) - network := simulation.NewNetwork(stopper, testConvergenceSize, true, zonepb.DefaultZoneConfigRef()) + network := simulation.NewNetwork(stopper, testConvergenceSize, true, config.DefaultZoneConfigRef()) const maxCycles = 100 if connectedCycle := network.RunUntilFullyConnected(); connectedCycle > maxCycles { @@ -93,7 +93,7 @@ func TestNetworkReachesEquilibrium(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) - network := simulation.NewNetwork(stopper, testReachesEquilibriumSize, true, zonepb.DefaultZoneConfigRef()) + network := simulation.NewNetwork(stopper, testReachesEquilibriumSize, true, config.DefaultZoneConfigRef()) var connsRefused int64 var cyclesWithoutChange int diff --git a/pkg/gossip/disable_merges_test.go b/pkg/gossip/disable_merges_test.go index d033701ae420..7af83a69516a 100644 --- a/pkg/gossip/disable_merges_test.go +++ b/pkg/gossip/disable_merges_test.go @@ -14,7 +14,7 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -35,7 +35,7 @@ func TestDisableMerges(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { g := NewTest(1, nil /* rpcContext */, nil, /* grpcServer */ - stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/pkg/gossip/gossip.go b/pkg/gossip/gossip.go index 973cb1961813..770d131b62d9 100644 --- a/pkg/gossip/gossip.go +++ b/pkg/gossip/gossip.go @@ -62,7 +62,6 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -274,7 +273,7 @@ type Gossip struct { lastConnectivity string - defaultZoneConfig *zonepb.ZoneConfig + defaultZoneConfig *config.ZoneConfig } // New creates an instance of a gossip node. @@ -297,7 +296,7 @@ func New( stopper *stop.Stopper, registry *metric.Registry, locality roachpb.Locality, - defaultZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, ) *Gossip { ambient.SetEventLog("gossip", "gossip") g := &Gossip{ @@ -358,7 +357,7 @@ func NewTest( grpcServer *grpc.Server, stopper *stop.Stopper, registry *metric.Registry, - defaultZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, ) *Gossip { return NewTestWithLocality(nodeID, rpcContext, grpcServer, stopper, registry, roachpb.Locality{}, defaultZoneConfig) } @@ -371,7 +370,7 @@ func NewTestWithLocality( stopper *stop.Stopper, registry *metric.Registry, locality roachpb.Locality, - defaultZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, ) *Gossip { c := &base.ClusterIDContainer{} n := &base.NodeIDContainer{} diff --git a/pkg/gossip/gossip_test.go b/pkg/gossip/gossip_test.go index 7e6e47aa53cb..208bafe5a1f4 100644 --- a/pkg/gossip/gossip_test.go +++ b/pkg/gossip/gossip_test.go @@ -20,7 +20,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -46,7 +46,7 @@ func TestGossipInfoStore(t *testing.T) { defer stopper.Stop(context.TODO()) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) rpcContext := rpc.NewInsecureTestingContext(clock, stopper) - g := NewTest(1, rpcContext, rpc.NewServer(rpcContext), stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + g := NewTest(1, rpcContext, rpc.NewServer(rpcContext), stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) slice := []byte("b") if err := g.AddInfo("s", slice, time.Hour); err != nil { t.Fatal(err) @@ -67,7 +67,7 @@ func TestGossipMoveNode(t *testing.T) { defer stopper.Stop(context.TODO()) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) rpcContext := rpc.NewInsecureTestingContext(clock, stopper) - g := NewTest(1, rpcContext, rpc.NewServer(rpcContext), stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + g := NewTest(1, rpcContext, rpc.NewServer(rpcContext), stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) var nodes []*roachpb.NodeDescriptor for i := 1; i <= 3; i++ { node := &roachpb.NodeDescriptor{ @@ -129,7 +129,7 @@ func TestGossipGetNextBootstrapAddress(t *testing.T) { clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) rpcContext := rpc.NewInsecureTestingContext(clock, stopper) server := rpc.NewServer(rpcContext) - g := NewTest(0, nil, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + g := NewTest(0, nil, server, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) g.setResolvers(resolvers) // Using specified resolvers, fetch bootstrap addresses 3 times @@ -190,7 +190,7 @@ func TestGossipLocalityResolver(t *testing.T) { var node2LocalityList []roachpb.LocalityAddress node2LocalityList = append(node2LocalityList, nodeLocalityAddress2) - g := NewTestWithLocality(1, rpcContext, rpc.NewServer(rpcContext), stopper, metric.NewRegistry(), gossipLocalityAdvertiseList, zonepb.DefaultZoneConfigRef()) + g := NewTestWithLocality(1, rpcContext, rpc.NewServer(rpcContext), stopper, metric.NewRegistry(), gossipLocalityAdvertiseList, config.DefaultZoneConfigRef()) node1 := &roachpb.NodeDescriptor{ NodeID: 1, Address: node1PublicAddressRPC, @@ -721,7 +721,7 @@ func TestGossipJoinTwoClusters(t *testing.T) { // node ID must be non-zero gnode := NewTest( - roachpb.NodeID(i+1), rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + roachpb.NodeID(i+1), rpcContext, server, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) g = append(g, gnode) gnode.SetStallInterval(interval) gnode.SetBootstrapInterval(interval) diff --git a/pkg/gossip/simulation/network.go b/pkg/gossip/simulation/network.go index c8512975b881..e9a0b3422708 100644 --- a/pkg/gossip/simulation/network.go +++ b/pkg/gossip/simulation/network.go @@ -17,7 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -63,7 +63,7 @@ type Network struct { // NewNetwork creates nodeCount gossip nodes. func NewNetwork( - stopper *stop.Stopper, nodeCount int, createResolvers bool, defaultZoneConfig *zonepb.ZoneConfig, + stopper *stop.Stopper, nodeCount int, createResolvers bool, defaultZoneConfig *config.ZoneConfig, ) *Network { log.Infof(context.TODO(), "simulating gossip network with %d nodes", nodeCount) @@ -107,7 +107,7 @@ func NewNetwork( } // CreateNode creates a simulation node and starts an RPC server for it. -func (n *Network) CreateNode(defaultZoneConfig *zonepb.ZoneConfig) (*Node, error) { +func (n *Network) CreateNode(defaultZoneConfig *config.ZoneConfig) (*Node, error) { server := rpc.NewServer(n.RPCContext) ln, err := net.Listen(util.IsolatedTestAddr.Network(), util.IsolatedTestAddr.String()) if err != nil { diff --git a/pkg/gossip/storage_test.go b/pkg/gossip/storage_test.go index 15ef6b4a608d..01534d34bc9f 100644 --- a/pkg/gossip/storage_test.go +++ b/pkg/gossip/storage_test.go @@ -18,7 +18,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" "github.com/cockroachdb/cockroach/pkg/gossip/simulation" @@ -97,7 +97,7 @@ func TestGossipStorage(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) - defaultZoneConfig := zonepb.DefaultZoneConfigRef() + defaultZoneConfig := config.DefaultZoneConfigRef() network := simulation.NewNetwork(stopper, 3, true, defaultZoneConfig) // Set storage for each of the nodes. @@ -216,7 +216,7 @@ func TestGossipStorageCleanup(t *testing.T) { defer stopper.Stop(context.TODO()) const numNodes = 3 - network := simulation.NewNetwork(stopper, numNodes, false, zonepb.DefaultZoneConfigRef()) + network := simulation.NewNetwork(stopper, numNodes, false, config.DefaultZoneConfigRef()) const notReachableAddr = "localhost:0" const invalidAddr = "10.0.0.1000:3333333" diff --git a/pkg/gossip/util.go b/pkg/gossip/util.go index 7ff4a1d19530..8ef97daeb145 100644 --- a/pkg/gossip/util.go +++ b/pkg/gossip/util.go @@ -15,7 +15,6 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -36,7 +35,7 @@ type SystemConfigDeltaFilter struct { func MakeSystemConfigDeltaFilter(keyPrefix roachpb.Key) SystemConfigDeltaFilter { return SystemConfigDeltaFilter{ keyPrefix: keyPrefix, - lastCfg: config.NewSystemConfig(zonepb.DefaultZoneConfigRef()), + lastCfg: config.NewSystemConfig(config.DefaultZoneConfigRef()), } } diff --git a/pkg/gossip/util_test.go b/pkg/gossip/util_test.go index 7c5e367973c2..7e7c36572388 100644 --- a/pkg/gossip/util_test.go +++ b/pkg/gossip/util_test.go @@ -18,7 +18,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -81,7 +80,7 @@ func TestSystemConfigDeltaFilter(t *testing.T) { rng, _ := randutil.NewPseudoRand() df := MakeSystemConfigDeltaFilter(nil) - cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) + cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) // Add one key. addKV(rng, cfg, 1) @@ -111,7 +110,7 @@ func TestSystemConfigDeltaFilterWithKeyPrefix(t *testing.T) { rng, _ := randutil.NewPseudoRand() df := MakeSystemConfigDeltaFilter(keyFromInt(12)) - cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) + cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) // Add one non-matching key. addKV(rng, cfg, 1) @@ -137,7 +136,7 @@ func BenchmarkSystemConfigDeltaFilter(b *testing.B) { rng, _ := randutil.NewPseudoRand() // Create two configs. - cfg1, cfg2 := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()), config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) + cfg1, cfg2 := config.NewSystemConfig(config.DefaultZoneConfigRef()), config.NewSystemConfig(config.DefaultZoneConfigRef()) for i := 0; i < 1000; i++ { key := i + 100000 // +100000 to match filter addKV(rng, cfg1, key) @@ -161,7 +160,7 @@ func BenchmarkSystemConfigDeltaFilter(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) + cfg := config.NewSystemConfig(config.DefaultZoneConfigRef()) cfg.Values = cfg1.Values if i%2 == 1 { cfg.Values = cfg2.Values diff --git a/pkg/internal/client/client_test.go b/pkg/internal/client/client_test.go index b42171a01a61..195e1dd718e7 100644 --- a/pkg/internal/client/client_test.go +++ b/pkg/internal/client/client_test.go @@ -23,7 +23,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -387,9 +387,9 @@ func TestClientGetAndPutProto(t *testing.T) { defer s.Stopper().Stop(context.TODO()) db := createTestClient(t, s) - zoneConfig := zonepb.ZoneConfig{ + zoneConfig := config.ZoneConfig{ NumReplicas: proto.Int32(2), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "mem"}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "mem"}}}}, RangeMinBytes: proto.Int64(1 << 10), // 1k RangeMaxBytes: proto.Int64(1 << 18), // 256k } @@ -399,7 +399,7 @@ func TestClientGetAndPutProto(t *testing.T) { t.Fatalf("unable to put proto: %s", err) } - var readZoneConfig zonepb.ZoneConfig + var readZoneConfig config.ZoneConfig if err := db.GetProto(context.TODO(), key, &readZoneConfig); err != nil { t.Fatalf("unable to get proto: %s", err) } diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index e593b070bfe9..5712100a20c5 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -308,10 +308,6 @@ const ( ZonesTableConfigColumnID = 2 ZonesTableConfigColFamID = 2 - DescriptorTablePrimaryKeyIndexID = 1 - DescriptorTableDescriptorColID = 2 - DescriptorTableDescriptorColFamID = 2 - // Reserved IDs for other system tables. Note that some of these IDs refer // to "Ranges" instead of a Table - these IDs are needed to store custom // configuration for non-table ranges (e.g. Zone Configs). diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 9fa1c94b5e54..fa6f6732868c 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -690,19 +690,6 @@ func DecodeTablePrefix(key roachpb.Key) ([]byte, uint64, error) { return encoding.DecodeUvarintAscending(key) } -// DescMetadataPrefix returns the key prefix for all descriptors. -func DescMetadataPrefix() []byte { - k := MakeTablePrefix(uint32(DescriptorTableID)) - return encoding.EncodeUvarintAscending(k, DescriptorTablePrimaryKeyIndexID) -} - -// DescMetadataKey returns the key for the descriptor. -func DescMetadataKey(descID uint32) roachpb.Key { - k := DescMetadataPrefix() - k = encoding.EncodeUvarintAscending(k, uint64(descID)) - return MakeFamilyKey(k, DescriptorTableDescriptorColFamID) -} - // DecodeDescMetadataID decodes a descriptor ID from a descriptor metadata key. func DecodeDescMetadataID(key roachpb.Key) (uint64, error) { // Extract object ID from key. @@ -988,16 +975,3 @@ func (b RangeIDPrefixBuf) RangeLastReplicaGCTimestampKey() roachpb.Key { func (b RangeIDPrefixBuf) RangeLastVerificationTimestampKeyDeprecated() roachpb.Key { return append(b.unreplicatedPrefix(), LocalRangeLastVerificationTimestampSuffixDeprecated...) } - -// ZoneKeyPrefix returns the key prefix for id's row in the system.zones table. -func ZoneKeyPrefix(id uint32) roachpb.Key { - k := MakeTablePrefix(uint32(ZonesTableID)) - k = encoding.EncodeUvarintAscending(k, uint64(ZonesTablePrimaryIndexID)) - return encoding.EncodeUvarintAscending(k, uint64(id)) -} - -// ZoneKey returns the key for id's entry in the system.zones table. -func ZoneKey(id uint32) roachpb.Key { - k := ZoneKeyPrefix(id) - return MakeFamilyKey(k, uint32(ZonesTableConfigColumnID)) -} diff --git a/pkg/kv/dist_sender_test.go b/pkg/kv/dist_sender_test.go index b67d7a85a33a..cf2abba142d5 100644 --- a/pkg/kv/dist_sender_test.go +++ b/pkg/kv/dist_sender_test.go @@ -25,7 +25,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/gossip/simulation" "github.com/cockroachdb/cockroach/pkg/internal/client" @@ -177,7 +177,7 @@ func makeGossip(t *testing.T, stopper *stop.Stopper, rpcContext *rpc.Context) *g server := rpc.NewServer(rpcContext) const nodeID = 1 - g := gossip.NewTest(nodeID, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + g := gossip.NewTest(nodeID, rpcContext, server, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) if err := g.SetNodeDescriptor(newNodeDesc(nodeID)); err != nil { t.Fatal(err) } @@ -1281,7 +1281,7 @@ func TestGetFirstRangeDescriptor(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) - n := simulation.NewNetwork(stopper, 3, true, zonepb.DefaultZoneConfigRef()) + n := simulation.NewNetwork(stopper, 3, true, config.DefaultZoneConfigRef()) for _, node := range n.Nodes { // TODO(spencer): remove the use of gossip/simulation here. node.Gossip.EnableSimulationCycler(false) diff --git a/pkg/kv/split_test.go b/pkg/kv/split_test.go index 4a84b9654a07..5380fc1e6b74 100644 --- a/pkg/kv/split_test.go +++ b/pkg/kv/split_test.go @@ -20,7 +20,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -175,7 +174,7 @@ func TestRangeSplitsWithConcurrentTxns(t *testing.T) { func TestRangeSplitsWithWritePressure(t *testing.T) { defer leaktest.AfterTest(t)() // Override default zone config. - cfg := zonepb.DefaultZoneConfigRef() + cfg := config.DefaultZoneConfigRef() cfg.RangeMaxBytes = proto.Int64(1 << 18) // Manually create the local test cluster so that the split queue diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 291af4a7b90b..39110e40172b 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -23,7 +23,7 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -1641,7 +1641,7 @@ func (s *adminServer) DataDistribution( target := string(tree.MustBeDString(row[0])) zcSQL := tree.MustBeDString(row[1]) zcBytes := tree.MustBeDBytes(row[2]) - var zcProto zonepb.ZoneConfig + var zcProto config.ZoneConfig if err := protoutil.Unmarshal([]byte(zcBytes), &zcProto); err != nil { return nil, s.serverError(err) } @@ -2033,29 +2033,29 @@ func (rs resultScanner) Scan(row tree.Datums, colName string, dst interface{}) e // if it exists. func (s *adminServer) queryZone( ctx context.Context, userName string, id sqlbase.ID, -) (zonepb.ZoneConfig, bool, error) { +) (config.ZoneConfig, bool, error) { const query = `SELECT config FROM system.zones WHERE id = $1` rows, _ /* cols */, err := s.server.internalExecutor.QueryWithUser( ctx, "admin-query-zone", nil /* txn */, userName, query, id, ) if err != nil { - return *zonepb.NewZoneConfig(), false, err + return *config.NewZoneConfig(), false, err } if len(rows) == 0 { - return *zonepb.NewZoneConfig(), false, nil + return *config.NewZoneConfig(), false, nil } var zoneBytes []byte scanner := resultScanner{} err = scanner.ScanIndex(rows[0], 0, &zoneBytes) if err != nil { - return *zonepb.NewZoneConfig(), false, err + return *config.NewZoneConfig(), false, err } - var zone zonepb.ZoneConfig + var zone config.ZoneConfig if err := protoutil.Unmarshal(zoneBytes, &zone); err != nil { - return *zonepb.NewZoneConfig(), false, err + return *config.NewZoneConfig(), false, err } return zone, true, nil } @@ -2065,14 +2065,14 @@ func (s *adminServer) queryZone( // ZoneConfig specified for the object IDs in the path. func (s *adminServer) queryZonePath( ctx context.Context, userName string, path []sqlbase.ID, -) (sqlbase.ID, zonepb.ZoneConfig, bool, error) { +) (sqlbase.ID, config.ZoneConfig, bool, error) { for i := len(path) - 1; i >= 0; i-- { zone, zoneExists, err := s.queryZone(ctx, userName, path[i]) if err != nil || zoneExists { return path[i], zone, true, err } } - return 0, *zonepb.NewZoneConfig(), false, nil + return 0, *config.NewZoneConfig(), false, nil } // queryNamespaceID queries for the ID of the namespace with the given name and diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index ccf321b25c0b..76db6981b767 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -27,7 +27,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -649,7 +649,7 @@ func TestAdminAPIZoneDetails(t *testing.T) { // Function to verify the zone for table "test.tbl" as returned by the Admin // API. verifyTblZone := func( - expectedZone zonepb.ZoneConfig, expectedLevel serverpb.ZoneConfigurationLevel, + expectedZone config.ZoneConfig, expectedLevel serverpb.ZoneConfigurationLevel, ) { var resp serverpb.TableDetailsResponse if err := getAdminJSONProto(s, "databases/test/tables/tbl", &resp); err != nil { @@ -669,7 +669,7 @@ func TestAdminAPIZoneDetails(t *testing.T) { // Function to verify the zone for database "test" as returned by the Admin // API. verifyDbZone := func( - expectedZone zonepb.ZoneConfig, expectedLevel serverpb.ZoneConfigurationLevel, + expectedZone config.ZoneConfig, expectedLevel serverpb.ZoneConfigurationLevel, ) { var resp serverpb.DatabaseDetailsResponse if err := getAdminJSONProto(s, "databases/test", &resp); err != nil { @@ -687,7 +687,7 @@ func TestAdminAPIZoneDetails(t *testing.T) { } // Function to store a zone config for a given object ID. - setZone := func(zoneCfg zonepb.ZoneConfig, id sqlbase.ID) { + setZone := func(zoneCfg config.ZoneConfig, id sqlbase.ID) { zoneBytes, err := protoutil.Marshal(&zoneCfg) if err != nil { t.Fatal(err) @@ -710,7 +710,7 @@ func TestAdminAPIZoneDetails(t *testing.T) { } // Apply zone configuration to database and check again. - dbZone := zonepb.ZoneConfig{ + dbZone := config.ZoneConfig{ RangeMinBytes: proto.Int64(456), } setZone(dbZone, idPath[1]) @@ -718,7 +718,7 @@ func TestAdminAPIZoneDetails(t *testing.T) { verifyTblZone(dbZone, serverpb.ZoneConfigurationLevel_DATABASE) // Apply zone configuration to table and check again. - tblZone := zonepb.ZoneConfig{ + tblZone := config.ZoneConfig{ RangeMinBytes: proto.Int64(789), } setZone(tblZone, idPath[2]) diff --git a/pkg/server/config.go b/pkg/server/config.go index 28136655fd8d..7ee059ecd877 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -20,7 +20,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/status" @@ -238,11 +238,11 @@ type Config struct { // DefaultZoneConfig is used to set the default zone config inside the server. // It can be overridden during tests by setting the DefaultZoneConfigOverride // server testing knob. - DefaultZoneConfig zonepb.ZoneConfig + DefaultZoneConfig config.ZoneConfig // DefaultSystemZoneConfig is used to set the default system zone config // inside the server. It can be overridden during tests by setting the // DefaultSystemZoneConfigOverride server testing knob. - DefaultSystemZoneConfig zonepb.ZoneConfig + DefaultSystemZoneConfig config.ZoneConfig // Locality is a description of the topography of the server. Locality roachpb.Locality @@ -342,8 +342,8 @@ func MakeConfig(ctx context.Context, st *cluster.Settings) Config { cfg := Config{ Config: new(base.Config), - DefaultZoneConfig: zonepb.DefaultZoneConfig(), - DefaultSystemZoneConfig: zonepb.DefaultSystemZoneConfig(), + DefaultZoneConfig: config.DefaultZoneConfig(), + DefaultSystemZoneConfig: config.DefaultSystemZoneConfig(), MaxOffset: MaxOffsetType(base.DefaultMaxClockOffset), Settings: st, CacheSize: DefaultCacheSize, diff --git a/pkg/server/diagnosticspb/diagnostics.pb.go b/pkg/server/diagnosticspb/diagnostics.pb.go index eb4c46acfd08..bca3eaf2a3f9 100644 --- a/pkg/server/diagnosticspb/diagnostics.pb.go +++ b/pkg/server/diagnosticspb/diagnostics.pb.go @@ -7,7 +7,7 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" import build "github.com/cockroachdb/cockroach/pkg/build" -import zonepb "github.com/cockroachdb/cockroach/pkg/config/zonepb" +import config "github.com/cockroachdb/cockroach/pkg/config" import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -35,7 +35,7 @@ type DiagnosticReport struct { Schema []sqlbase.TableDescriptor `protobuf:"bytes,3,rep,name=schema,proto3" json:"schema"` SqlStats []roachpb.CollectedStatementStatistics `protobuf:"bytes,4,rep,name=sql_stats,json=sqlStats,proto3" json:"sql_stats"` AlteredSettings map[string]string `protobuf:"bytes,6,rep,name=altered_settings,json=alteredSettings,proto3" json:"altered_settings,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - ZoneConfigs map[int64]zonepb.ZoneConfig `protobuf:"bytes,8,rep,name=zone_configs,json=zoneConfigs,proto3" json:"zone_configs" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ZoneConfigs map[int64]config.ZoneConfig `protobuf:"bytes,8,rep,name=zone_configs,json=zoneConfigs,proto3" json:"zone_configs" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` FeatureUsage map[string]int32 `protobuf:"bytes,9,rep,name=feature_usage,json=featureUsage,proto3" json:"feature_usage" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` LegacyUnimplementedErrors map[string]int64 `protobuf:"bytes,5,rep,name=legacy_unimplemented_errors,json=legacyUnimplementedErrors,proto3" json:"legacy_unimplemented_errors,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` LegacyErrorCounts map[string]int64 `protobuf:"bytes,7,rep,name=legacy_error_counts,json=legacyErrorCounts,proto3" json:"legacy_error_counts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` @@ -45,7 +45,7 @@ func (m *DiagnosticReport) Reset() { *m = DiagnosticReport{} } func (m *DiagnosticReport) String() string { return proto.CompactTextString(m) } func (*DiagnosticReport) ProtoMessage() {} func (*DiagnosticReport) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5c3b35b5aaeee3d9, []int{0} + return fileDescriptor_diagnostics_e9c218b3559e7579, []int{0} } func (m *DiagnosticReport) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -88,7 +88,7 @@ func (m *NodeInfo) Reset() { *m = NodeInfo{} } func (m *NodeInfo) String() string { return proto.CompactTextString(m) } func (*NodeInfo) ProtoMessage() {} func (*NodeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5c3b35b5aaeee3d9, []int{1} + return fileDescriptor_diagnostics_e9c218b3559e7579, []int{1} } func (m *NodeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -129,7 +129,7 @@ func (m *StoreInfo) Reset() { *m = StoreInfo{} } func (m *StoreInfo) String() string { return proto.CompactTextString(m) } func (*StoreInfo) ProtoMessage() {} func (*StoreInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5c3b35b5aaeee3d9, []int{2} + return fileDescriptor_diagnostics_e9c218b3559e7579, []int{2} } func (m *StoreInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -167,7 +167,7 @@ func (m *CPUInfo) Reset() { *m = CPUInfo{} } func (m *CPUInfo) String() string { return proto.CompactTextString(m) } func (*CPUInfo) ProtoMessage() {} func (*CPUInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5c3b35b5aaeee3d9, []int{3} + return fileDescriptor_diagnostics_e9c218b3559e7579, []int{3} } func (m *CPUInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -205,7 +205,7 @@ func (m *HardwareInfo) Reset() { *m = HardwareInfo{} } func (m *HardwareInfo) String() string { return proto.CompactTextString(m) } func (*HardwareInfo) ProtoMessage() {} func (*HardwareInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5c3b35b5aaeee3d9, []int{4} + return fileDescriptor_diagnostics_e9c218b3559e7579, []int{4} } func (m *HardwareInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -240,7 +240,7 @@ func (m *OSInfo) Reset() { *m = OSInfo{} } func (m *OSInfo) String() string { return proto.CompactTextString(m) } func (*OSInfo) ProtoMessage() {} func (*OSInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5c3b35b5aaeee3d9, []int{5} + return fileDescriptor_diagnostics_e9c218b3559e7579, []int{5} } func (m *OSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -274,7 +274,7 @@ func (m *MemInfo) Reset() { *m = MemInfo{} } func (m *MemInfo) String() string { return proto.CompactTextString(m) } func (*MemInfo) ProtoMessage() {} func (*MemInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5c3b35b5aaeee3d9, []int{6} + return fileDescriptor_diagnostics_e9c218b3559e7579, []int{6} } func (m *MemInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -308,7 +308,7 @@ func (m *TopologyInfo) Reset() { *m = TopologyInfo{} } func (m *TopologyInfo) String() string { return proto.CompactTextString(m) } func (*TopologyInfo) ProtoMessage() {} func (*TopologyInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5c3b35b5aaeee3d9, []int{7} + return fileDescriptor_diagnostics_e9c218b3559e7579, []int{7} } func (m *TopologyInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -339,7 +339,7 @@ func init() { proto.RegisterMapType((map[string]int32)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.FeatureUsageEntry") proto.RegisterMapType((map[string]int64)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.LegacyErrorCountsEntry") proto.RegisterMapType((map[string]int64)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.LegacyUnimplementedErrorsEntry") - proto.RegisterMapType((map[int64]zonepb.ZoneConfig)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.ZoneConfigsEntry") + proto.RegisterMapType((map[int64]config.ZoneConfig)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.ZoneConfigsEntry") proto.RegisterType((*NodeInfo)(nil), "cockroach.server.diagnosticspb.NodeInfo") proto.RegisterType((*StoreInfo)(nil), "cockroach.server.diagnosticspb.StoreInfo") proto.RegisterType((*CPUInfo)(nil), "cockroach.server.diagnosticspb.CPUInfo") @@ -1678,10 +1678,10 @@ func (m *DiagnosticReport) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ZoneConfigs == nil { - m.ZoneConfigs = make(map[int64]zonepb.ZoneConfig) + m.ZoneConfigs = make(map[int64]config.ZoneConfig) } var mapkey int64 - mapvalue := &zonepb.ZoneConfig{} + mapvalue := &config.ZoneConfig{} for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -1741,7 +1741,7 @@ func (m *DiagnosticReport) Unmarshal(dAtA []byte) error { if postmsgIndex > l { return io.ErrUnexpectedEOF } - mapvalue = &zonepb.ZoneConfig{} + mapvalue = &config.ZoneConfig{} if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { return err } @@ -3259,88 +3259,88 @@ var ( ) func init() { - proto.RegisterFile("server/diagnosticspb/diagnostics.proto", fileDescriptor_diagnostics_5c3b35b5aaeee3d9) -} - -var fileDescriptor_diagnostics_5c3b35b5aaeee3d9 = []byte{ - // 1264 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0x41, 0x6f, 0x1b, 0xc5, - 0x17, 0x8f, 0xbd, 0xf6, 0xda, 0x1e, 0xa7, 0xfd, 0xa7, 0xd3, 0xfc, 0xc3, 0x92, 0x56, 0x4e, 0x31, - 0xa2, 0x14, 0x81, 0x6c, 0xb5, 0x05, 0x09, 0x10, 0x55, 0x15, 0x27, 0xa5, 0x54, 0x2a, 0x2d, 0xda, - 0xb4, 0x1c, 0x7a, 0xc0, 0x1a, 0xef, 0x8e, 0x37, 0x4b, 0x66, 0x77, 0x36, 0x33, 0xb3, 0xae, 0x9c, - 0xaf, 0xc0, 0x01, 0x2e, 0x1c, 0x90, 0xf8, 0x3a, 0x48, 0x3d, 0xf6, 0xd8, 0x53, 0x04, 0xe9, 0xb7, - 0xe0, 0x84, 0xe6, 0xcd, 0xac, 0xb3, 0x71, 0xdb, 0xb8, 0x41, 0xe2, 0xe4, 0x79, 0xef, 0xcd, 0xfb, - 0xbd, 0xf9, 0xbd, 0x79, 0xf3, 0x5b, 0xa3, 0xab, 0x92, 0x8a, 0x09, 0x15, 0xfd, 0x30, 0x26, 0x51, - 0xca, 0xa5, 0x8a, 0x03, 0x99, 0x8d, 0xca, 0x56, 0x2f, 0x13, 0x5c, 0x71, 0xdc, 0x09, 0x78, 0xb0, - 0x27, 0x38, 0x09, 0x76, 0x7b, 0x26, 0xa3, 0x77, 0x22, 0x63, 0x7d, 0x65, 0x94, 0xc7, 0x2c, 0xec, - 0xc7, 0xe9, 0x98, 0x9b, 0x8c, 0x75, 0x2f, 0xe0, 0xe9, 0x38, 0x8e, 0xfa, 0x07, 0x3c, 0xa5, 0xd9, - 0x08, 0x7e, 0x6c, 0x64, 0x35, 0xe2, 0x11, 0x87, 0x65, 0x5f, 0xaf, 0xac, 0xf7, 0x1d, 0x40, 0xcf, - 0x46, 0x7d, 0x92, 0x65, 0x43, 0xa9, 0x88, 0xb2, 0xa5, 0xd7, 0x2f, 0xcb, 0x7d, 0xd6, 0x97, 0xfb, - 0x6c, 0x44, 0x24, 0xed, 0x4b, 0x25, 0xf2, 0x40, 0xe5, 0x82, 0x86, 0x36, 0xba, 0x56, 0xa4, 0x25, - 0x54, 0x91, 0x90, 0x28, 0x62, 0xfc, 0xdd, 0x9f, 0x10, 0x5a, 0xd9, 0x9e, 0x1d, 0xd1, 0xa7, 0x19, - 0x17, 0x0a, 0x0f, 0x50, 0x2d, 0xe5, 0x21, 0xf5, 0x2a, 0x57, 0x2a, 0xd7, 0xda, 0x37, 0xae, 0xf5, - 0x4e, 0x27, 0xd5, 0x7b, 0xc0, 0x43, 0x7a, 0x2f, 0x1d, 0xf3, 0x41, 0xed, 0xd9, 0xe1, 0xc6, 0x92, - 0x0f, 0xb9, 0xf8, 0x2e, 0x72, 0xa5, 0xe2, 0x82, 0x4a, 0xaf, 0x7a, 0xc5, 0xb9, 0xd6, 0xbe, 0xf1, - 0xd1, 0x22, 0x94, 0x1d, 0xbd, 0xbb, 0x04, 0x63, 0xd3, 0xf1, 0x36, 0x72, 0x65, 0xb0, 0x4b, 0x13, - 0xe2, 0x39, 0x00, 0x74, 0xb5, 0x0c, 0xb4, 0xcf, 0x7a, 0x96, 0x72, 0xef, 0x11, 0x19, 0x31, 0xba, - 0x4d, 0x65, 0x20, 0xe2, 0x4c, 0x71, 0x31, 0x43, 0x81, 0x5c, 0xfc, 0x00, 0xb5, 0xe4, 0x3e, 0x33, - 0x0d, 0xf3, 0x6a, 0x00, 0xf4, 0xf1, 0x1c, 0xd0, 0x16, 0x67, 0x8c, 0x06, 0x8a, 0x86, 0x3b, 0x8a, - 0x28, 0x9a, 0xd0, 0x54, 0xe9, 0x45, 0x0c, 0xe7, 0xb3, 0x68, 0x4d, 0xb9, 0xcf, 0xb4, 0x53, 0xe2, - 0x9f, 0x2b, 0xe8, 0x12, 0xa3, 0x11, 0x09, 0xa6, 0xc3, 0x3c, 0x8d, 0x93, 0x8c, 0x41, 0x02, 0x0d, - 0x87, 0x54, 0x08, 0x2e, 0xa4, 0x57, 0x87, 0x12, 0x0f, 0x17, 0x91, 0x9e, 0x6f, 0x7d, 0xef, 0x3e, - 0x60, 0x3e, 0x2e, 0x43, 0xde, 0x01, 0xc4, 0x3b, 0xa9, 0x12, 0x53, 0xff, 0x5d, 0xf6, 0xa6, 0x38, - 0xce, 0xd0, 0x0a, 0x61, 0x8a, 0x0a, 0x1a, 0x0e, 0x25, 0x55, 0x2a, 0x4e, 0x23, 0xe9, 0xb9, 0x70, - 0x8a, 0x3b, 0x67, 0x3e, 0xc5, 0xa6, 0x01, 0xda, 0xb1, 0x38, 0xa6, 0xf6, 0xff, 0xc8, 0x49, 0x2f, - 0x7e, 0x8a, 0x2e, 0xda, 0x16, 0x00, 0xe9, 0x61, 0xc0, 0xf3, 0x54, 0x49, 0xaf, 0x01, 0x45, 0xef, - 0xfe, 0x4b, 0xea, 0xc0, 0x66, 0x0b, 0x90, 0x4c, 0xd9, 0x0b, 0x6c, 0xde, 0x8f, 0x7f, 0x44, 0xcb, - 0xfa, 0x9d, 0x0c, 0xcd, 0xd3, 0x91, 0x5e, 0x13, 0x2a, 0x6e, 0x9e, 0xb9, 0xe2, 0x13, 0x9e, 0xd2, - 0x2d, 0x83, 0x01, 0xb5, 0xec, 0x2d, 0xb7, 0x0f, 0x8e, 0xfd, 0x38, 0x41, 0xe7, 0xc6, 0x94, 0xe8, - 0xa7, 0x34, 0xcc, 0x25, 0x89, 0xa8, 0xd7, 0x82, 0x62, 0x83, 0x33, 0x17, 0xfb, 0xda, 0xa0, 0x3c, - 0xd6, 0x20, 0xe5, 0x6a, 0xcb, 0xe3, 0x52, 0x60, 0x7d, 0x80, 0x56, 0x5f, 0xd7, 0x7c, 0xbc, 0x82, - 0x9c, 0x3d, 0x3a, 0x85, 0x17, 0xd9, 0xf2, 0xf5, 0x12, 0xaf, 0xa2, 0xfa, 0x84, 0xb0, 0x9c, 0x7a, - 0x55, 0xf0, 0x19, 0xe3, 0xcb, 0xea, 0xe7, 0x95, 0xf5, 0x00, 0xad, 0xcc, 0x33, 0x2b, 0xe7, 0x3b, - 0x26, 0xff, 0x8b, 0x72, 0x7e, 0xfb, 0xc6, 0xfb, 0x25, 0x42, 0xa6, 0xaf, 0x3d, 0x23, 0x49, 0xa5, - 0x2e, 0x95, 0x8b, 0xdc, 0x46, 0x17, 0x5e, 0x61, 0xb4, 0xe8, 0x94, 0xf5, 0x32, 0xc0, 0x7d, 0xd4, - 0x39, 0x7d, 0xd8, 0x17, 0xa1, 0x39, 0x65, 0xb4, 0x6d, 0xb4, 0xf6, 0xfa, 0xf9, 0x39, 0x0b, 0x4a, - 0xf7, 0x8f, 0x1a, 0x6a, 0x16, 0x6a, 0x86, 0x9f, 0xa0, 0x86, 0x56, 0xb2, 0x61, 0x1c, 0x42, 0x72, - 0x7d, 0xb0, 0x79, 0x74, 0xb8, 0xe1, 0x42, 0x78, 0xfb, 0xef, 0xc3, 0x8d, 0x9b, 0x51, 0xac, 0x76, - 0xf3, 0x51, 0x2f, 0xe0, 0x49, 0x7f, 0xd6, 0xba, 0x70, 0x74, 0xbc, 0xee, 0x67, 0x7b, 0x51, 0xdf, - 0xca, 0xae, 0xd1, 0xc8, 0x6d, 0xdf, 0xd5, 0x88, 0xf7, 0x42, 0x7d, 0x84, 0xd1, 0x54, 0x81, 0x38, - 0xc2, 0x11, 0xc0, 0xc0, 0x97, 0x50, 0x6b, 0x8f, 0x4e, 0xcd, 0x3b, 0xf2, 0x1c, 0x88, 0x34, 0xf7, - 0xe8, 0x14, 0xd8, 0xe0, 0x0d, 0xd4, 0x16, 0x24, 0x8d, 0xa8, 0x0d, 0xd7, 0x20, 0x8c, 0xc0, 0x65, - 0x36, 0xdc, 0x42, 0x4d, 0xc6, 0x03, 0xc2, 0x62, 0x35, 0xf5, 0xea, 0x70, 0xa7, 0x97, 0x4a, 0x77, - 0x5a, 0x1c, 0xe4, 0xbe, 0xdd, 0x52, 0x28, 0x5a, 0x91, 0x82, 0x1f, 0xa0, 0xe6, 0x2e, 0x11, 0xe1, - 0x53, 0x22, 0xa8, 0xe7, 0x42, 0xfa, 0x27, 0x8b, 0x66, 0xfc, 0x1b, 0xbb, 0xbf, 0xa4, 0xda, 0x33, - 0x0c, 0xfc, 0x15, 0xaa, 0x72, 0x2d, 0x06, 0x95, 0x79, 0xcd, 0x7e, 0x1d, 0xd2, 0xc3, 0x9d, 0x12, - 0x46, 0x95, 0x4b, 0x7c, 0x1d, 0xd5, 0xe1, 0x53, 0xe9, 0x35, 0x01, 0xe0, 0xff, 0x25, 0x00, 0xf0, - 0xf7, 0x4a, 0xfb, 0xcd, 0x4e, 0xbc, 0x86, 0xdc, 0x3c, 0x53, 0x71, 0xa2, 0x9f, 0xa8, 0xee, 0x8d, - 0xb5, 0xf0, 0x7b, 0x68, 0x99, 0xc5, 0x01, 0x4d, 0x25, 0x1d, 0xaa, 0x69, 0x46, 0x3d, 0x04, 0x93, - 0xd0, 0xb6, 0xbe, 0x47, 0xd3, 0x8c, 0x6a, 0xee, 0x8a, 0x67, 0x9c, 0xf1, 0x68, 0xea, 0xb5, 0xdf, - 0x8e, 0xfb, 0x23, 0xbb, 0xbf, 0xcc, 0xbd, 0xc0, 0xe8, 0xfe, 0xee, 0xa0, 0xd6, 0xec, 0x7b, 0xf6, - 0x9f, 0x0e, 0xd2, 0x0f, 0xa8, 0x09, 0xdf, 0x49, 0x0d, 0x0e, 0x4f, 0x6c, 0xb0, 0x75, 0x74, 0xb8, - 0xd1, 0x30, 0xc5, 0x35, 0xfa, 0xa7, 0x67, 0x42, 0xb7, 0x79, 0x7e, 0x03, 0x40, 0xcb, 0x83, 0xea, - 0xbc, 0x71, 0x50, 0x6b, 0xa7, 0x0f, 0x6a, 0xfd, 0x95, 0x41, 0x5d, 0x47, 0xcd, 0x80, 0x64, 0x24, - 0xd0, 0x83, 0xea, 0x9a, 0xe4, 0xc2, 0xc6, 0x97, 0x51, 0x8b, 0x4c, 0x48, 0xcc, 0xf4, 0xc7, 0x1c, - 0x86, 0xc7, 0xf1, 0x8f, 0x1d, 0x18, 0xa3, 0x5a, 0x2e, 0xa9, 0x19, 0x0a, 0xc7, 0x87, 0x35, 0xbe, - 0x8e, 0x56, 0x69, 0x1a, 0x88, 0x69, 0xa6, 0x62, 0x9e, 0x0e, 0x09, 0x8b, 0xb8, 0x88, 0xd5, 0x6e, - 0x62, 0x87, 0xe0, 0xe2, 0x71, 0x6c, 0xb3, 0x08, 0x75, 0x7f, 0xad, 0xa0, 0xc6, 0xd6, 0x77, 0x8f, - 0xe1, 0x72, 0xd6, 0x90, 0x9b, 0xe6, 0x49, 0x90, 0xe5, 0xe6, 0x6e, 0x7c, 0x6b, 0x61, 0x0f, 0x35, - 0x24, 0x0f, 0xf6, 0xa8, 0x92, 0x56, 0xba, 0x0a, 0x53, 0xb7, 0x24, 0x80, 0x3f, 0x36, 0x8e, 0x91, - 0x34, 0x30, 0xb4, 0x37, 0xe1, 0x21, 0x65, 0xd0, 0x8e, 0x96, 0x6f, 0x0c, 0x2d, 0x3e, 0xc9, 0xee, - 0x01, 0xf4, 0xa0, 0xea, 0xeb, 0xa5, 0x26, 0x6f, 0x05, 0xdf, 0x7c, 0x9e, 0x5b, 0xfe, 0xcc, 0xee, - 0xfe, 0x56, 0x45, 0xcb, 0xe5, 0x37, 0x85, 0xaf, 0xa2, 0xf3, 0x93, 0x58, 0xa8, 0x9c, 0xb0, 0xf8, - 0x80, 0x68, 0x0e, 0x56, 0xc6, 0xe6, 0xbc, 0xf8, 0x36, 0x72, 0x34, 0x03, 0xa3, 0xe4, 0x1f, 0x2e, - 0x1a, 0x5d, 0x4b, 0xdd, 0x4e, 0xad, 0xce, 0xd4, 0x00, 0x09, 0x4d, 0x80, 0xd1, 0x5b, 0x00, 0x7c, - 0x4b, 0x93, 0x32, 0x40, 0x42, 0x13, 0x7d, 0x6f, 0x8c, 0x93, 0x90, 0x4c, 0xa2, 0xeb, 0x9f, 0x41, - 0x0b, 0xaa, 0xfe, 0xb1, 0x43, 0x93, 0xce, 0x04, 0x9f, 0xc4, 0x21, 0x15, 0xd0, 0x8b, 0x96, 0x3f, - 0xb3, 0xf1, 0x07, 0xe8, 0x7c, 0x9c, 0x4a, 0x45, 0xd2, 0x80, 0x0e, 0x03, 0x46, 0xa4, 0x84, 0x99, - 0x68, 0xf9, 0xe7, 0x0a, 0xef, 0x96, 0x76, 0x76, 0xbf, 0x47, 0xae, 0x11, 0x09, 0x7d, 0x63, 0x63, - 0x92, 0xc4, 0xac, 0xd0, 0x74, 0x6b, 0x41, 0x11, 0x46, 0xd4, 0x98, 0x8b, 0xc4, 0x7e, 0x13, 0x67, - 0xb6, 0xbe, 0xcd, 0x09, 0x15, 0x52, 0x77, 0xd0, 0x81, 0x50, 0x61, 0x76, 0x6f, 0xa1, 0x86, 0xa5, - 0xa3, 0xaf, 0x50, 0x71, 0x45, 0x18, 0xe0, 0xd6, 0x7c, 0x63, 0x9c, 0x9c, 0xc8, 0x2a, 0x44, 0x8e, - 0x1d, 0xdd, 0x01, 0x5a, 0x2e, 0x2b, 0xc1, 0x09, 0xa6, 0x95, 0x39, 0xa6, 0x6b, 0xc8, 0x15, 0x34, - 0xd2, 0x67, 0x30, 0xc7, 0xb3, 0xd6, 0xa0, 0xff, 0xec, 0xaf, 0xce, 0xd2, 0xb3, 0xa3, 0x4e, 0xe5, - 0xf9, 0x51, 0xa7, 0xf2, 0xe2, 0xa8, 0x53, 0xf9, 0xf3, 0xa8, 0x53, 0xf9, 0xe5, 0x65, 0x67, 0xe9, - 0xf9, 0xcb, 0xce, 0xd2, 0x8b, 0x97, 0x9d, 0xa5, 0x27, 0xe7, 0x4e, 0xf4, 0x7f, 0xe4, 0xc2, 0x7f, - 0xf7, 0x9b, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x66, 0x7a, 0x5d, 0xa8, 0x96, 0x0c, 0x00, 0x00, + proto.RegisterFile("server/diagnosticspb/diagnostics.proto", fileDescriptor_diagnostics_e9c218b3559e7579) +} + +var fileDescriptor_diagnostics_e9c218b3559e7579 = []byte{ + // 1256 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0x5f, 0x6f, 0x1b, 0x45, + 0x10, 0x8f, 0x7d, 0xf6, 0xd9, 0x5e, 0xa7, 0x25, 0xdd, 0x86, 0x70, 0xa4, 0x95, 0x53, 0x2c, 0x51, + 0x8a, 0x40, 0xb6, 0x9a, 0x82, 0x84, 0x10, 0x55, 0x15, 0x27, 0xa5, 0x54, 0x2a, 0x2d, 0xba, 0xb4, + 0x3c, 0x54, 0x08, 0x6b, 0x7d, 0xb7, 0xbe, 0x1c, 0xd9, 0xbb, 0xbd, 0xec, 0xee, 0xb9, 0x72, 0x9e, + 0x79, 0x87, 0x17, 0x1e, 0x90, 0xf8, 0x3a, 0x48, 0x7d, 0xec, 0x63, 0x9f, 0x22, 0x48, 0xbf, 0x05, + 0x4f, 0x68, 0x67, 0xf7, 0x9c, 0x8b, 0xfb, 0xc7, 0x0d, 0x12, 0x6f, 0x3b, 0x33, 0x3b, 0xbf, 0x99, + 0xdf, 0xcc, 0xec, 0xdc, 0xa1, 0xab, 0x92, 0x8a, 0x09, 0x15, 0xfd, 0x30, 0x26, 0x51, 0xca, 0xa5, + 0x8a, 0x03, 0x99, 0x8d, 0xca, 0x52, 0x2f, 0x13, 0x5c, 0x71, 0xdc, 0x09, 0x78, 0xb0, 0x2f, 0x38, + 0x09, 0xf6, 0x7a, 0xc6, 0xa3, 0x77, 0xca, 0x63, 0x7d, 0x65, 0x94, 0xc7, 0x2c, 0xec, 0xc7, 0xe9, + 0x98, 0x1b, 0x8f, 0xf5, 0x0b, 0x01, 0x4f, 0xc7, 0x71, 0xd4, 0x3f, 0xe4, 0x29, 0xb5, 0xaa, 0xd5, + 0x88, 0x47, 0x1c, 0x8e, 0x7d, 0x7d, 0xb2, 0xda, 0xf7, 0x00, 0x36, 0x1b, 0xf5, 0x49, 0x96, 0x0d, + 0xa5, 0x22, 0xca, 0xc6, 0x5c, 0xbf, 0x2c, 0x0f, 0x58, 0x5f, 0x1e, 0xb0, 0x11, 0x91, 0xb4, 0x2f, + 0x95, 0xc8, 0x03, 0x95, 0x0b, 0x1a, 0x5a, 0xeb, 0x5a, 0xe1, 0x96, 0x50, 0x45, 0x42, 0xa2, 0x88, + 0xd1, 0x77, 0x7f, 0x46, 0x68, 0x65, 0x67, 0x96, 0x9b, 0x4f, 0x33, 0x2e, 0x14, 0x1e, 0xa0, 0x5a, + 0xca, 0x43, 0xea, 0x55, 0xae, 0x54, 0xae, 0xb5, 0x37, 0xaf, 0xf5, 0xde, 0xcc, 0xa6, 0x77, 0x9f, + 0x87, 0xf4, 0x6e, 0x3a, 0xe6, 0x83, 0xda, 0xd3, 0xa3, 0x8d, 0x25, 0x1f, 0x7c, 0xf1, 0x1d, 0xe4, + 0x4a, 0xc5, 0x05, 0x95, 0x5e, 0xf5, 0x8a, 0x73, 0xad, 0xbd, 0xf9, 0xf1, 0x22, 0x94, 0x5d, 0x7d, + 0xbb, 0x04, 0x63, 0xdd, 0xf1, 0x0e, 0x72, 0x65, 0xb0, 0x47, 0x13, 0xe2, 0x39, 0x00, 0x74, 0xb5, + 0x0c, 0x74, 0xc0, 0x7a, 0x96, 0x72, 0xef, 0x21, 0x19, 0x31, 0xba, 0x43, 0x65, 0x20, 0xe2, 0x4c, + 0x71, 0x31, 0x43, 0x01, 0x5f, 0x7c, 0x1f, 0xb5, 0xe4, 0x01, 0x33, 0x05, 0xf3, 0x6a, 0x00, 0xf4, + 0xc9, 0x1c, 0xd0, 0x36, 0x67, 0x8c, 0x06, 0x8a, 0x86, 0xbb, 0x8a, 0x28, 0x9a, 0xd0, 0x54, 0xe9, + 0x43, 0x0c, 0xf9, 0x59, 0xb4, 0xa6, 0x3c, 0x60, 0x5a, 0x29, 0xf1, 0x2f, 0x15, 0x74, 0x89, 0xd1, + 0x88, 0x04, 0xd3, 0x61, 0x9e, 0xc6, 0x49, 0xc6, 0xc0, 0x81, 0x86, 0x43, 0x2a, 0x04, 0x17, 0xd2, + 0xab, 0x43, 0x88, 0x07, 0x8b, 0x48, 0xcf, 0x97, 0xbe, 0x77, 0x0f, 0x30, 0x1f, 0x95, 0x21, 0x6f, + 0x03, 0xe2, 0xed, 0x54, 0x89, 0xa9, 0xff, 0x3e, 0x7b, 0x9d, 0x1d, 0x67, 0x68, 0x85, 0x30, 0x45, + 0x05, 0x0d, 0x87, 0x92, 0x2a, 0x15, 0xa7, 0x91, 0xf4, 0x5c, 0xc8, 0xe2, 0xf6, 0x99, 0xb3, 0xd8, + 0x32, 0x40, 0xbb, 0x16, 0xc7, 0xc4, 0x7e, 0x87, 0x9c, 0xd6, 0xe2, 0x27, 0xe8, 0xa2, 0x2d, 0x01, + 0x90, 0x1e, 0x06, 0x3c, 0x4f, 0x95, 0xf4, 0x1a, 0x10, 0xf4, 0xce, 0x7f, 0xa4, 0x0e, 0x6c, 0xb6, + 0x01, 0xc9, 0x84, 0xbd, 0xc0, 0xe6, 0xf5, 0xf8, 0x27, 0xb4, 0xac, 0xdf, 0xc9, 0xd0, 0xbc, 0x19, + 0xe9, 0x35, 0x21, 0xe2, 0xd6, 0x99, 0x23, 0x3e, 0xe6, 0x29, 0xdd, 0x36, 0x18, 0x10, 0xcb, 0x76, + 0xb9, 0x7d, 0x78, 0xa2, 0xc7, 0x09, 0x3a, 0x37, 0xa6, 0x44, 0x3f, 0xa5, 0x61, 0x2e, 0x49, 0x44, + 0xbd, 0x16, 0x04, 0x1b, 0x9c, 0x39, 0xd8, 0xd7, 0x06, 0xe5, 0x91, 0x06, 0x29, 0x47, 0x5b, 0x1e, + 0x97, 0x0c, 0xeb, 0x03, 0xb4, 0xfa, 0xaa, 0xe2, 0xe3, 0x15, 0xe4, 0xec, 0xd3, 0x29, 0xbc, 0xc8, + 0x96, 0xaf, 0x8f, 0x78, 0x15, 0xd5, 0x27, 0x84, 0xe5, 0xd4, 0xab, 0x82, 0xce, 0x08, 0x5f, 0x56, + 0xbf, 0xa8, 0xac, 0xff, 0x80, 0x56, 0xe6, 0x99, 0x95, 0xfd, 0x1d, 0xe3, 0xbf, 0x59, 0xf6, 0x6f, + 0x6f, 0x5e, 0x2e, 0x11, 0x32, 0x75, 0x2d, 0x95, 0xa7, 0x8c, 0x7e, 0x0b, 0x5d, 0x78, 0x89, 0xca, + 0xa2, 0xf4, 0xea, 0x65, 0x80, 0x7b, 0xa8, 0xf3, 0xe6, 0x29, 0x5f, 0x84, 0xe6, 0x94, 0xd1, 0x76, + 0xd0, 0xda, 0xab, 0x07, 0xe7, 0x2c, 0x28, 0xdd, 0x3f, 0x6b, 0xa8, 0x59, 0xac, 0x31, 0xfc, 0x18, + 0x35, 0xf4, 0x0a, 0x1b, 0xc6, 0x21, 0x38, 0xd7, 0x07, 0x5b, 0xc7, 0x47, 0x1b, 0x2e, 0x98, 0x77, + 0xfe, 0x39, 0xda, 0xb8, 0x11, 0xc5, 0x6a, 0x2f, 0x1f, 0xf5, 0x02, 0x9e, 0xf4, 0x67, 0x35, 0x0b, + 0x47, 0x27, 0xe7, 0x7e, 0xb6, 0x1f, 0xf5, 0xed, 0xbe, 0x35, 0xcb, 0x71, 0xc7, 0x77, 0x35, 0xe2, + 0xdd, 0x50, 0xa7, 0x30, 0x9a, 0x2a, 0xd8, 0x8a, 0x90, 0x02, 0x08, 0xf8, 0x12, 0x6a, 0xed, 0xd3, + 0xa9, 0x79, 0x40, 0x9e, 0x03, 0x96, 0xe6, 0x3e, 0x9d, 0x02, 0x1b, 0xbc, 0x81, 0xda, 0x82, 0xa4, + 0x11, 0xb5, 0xe6, 0x1a, 0x98, 0x11, 0xa8, 0xcc, 0x85, 0x9b, 0xa8, 0xc9, 0x78, 0x40, 0x58, 0xac, + 0xa6, 0x5e, 0x1d, 0x9a, 0x79, 0xa9, 0xd4, 0xcc, 0x22, 0x91, 0x7b, 0xf6, 0x4a, 0xb1, 0xca, 0x0a, + 0x17, 0x7c, 0x1f, 0x35, 0xf7, 0x88, 0x08, 0x9f, 0x10, 0x41, 0x3d, 0x17, 0xdc, 0x3f, 0x5d, 0x34, + 0xdc, 0xdf, 0xd8, 0xfb, 0xa5, 0x75, 0x3d, 0xc3, 0xc0, 0x5f, 0xa1, 0x2a, 0xd7, 0x5b, 0xa0, 0x32, + 0xbf, 0xac, 0x5f, 0x85, 0xf4, 0x60, 0xb7, 0x84, 0x51, 0xe5, 0x12, 0x5f, 0x47, 0x75, 0xf8, 0x38, + 0x7a, 0x4d, 0x00, 0x78, 0xb7, 0x04, 0x00, 0xfa, 0x5e, 0xe9, 0xbe, 0xb9, 0x89, 0xd7, 0x90, 0x9b, + 0x67, 0x2a, 0x4e, 0xf4, 0xdb, 0xd4, 0xb5, 0xb1, 0x12, 0xfe, 0x00, 0x2d, 0xb3, 0x38, 0xa0, 0xa9, + 0xa4, 0x43, 0x35, 0xcd, 0xa8, 0x87, 0x60, 0x12, 0xda, 0x56, 0xf7, 0x70, 0x9a, 0x51, 0xcd, 0x5d, + 0xf1, 0x8c, 0x33, 0x1e, 0x4d, 0xbd, 0xf6, 0xdb, 0x71, 0x7f, 0x68, 0xef, 0x97, 0xb9, 0x17, 0x18, + 0xdd, 0x3f, 0x1c, 0xd4, 0x9a, 0x7d, 0xc8, 0xfe, 0xd7, 0x41, 0xfa, 0x11, 0x35, 0xe1, 0x03, 0xa9, + 0xc1, 0xe1, 0x89, 0x0d, 0xb6, 0x8f, 0x8f, 0x36, 0x1a, 0x26, 0xb8, 0x46, 0xff, 0xec, 0x4c, 0xe8, + 0xd6, 0xcf, 0x6f, 0x00, 0x68, 0x79, 0x50, 0x9d, 0xd7, 0x0e, 0x6a, 0xed, 0xcd, 0x83, 0x5a, 0x7f, + 0x69, 0x50, 0xd7, 0x51, 0x33, 0x20, 0x19, 0x09, 0xf4, 0xa0, 0xba, 0xc6, 0xb9, 0x90, 0xf1, 0x65, + 0xd4, 0x22, 0x13, 0x12, 0x33, 0xfd, 0x15, 0x87, 0xe1, 0x71, 0xfc, 0x13, 0x05, 0xc6, 0xa8, 0x96, + 0x4b, 0x6a, 0x86, 0xc2, 0xf1, 0xe1, 0x8c, 0xaf, 0xa3, 0x55, 0x9a, 0x06, 0x62, 0x9a, 0xa9, 0x98, + 0xa7, 0x43, 0xc2, 0x22, 0x2e, 0x62, 0xb5, 0x97, 0xd8, 0x21, 0xb8, 0x78, 0x62, 0xdb, 0x2a, 0x4c, + 0xdd, 0xdf, 0x2a, 0xa8, 0xb1, 0xfd, 0xdd, 0x23, 0x68, 0xce, 0x1a, 0x72, 0xd3, 0x3c, 0x09, 0xb2, + 0xdc, 0xf4, 0xc6, 0xb7, 0x12, 0xf6, 0x50, 0x43, 0xf2, 0x60, 0x9f, 0x2a, 0x69, 0x57, 0x57, 0x21, + 0xea, 0x92, 0x04, 0xf0, 0x47, 0xe3, 0x98, 0x95, 0x06, 0x82, 0xd6, 0x26, 0x3c, 0xa4, 0x0c, 0xca, + 0xd1, 0xf2, 0x8d, 0xa0, 0x97, 0x4f, 0xb2, 0x77, 0x08, 0x35, 0xa8, 0xfa, 0xfa, 0xa8, 0xc9, 0xdb, + 0x4d, 0x6f, 0xbe, 0xcb, 0x2d, 0x7f, 0x26, 0x77, 0x7f, 0xaf, 0xa2, 0xe5, 0xf2, 0x9b, 0xc2, 0x57, + 0xd1, 0xf9, 0x49, 0x2c, 0x54, 0x4e, 0x58, 0x7c, 0x48, 0x34, 0x07, 0xbb, 0xc6, 0xe6, 0xb4, 0xf8, + 0x16, 0x72, 0x34, 0x03, 0xb3, 0xc2, 0x3f, 0x5a, 0x34, 0xba, 0x96, 0xba, 0x9d, 0x5a, 0xed, 0xa9, + 0x01, 0x12, 0x9a, 0x00, 0xa3, 0xb7, 0x00, 0xf8, 0x96, 0x26, 0x65, 0x80, 0x84, 0x26, 0xba, 0x6f, + 0x8c, 0x93, 0x90, 0x4c, 0xa2, 0xeb, 0x9f, 0x43, 0x09, 0xaa, 0xfe, 0x89, 0x42, 0x93, 0xce, 0x04, + 0x9f, 0xc4, 0x21, 0x15, 0x50, 0x8b, 0x96, 0x3f, 0x93, 0xf1, 0x87, 0xe8, 0x7c, 0x9c, 0x4a, 0x45, + 0xd2, 0x80, 0x0e, 0x03, 0x46, 0xa4, 0x84, 0x99, 0x68, 0xf9, 0xe7, 0x0a, 0xed, 0xb6, 0x56, 0x76, + 0xbf, 0x47, 0xae, 0x59, 0x12, 0xba, 0x63, 0x63, 0x92, 0xc4, 0xac, 0xd8, 0xe9, 0x56, 0x82, 0x20, + 0x8c, 0xa8, 0x31, 0x17, 0x89, 0xfd, 0x18, 0xce, 0x64, 0xdd, 0xcd, 0x09, 0x15, 0x52, 0x57, 0xd0, + 0x01, 0x53, 0x21, 0x76, 0x6f, 0xa2, 0x86, 0xa5, 0xa3, 0x5b, 0xa8, 0xb8, 0x22, 0x0c, 0x70, 0x6b, + 0xbe, 0x11, 0x4e, 0x4f, 0x64, 0x15, 0x2c, 0x27, 0x8a, 0xee, 0x00, 0x2d, 0x97, 0x37, 0xc1, 0x29, + 0xa6, 0x95, 0x39, 0xa6, 0x6b, 0xc8, 0x15, 0x34, 0xd2, 0x39, 0x98, 0xf4, 0xac, 0x34, 0xe8, 0x3f, + 0xfd, 0xbb, 0xb3, 0xf4, 0xf4, 0xb8, 0x53, 0x79, 0x76, 0xdc, 0xa9, 0x3c, 0x3f, 0xee, 0x54, 0xfe, + 0x3a, 0xee, 0x54, 0x7e, 0x7d, 0xd1, 0x59, 0x7a, 0xf6, 0xa2, 0xb3, 0xf4, 0xfc, 0x45, 0x67, 0xe9, + 0xf1, 0xb9, 0x53, 0xf5, 0x1f, 0xb9, 0xf0, 0xd3, 0x7e, 0xe3, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, + 0x0f, 0x72, 0xb7, 0x96, 0x88, 0x0c, 0x00, 0x00, } diff --git a/pkg/server/diagnosticspb/diagnostics.proto b/pkg/server/diagnosticspb/diagnostics.proto index 59ba38f1eb37..50d142bdbc33 100644 --- a/pkg/server/diagnosticspb/diagnostics.proto +++ b/pkg/server/diagnosticspb/diagnostics.proto @@ -13,7 +13,7 @@ package cockroach.server.diagnosticspb; option go_package = "diagnosticspb"; import "build/info.proto"; -import "config/zonepb/zone.proto"; +import "config/zone.proto"; import "gogoproto/gogo.proto"; import "roachpb/app_stats.proto"; import "sql/sqlbase/structured.proto"; @@ -25,7 +25,7 @@ message DiagnosticReport { repeated sql.sqlbase.TableDescriptor schema = 3 [(gogoproto.nullable) = false]; repeated sql.CollectedStatementStatistics sql_stats = 4 [(gogoproto.nullable) = false]; map altered_settings = 6; - map zone_configs = 8 [(gogoproto.nullable) = false]; + map zone_configs = 8 [(gogoproto.nullable) = false]; map feature_usage = 9 [(gogoproto.nullable) = false]; diff --git a/pkg/server/node.go b/pkg/server/node.go index 2d4bd01c7081..9c277a1dba41 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" @@ -194,7 +193,7 @@ func allocateStoreIDs( // GetBootstrapSchema returns the schema which will be used to bootstrap a new // server. func GetBootstrapSchema( - defaultZoneConfig *zonepb.ZoneConfig, defaultSystemZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, defaultSystemZoneConfig *config.ZoneConfig, ) sqlbase.MetadataSchema { return sqlbase.MakeMetadataSchema(defaultZoneConfig, defaultSystemZoneConfig) } @@ -211,8 +210,8 @@ func bootstrapCluster( ctx context.Context, engines []engine.Engine, bootstrapVersion cluster.ClusterVersion, - defaultZoneConfig *zonepb.ZoneConfig, - defaultSystemZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, + defaultSystemZoneConfig *config.ZoneConfig, ) (uuid.UUID, error) { clusterID := uuid.MakeV4() // TODO(andrei): It'd be cool if this method wouldn't do anything to engines @@ -311,8 +310,8 @@ func (n *Node) bootstrapCluster( ctx context.Context, engines []engine.Engine, bootstrapVersion cluster.ClusterVersion, - defaultZoneConfig *zonepb.ZoneConfig, - defaultSystemZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, + defaultSystemZoneConfig *config.ZoneConfig, ) error { if n.initialBoot || n.clusterID.Get() != uuid.Nil { return fmt.Errorf("cluster has already been initialized with ID %s", n.clusterID.Get()) diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index db9c776a1ba3..e22496e8dd9b 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" "github.com/cockroachdb/cockroach/pkg/internal/client" @@ -209,7 +208,7 @@ func TestBootstrapCluster(t *testing.T) { e := engine.NewInMem(roachpb.Attributes{}, 1<<20) defer e.Close() if _, err := bootstrapCluster( - ctx, []engine.Engine{e}, cluster.TestingClusterVersion, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + ctx, []engine.Engine{e}, cluster.TestingClusterVersion, config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef(), ); err != nil { t.Fatal(err) } @@ -237,7 +236,7 @@ func TestBootstrapCluster(t *testing.T) { } // Add the initial keys for sql. - kvs, tableSplits := GetBootstrapSchema(zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()).GetInitialValues() + kvs, tableSplits := GetBootstrapSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()).GetInitialValues() for _, kv := range kvs { expectedKeys = append(expectedKeys, kv.Key) } @@ -264,7 +263,7 @@ func TestBootstrapNewStore(t *testing.T) { ctx := context.Background() e := engine.NewInMem(roachpb.Attributes{}, 1<<20) if _, err := bootstrapCluster( - ctx, []engine.Engine{e}, cluster.TestingClusterVersion, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + ctx, []engine.Engine{e}, cluster.TestingClusterVersion, config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef(), ); err != nil { t.Fatal(err) } @@ -319,7 +318,7 @@ func TestNodeJoin(t *testing.T) { engineStopper.AddCloser(e) if _, err := bootstrapCluster( - ctx, []engine.Engine{e}, cluster.TestingClusterVersion, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + ctx, []engine.Engine{e}, cluster.TestingClusterVersion, config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef(), ); err != nil { t.Fatal(err) } @@ -390,7 +389,7 @@ func TestCorruptedClusterID(t *testing.T) { defer e.Close() if _, err := bootstrapCluster( - ctx, []engine.Engine{e}, cluster.TestingClusterVersion, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + ctx, []engine.Engine{e}, cluster.TestingClusterVersion, config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef(), ); err != nil { t.Fatal(err) } @@ -734,7 +733,7 @@ func TestStartNodeWithLocality(t *testing.T) { e := engine.NewInMem(roachpb.Attributes{}, 1<<20) defer e.Close() if _, err := bootstrapCluster( - ctx, []engine.Engine{e}, cluster.TestingClusterVersion, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), + ctx, []engine.Engine{e}, cluster.TestingClusterVersion, config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef(), ); err != nil { t.Fatal(err) } diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 969ff56499e2..9f5eca1703d1 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -29,7 +29,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -74,7 +73,7 @@ func TestSelfBootstrap(t *testing.T) { func TestHealthCheck(t *testing.T) { defer leaktest.AfterTest(t)() - cfg := zonepb.DefaultZoneConfig() + cfg := config.DefaultZoneConfig() cfg.NumReplicas = proto.Int32(1) s, err := serverutils.StartServerRaw(base.TestServerArgs{ Knobs: base.TestingKnobs{ diff --git a/pkg/server/serverpb/admin.pb.go b/pkg/server/serverpb/admin.pb.go index 7c95a384c018..2737e8f76209 100644 --- a/pkg/server/serverpb/admin.pb.go +++ b/pkg/server/serverpb/admin.pb.go @@ -6,7 +6,7 @@ package serverpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import zonepb "github.com/cockroachdb/cockroach/pkg/config/zonepb" +import config "github.com/cockroachdb/cockroach/pkg/config" import jobspb "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" import enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" import storagepb "github.com/cockroachdb/cockroach/pkg/storage/storagepb" @@ -73,7 +73,7 @@ func (x ZoneConfigurationLevel) String() string { return proto.EnumName(ZoneConfigurationLevel_name, int32(x)) } func (ZoneConfigurationLevel) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_admin_247dca19c09077f7, []int{0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{0} } type DrainMode int32 @@ -100,7 +100,7 @@ func (x DrainMode) String() string { return proto.EnumName(DrainMode_name, int32(x)) } func (DrainMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_admin_247dca19c09077f7, []int{1} + return fileDescriptor_admin_19a5a4e012fc0559, []int{1} } // DatabasesRequest requests a list of databases. @@ -111,7 +111,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_247dca19c09077f7, []int{0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{0} } func (m *DatabasesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -145,7 +145,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_247dca19c09077f7, []int{1} + return fileDescriptor_admin_19a5a4e012fc0559, []int{1} } func (m *DatabasesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -181,7 +181,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_247dca19c09077f7, []int{2} + return fileDescriptor_admin_19a5a4e012fc0559, []int{2} } func (m *DatabaseDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -219,7 +219,7 @@ type DatabaseDetailsResponse struct { // the 'target_id' field of events. DescriptorID int64 `protobuf:"varint,3,opt,name=descriptor_id,json=descriptorId,proto3" json:"descriptor_id,omitempty"` // The zone configuration in effect for this database. - ZoneConfig zonepb.ZoneConfig `protobuf:"bytes,4,opt,name=zone_config,json=zoneConfig,proto3" json:"zone_config"` + ZoneConfig config.ZoneConfig `protobuf:"bytes,4,opt,name=zone_config,json=zoneConfig,proto3" json:"zone_config"` // The level at which this object's zone configuration is set. ZoneConfigLevel ZoneConfigurationLevel `protobuf:"varint,5,opt,name=zone_config_level,json=zoneConfigLevel,proto3,enum=cockroach.server.serverpb.ZoneConfigurationLevel" json:"zone_config_level,omitempty"` } @@ -228,7 +228,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_247dca19c09077f7, []int{3} + return fileDescriptor_admin_19a5a4e012fc0559, []int{3} } func (m *DatabaseDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -264,7 +264,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_247dca19c09077f7, []int{3, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{3, 0} } func (m *DatabaseDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -301,7 +301,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_247dca19c09077f7, []int{4} + return fileDescriptor_admin_19a5a4e012fc0559, []int{4} } func (m *TableDetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -345,7 +345,7 @@ type TableDetailsResponse struct { // executed. CreateTableStatement string `protobuf:"bytes,5,opt,name=create_table_statement,json=createTableStatement,proto3" json:"create_table_statement,omitempty"` // The zone configuration in effect for this table. - ZoneConfig zonepb.ZoneConfig `protobuf:"bytes,6,opt,name=zone_config,json=zoneConfig,proto3" json:"zone_config"` + ZoneConfig config.ZoneConfig `protobuf:"bytes,6,opt,name=zone_config,json=zoneConfig,proto3" json:"zone_config"` // The level at which this object's zone configuration is set. ZoneConfigLevel ZoneConfigurationLevel `protobuf:"varint,7,opt,name=zone_config_level,json=zoneConfigLevel,proto3,enum=cockroach.server.serverpb.ZoneConfigurationLevel" json:"zone_config_level,omitempty"` // descriptor_id is an identifier used to uniquely identify this table. @@ -358,7 +358,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_247dca19c09077f7, []int{5} + return fileDescriptor_admin_19a5a4e012fc0559, []int{5} } func (m *TableDetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -395,7 +395,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_247dca19c09077f7, []int{5, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{5, 0} } func (m *TableDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -439,7 +439,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_247dca19c09077f7, []int{5, 1} + return fileDescriptor_admin_19a5a4e012fc0559, []int{5, 1} } func (m *TableDetailsResponse_Column) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -485,7 +485,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_247dca19c09077f7, []int{5, 2} + return fileDescriptor_admin_19a5a4e012fc0559, []int{5, 2} } func (m *TableDetailsResponse_Index) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,7 +523,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_247dca19c09077f7, []int{6} + return fileDescriptor_admin_19a5a4e012fc0559, []int{6} } func (m *TableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -577,7 +577,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_247dca19c09077f7, []int{7} + return fileDescriptor_admin_19a5a4e012fc0559, []int{7} } func (m *TableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -615,7 +615,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_247dca19c09077f7, []int{7, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{7, 0} } func (m *TableStatsResponse_MissingNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -649,7 +649,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_247dca19c09077f7, []int{8} + return fileDescriptor_admin_19a5a4e012fc0559, []int{8} } func (m *NonTableStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -688,7 +688,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_247dca19c09077f7, []int{9} + return fileDescriptor_admin_19a5a4e012fc0559, []int{9} } func (m *NonTableStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -721,7 +721,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_247dca19c09077f7, []int{10} + return fileDescriptor_admin_19a5a4e012fc0559, []int{10} } func (m *UsersRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -756,7 +756,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_247dca19c09077f7, []int{11} + return fileDescriptor_admin_19a5a4e012fc0559, []int{11} } func (m *UsersResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -790,7 +790,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_247dca19c09077f7, []int{11, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{11, 0} } func (m *UsersResponse_User) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -831,7 +831,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_247dca19c09077f7, []int{12} + return fileDescriptor_admin_19a5a4e012fc0559, []int{12} } func (m *EventsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -866,7 +866,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_247dca19c09077f7, []int{13} + return fileDescriptor_admin_19a5a4e012fc0559, []int{13} } func (m *EventsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -911,7 +911,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_247dca19c09077f7, []int{13, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{13, 0} } func (m *EventsResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -947,7 +947,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_247dca19c09077f7, []int{14} + return fileDescriptor_admin_19a5a4e012fc0559, []int{14} } func (m *SetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -980,7 +980,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_247dca19c09077f7, []int{15} + return fileDescriptor_admin_19a5a4e012fc0559, []int{15} } func (m *SetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1015,7 +1015,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_247dca19c09077f7, []int{16} + return fileDescriptor_admin_19a5a4e012fc0559, []int{16} } func (m *GetUIDataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1052,7 +1052,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_247dca19c09077f7, []int{17} + return fileDescriptor_admin_19a5a4e012fc0559, []int{17} } func (m *GetUIDataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1088,7 +1088,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_247dca19c09077f7, []int{17, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{17, 0} } func (m *GetUIDataResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1121,7 +1121,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_247dca19c09077f7, []int{18} + return fileDescriptor_admin_19a5a4e012fc0559, []int{18} } func (m *ClusterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1160,7 +1160,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_247dca19c09077f7, []int{19} + return fileDescriptor_admin_19a5a4e012fc0559, []int{19} } func (m *ClusterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1204,7 +1204,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_247dca19c09077f7, []int{20} + return fileDescriptor_admin_19a5a4e012fc0559, []int{20} } func (m *DrainRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1239,7 +1239,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_247dca19c09077f7, []int{21} + return fileDescriptor_admin_19a5a4e012fc0559, []int{21} } func (m *DrainResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1274,7 +1274,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_247dca19c09077f7, []int{22} + return fileDescriptor_admin_19a5a4e012fc0559, []int{22} } func (m *DecommissionStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1312,7 +1312,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_247dca19c09077f7, []int{23} + return fileDescriptor_admin_19a5a4e012fc0559, []int{23} } func (m *DecommissionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1347,7 +1347,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_247dca19c09077f7, []int{24} + return fileDescriptor_admin_19a5a4e012fc0559, []int{24} } func (m *DecommissionStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1385,7 +1385,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_247dca19c09077f7, []int{24, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{24, 0} } func (m *DecommissionStatusResponse_Status) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1421,7 +1421,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_247dca19c09077f7, []int{25} + return fileDescriptor_admin_19a5a4e012fc0559, []int{25} } func (m *SettingsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1455,7 +1455,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_247dca19c09077f7, []int{26} + return fileDescriptor_admin_19a5a4e012fc0559, []int{26} } func (m *SettingsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1490,7 +1490,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_247dca19c09077f7, []int{26, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{26, 0} } func (m *SettingsResponse_Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1523,7 +1523,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_247dca19c09077f7, []int{27} + return fileDescriptor_admin_19a5a4e012fc0559, []int{27} } func (m *HealthRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1558,7 +1558,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_247dca19c09077f7, []int{28} + return fileDescriptor_admin_19a5a4e012fc0559, []int{28} } func (m *HealthResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1591,7 +1591,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_247dca19c09077f7, []int{29} + return fileDescriptor_admin_19a5a4e012fc0559, []int{29} } func (m *LivenessRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1626,7 +1626,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_247dca19c09077f7, []int{30} + return fileDescriptor_admin_19a5a4e012fc0559, []int{30} } func (m *LivenessResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1662,7 +1662,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_247dca19c09077f7, []int{31} + return fileDescriptor_admin_19a5a4e012fc0559, []int{31} } func (m *JobsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1696,7 +1696,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_247dca19c09077f7, []int{32} + return fileDescriptor_admin_19a5a4e012fc0559, []int{32} } func (m *JobsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1749,7 +1749,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_247dca19c09077f7, []int{32, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{32, 0} } func (m *JobsResponse_Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1782,7 +1782,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_247dca19c09077f7, []int{33} + return fileDescriptor_admin_19a5a4e012fc0559, []int{33} } func (m *LocationsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1816,7 +1816,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_247dca19c09077f7, []int{34} + return fileDescriptor_admin_19a5a4e012fc0559, []int{34} } func (m *LocationsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1852,7 +1852,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_247dca19c09077f7, []int{34, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{34, 0} } func (m *LocationsResponse_Location) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1894,7 +1894,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_247dca19c09077f7, []int{35} + return fileDescriptor_admin_19a5a4e012fc0559, []int{35} } func (m *RangeLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1928,7 +1928,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_247dca19c09077f7, []int{36} + return fileDescriptor_admin_19a5a4e012fc0559, []int{36} } func (m *RangeLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1968,7 +1968,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_247dca19c09077f7, []int{36, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{36, 0} } func (m *RangeLogResponse_PrettyInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2002,7 +2002,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_247dca19c09077f7, []int{36, 1} + return fileDescriptor_admin_19a5a4e012fc0559, []int{36, 1} } func (m *RangeLogResponse_Event) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2037,7 +2037,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_247dca19c09077f7, []int{37} + return fileDescriptor_admin_19a5a4e012fc0559, []int{37} } func (m *QueryPlanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2072,7 +2072,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_247dca19c09077f7, []int{38} + return fileDescriptor_admin_19a5a4e012fc0559, []int{38} } func (m *QueryPlanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2104,7 +2104,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_247dca19c09077f7, []int{39} + return fileDescriptor_admin_19a5a4e012fc0559, []int{39} } func (m *DataDistributionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2140,7 +2140,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_247dca19c09077f7, []int{40} + return fileDescriptor_admin_19a5a4e012fc0559, []int{40} } func (m *DataDistributionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2169,7 +2169,7 @@ type DataDistributionResponse_ZoneConfig struct { // target is the object the zone config applies to, e.g. "DATABASE db" or // "PARTITION north_america OF TABLE users". Target string `protobuf:"bytes,1,opt,name=target,proto3" json:"target,omitempty"` - Config zonepb.ZoneConfig `protobuf:"bytes,2,opt,name=config,proto3" json:"config"` + Config config.ZoneConfig `protobuf:"bytes,2,opt,name=config,proto3" json:"config"` // config_sql is the SQL representation of config. ConfigSQL string `protobuf:"bytes,4,opt,name=config_sql,json=configSql,proto3" json:"config_sql,omitempty"` } @@ -2178,7 +2178,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_247dca19c09077f7, []int{40, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{40, 0} } func (m *DataDistributionResponse_ZoneConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2213,7 +2213,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_247dca19c09077f7, []int{40, 1} + return fileDescriptor_admin_19a5a4e012fc0559, []int{40, 1} } func (m *DataDistributionResponse_TableInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2247,7 +2247,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_247dca19c09077f7, []int{40, 2} + return fileDescriptor_admin_19a5a4e012fc0559, []int{40, 2} } func (m *DataDistributionResponse_DatabaseInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2280,7 +2280,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_247dca19c09077f7, []int{41} + return fileDescriptor_admin_19a5a4e012fc0559, []int{41} } func (m *MetricMetadataRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2314,7 +2314,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_247dca19c09077f7, []int{42} + return fileDescriptor_admin_19a5a4e012fc0559, []int{42} } func (m *MetricMetadataResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2357,7 +2357,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_247dca19c09077f7, []int{43} + return fileDescriptor_admin_19a5a4e012fc0559, []int{43} } func (m *EnqueueRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2390,7 +2390,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_247dca19c09077f7, []int{44} + return fileDescriptor_admin_19a5a4e012fc0559, []int{44} } func (m *EnqueueRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2427,7 +2427,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_247dca19c09077f7, []int{44, 0} + return fileDescriptor_admin_19a5a4e012fc0559, []int{44, 0} } func (m *EnqueueRangeResponse_Details) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2460,7 +2460,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_247dca19c09077f7, []int{45} + return fileDescriptor_admin_19a5a4e012fc0559, []int{45} } func (m *ChartCatalogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2494,7 +2494,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_247dca19c09077f7, []int{46} + return fileDescriptor_admin_19a5a4e012fc0559, []int{46} } func (m *ChartCatalogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -15540,260 +15540,260 @@ var ( ErrIntOverflowAdmin = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_247dca19c09077f7) } +func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_19a5a4e012fc0559) } -var fileDescriptor_admin_247dca19c09077f7 = []byte{ - // 4025 bytes of a gzipped FileDescriptorProto +var fileDescriptor_admin_19a5a4e012fc0559 = []byte{ + // 4019 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x4d, 0x73, 0x1b, 0xc9, - 0x75, 0x1a, 0x80, 0x00, 0x81, 0x47, 0x80, 0x04, 0x5b, 0x14, 0x05, 0x41, 0x32, 0xc1, 0x1d, 0x79, - 0xbd, 0x34, 0x77, 0x0d, 0xac, 0x28, 0xed, 0xae, 0xa3, 0xd5, 0xc6, 0x21, 0x48, 0x16, 0x17, 0x5a, - 0x4a, 0xbb, 0x1a, 0x50, 0xda, 0xb2, 0x9c, 0xcd, 0x64, 0x80, 0x69, 0x82, 0x63, 0x0e, 0x66, 0xc0, - 0x99, 0x01, 0x25, 0xee, 0xc6, 0xae, 0xd8, 0xf9, 0xa8, 0x9c, 0x52, 0x5b, 0x71, 0x6e, 0x5b, 0x95, - 0x2a, 0xa7, 0x52, 0xc9, 0x21, 0x97, 0xf8, 0x96, 0xe4, 0x90, 0xf3, 0x56, 0x0e, 0xb6, 0xab, 0x72, - 0x71, 0x0e, 0xa1, 0x13, 0x6e, 0x0e, 0xa9, 0xfc, 0x80, 0x1c, 0x72, 0x49, 0xaa, 0xbf, 0x66, 0x1a, - 0xe0, 0x68, 0x04, 0x50, 0x5e, 0x1f, 0x48, 0x4c, 0xbf, 0xd7, 0xfd, 0xfa, 0xf5, 0xeb, 0xd7, 0xaf, - 0xdf, 0x47, 0xc3, 0x55, 0x1f, 0x7b, 0x47, 0xd8, 0xab, 0xb3, 0x9f, 0x7e, 0xbb, 0x6e, 0x98, 0x3d, - 0xcb, 0xa9, 0xf5, 0x3d, 0x37, 0x70, 0xd1, 0x95, 0x8e, 0xdb, 0x39, 0xf0, 0x5c, 0xa3, 0xb3, 0x5f, - 0x63, 0xf8, 0x9a, 0xe8, 0x56, 0x29, 0x77, 0x5c, 0x67, 0xcf, 0xea, 0xd6, 0x3f, 0x76, 0x1d, 0xdc, - 0x6f, 0xd3, 0x1f, 0x36, 0xa8, 0xb2, 0xf8, 0x5d, 0xb7, 0xed, 0xd7, 0xc9, 0xbf, 0x7e, 0x9b, 0xfe, - 0x70, 0xf8, 0xb5, 0xd1, 0x99, 0xfc, 0xc0, 0x08, 0x06, 0x02, 0xab, 0xfa, 0x81, 0xeb, 0x19, 0x5d, - 0x5c, 0xc7, 0x4e, 0xd7, 0x72, 0xc4, 0x4f, 0xbf, 0x5d, 0xef, 0x1d, 0x75, 0x3a, 0xbc, 0xcf, 0xb2, - 0xe8, 0xc3, 0x7f, 0xfb, 0xed, 0xba, 0x6d, 0x1d, 0x61, 0x07, 0xfb, 0x82, 0xca, 0xd5, 0x98, 0x1e, - 0x6e, 0x97, 0x23, 0x97, 0x02, 0xbf, 0xde, 0x31, 0x02, 0xc3, 0x76, 0xbb, 0xf5, 0xce, 0xbe, 0xe1, - 0x05, 0x3a, 0x6f, 0x71, 0x7c, 0x79, 0x10, 0x58, 0x76, 0xbd, 0x87, 0x03, 0xcf, 0xea, 0xf0, 0x1f, - 0x8e, 0x59, 0xe8, 0xba, 0x5d, 0x97, 0x7e, 0xd6, 0xc9, 0x97, 0x58, 0x50, 0xd7, 0x75, 0xbb, 0x36, - 0xae, 0x1b, 0x7d, 0xab, 0x6e, 0x38, 0x8e, 0x1b, 0x18, 0x81, 0xe5, 0x3a, 0x82, 0x95, 0x2a, 0xc7, - 0xd2, 0x56, 0x7b, 0xb0, 0x57, 0x0f, 0xac, 0x1e, 0xf6, 0x03, 0xa3, 0xd7, 0x67, 0x1d, 0x54, 0x04, - 0xa5, 0x4d, 0x23, 0x30, 0xda, 0x86, 0x8f, 0x7d, 0x0d, 0x1f, 0x0e, 0xb0, 0x1f, 0xa8, 0x37, 0x60, - 0x5e, 0x82, 0xf9, 0x7d, 0xd7, 0xf1, 0x31, 0xba, 0x06, 0x79, 0x53, 0x00, 0xcb, 0xca, 0x72, 0x7a, - 0x25, 0xaf, 0x45, 0x00, 0xf5, 0x16, 0x2c, 0x8a, 0x21, 0x9b, 0x38, 0x30, 0x2c, 0x5b, 0x10, 0x43, - 0x15, 0xc8, 0x89, 0x6e, 0x65, 0x65, 0x59, 0x59, 0xc9, 0x6b, 0x61, 0x5b, 0xfd, 0x49, 0x1a, 0x2e, - 0x9f, 0x19, 0xc6, 0xe7, 0x7b, 0x04, 0xd9, 0xae, 0x67, 0x38, 0x01, 0x9b, 0x6c, 0x66, 0xed, 0x9b, - 0xb5, 0x67, 0xaa, 0x41, 0xed, 0x19, 0x34, 0x6a, 0xdb, 0x84, 0x40, 0x63, 0xea, 0xf3, 0x93, 0xea, - 0x05, 0x8d, 0x53, 0x43, 0x55, 0x98, 0x09, 0x8c, 0xb6, 0x8d, 0x75, 0xc7, 0xe8, 0x61, 0xbf, 0x9c, - 0xa2, 0x2b, 0x01, 0x0a, 0xba, 0x4f, 0x20, 0xe8, 0x0d, 0x28, 0x9a, 0xd8, 0xef, 0x78, 0x56, 0x3f, - 0x70, 0x3d, 0xdd, 0x32, 0xcb, 0xe9, 0x65, 0x65, 0x25, 0xdd, 0x28, 0x9d, 0x9e, 0x54, 0x0b, 0x9b, - 0x21, 0xa2, 0xb9, 0xa9, 0x15, 0xa2, 0x6e, 0x4d, 0x13, 0xdd, 0x85, 0x19, 0xa2, 0x7e, 0x3a, 0xd3, - 0xc8, 0xf2, 0xd4, 0xb2, 0xb2, 0x32, 0xb3, 0x76, 0x5d, 0x62, 0x9a, 0x21, 0x6a, 0x4c, 0x55, 0x6b, - 0x8f, 0x5d, 0x07, 0x6f, 0x50, 0x08, 0xe7, 0x0f, 0x3e, 0x0e, 0x21, 0xe8, 0x23, 0x98, 0x97, 0x68, - 0xe9, 0x36, 0x3e, 0xc2, 0x76, 0x39, 0xb3, 0xac, 0xac, 0xcc, 0xae, 0xdd, 0x48, 0x10, 0x43, 0x44, - 0x73, 0xe0, 0x51, 0x2d, 0xd8, 0x21, 0x03, 0xb5, 0xb9, 0x88, 0x32, 0x05, 0x54, 0xde, 0x86, 0x0c, - 0x95, 0x0c, 0x42, 0x30, 0x35, 0xf0, 0xb1, 0xc7, 0xf7, 0x85, 0x7e, 0xa3, 0x25, 0x80, 0xbe, 0x67, - 0x1d, 0x59, 0x36, 0xee, 0x46, 0xe2, 0x89, 0x20, 0xea, 0x36, 0x5c, 0xdc, 0x25, 0xc2, 0x1a, 0x7f, - 0x9b, 0xd1, 0x02, 0x64, 0xa8, 0x7c, 0xcb, 0x29, 0x8a, 0x60, 0x0d, 0xf5, 0x8f, 0x72, 0xb0, 0x30, - 0x4c, 0x89, 0xef, 0x7c, 0x6b, 0x64, 0xe7, 0xdf, 0x48, 0x58, 0x72, 0x1c, 0x81, 0xd8, 0x6d, 0x7f, - 0x04, 0xd3, 0x1d, 0xd7, 0x1e, 0xf4, 0x1c, 0xb6, 0xa6, 0x99, 0xb5, 0x37, 0x27, 0xa5, 0xba, 0x41, - 0x87, 0x73, 0xb2, 0x82, 0x18, 0x7a, 0x08, 0xd3, 0x96, 0x63, 0xe2, 0xa7, 0xd8, 0x2f, 0xa7, 0xcf, - 0xc7, 0x6d, 0x93, 0x0c, 0x17, 0x64, 0x39, 0x2d, 0xa2, 0xa5, 0x9e, 0xe1, 0x74, 0x89, 0x0a, 0x0c, - 0x9c, 0x80, 0x6a, 0x53, 0x5a, 0x03, 0x0a, 0xda, 0x20, 0x10, 0x74, 0x0b, 0x16, 0x3b, 0x1e, 0x36, - 0x02, 0xac, 0x33, 0x6d, 0x26, 0x66, 0x0c, 0xf7, 0xb0, 0x13, 0x50, 0x3d, 0xc9, 0x6b, 0x0b, 0x0c, - 0x4b, 0x67, 0x6c, 0x09, 0xdc, 0xa8, 0x92, 0x66, 0x7f, 0xe5, 0x4a, 0x3a, 0xfd, 0xab, 0x52, 0xd2, - 0xb3, 0xc7, 0x30, 0x37, 0xce, 0x31, 0x7c, 0x21, 0xdd, 0xae, 0xfc, 0x93, 0x02, 0x59, 0xb6, 0xcd, - 0x64, 0x38, 0x31, 0x10, 0x62, 0x38, 0xf9, 0x26, 0xb0, 0xe0, 0xb8, 0x2f, 0xd4, 0x98, 0x7e, 0x13, - 0xbd, 0x77, 0x06, 0xb6, 0x4d, 0xd5, 0x9b, 0x18, 0x8a, 0x9c, 0x16, 0xb6, 0xd1, 0x75, 0xb2, 0x84, - 0x3d, 0x63, 0x60, 0x07, 0xfa, 0x91, 0x61, 0x0f, 0x30, 0xdd, 0xc6, 0x3c, 0x61, 0x98, 0x02, 0x1f, - 0x11, 0x18, 0xba, 0x09, 0x97, 0xba, 0xd8, 0xc1, 0x4c, 0x16, 0x3a, 0x7e, 0xda, 0xf7, 0xb0, 0xef, - 0x5b, 0xae, 0x23, 0xf6, 0x31, 0x42, 0x6e, 0x85, 0x38, 0xb4, 0x08, 0xd9, 0x7d, 0xcb, 0x34, 0xb1, - 0x43, 0xb7, 0x30, 0xa7, 0xf1, 0x56, 0xe5, 0x27, 0x0a, 0x64, 0xa8, 0x3e, 0xc5, 0xf2, 0xbf, 0x08, - 0xd9, 0x81, 0x63, 0x1d, 0x0e, 0xd8, 0x0a, 0x72, 0x1a, 0x6f, 0xa1, 0x12, 0xa4, 0x7d, 0x7c, 0xc8, - 0xec, 0x9c, 0x46, 0x3e, 0x49, 0x4f, 0xa6, 0xe0, 0x9c, 0x65, 0xde, 0xa2, 0x97, 0x80, 0xe5, 0xe1, - 0x4e, 0x10, 0x31, 0x18, 0x01, 0x50, 0x19, 0xa6, 0xc9, 0x8d, 0x67, 0x39, 0x5d, 0xce, 0x96, 0x68, - 0x12, 0x29, 0x59, 0xbd, 0xbe, 0x6d, 0x75, 0xac, 0x80, 0xaa, 0x48, 0x4e, 0x0b, 0xdb, 0xea, 0x16, - 0xcc, 0x87, 0x5a, 0xfa, 0x02, 0xe6, 0xe4, 0xef, 0xd3, 0x80, 0x64, 0x3a, 0xdc, 0x98, 0x8c, 0x1c, - 0x24, 0xe5, 0xcc, 0x41, 0xba, 0x0e, 0x45, 0x0f, 0x13, 0x56, 0x0c, 0xde, 0x25, 0x45, 0xbb, 0x14, - 0x38, 0x90, 0x75, 0xfa, 0x0a, 0x80, 0xe3, 0x9a, 0x82, 0x08, 0x13, 0x54, 0x9e, 0x40, 0x18, 0xfa, - 0x5d, 0xc8, 0x90, 0xf3, 0xe7, 0x73, 0xab, 0xff, 0x9a, 0xac, 0xfe, 0xcc, 0x05, 0xa8, 0x31, 0x4f, - 0xa2, 0x26, 0x1c, 0x8a, 0xda, 0xbd, 0x47, 0x1b, 0x1b, 0x94, 0x53, 0x7e, 0xb2, 0x18, 0x01, 0x64, - 0x42, 0xb1, 0x67, 0xf9, 0xbe, 0xe5, 0x74, 0x75, 0x42, 0xde, 0x2f, 0x67, 0xa8, 0x51, 0xf9, 0x8d, - 0xe7, 0x19, 0x95, 0xa1, 0x45, 0xd7, 0xee, 0x31, 0x12, 0xf7, 0x5d, 0x13, 0x73, 0xf2, 0x85, 0x5e, - 0x04, 0xf2, 0x89, 0xf1, 0x30, 0xfa, 0x7d, 0xcf, 0x7d, 0x6a, 0xf5, 0x88, 0x05, 0x31, 0x2d, 0xff, - 0x40, 0x6f, 0x1f, 0x07, 0xd8, 0xa7, 0xfb, 0x36, 0xa5, 0x2d, 0x48, 0xd8, 0x4d, 0xcb, 0x3f, 0x68, - 0x10, 0x5c, 0xe5, 0x43, 0x98, 0x91, 0x08, 0xa3, 0xeb, 0x30, 0x4d, 0x65, 0x62, 0x99, 0x6c, 0x87, - 0x1a, 0x70, 0x7a, 0x52, 0xcd, 0x12, 0x54, 0x73, 0x53, 0xcb, 0x12, 0x54, 0xd3, 0x24, 0xd2, 0xc5, - 0x9e, 0xe7, 0x7a, 0x7a, 0x0f, 0xfb, 0xbe, 0xd1, 0x15, 0x7b, 0x56, 0xa0, 0xc0, 0x7b, 0x0c, 0xa6, - 0x2e, 0xc2, 0xc2, 0x7d, 0xd7, 0x39, 0xa3, 0x04, 0xea, 0xcf, 0x14, 0xb8, 0x34, 0x82, 0xe0, 0xbb, - 0xfa, 0x6d, 0x98, 0x27, 0x8e, 0x8c, 0xee, 0x63, 0xcf, 0xc2, 0xbe, 0xce, 0x84, 0xaf, 0x50, 0xe1, - 0x7f, 0x63, 0x22, 0x51, 0x69, 0x73, 0x84, 0x4e, 0x8b, 0x92, 0xa1, 0x08, 0xf4, 0x1d, 0x40, 0x96, - 0x13, 0x60, 0xcf, 0x31, 0x6c, 0x7d, 0xe0, 0x63, 0x4e, 0x3b, 0x75, 0x1e, 0xda, 0x25, 0x41, 0xe8, - 0xa1, 0xcf, 0x30, 0xea, 0x2c, 0x14, 0x1e, 0xfa, 0xd8, 0x0b, 0x57, 0xf8, 0x7d, 0x28, 0xf2, 0x36, - 0x5f, 0x58, 0x13, 0x32, 0xc4, 0x52, 0x89, 0xab, 0x2f, 0x69, 0xc2, 0xa1, 0x81, 0xb4, 0x25, 0x54, - 0x89, 0x52, 0xa8, 0xa8, 0x30, 0x45, 0x80, 0xe4, 0x28, 0x11, 0x80, 0x64, 0x0d, 0xc2, 0xb6, 0xfa, - 0x08, 0x8a, 0x5b, 0x47, 0xd8, 0x89, 0xce, 0x9d, 0x30, 0x71, 0x8a, 0x64, 0xe2, 0xae, 0x42, 0x3e, - 0x30, 0xbc, 0x2e, 0x0e, 0xc8, 0x56, 0xb3, 0xd3, 0x91, 0x63, 0x80, 0xa6, 0x49, 0x0e, 0xa3, 0x6d, - 0xf5, 0x2c, 0x76, 0x28, 0x32, 0x1a, 0x6b, 0xa8, 0xff, 0x9d, 0x82, 0x59, 0x41, 0x98, 0xaf, 0xec, - 0x1e, 0x64, 0x31, 0x85, 0xf0, 0xa5, 0xd5, 0x13, 0x96, 0x36, 0x3c, 0x94, 0x35, 0xc5, 0x7d, 0xce, - 0x88, 0x54, 0xfe, 0x24, 0x05, 0x19, 0x0a, 0x47, 0x0d, 0xc8, 0x87, 0x4e, 0x2d, 0xd7, 0x81, 0x4a, - 0x8d, 0xb9, 0xbd, 0x35, 0xe1, 0xf6, 0xd6, 0x76, 0x45, 0x8f, 0x46, 0x8e, 0x90, 0xf9, 0xf4, 0x97, - 0x55, 0x45, 0x8b, 0x86, 0x91, 0xf3, 0x4d, 0xe9, 0xea, 0x92, 0x7d, 0xcf, 0x53, 0xc8, 0x2e, 0x91, - 0xc0, 0xd7, 0x65, 0x09, 0x30, 0x77, 0xb0, 0x70, 0x7a, 0x52, 0xcd, 0xed, 0x32, 0x29, 0x6c, 0x4a, - 0xf2, 0x58, 0x03, 0x62, 0x39, 0x5c, 0x2f, 0x20, 0x47, 0xd8, 0x32, 0xd9, 0xcd, 0xdd, 0x98, 0x3b, - 0x3d, 0xa9, 0xce, 0x68, 0x02, 0xde, 0xdc, 0xd4, 0x66, 0xc2, 0x4e, 0x4d, 0x93, 0x08, 0xdd, 0x72, - 0xf6, 0x5c, 0x6e, 0x50, 0xe9, 0x37, 0x99, 0x92, 0x59, 0x67, 0x42, 0x84, 0x9c, 0xca, 0x02, 0x9b, - 0xf2, 0x21, 0x05, 0x92, 0x29, 0x19, 0xba, 0x69, 0xaa, 0x7f, 0xab, 0x40, 0xa9, 0x85, 0x83, 0x87, - 0x4d, 0xe2, 0x06, 0x8b, 0x8d, 0xfc, 0x36, 0xc0, 0x01, 0x3e, 0x66, 0xf7, 0x8e, 0x10, 0xf9, 0xed, - 0x04, 0x91, 0x8f, 0x12, 0xa8, 0xbd, 0x87, 0x8f, 0xe9, 0x05, 0xe5, 0x6f, 0x39, 0x81, 0x77, 0xac, - 0xe5, 0x0f, 0x44, 0xbb, 0x72, 0x07, 0x66, 0x87, 0x91, 0xe4, 0x02, 0x39, 0xc0, 0xc7, 0x5c, 0x69, - 0xc8, 0x27, 0x51, 0x0b, 0x76, 0xe5, 0x11, 0x59, 0x16, 0x34, 0xd6, 0xb8, 0x9d, 0xfa, 0xa6, 0xa2, - 0x5e, 0x84, 0x79, 0x69, 0x2e, 0xb6, 0xc3, 0xea, 0xd7, 0xa0, 0xb4, 0x3d, 0xba, 0x02, 0x04, 0x53, - 0x07, 0xf8, 0x58, 0xc4, 0x1a, 0xf4, 0x5b, 0xfd, 0x59, 0x0a, 0xe6, 0xb7, 0x47, 0x47, 0xa3, 0xdf, - 0x8d, 0x59, 0xeb, 0xdb, 0x09, 0x6b, 0x3d, 0x43, 0x61, 0x64, 0xb1, 0x5c, 0xd5, 0xa4, 0x25, 0xef, - 0x41, 0x86, 0xdd, 0xd6, 0xe1, 0xba, 0x14, 0x69, 0x5d, 0x68, 0x1b, 0x0a, 0xb6, 0xe1, 0x07, 0xfa, - 0xa0, 0x6f, 0x1a, 0x01, 0x36, 0xb9, 0xb5, 0x18, 0x4f, 0x0b, 0x67, 0xc8, 0xc8, 0x87, 0x6c, 0x60, - 0xa5, 0x3f, 0x86, 0x68, 0xdf, 0x95, 0x45, 0x3b, 0xb3, 0xb6, 0x36, 0xd1, 0x42, 0x29, 0x69, 0x79, - 0x3b, 0x4a, 0x30, 0xbb, 0x61, 0x0f, 0xfc, 0x00, 0x7b, 0xc2, 0x26, 0x7d, 0xa6, 0xc0, 0x5c, 0x08, - 0xe2, 0x12, 0x7e, 0x0d, 0xa0, 0xc3, 0x40, 0x91, 0xb9, 0x2f, 0x9e, 0x9e, 0x54, 0xf3, 0xbc, 0x63, - 0x73, 0x53, 0xcb, 0xf3, 0x0e, 0x4d, 0x13, 0xbd, 0x0a, 0xf3, 0xd1, 0x19, 0xc0, 0x0e, 0x31, 0x8c, - 0x26, 0x77, 0x39, 0x4a, 0x21, 0x62, 0x8b, 0xc1, 0xd1, 0x37, 0x00, 0x61, 0x62, 0x26, 0xfb, 0x9e, - 0xe5, 0xe3, 0xb0, 0x37, 0x73, 0xa5, 0xe6, 0x23, 0x0c, 0xef, 0xae, 0xee, 0x40, 0x61, 0xd3, 0x33, - 0x2c, 0x47, 0x68, 0xc9, 0x2c, 0xa4, 0x5c, 0x87, 0xee, 0x79, 0x46, 0x4b, 0xb9, 0x0e, 0x91, 0x97, - 0xbb, 0xb7, 0x47, 0x7d, 0xbb, 0x8c, 0x46, 0x3e, 0x89, 0xfd, 0xf3, 0xf7, 0x07, 0x81, 0xe9, 0x3e, - 0x71, 0x84, 0x87, 0x26, 0xda, 0x6a, 0x15, 0x8a, 0x9c, 0x1a, 0x5f, 0xe8, 0x08, 0x39, 0xf5, 0x63, - 0xb8, 0xb2, 0x89, 0x3b, 0x6e, 0x8f, 0x5e, 0x9f, 0xae, 0xd3, 0xa2, 0xc9, 0x02, 0x31, 0xf7, 0x47, - 0x90, 0xe3, 0x37, 0x20, 0xd3, 0xba, 0x4c, 0xa3, 0x71, 0x7a, 0x52, 0x9d, 0x66, 0x57, 0xa0, 0xff, - 0xbf, 0x27, 0xd5, 0x9b, 0x5d, 0x2b, 0xd8, 0x1f, 0xb4, 0x6b, 0x1d, 0xb7, 0x57, 0x0f, 0x77, 0xc9, - 0x6c, 0x47, 0xdf, 0xf5, 0xfe, 0x41, 0xb7, 0x4e, 0xbf, 0xfa, 0xed, 0x1a, 0xbf, 0x3a, 0xa7, 0xd9, - 0xd5, 0xe9, 0xab, 0x7f, 0xa1, 0xc0, 0x45, 0x79, 0xf2, 0x5f, 0xcf, 0xb4, 0x68, 0x05, 0xe6, 0x4c, - 0x69, 0x56, 0xe2, 0xcd, 0xb1, 0xbd, 0x1b, 0x05, 0xab, 0x5f, 0xa4, 0xa0, 0x12, 0x27, 0x1d, 0x2e, - 0xcb, 0xc7, 0x90, 0x65, 0xc9, 0x15, 0x1e, 0x71, 0xdd, 0x49, 0x8a, 0xe0, 0x9f, 0x49, 0xa6, 0xc6, - 0x9a, 0xc2, 0xfc, 0x33, 0x8a, 0x95, 0xff, 0x52, 0x20, 0xcb, 0x10, 0xe8, 0xf1, 0xb0, 0x1f, 0x92, - 0x69, 0xac, 0x47, 0x7e, 0xc8, 0x79, 0x85, 0x21, 0xdc, 0x97, 0xcb, 0x30, 0x6d, 0xf9, 0xba, 0x6d, - 0x1d, 0x85, 0x2e, 0xb3, 0xe5, 0xef, 0x58, 0x47, 0xf8, 0xac, 0xd7, 0x98, 0x8e, 0xf1, 0x1a, 0x63, - 0x24, 0x39, 0x15, 0x2b, 0x49, 0xea, 0xee, 0x12, 0x3d, 0x24, 0x5d, 0x32, 0x4c, 0x47, 0x45, 0x5b, - 0x7d, 0x19, 0xe6, 0x5a, 0x38, 0x20, 0x47, 0xc6, 0x4f, 0x32, 0x8d, 0xff, 0x90, 0xa2, 0xb7, 0x00, - 0xef, 0xc7, 0xb7, 0x40, 0x9f, 0xfc, 0x16, 0x18, 0x22, 0xf0, 0x5c, 0xc3, 0xd8, 0x8a, 0x35, 0x8c, - 0x79, 0x61, 0x18, 0xe3, 0x22, 0xa6, 0x65, 0x98, 0x11, 0x11, 0x1b, 0x89, 0x22, 0xd2, 0x14, 0x25, - 0x83, 0x2a, 0xee, 0x18, 0x56, 0x70, 0x7b, 0xd8, 0x0a, 0xde, 0x98, 0x64, 0x51, 0x67, 0x8c, 0xe0, - 0x1c, 0x14, 0xdf, 0xc5, 0x86, 0x1d, 0xec, 0x0b, 0x1b, 0x58, 0x82, 0x59, 0x01, 0xe0, 0x37, 0xd4, - 0x3c, 0xcc, 0xed, 0xf0, 0x2c, 0x9f, 0xe8, 0xf4, 0xd3, 0x14, 0x94, 0x22, 0x18, 0x97, 0xf8, 0x3a, - 0x80, 0xc8, 0x06, 0x86, 0x12, 0xbf, 0x1a, 0x13, 0x0f, 0x88, 0x81, 0x22, 0xb0, 0x8e, 0x06, 0xa1, - 0x3f, 0x53, 0x20, 0xc7, 0xd4, 0x1c, 0x8b, 0xa3, 0x93, 0xe4, 0xff, 0x8f, 0xb2, 0xc0, 0x0f, 0x8c, - 0xd8, 0xb2, 0xb7, 0x09, 0xfd, 0x1f, 0xfe, 0xf2, 0x7c, 0xe7, 0x20, 0xe4, 0xa3, 0xd2, 0x86, 0xe2, - 0x10, 0x5d, 0x79, 0x4b, 0x32, 0x6c, 0x4b, 0xde, 0x96, 0xb7, 0x64, 0x76, 0xed, 0xe5, 0x98, 0x55, - 0x13, 0xb2, 0x82, 0x5f, 0x7e, 0xcc, 0xa5, 0x6d, 0xe8, 0xc3, 0xcc, 0x5d, 0xb7, 0x1d, 0x6a, 0x79, - 0xe8, 0x5a, 0x2a, 0x92, 0x6b, 0x49, 0x42, 0xd3, 0xd0, 0xaa, 0xd0, 0xd0, 0x94, 0xb5, 0xd0, 0x4d, - 0xae, 0x6a, 0x69, 0x3a, 0x79, 0x55, 0x9e, 0xfc, 0xd0, 0xae, 0xd1, 0x0c, 0x30, 0xcb, 0x06, 0xd7, - 0x88, 0x4b, 0xc7, 0x74, 0x51, 0xfd, 0x69, 0x16, 0x0a, 0x6c, 0x4a, 0xbe, 0x7d, 0x5b, 0x30, 0x45, - 0x7a, 0xf1, 0x8d, 0x7b, 0x35, 0x41, 0xec, 0xf2, 0x30, 0xd2, 0xe0, 0x1b, 0x49, 0x87, 0x57, 0xfe, - 0x27, 0x03, 0xe9, 0xbb, 0x6e, 0x1b, 0x2d, 0x42, 0x8a, 0x9b, 0xa5, 0x74, 0x23, 0x7b, 0x7a, 0x52, - 0x4d, 0x35, 0x37, 0xb5, 0x94, 0x65, 0x9e, 0xef, 0x5c, 0x0c, 0x79, 0xf2, 0x53, 0xc3, 0x9e, 0x3c, - 0x72, 0x61, 0x76, 0x28, 0x5d, 0xc2, 0x22, 0xc7, 0x62, 0xe3, 0xdd, 0xd3, 0x93, 0x6a, 0x51, 0xce, - 0x97, 0x8c, 0x7f, 0x41, 0xf8, 0x87, 0x36, 0xf9, 0x23, 0x31, 0x77, 0xad, 0xb9, 0xa9, 0x15, 0xe5, - 0x3c, 0x8b, 0x2f, 0xed, 0x43, 0x76, 0x68, 0x1f, 0x6e, 0xc3, 0x34, 0x4b, 0x3d, 0x99, 0x34, 0xd2, - 0x4f, 0x76, 0x83, 0xa6, 0xa8, 0x0b, 0x24, 0x06, 0x90, 0xb1, 0x7e, 0x60, 0x78, 0x64, 0x6c, 0x6e, - 0xdc, 0xb1, 0x7c, 0x00, 0xba, 0x03, 0xb9, 0x3d, 0xcb, 0xb1, 0xfc, 0x7d, 0x6c, 0x96, 0xf3, 0x63, - 0x0e, 0x0e, 0x47, 0x90, 0xd1, 0x3d, 0xd7, 0xb4, 0xf6, 0x2c, 0x6c, 0x96, 0x61, 0xdc, 0xd1, 0x62, - 0x04, 0xf1, 0x61, 0xf6, 0x3c, 0x83, 0x26, 0x41, 0xf4, 0x8e, 0xdb, 0xeb, 0xdb, 0x98, 0x2c, 0x61, - 0x66, 0x59, 0x59, 0x49, 0x69, 0xf3, 0x02, 0xb3, 0x21, 0x10, 0x44, 0xb1, 0x69, 0xfc, 0x5b, 0x2e, - 0x30, 0x5b, 0x49, 0x1b, 0xe8, 0x01, 0x5c, 0xdc, 0xb7, 0xba, 0xfb, 0x4f, 0x0c, 0xe2, 0x65, 0x45, - 0x11, 0x4d, 0x71, 0x4c, 0x6e, 0x50, 0x38, 0x38, 0xc4, 0x10, 0x47, 0x2c, 0x22, 0x69, 0xe2, 0x8e, - 0xd5, 0x33, 0xec, 0xf2, 0x2c, 0x9d, 0xb4, 0x14, 0x22, 0x36, 0x19, 0x1c, 0xbd, 0x0c, 0xb3, 0xde, - 0xc0, 0x21, 0x57, 0x8e, 0xce, 0x37, 0x76, 0x8e, 0xf6, 0x2c, 0x72, 0x28, 0xbf, 0x6e, 0xaf, 0x41, - 0x3e, 0xca, 0x35, 0x96, 0x58, 0xa4, 0x14, 0x02, 0x54, 0x04, 0xa5, 0x1d, 0xb7, 0xc3, 0x4a, 0x10, - 0xc2, 0x4e, 0xfe, 0x9f, 0x02, 0xf3, 0x12, 0x30, 0x0c, 0xe1, 0xf3, 0xb6, 0x00, 0x8e, 0x91, 0xe8, - 0x3d, 0x43, 0x20, 0x84, 0x88, 0x4b, 0x29, 0xa4, 0x56, 0xf9, 0x53, 0x05, 0x72, 0x02, 0x8b, 0x5e, - 0x82, 0x02, 0xc1, 0xd8, 0x56, 0x70, 0xac, 0x47, 0x77, 0xc8, 0x8c, 0x80, 0xbd, 0x87, 0x8f, 0xc9, - 0xca, 0xc3, 0x2e, 0x91, 0x05, 0xcb, 0x6b, 0x45, 0x01, 0x65, 0x57, 0x5c, 0x05, 0x72, 0xb6, 0x11, - 0x58, 0xc1, 0xc0, 0x64, 0x56, 0x46, 0xd1, 0xc2, 0x36, 0x91, 0x8a, 0xed, 0x3a, 0x5d, 0x86, 0x9c, - 0xa2, 0xc8, 0x08, 0xa0, 0x36, 0x60, 0x4e, 0x33, 0x9c, 0x2e, 0xde, 0x71, 0xbb, 0xc2, 0xb8, 0x5d, - 0x81, 0x1c, 0xcb, 0x4b, 0x09, 0xfb, 0xa0, 0x4d, 0xd3, 0xb6, 0x1c, 0x52, 0xa7, 0xe4, 0x90, 0xfa, - 0xdf, 0xd2, 0x50, 0x8a, 0x88, 0x70, 0x21, 0xbe, 0x1f, 0x06, 0xd5, 0xec, 0x9e, 0x48, 0xba, 0x06, - 0x47, 0x07, 0xc7, 0x86, 0xd5, 0xff, 0xac, 0x00, 0x7c, 0xe0, 0xe1, 0x20, 0x38, 0x6e, 0x92, 0x28, - 0xf4, 0x25, 0x28, 0xf0, 0x98, 0x46, 0x27, 0xa7, 0x5f, 0x08, 0x8f, 0xc3, 0x88, 0x59, 0x21, 0x0b, - 0x71, 0xf0, 0x13, 0x86, 0x66, 0x62, 0x9b, 0x76, 0xf0, 0x13, 0x8a, 0xba, 0x0e, 0x45, 0xc3, 0x34, - 0xb1, 0xa9, 0x73, 0xaf, 0x88, 0xdb, 0xb4, 0x02, 0x05, 0x6a, 0x0c, 0x86, 0x5e, 0x81, 0x39, 0x0f, - 0xf7, 0xdc, 0x23, 0xa9, 0x1b, 0xb3, 0x6d, 0xb3, 0x1c, 0x2c, 0x3a, 0x2e, 0x42, 0xd6, 0xc3, 0x86, - 0x1f, 0x26, 0x1e, 0x79, 0x0b, 0x95, 0x61, 0xda, 0x64, 0x29, 0x75, 0x6e, 0x89, 0x44, 0xb3, 0xf2, - 0x57, 0x8a, 0xc8, 0x11, 0xdc, 0x81, 0x0c, 0x5d, 0x20, 0xcf, 0x0f, 0x2c, 0xc7, 0x5c, 0x4d, 0x42, - 0x3c, 0xb2, 0x54, 0xd8, 0x20, 0xf4, 0x11, 0xcc, 0xf4, 0xa9, 0x4c, 0x74, 0x1a, 0xa6, 0x33, 0x8f, - 0xe3, 0xcd, 0x49, 0x44, 0x1d, 0x89, 0x54, 0xdc, 0xf7, 0xfd, 0x10, 0x72, 0x77, 0x2a, 0xa7, 0x94, - 0x52, 0xea, 0x0a, 0x94, 0x1e, 0x0c, 0xb0, 0x77, 0xfc, 0x81, 0x6d, 0x38, 0xd2, 0x0d, 0x78, 0x48, - 0x60, 0xc2, 0xa9, 0xa2, 0x0d, 0xb5, 0x0f, 0xf3, 0x52, 0x4f, 0xae, 0x09, 0xdf, 0x81, 0xab, 0xa6, - 0xe5, 0x07, 0xfe, 0xa1, 0xad, 0xf7, 0xf7, 0x8f, 0x7d, 0xab, 0x63, 0xd8, 0x3a, 0xed, 0xae, 0xf7, - 0x6d, 0xc3, 0xe1, 0x21, 0xdb, 0xb5, 0xd3, 0x93, 0x6a, 0x79, 0xd3, 0xf2, 0x83, 0xd6, 0x83, 0x9d, - 0x0f, 0x78, 0xaf, 0x88, 0x54, 0x99, 0x13, 0x38, 0x83, 0x51, 0xaf, 0xb0, 0x32, 0x1d, 0x19, 0xe9, - 0x59, 0xed, 0x41, 0x10, 0x05, 0x23, 0xea, 0xdf, 0x01, 0x94, 0xcf, 0xe2, 0x38, 0x53, 0x7d, 0x28, - 0x8a, 0xac, 0x2d, 0x13, 0x1d, 0x3b, 0xe7, 0x5b, 0xcf, 0x29, 0xe5, 0xc5, 0xd1, 0x0a, 0x6b, 0x7c, - 0x44, 0x64, 0xb2, 0x33, 0x5a, 0x30, 0x25, 0x04, 0xea, 0x41, 0x41, 0x2a, 0x4a, 0x88, 0x9a, 0xcc, - 0xe6, 0x79, 0x26, 0x8c, 0x0a, 0x15, 0x43, 0xce, 0xef, 0x4c, 0x54, 0xa8, 0xf0, 0x2b, 0x9f, 0x29, - 0x00, 0x51, 0x3f, 0xa2, 0xa2, 0x2c, 0x11, 0xc4, 0x37, 0x8c, 0xb7, 0xd0, 0x3a, 0x64, 0x79, 0xc5, - 0x25, 0x35, 0x69, 0xc5, 0x85, 0x0f, 0xa4, 0x11, 0x38, 0x2b, 0xb4, 0xf8, 0x87, 0x36, 0x3b, 0x21, - 0x3c, 0x02, 0xa7, 0xd0, 0xd6, 0x83, 0x1d, 0x2d, 0xcf, 0x3a, 0xb4, 0x0e, 0xed, 0xbb, 0x53, 0xb9, - 0x74, 0x69, 0xaa, 0xf2, 0xc7, 0x69, 0xc8, 0xd3, 0xb4, 0x24, 0x15, 0xcd, 0xe7, 0x0a, 0x94, 0x87, - 0x62, 0x16, 0xbd, 0x7d, 0xac, 0x47, 0x91, 0x13, 0x91, 0xd3, 0x87, 0xe7, 0x91, 0x53, 0x38, 0x43, - 0x4d, 0x93, 0x82, 0x9f, 0xc6, 0x31, 0x75, 0x21, 0x4d, 0x26, 0xba, 0xb7, 0xce, 0xeb, 0x80, 0x2e, - 0x78, 0x31, 0x34, 0xd1, 0x57, 0x61, 0x56, 0x2e, 0x3d, 0x85, 0x69, 0xc9, 0x42, 0xb4, 0x37, 0x4d, - 0x13, 0x7d, 0x0b, 0xc0, 0xf4, 0xdc, 0x7e, 0x1f, 0x9b, 0xba, 0xc1, 0x02, 0xb4, 0x71, 0xee, 0xd1, - 0x3c, 0x1f, 0xb3, 0x1e, 0x54, 0xb6, 0xe1, 0xca, 0x33, 0x97, 0x14, 0xe3, 0xff, 0x0e, 0xe5, 0xbc, - 0xd2, 0x92, 0x63, 0x5b, 0xf9, 0x41, 0x0a, 0x0a, 0xb2, 0xfe, 0xa2, 0x00, 0x58, 0xc5, 0x59, 0x3e, - 0x15, 0xef, 0xbf, 0xe8, 0xa9, 0x88, 0x76, 0x62, 0x28, 0x58, 0x0b, 0x04, 0xb4, 0xf2, 0x09, 0xcc, - 0x0e, 0x77, 0x89, 0x89, 0xab, 0x5a, 0xc3, 0x71, 0xd5, 0x3b, 0x2f, 0xa4, 0x11, 0x43, 0x32, 0x50, - 0xa2, 0x57, 0x05, 0x49, 0x0c, 0x3c, 0x1a, 0x66, 0xe0, 0xb7, 0x5e, 0x54, 0x2a, 0x32, 0x0f, 0xdf, - 0x87, 0xd2, 0xe8, 0xa9, 0x8e, 0xe1, 0x60, 0x77, 0x98, 0x83, 0xdf, 0x7c, 0x31, 0xe3, 0x21, 0xcd, - 0x7f, 0x77, 0x2a, 0x97, 0x2a, 0xa5, 0xd5, 0xcb, 0x70, 0xe9, 0x1e, 0x7d, 0xd7, 0x71, 0x0f, 0x07, - 0x86, 0x19, 0x65, 0x3c, 0xd5, 0x7f, 0x55, 0x60, 0x71, 0x14, 0xc3, 0x2d, 0xa9, 0x01, 0xb9, 0x1e, - 0x87, 0x71, 0x75, 0xf9, 0x56, 0x02, 0x5b, 0xf1, 0x44, 0x6a, 0x02, 0x20, 0xab, 0x47, 0x48, 0xb6, - 0xf2, 0xdb, 0x50, 0x1c, 0xea, 0x10, 0x23, 0x99, 0x37, 0x86, 0x25, 0x23, 0x07, 0x59, 0x83, 0xc0, - 0xb2, 0x6b, 0xfc, 0xb9, 0x4a, 0x38, 0xb1, 0x14, 0xdb, 0xfd, 0x28, 0x05, 0x17, 0xb7, 0x9c, 0xc3, - 0x01, 0x1e, 0x60, 0x7a, 0x3b, 0x8a, 0x2b, 0xee, 0xcb, 0xcc, 0xde, 0xb0, 0xeb, 0x33, 0x74, 0xe7, - 0x58, 0x03, 0xfd, 0x8e, 0xe4, 0x79, 0xd1, 0xa2, 0x45, 0x63, 0xe3, 0xf4, 0xa4, 0x3a, 0x4d, 0xb9, - 0xa2, 0x73, 0xde, 0x9a, 0x68, 0x4e, 0x3e, 0x2e, 0x72, 0xdf, 0x56, 0x61, 0xde, 0x3f, 0xb0, 0xfa, - 0xba, 0xbf, 0xef, 0x0e, 0x6c, 0x53, 0x67, 0x1c, 0xf0, 0xbc, 0x0f, 0x41, 0xb4, 0x28, 0xfc, 0x01, - 0x01, 0xab, 0x7f, 0x9d, 0x82, 0x85, 0x61, 0xa9, 0xf0, 0xfd, 0x7e, 0x10, 0x39, 0x35, 0x6c, 0xbb, - 0xdf, 0x4a, 0x2a, 0x97, 0xc4, 0x50, 0xa8, 0x89, 0x67, 0x06, 0xa1, 0x37, 0xf4, 0x8f, 0x0a, 0x4c, - 0x73, 0xe0, 0x97, 0x2a, 0xf5, 0x77, 0x46, 0x7c, 0xd2, 0x97, 0x93, 0x8a, 0x66, 0x9e, 0xd1, 0xc1, - 0xd4, 0xe3, 0x12, 0x1e, 0x68, 0x14, 0x1c, 0xa5, 0xa5, 0xe0, 0x48, 0xbd, 0x04, 0x17, 0x37, 0xf6, - 0x0d, 0x2f, 0xd8, 0x60, 0x6f, 0xa5, 0xc4, 0x89, 0x79, 0x0c, 0x0b, 0xc3, 0x60, 0x2e, 0xbe, 0x06, - 0x4c, 0xf3, 0x57, 0x55, 0x5c, 0x7c, 0xaa, 0xc4, 0x44, 0xe0, 0xd7, 0xc4, 0x93, 0x2b, 0x3a, 0xb8, - 0xc5, 0xca, 0xd7, 0xe1, 0xcb, 0x0e, 0x86, 0x5b, 0x7d, 0x0f, 0x16, 0xe3, 0xdf, 0x2a, 0xa0, 0x19, - 0x98, 0x7e, 0x78, 0xff, 0xbd, 0xfb, 0xef, 0x7f, 0x78, 0xbf, 0x74, 0x81, 0x34, 0x36, 0x76, 0x1e, - 0xb6, 0x76, 0xb7, 0xb4, 0x92, 0x82, 0x0a, 0x90, 0xdb, 0x5c, 0xdf, 0x5d, 0x6f, 0xac, 0xb7, 0xb6, - 0x4a, 0x29, 0x94, 0x87, 0xcc, 0xee, 0x7a, 0x63, 0x67, 0xab, 0x94, 0x5e, 0xbd, 0x0e, 0x79, 0x9a, - 0x68, 0xbe, 0xe7, 0x9a, 0x18, 0x01, 0x64, 0x37, 0x76, 0x9a, 0x5b, 0xf7, 0x77, 0x4b, 0x17, 0xc8, - 0xf7, 0xce, 0xd6, 0x7a, 0x6b, 0xab, 0x55, 0x52, 0xd6, 0x7e, 0x5c, 0x81, 0xcc, 0xba, 0xd9, 0xb3, - 0x1c, 0x14, 0x40, 0x86, 0x96, 0xf7, 0xd0, 0x2b, 0xcf, 0x2f, 0x00, 0x52, 0x49, 0x54, 0x56, 0xc6, - 0xad, 0x14, 0xaa, 0xe5, 0x1f, 0xfe, 0xcb, 0x7f, 0xfe, 0x28, 0x85, 0x50, 0xa9, 0xae, 0xd3, 0x57, - 0x76, 0xf5, 0xa3, 0x1b, 0x75, 0x5a, 0x31, 0x44, 0x7f, 0xa8, 0x40, 0x3e, 0x7c, 0xf8, 0x85, 0x5e, - 0x1d, 0xe3, 0xc1, 0x55, 0x38, 0xfd, 0x6b, 0xe3, 0x75, 0xe6, 0x2c, 0x5c, 0xa3, 0x2c, 0x2c, 0xa2, - 0x05, 0x89, 0x85, 0xf0, 0x2d, 0x19, 0xfa, 0xb1, 0x02, 0x73, 0x23, 0x2f, 0xba, 0xd0, 0x8d, 0x49, - 0x5e, 0x7f, 0x31, 0x96, 0xd6, 0x26, 0x7f, 0x30, 0xa6, 0xbe, 0x42, 0x19, 0x7b, 0x09, 0x55, 0xe3, - 0x18, 0xab, 0x7f, 0x22, 0x3e, 0xbf, 0x87, 0xfe, 0x46, 0x81, 0x82, 0xfc, 0x9a, 0x07, 0xd5, 0xc6, - 0x7e, 0xf6, 0xc3, 0xb8, 0xab, 0x4f, 0xf8, 0x4c, 0x48, 0x7d, 0x93, 0xb2, 0xf6, 0x3a, 0xaa, 0x3d, - 0x87, 0xb5, 0x3a, 0xbd, 0xef, 0xfd, 0xfa, 0x27, 0xf4, 0x97, 0x72, 0x0a, 0x51, 0x6d, 0x1a, 0xbd, - 0x36, 0x66, 0x09, 0x9b, 0x71, 0x39, 0x59, 0xc1, 0x5b, 0xbd, 0x43, 0x79, 0x7c, 0x13, 0xdd, 0x9a, - 0x8c, 0xc7, 0x3a, 0x7b, 0xfb, 0xf0, 0xe7, 0x0a, 0x14, 0x87, 0xca, 0xfd, 0x28, 0x49, 0x48, 0x71, - 0x2f, 0x06, 0x2a, 0xaf, 0x8f, 0x3f, 0x80, 0xb3, 0xbc, 0x4c, 0x59, 0xae, 0xa0, 0xb2, 0xc4, 0xb2, - 0xe3, 0x3a, 0x8c, 0x41, 0xca, 0xc4, 0x53, 0xc8, 0xb2, 0x7a, 0x34, 0x5a, 0x19, 0xa3, 0x64, 0xcd, - 0xf8, 0xf8, 0xfa, 0xd8, 0xc5, 0x6d, 0xf5, 0x0a, 0x65, 0xe0, 0x22, 0x9a, 0x97, 0x18, 0xe0, 0xa6, - 0x90, 0x9c, 0xc7, 0xb0, 0x56, 0x9a, 0x78, 0x1e, 0x47, 0xab, 0xb7, 0x89, 0xe7, 0xf1, 0x6c, 0xf9, - 0x95, 0x9f, 0x47, 0x55, 0xe6, 0x61, 0x60, 0x91, 0xed, 0xba, 0xad, 0xac, 0xa2, 0x1f, 0x28, 0x90, - 0xdf, 0x1e, 0x8b, 0x8d, 0xed, 0x49, 0xd8, 0x38, 0x53, 0x9c, 0x8c, 0x15, 0x05, 0x63, 0x03, 0xfd, - 0x1e, 0x4c, 0xf3, 0x52, 0x23, 0x4a, 0x92, 0xed, 0x70, 0x29, 0xb3, 0xb2, 0x3a, 0x4e, 0x57, 0x3e, - 0x79, 0x85, 0x4e, 0xbe, 0x80, 0x90, 0x34, 0x39, 0x2f, 0x69, 0xa2, 0xdf, 0x57, 0x20, 0x27, 0xaa, - 0x08, 0x68, 0x75, 0xac, 0x52, 0x03, 0x63, 0xe0, 0xd5, 0x09, 0xca, 0x12, 0xea, 0x55, 0xca, 0xc1, - 0x25, 0x74, 0x51, 0xe2, 0xc0, 0x17, 0xb3, 0x3e, 0x85, 0x2c, 0xab, 0x48, 0x24, 0x6a, 0xe1, 0x50, - 0x15, 0x23, 0x51, 0x0b, 0x47, 0xca, 0x1b, 0x71, 0xa2, 0xdf, 0x67, 0xf3, 0x91, 0xc5, 0x8b, 0x9c, - 0x7d, 0xe2, 0xe2, 0x47, 0xea, 0x23, 0x89, 0x8b, 0x1f, 0x2d, 0x5a, 0xc4, 0x2e, 0x5e, 0xd4, 0x44, - 0x50, 0x1f, 0xa6, 0xee, 0xba, 0x6d, 0x1f, 0x7d, 0xed, 0xb9, 0xf9, 0x78, 0x36, 0xf3, 0x2b, 0x63, - 0xe6, 0xed, 0xd5, 0xcb, 0x74, 0xd6, 0x79, 0x34, 0x27, 0xcd, 0xfa, 0x5d, 0x32, 0x13, 0x39, 0x7a, - 0x61, 0xca, 0x31, 0x51, 0xe7, 0x47, 0xd3, 0x9d, 0x89, 0x3a, 0x7f, 0x26, 0x8b, 0x19, 0x7b, 0x15, - 0x86, 0x99, 0x4c, 0xca, 0x46, 0x98, 0x86, 0x49, 0x64, 0x63, 0x34, 0x77, 0x94, 0xc8, 0xc6, 0x99, - 0xf4, 0x51, 0x2c, 0x1b, 0x34, 0x7d, 0xd4, 0x27, 0x13, 0x1f, 0x43, 0x86, 0x7a, 0x2f, 0x89, 0xee, - 0x88, 0x5c, 0x96, 0x4f, 0x74, 0x47, 0x86, 0x2a, 0xee, 0x62, 0xe3, 0x55, 0xd9, 0x1d, 0xa1, 0xb5, - 0xcf, 0xdb, 0xca, 0xea, 0xeb, 0x0a, 0x7a, 0x02, 0x05, 0xb9, 0x36, 0x9c, 0x78, 0xcf, 0xc6, 0x14, - 0xcb, 0x2b, 0x6f, 0x9c, 0xab, 0xe8, 0xac, 0x5e, 0x40, 0x7f, 0xa0, 0x00, 0x3a, 0xdb, 0x01, 0xdd, - 0x9a, 0x90, 0xde, 0x0b, 0x72, 0xf1, 0x97, 0x0a, 0xe4, 0x44, 0x36, 0x31, 0xf1, 0xf0, 0x8d, 0xe4, - 0x97, 0x13, 0x0f, 0xdf, 0x68, 0x7a, 0x52, 0x7d, 0x87, 0xee, 0xc1, 0x5b, 0x43, 0x87, 0x8f, 0x86, - 0x33, 0xb6, 0xdb, 0x7d, 0xbc, 0x8c, 0x96, 0x62, 0xc0, 0xf5, 0x4f, 0x44, 0x10, 0xf5, 0x3d, 0xe2, - 0xb0, 0x95, 0x46, 0x23, 0x61, 0xb4, 0x36, 0x51, 0xd8, 0xcc, 0x98, 0xbe, 0x79, 0x8e, 0x50, 0x5b, - 0xfd, 0x2a, 0x65, 0x7e, 0x09, 0x5d, 0x1b, 0x71, 0x3a, 0x74, 0x53, 0x66, 0xe7, 0x33, 0x05, 0xe6, - 0xd7, 0x6d, 0x7b, 0x38, 0x32, 0x46, 0xaf, 0x4f, 0x10, 0x44, 0x33, 0x16, 0x6f, 0x4c, 0x1c, 0x76, - 0xab, 0x2f, 0x51, 0x06, 0xaf, 0xa2, 0x2b, 0x12, 0x83, 0x2c, 0x52, 0x16, 0xb1, 0x37, 0xfa, 0x54, - 0x81, 0x82, 0x1c, 0xc8, 0x24, 0xaa, 0x79, 0x4c, 0x20, 0x94, 0xe8, 0x4e, 0xc6, 0x45, 0x48, 0x6a, - 0x95, 0x32, 0x75, 0x05, 0x5d, 0x96, 0xaf, 0x3b, 0xd2, 0x91, 0x87, 0x3f, 0xc4, 0x1b, 0x2b, 0xc8, - 0x81, 0x65, 0x22, 0x4b, 0x31, 0x91, 0x7d, 0x22, 0x4b, 0x71, 0x11, 0xab, 0x7a, 0x9d, 0xb2, 0xf4, - 0x15, 0x55, 0x76, 0xc5, 0x30, 0xeb, 0xa8, 0x53, 0x65, 0xbb, 0xad, 0xac, 0x36, 0x56, 0x3f, 0xff, - 0x8f, 0xa5, 0x0b, 0x9f, 0x9f, 0x2e, 0x29, 0x3f, 0x3f, 0x5d, 0x52, 0x7e, 0x71, 0xba, 0xa4, 0xfc, - 0xfb, 0xe9, 0x92, 0xf2, 0xe9, 0x17, 0x4b, 0x17, 0x7e, 0xfe, 0xc5, 0xd2, 0x85, 0x5f, 0x7c, 0xb1, - 0x74, 0xe1, 0x71, 0x4e, 0xcc, 0xd2, 0xce, 0xd2, 0x24, 0xdf, 0xcd, 0xff, 0x0f, 0x00, 0x00, 0xff, - 0xff, 0x70, 0x58, 0x0e, 0xba, 0x5e, 0x34, 0x00, 0x00, + 0x75, 0x1a, 0x80, 0xf8, 0x7a, 0x04, 0x48, 0xb0, 0x45, 0x51, 0x10, 0x24, 0x13, 0xdc, 0x91, 0xd7, + 0x4b, 0x6b, 0xd7, 0xc0, 0x8a, 0xd2, 0xee, 0x3a, 0x5a, 0x6d, 0x1c, 0x82, 0x64, 0x71, 0xa1, 0xa5, + 0xb4, 0xab, 0x01, 0xa5, 0x2d, 0xcb, 0xd9, 0x4c, 0x06, 0x98, 0x26, 0x38, 0xe6, 0x60, 0x06, 0x9c, + 0x19, 0x50, 0xe2, 0x6e, 0xec, 0x8a, 0x9d, 0x54, 0x2a, 0xa7, 0xd4, 0x56, 0xec, 0x9b, 0xab, 0x52, + 0xe5, 0x54, 0x2a, 0xa9, 0x24, 0xa7, 0xdc, 0x92, 0x1c, 0x72, 0xde, 0xca, 0xc1, 0x76, 0x55, 0x2e, + 0xce, 0x21, 0x74, 0xc2, 0xcd, 0x21, 0x95, 0x1f, 0x90, 0x43, 0x0e, 0x49, 0xaa, 0xbf, 0x66, 0x1a, + 0xe0, 0xec, 0x08, 0xa0, 0xec, 0x1c, 0x48, 0x4c, 0xbf, 0xd7, 0xfd, 0xfa, 0xf5, 0xeb, 0xd7, 0xaf, + 0xdf, 0x47, 0xc3, 0x55, 0x1f, 0x7b, 0x47, 0xd8, 0x6b, 0xb0, 0x9f, 0x41, 0xa7, 0x61, 0x98, 0x7d, + 0xcb, 0xa9, 0x0f, 0x3c, 0x37, 0x70, 0xd1, 0x95, 0xae, 0xdb, 0x3d, 0xf0, 0x5c, 0xa3, 0xbb, 0x5f, + 0x67, 0xf8, 0xba, 0xe8, 0x56, 0x5d, 0xe8, 0xba, 0xce, 0x9e, 0xd5, 0x6b, 0x7c, 0xec, 0x3a, 0x98, + 0xf5, 0xae, 0x2e, 0x7d, 0xdb, 0xed, 0xf8, 0x0d, 0xf2, 0x6f, 0xd0, 0xa1, 0x3f, 0x1c, 0x7e, 0x6d, + 0x7c, 0x0a, 0x3f, 0x30, 0x82, 0xa1, 0xc0, 0xaa, 0x7e, 0xe0, 0x7a, 0x46, 0x0f, 0x37, 0xb0, 0xd3, + 0xb3, 0x1c, 0xf1, 0x33, 0xe8, 0x34, 0xfa, 0x47, 0xdd, 0x2e, 0xef, 0xb3, 0x22, 0xfa, 0xf0, 0xdf, + 0x41, 0xa7, 0x61, 0x5b, 0x47, 0xd8, 0xc1, 0xbe, 0xa0, 0x72, 0x35, 0xa6, 0x87, 0xdb, 0xe3, 0xc8, + 0xe5, 0xc0, 0x6f, 0x74, 0x8d, 0xc0, 0xb0, 0xdd, 0x5e, 0xa3, 0xbb, 0x6f, 0x78, 0x81, 0xce, 0x5b, + 0x1c, 0x5f, 0x19, 0x06, 0x96, 0xdd, 0xe8, 0xe3, 0xc0, 0xb3, 0xba, 0xfc, 0x87, 0x63, 0x16, 0x7b, + 0x6e, 0xcf, 0xa5, 0x9f, 0x0d, 0xf2, 0x25, 0x16, 0xd4, 0x73, 0xdd, 0x9e, 0x8d, 0x1b, 0xc6, 0xc0, + 0x6a, 0x18, 0x8e, 0xe3, 0x06, 0x46, 0x60, 0xb9, 0x8e, 0x60, 0xa5, 0xc6, 0xb1, 0xb4, 0xd5, 0x19, + 0xee, 0x35, 0x02, 0xab, 0x8f, 0xfd, 0xc0, 0xe8, 0x0f, 0x58, 0x07, 0x15, 0x41, 0x79, 0xd3, 0x08, + 0x8c, 0x8e, 0xe1, 0x63, 0x5f, 0xc3, 0x87, 0x43, 0xec, 0x07, 0xea, 0x4d, 0x58, 0x90, 0x60, 0xfe, + 0xc0, 0x75, 0x7c, 0x8c, 0xae, 0x41, 0xc1, 0x14, 0xc0, 0x8a, 0xb2, 0x92, 0x5e, 0x2d, 0x68, 0x11, + 0x40, 0xbd, 0x0d, 0x4b, 0x62, 0xc8, 0x26, 0x0e, 0x0c, 0xcb, 0x16, 0xc4, 0x50, 0x15, 0xf2, 0xa2, + 0x5b, 0x45, 0x59, 0x51, 0x56, 0x0b, 0x5a, 0xd8, 0x56, 0xff, 0x2a, 0x0d, 0x97, 0xcf, 0x0c, 0xe3, + 0xf3, 0x3d, 0x86, 0x6c, 0xcf, 0x33, 0x9c, 0x80, 0x4d, 0x36, 0xbb, 0xf6, 0xf5, 0xfa, 0x17, 0xee, + 0x7f, 0xfd, 0x0b, 0x68, 0xd4, 0xb7, 0x09, 0x81, 0xe6, 0xcc, 0x67, 0x27, 0xb5, 0x0b, 0x1a, 0xa7, + 0x86, 0x6a, 0x30, 0x1b, 0x18, 0x1d, 0x1b, 0xeb, 0x8e, 0xd1, 0xc7, 0x7e, 0x25, 0x45, 0x57, 0x02, + 0x14, 0xf4, 0x80, 0x40, 0xd0, 0x1b, 0x50, 0x32, 0xb1, 0xdf, 0xf5, 0xac, 0x41, 0xe0, 0x7a, 0xba, + 0x65, 0x56, 0xd2, 0x2b, 0xca, 0x6a, 0xba, 0x59, 0x3e, 0x3d, 0xa9, 0x15, 0x37, 0x43, 0x44, 0x6b, + 0x53, 0x2b, 0x46, 0xdd, 0x5a, 0x26, 0xda, 0x80, 0x59, 0xa2, 0x7e, 0x3a, 0x53, 0xc5, 0xca, 0xcc, + 0x8a, 0xb2, 0x3a, 0xbb, 0x76, 0x4d, 0x62, 0x9a, 0x21, 0xea, 0x4f, 0x5c, 0x07, 0x6f, 0xd0, 0x4f, + 0xce, 0x18, 0x7c, 0x1c, 0x42, 0xd0, 0x47, 0xb0, 0x20, 0x11, 0xd1, 0x6d, 0x7c, 0x84, 0xed, 0x4a, + 0x66, 0x45, 0x59, 0x9d, 0x5b, 0xbb, 0x99, 0xb0, 0xfe, 0x88, 0xe6, 0xd0, 0xa3, 0xdb, 0xbf, 0x43, + 0x06, 0x6a, 0xf3, 0x11, 0x65, 0x0a, 0xa8, 0xbe, 0x0d, 0x19, 0x2a, 0x12, 0x84, 0x60, 0x66, 0xe8, + 0x63, 0x8f, 0x6f, 0x08, 0xfd, 0x46, 0xcb, 0x00, 0x03, 0xcf, 0x3a, 0xb2, 0x6c, 0xdc, 0x8b, 0xe4, + 0x12, 0x41, 0xd4, 0x6d, 0xb8, 0xb8, 0x4b, 0xa4, 0x34, 0xf9, 0xfe, 0xa2, 0x45, 0xc8, 0x50, 0xc1, + 0x56, 0x52, 0x14, 0xc1, 0x1a, 0xea, 0xff, 0xe4, 0x60, 0x71, 0x94, 0x12, 0xdf, 0xf2, 0xf6, 0xd8, + 0x96, 0xbf, 0x91, 0xb0, 0xe4, 0x38, 0x02, 0xb1, 0xfb, 0xfd, 0x18, 0x72, 0x5d, 0xd7, 0x1e, 0xf6, + 0x1d, 0xb6, 0xa6, 0xd9, 0xb5, 0x37, 0xa7, 0xa5, 0xba, 0x41, 0x87, 0x73, 0xb2, 0x82, 0x18, 0x7a, + 0x04, 0x39, 0xcb, 0x31, 0xf1, 0x33, 0xec, 0x57, 0xd2, 0xe7, 0xe3, 0xb6, 0x45, 0x86, 0x0b, 0xb2, + 0x9c, 0x16, 0x51, 0x4f, 0xcf, 0x70, 0x7a, 0x44, 0x05, 0x86, 0x4e, 0x40, 0xd5, 0x28, 0xad, 0x01, + 0x05, 0x6d, 0x10, 0x08, 0xba, 0x0d, 0x4b, 0x5d, 0x0f, 0x1b, 0x01, 0xd6, 0x99, 0x1a, 0x13, 0xfb, + 0x85, 0xfb, 0xd8, 0x09, 0xa8, 0x9e, 0x14, 0xb4, 0x45, 0x86, 0xa5, 0x33, 0xb6, 0x05, 0x6e, 0x5c, + 0x3b, 0xb3, 0xbf, 0x3c, 0xed, 0xcc, 0xfd, 0xb2, 0xb4, 0xf3, 0xec, 0xc1, 0xcb, 0x4f, 0x72, 0xf0, + 0x5e, 0x48, 0xa9, 0xab, 0xff, 0xa0, 0x40, 0x96, 0xed, 0x2f, 0x19, 0x4e, 0x4c, 0x82, 0x18, 0x4e, + 0xbe, 0x09, 0x2c, 0x38, 0x1e, 0x08, 0xfd, 0xa5, 0xdf, 0x44, 0xe1, 0x9d, 0xa1, 0x6d, 0x53, 0xbd, + 0x26, 0xa6, 0x21, 0xaf, 0x85, 0x6d, 0x74, 0x9d, 0x2c, 0x61, 0xcf, 0x18, 0xda, 0x81, 0x7e, 0x64, + 0xd8, 0x43, 0x4c, 0xf7, 0xaf, 0x40, 0x18, 0xa6, 0xc0, 0xc7, 0x04, 0x86, 0x6e, 0xc1, 0xa5, 0x1e, + 0x76, 0x30, 0x93, 0x85, 0x8e, 0x9f, 0x0d, 0x3c, 0xec, 0xfb, 0x96, 0xeb, 0x88, 0x0d, 0x8c, 0x90, + 0x5b, 0x21, 0x0e, 0x2d, 0x41, 0x76, 0xdf, 0x32, 0x4d, 0xec, 0xd0, 0xbd, 0xcb, 0x6b, 0xbc, 0x55, + 0xfd, 0x1b, 0x05, 0x32, 0x54, 0x91, 0x62, 0xf9, 0x5f, 0x82, 0xec, 0xd0, 0xb1, 0x0e, 0x87, 0x6c, + 0x05, 0x79, 0x8d, 0xb7, 0x50, 0x19, 0xd2, 0x3e, 0x3e, 0x64, 0x96, 0x4d, 0x23, 0x9f, 0xa4, 0x27, + 0xd3, 0x6c, 0xce, 0x32, 0x6f, 0x51, 0xb3, 0x6f, 0x79, 0xb8, 0x1b, 0x44, 0x0c, 0x46, 0x00, 0x54, + 0x81, 0x1c, 0xb9, 0xe3, 0x2c, 0xa7, 0xc7, 0xd9, 0x12, 0x4d, 0x22, 0x25, 0xab, 0x3f, 0xb0, 0xad, + 0xae, 0x15, 0x50, 0x15, 0xc9, 0x6b, 0x61, 0x5b, 0xdd, 0x82, 0x85, 0x50, 0x3d, 0x5f, 0xc0, 0x8e, + 0xfc, 0x6d, 0x1a, 0x90, 0x4c, 0x87, 0x5b, 0x91, 0xb1, 0x13, 0xa4, 0x9c, 0x39, 0x41, 0xd7, 0xa1, + 0xe4, 0x61, 0xc2, 0x8a, 0xc1, 0xbb, 0xa4, 0x68, 0x97, 0x22, 0x07, 0xb2, 0x4e, 0x5f, 0x02, 0x70, + 0x5c, 0x53, 0x10, 0x61, 0x82, 0x2a, 0x10, 0x08, 0x43, 0xbf, 0x0b, 0x19, 0x72, 0xf0, 0x7c, 0x6e, + 0xe7, 0x5f, 0x93, 0xd5, 0x9f, 0x5d, 0xfa, 0x75, 0xe6, 0x3b, 0xd4, 0x85, 0x0b, 0x51, 0xbf, 0xff, + 0x78, 0x63, 0x83, 0x72, 0xca, 0x4f, 0x16, 0x23, 0x80, 0x4c, 0x28, 0xf5, 0x2d, 0xdf, 0xb7, 0x9c, + 0x9e, 0x4e, 0xc8, 0xfb, 0x95, 0x0c, 0xb5, 0x26, 0xbf, 0xf6, 0x3c, 0x6b, 0x32, 0xb2, 0xe8, 0xfa, + 0x7d, 0x46, 0xe2, 0x81, 0x6b, 0x62, 0x4e, 0xbe, 0xd8, 0x8f, 0x40, 0x3e, 0xb1, 0x1a, 0xc6, 0x60, + 0xe0, 0xb9, 0xcf, 0xac, 0x3e, 0x31, 0x1d, 0xa6, 0xe5, 0x1f, 0xe8, 0x9d, 0xe3, 0x00, 0xfb, 0x74, + 0xdf, 0x66, 0xb4, 0x45, 0x09, 0xbb, 0x69, 0xf9, 0x07, 0x4d, 0x82, 0xab, 0x7e, 0x08, 0xb3, 0x12, + 0x61, 0x74, 0x1d, 0x72, 0x54, 0x26, 0x96, 0xc9, 0x76, 0xa8, 0x09, 0xa7, 0x27, 0xb5, 0x2c, 0x41, + 0xb5, 0x36, 0xb5, 0x2c, 0x41, 0xb5, 0x4c, 0x22, 0x5d, 0xec, 0x79, 0xae, 0xa7, 0xf7, 0xb1, 0xef, + 0x1b, 0x3d, 0xb1, 0x67, 0x45, 0x0a, 0xbc, 0xcf, 0x60, 0xea, 0x12, 0x2c, 0x3e, 0x70, 0x9d, 0x33, + 0x4a, 0xa0, 0xfe, 0x54, 0x81, 0x4b, 0x63, 0x08, 0xbe, 0xab, 0xdf, 0x84, 0x05, 0xe2, 0xba, 0xe8, + 0x3e, 0xf6, 0x2c, 0xec, 0xeb, 0x4c, 0xf8, 0x0a, 0x15, 0xfe, 0xd7, 0xa6, 0x12, 0x95, 0x36, 0x4f, + 0xe8, 0xb4, 0x29, 0x19, 0x8a, 0x40, 0xdf, 0x02, 0x64, 0x39, 0x01, 0xf6, 0x1c, 0xc3, 0xd6, 0x87, + 0x3e, 0xe6, 0xb4, 0x53, 0xe7, 0xa1, 0x5d, 0x16, 0x84, 0x1e, 0xf9, 0x0c, 0xa3, 0xce, 0x41, 0xf1, + 0x91, 0x8f, 0xbd, 0x70, 0x85, 0xdf, 0x85, 0x12, 0x6f, 0xf3, 0x85, 0xb5, 0x20, 0x43, 0x2c, 0x95, + 0xb8, 0xf3, 0x92, 0x26, 0x1c, 0x19, 0x48, 0x5b, 0x42, 0x95, 0x28, 0x85, 0xaa, 0x0a, 0x33, 0x04, + 0x48, 0x8e, 0x12, 0x01, 0x48, 0xd6, 0x20, 0x6c, 0xab, 0x8f, 0xa1, 0xb4, 0x75, 0x84, 0x9d, 0xe8, + 0xdc, 0x09, 0x13, 0xa7, 0x48, 0x26, 0xee, 0x2a, 0x14, 0x02, 0xc3, 0xeb, 0xe1, 0x80, 0x6c, 0x35, + 0x3b, 0x1d, 0x79, 0x06, 0x68, 0x99, 0xe4, 0x30, 0xda, 0x56, 0xdf, 0x62, 0x87, 0x22, 0xa3, 0xb1, + 0x86, 0xfa, 0x9f, 0x29, 0x98, 0x13, 0x84, 0xf9, 0xca, 0xee, 0x43, 0x16, 0x53, 0x08, 0x5f, 0x5a, + 0x23, 0x61, 0x69, 0xa3, 0x43, 0x59, 0x53, 0x5c, 0xe4, 0x8c, 0x48, 0xf5, 0x0f, 0x53, 0x90, 0xa1, + 0x70, 0xd4, 0x84, 0x42, 0xe8, 0xc6, 0x72, 0x1d, 0xa8, 0xd6, 0x99, 0xa3, 0x5b, 0x17, 0x8e, 0x6e, + 0x7d, 0x57, 0xf4, 0x68, 0xe6, 0x09, 0x99, 0x4f, 0x7f, 0x51, 0x53, 0xb4, 0x68, 0x18, 0x39, 0xdf, + 0x94, 0xae, 0x2e, 0xd9, 0xf7, 0x02, 0x85, 0xec, 0x12, 0x09, 0x7c, 0x55, 0x96, 0x00, 0x73, 0x00, + 0x8b, 0xa7, 0x27, 0xb5, 0xfc, 0x2e, 0x93, 0xc2, 0xa6, 0x24, 0x8f, 0x35, 0x20, 0x96, 0xc3, 0xf5, + 0x02, 0x72, 0x84, 0x2d, 0x93, 0x5d, 0xd9, 0xcd, 0xf9, 0xd3, 0x93, 0xda, 0xac, 0x26, 0xe0, 0xad, + 0x4d, 0x6d, 0x36, 0xec, 0xd4, 0x32, 0x89, 0xd0, 0x2d, 0x67, 0xcf, 0xe5, 0x06, 0x95, 0x7e, 0x93, + 0x29, 0x99, 0x75, 0x26, 0x44, 0xc8, 0xa9, 0x2c, 0xb2, 0x29, 0x1f, 0x51, 0x20, 0x99, 0x92, 0xa1, + 0x5b, 0xa6, 0xfa, 0xd7, 0x0a, 0x94, 0xdb, 0x38, 0x78, 0xd4, 0x22, 0x8e, 0xaf, 0xd8, 0xc8, 0x6f, + 0x02, 0x1c, 0xe0, 0x63, 0x76, 0xef, 0x08, 0x91, 0xdf, 0x49, 0x10, 0xf9, 0x38, 0x81, 0xfa, 0x7b, + 0xf8, 0x98, 0x5e, 0x50, 0xfe, 0x96, 0x13, 0x78, 0xc7, 0x5a, 0xe1, 0x40, 0xb4, 0xab, 0x77, 0x61, + 0x6e, 0x14, 0x49, 0x2e, 0x90, 0x03, 0x7c, 0xcc, 0x95, 0x86, 0x7c, 0x12, 0xb5, 0x60, 0x57, 0x1e, + 0x91, 0x65, 0x51, 0x63, 0x8d, 0x3b, 0xa9, 0xaf, 0x2b, 0xea, 0x45, 0x58, 0x90, 0xe6, 0x62, 0x3b, + 0xac, 0x7e, 0x05, 0xca, 0xdb, 0xe3, 0x2b, 0x40, 0x30, 0x73, 0x80, 0x8f, 0x45, 0x74, 0x41, 0xbf, + 0xd5, 0x9f, 0xa6, 0x60, 0x61, 0x7b, 0x7c, 0x34, 0xfa, 0xed, 0x98, 0xb5, 0xbe, 0x9d, 0xb0, 0xd6, + 0x33, 0x14, 0xc6, 0x16, 0xcb, 0x55, 0x4d, 0x5a, 0xf2, 0x1e, 0x64, 0xd8, 0x6d, 0x1d, 0xae, 0x4b, + 0x91, 0xd6, 0x85, 0xb6, 0xa1, 0x68, 0x1b, 0x7e, 0xa0, 0x0f, 0x07, 0xa6, 0x11, 0x60, 0x93, 0x5b, + 0x8b, 0xc9, 0xb4, 0x70, 0x96, 0x8c, 0x7c, 0xc4, 0x06, 0x56, 0x07, 0x13, 0x88, 0xf6, 0x5d, 0x59, + 0xb4, 0xb3, 0x6b, 0x6b, 0x53, 0x2d, 0x94, 0x92, 0x96, 0xb7, 0xa3, 0x0c, 0x73, 0x1b, 0xf6, 0xd0, + 0x0f, 0xb0, 0x27, 0x6c, 0xd2, 0x8f, 0x14, 0x98, 0x0f, 0x41, 0x5c, 0xc2, 0xaf, 0x01, 0x74, 0x19, + 0x28, 0x32, 0xf7, 0xa5, 0xd3, 0x93, 0x5a, 0x81, 0x77, 0x6c, 0x6d, 0x6a, 0x05, 0xde, 0xa1, 0x65, + 0xa2, 0x57, 0x61, 0x21, 0x3a, 0x03, 0xd8, 0x21, 0x86, 0xd1, 0xe4, 0x2e, 0x47, 0x39, 0x44, 0x6c, + 0x31, 0x38, 0xfa, 0x1a, 0x20, 0x4c, 0xcc, 0xe4, 0xc0, 0xb3, 0x7c, 0x1c, 0xf6, 0x66, 0xae, 0xd4, + 0x42, 0x84, 0xe1, 0xdd, 0xd5, 0x1d, 0x28, 0x6e, 0x7a, 0x86, 0xe5, 0x08, 0x2d, 0x99, 0x83, 0x94, + 0xeb, 0xd0, 0x3d, 0xcf, 0x68, 0x29, 0xd7, 0x21, 0xf2, 0x72, 0xf7, 0xf6, 0xa8, 0x6f, 0x97, 0xd1, + 0xc8, 0x27, 0xb1, 0x7f, 0xfe, 0xfe, 0x30, 0x30, 0xdd, 0xa7, 0x8e, 0xf0, 0xd0, 0x44, 0x5b, 0xad, + 0x41, 0x89, 0x53, 0xe3, 0x0b, 0x1d, 0x23, 0xa7, 0x7e, 0x0c, 0x57, 0x36, 0x71, 0xd7, 0xed, 0xd3, + 0xeb, 0xd3, 0x75, 0xda, 0x34, 0x3d, 0x20, 0xe6, 0xfe, 0x08, 0xf2, 0xfc, 0x06, 0x64, 0x5a, 0x97, + 0x69, 0x36, 0x4f, 0x4f, 0x6a, 0x39, 0x76, 0x05, 0xfa, 0xff, 0x7d, 0x52, 0xbb, 0xd5, 0xb3, 0x82, + 0xfd, 0x61, 0xa7, 0xde, 0x75, 0xfb, 0x8d, 0x70, 0x97, 0xcc, 0x4e, 0xf4, 0xdd, 0x18, 0x1c, 0xf4, + 0x1a, 0xf4, 0x6b, 0xd0, 0xa9, 0xf3, 0xab, 0x33, 0xc7, 0xae, 0x4e, 0x5f, 0xfd, 0x13, 0x05, 0x2e, + 0xca, 0x93, 0xff, 0xff, 0x4c, 0x8b, 0x56, 0x61, 0xde, 0x94, 0x66, 0x25, 0xde, 0x1c, 0xdb, 0xbb, + 0x71, 0xb0, 0xfa, 0x79, 0x0a, 0xaa, 0x71, 0xd2, 0xe1, 0xb2, 0x7c, 0x02, 0x59, 0x96, 0x4e, 0xe1, + 0xa1, 0xd6, 0xdd, 0xa4, 0x98, 0xfd, 0x0b, 0xc9, 0xd4, 0x59, 0x53, 0x98, 0x7f, 0x46, 0xb1, 0xfa, + 0x1f, 0x0a, 0x64, 0x19, 0x02, 0x3d, 0x19, 0xf5, 0x43, 0x32, 0xcd, 0xf5, 0xc8, 0x0f, 0x39, 0xaf, + 0x30, 0x84, 0xfb, 0x72, 0x19, 0x72, 0x96, 0xaf, 0xdb, 0xd6, 0x51, 0xe8, 0x32, 0x5b, 0xfe, 0x8e, + 0x75, 0x84, 0xcf, 0x7a, 0x8d, 0xe9, 0x18, 0xaf, 0x31, 0x46, 0x92, 0x33, 0xb1, 0x92, 0xa4, 0xee, + 0x2e, 0xd1, 0x43, 0xd2, 0x25, 0xc3, 0x74, 0x54, 0xb4, 0xd5, 0x97, 0x61, 0xbe, 0x8d, 0x03, 0x72, + 0x64, 0xfc, 0x24, 0xd3, 0xf8, 0x77, 0x29, 0x7a, 0x0b, 0xf0, 0x7e, 0x7c, 0x0b, 0xf4, 0xe9, 0x6f, + 0x81, 0x11, 0x02, 0xcf, 0x35, 0x8c, 0xed, 0x58, 0xc3, 0x58, 0x10, 0x86, 0x31, 0x2e, 0x62, 0x5a, + 0x81, 0x59, 0x11, 0xb1, 0x91, 0x28, 0x22, 0x4d, 0x51, 0x32, 0xa8, 0xea, 0x4e, 0x60, 0x05, 0xb7, + 0x47, 0xad, 0xe0, 0xcd, 0x69, 0x16, 0x75, 0xc6, 0x08, 0xce, 0x43, 0xe9, 0x5d, 0x6c, 0xd8, 0xc1, + 0xbe, 0xb0, 0x81, 0x65, 0x98, 0x13, 0x00, 0x7e, 0x43, 0x2d, 0xc0, 0xfc, 0x0e, 0xcf, 0xeb, 0x89, + 0x4e, 0x3f, 0x49, 0x41, 0x39, 0x82, 0x71, 0x89, 0xaf, 0x03, 0x88, 0xfc, 0x5f, 0x28, 0xf1, 0xab, + 0x31, 0xf1, 0x80, 0x18, 0x28, 0x02, 0xeb, 0x68, 0x10, 0xfa, 0x63, 0x05, 0xf2, 0x4c, 0xcd, 0xb1, + 0x38, 0x3a, 0x49, 0xfe, 0xff, 0x38, 0x0b, 0xfc, 0xc0, 0x88, 0x2d, 0x7b, 0x9b, 0xd0, 0xff, 0xfe, + 0x2f, 0xce, 0x77, 0x0e, 0x42, 0x3e, 0xaa, 0x1d, 0x28, 0x8d, 0xd0, 0x95, 0xb7, 0x24, 0xc3, 0xb6, + 0xe4, 0x6d, 0x79, 0x4b, 0xe6, 0xd6, 0x5e, 0x8e, 0x59, 0x35, 0x21, 0x2b, 0xf8, 0xe5, 0xc7, 0x5c, + 0xda, 0x86, 0x01, 0xcc, 0xde, 0x73, 0x3b, 0xa1, 0x96, 0x87, 0xae, 0xa5, 0x22, 0xb9, 0x96, 0x24, + 0x34, 0x0d, 0xad, 0x0a, 0x0d, 0x4d, 0x59, 0x0b, 0xdd, 0xe2, 0xaa, 0x96, 0xa6, 0x93, 0xd7, 0xe4, + 0xc9, 0x0f, 0xed, 0x3a, 0xcd, 0xf9, 0xb2, 0xfc, 0x6f, 0x9d, 0xb8, 0x74, 0x4c, 0x17, 0xd5, 0x9f, + 0x64, 0xa1, 0xc8, 0xa6, 0xe4, 0xdb, 0xb7, 0x05, 0x33, 0xa4, 0x17, 0xdf, 0xb8, 0x57, 0x13, 0xc4, + 0x2e, 0x0f, 0x23, 0x0d, 0xbe, 0x91, 0x74, 0x78, 0xf5, 0xbf, 0x32, 0x90, 0xbe, 0xe7, 0x76, 0xd0, + 0x12, 0xa4, 0xb8, 0x59, 0x4a, 0x37, 0xb3, 0xa7, 0x27, 0xb5, 0x54, 0x6b, 0x53, 0x4b, 0x59, 0xe6, + 0xf9, 0xce, 0xc5, 0x88, 0x27, 0x3f, 0x33, 0xea, 0xc9, 0x23, 0x17, 0xe6, 0x46, 0xd2, 0x25, 0x2c, + 0x72, 0x2c, 0x35, 0xdf, 0x3d, 0x3d, 0xa9, 0x95, 0xe4, 0x7c, 0xc9, 0xe4, 0x17, 0x84, 0x7f, 0x68, + 0x93, 0x3f, 0x12, 0x73, 0xd7, 0x5b, 0x9b, 0x5a, 0x49, 0xce, 0xb3, 0xf8, 0xd2, 0x3e, 0x64, 0x47, + 0xf6, 0xe1, 0x0e, 0xe4, 0x58, 0xce, 0xc9, 0xa4, 0x91, 0x7e, 0xb2, 0x1b, 0x34, 0x43, 0x5d, 0x20, + 0x31, 0x80, 0x8c, 0xf5, 0x03, 0xc3, 0x23, 0x63, 0xf3, 0x93, 0x8e, 0xe5, 0x03, 0xd0, 0x5d, 0xc8, + 0xef, 0x59, 0x8e, 0xe5, 0xef, 0x63, 0xb3, 0x52, 0x98, 0x70, 0x70, 0x38, 0x82, 0x8c, 0xee, 0xbb, + 0xa6, 0xb5, 0x67, 0x61, 0xb3, 0x02, 0x93, 0x8e, 0x16, 0x23, 0x88, 0x0f, 0xb3, 0xe7, 0x19, 0x34, + 0x09, 0xa2, 0x77, 0xdd, 0xfe, 0xc0, 0xc6, 0x64, 0x09, 0xb3, 0x2b, 0xca, 0x6a, 0x4a, 0x5b, 0x10, + 0x98, 0x0d, 0x81, 0x20, 0x8a, 0x4d, 0xe3, 0xdf, 0x4a, 0x91, 0xd9, 0x4a, 0xda, 0x40, 0x0f, 0xe1, + 0xe2, 0xbe, 0xd5, 0xdb, 0x7f, 0x6a, 0x10, 0x2f, 0x2b, 0x8a, 0x68, 0x4a, 0x13, 0x72, 0x83, 0xc2, + 0xc1, 0x21, 0x86, 0x38, 0x62, 0x11, 0x49, 0x13, 0x77, 0xad, 0xbe, 0x61, 0x57, 0xe6, 0xe8, 0xa4, + 0xe5, 0x10, 0xb1, 0xc9, 0xe0, 0xe8, 0x65, 0x98, 0xf3, 0x86, 0x0e, 0xb9, 0x72, 0x74, 0xbe, 0xb1, + 0xf3, 0xb4, 0x67, 0x89, 0x43, 0xf9, 0x75, 0x7b, 0x0d, 0x0a, 0x51, 0x92, 0xb1, 0xcc, 0x22, 0xa5, + 0x10, 0xa0, 0x22, 0x28, 0xef, 0xb8, 0x5d, 0x56, 0x74, 0x10, 0x76, 0xf2, 0x7f, 0x15, 0x58, 0x90, + 0x80, 0x61, 0x08, 0x5f, 0xb0, 0x05, 0x70, 0x82, 0x0c, 0xef, 0x19, 0x02, 0x21, 0x44, 0x5c, 0x4a, + 0x21, 0xb5, 0xea, 0x1f, 0x29, 0x90, 0x17, 0x58, 0xf4, 0x12, 0x14, 0x09, 0xc6, 0xb6, 0x82, 0x63, + 0x3d, 0xba, 0x43, 0x66, 0x05, 0xec, 0x3d, 0x7c, 0x4c, 0x56, 0x1e, 0x76, 0x89, 0x2c, 0x58, 0x41, + 0x2b, 0x09, 0x28, 0xbb, 0xe2, 0xaa, 0x90, 0xb7, 0x8d, 0xc0, 0x0a, 0x86, 0x26, 0xb3, 0x32, 0x8a, + 0x16, 0xb6, 0x89, 0x54, 0x6c, 0xd7, 0xe9, 0x31, 0xe4, 0x0c, 0x45, 0x46, 0x00, 0xb5, 0x09, 0xf3, + 0x9a, 0xe1, 0xf4, 0xf0, 0x8e, 0xdb, 0x13, 0xc6, 0xed, 0x0a, 0xe4, 0x59, 0x5e, 0x4a, 0xd8, 0x07, + 0x2d, 0x47, 0xdb, 0x72, 0x48, 0x9d, 0x92, 0x43, 0xea, 0x7f, 0x49, 0x43, 0x39, 0x22, 0xc2, 0x85, + 0xf8, 0x7e, 0x18, 0x54, 0xb3, 0x7b, 0x22, 0xe9, 0x1a, 0x1c, 0x1f, 0x1c, 0x1b, 0x56, 0xff, 0xa3, + 0x02, 0xf0, 0x81, 0x87, 0x83, 0xe0, 0xb8, 0x45, 0xa2, 0xd0, 0x97, 0xa0, 0xc8, 0x63, 0x1a, 0x9d, + 0x9c, 0x7e, 0x21, 0x3c, 0x0e, 0x23, 0x66, 0x85, 0x2c, 0xc4, 0xc1, 0x4f, 0x19, 0x9a, 0x89, 0x2d, + 0xe7, 0xe0, 0xa7, 0x14, 0x75, 0x1d, 0x4a, 0x86, 0x69, 0x62, 0x53, 0xe7, 0x5e, 0x11, 0xb7, 0x69, + 0x45, 0x0a, 0xd4, 0x18, 0x0c, 0xbd, 0x02, 0xf3, 0x1e, 0xee, 0xbb, 0x47, 0x52, 0x37, 0x66, 0xdb, + 0xe6, 0x38, 0x58, 0x74, 0x5c, 0x82, 0xac, 0x87, 0x0d, 0x3f, 0x4c, 0x3c, 0xf2, 0x16, 0xaa, 0x40, + 0xce, 0x64, 0xb9, 0x74, 0x6e, 0x89, 0x44, 0xb3, 0xfa, 0x67, 0x8a, 0xc8, 0x11, 0xdc, 0x85, 0x0c, + 0x5d, 0x20, 0xcf, 0x0f, 0xac, 0xc4, 0x5c, 0x4d, 0x42, 0x3c, 0xb2, 0x54, 0xd8, 0x20, 0xf4, 0x11, + 0xcc, 0x0e, 0xa8, 0x4c, 0x74, 0x1a, 0xa6, 0x33, 0x8f, 0xe3, 0xcd, 0x69, 0x44, 0x1d, 0x89, 0x54, + 0xdc, 0xf7, 0x83, 0x10, 0x72, 0x6f, 0x26, 0xaf, 0x94, 0x53, 0xea, 0x2a, 0x94, 0x1f, 0x0e, 0xb1, + 0x77, 0xfc, 0x81, 0x6d, 0x38, 0xd2, 0x0d, 0x78, 0x48, 0x60, 0xc2, 0xa9, 0xa2, 0x0d, 0x75, 0x00, + 0x0b, 0x52, 0x4f, 0xae, 0x09, 0xdf, 0x82, 0xab, 0xa6, 0xe5, 0x07, 0xfe, 0xa1, 0xad, 0x0f, 0xf6, + 0x8f, 0x7d, 0xab, 0x6b, 0xd8, 0x3a, 0xed, 0xae, 0x0f, 0x6c, 0xc3, 0xe1, 0x21, 0xdb, 0xb5, 0xd3, + 0x93, 0x5a, 0x65, 0xd3, 0xf2, 0x83, 0xf6, 0xc3, 0x9d, 0x0f, 0x78, 0xaf, 0x88, 0x54, 0x85, 0x13, + 0x38, 0x83, 0x51, 0xaf, 0xb0, 0xc2, 0x1c, 0x19, 0xe9, 0x59, 0x9d, 0x61, 0x10, 0x05, 0x23, 0xea, + 0x5f, 0x02, 0x54, 0xce, 0xe2, 0x38, 0x53, 0x03, 0x28, 0x89, 0xac, 0x2d, 0x13, 0x1d, 0x3b, 0xe7, + 0x5b, 0xcf, 0x29, 0xde, 0xc5, 0xd1, 0x0a, 0xab, 0x7a, 0x44, 0x64, 0xb2, 0x33, 0x5a, 0x34, 0x25, + 0x04, 0xea, 0x43, 0x51, 0x2a, 0x4a, 0x88, 0x62, 0xcc, 0xe6, 0x79, 0x26, 0x8c, 0x0a, 0x15, 0x23, + 0xce, 0xef, 0x6c, 0x54, 0xa8, 0xf0, 0xab, 0x3f, 0x54, 0x00, 0xa2, 0x7e, 0x44, 0x45, 0x59, 0x22, + 0x88, 0x6f, 0x18, 0x6f, 0xa1, 0x3b, 0x90, 0xe5, 0xa5, 0x96, 0xd4, 0xc4, 0xa5, 0x16, 0x3e, 0x82, + 0x86, 0xde, 0xac, 0xc2, 0xe2, 0x1f, 0xda, 0xec, 0x68, 0xf0, 0xd0, 0x9b, 0x42, 0xdb, 0x0f, 0x77, + 0xb4, 0x02, 0xeb, 0xd0, 0x3e, 0xb4, 0xef, 0xcd, 0xe4, 0xd3, 0xe5, 0x99, 0xea, 0x1f, 0xa4, 0xa1, + 0x40, 0xf3, 0x91, 0x54, 0x26, 0x9f, 0x29, 0x50, 0x19, 0x09, 0x56, 0xf4, 0xce, 0xb1, 0x1e, 0x85, + 0x4c, 0x44, 0x40, 0x1f, 0x9e, 0x47, 0x40, 0xe1, 0x0c, 0x75, 0x4d, 0x8a, 0x7a, 0x9a, 0xc7, 0xd4, + 0x77, 0x34, 0x99, 0xcc, 0xde, 0x3a, 0xaf, 0xe7, 0xb9, 0xe8, 0xc5, 0xd0, 0x44, 0x5f, 0x86, 0x39, + 0xb9, 0xe6, 0x14, 0xe6, 0x23, 0x8b, 0xd1, 0xa6, 0xb4, 0x4c, 0xf4, 0x0d, 0x00, 0xd3, 0x73, 0x07, + 0x03, 0x6c, 0xea, 0x06, 0x8b, 0xcc, 0x26, 0xb9, 0x40, 0x0b, 0x7c, 0xcc, 0x7a, 0x50, 0xdd, 0x86, + 0x2b, 0x5f, 0xb8, 0xa4, 0x18, 0xc7, 0x77, 0x24, 0xd9, 0x95, 0x96, 0x3c, 0xda, 0xea, 0xf7, 0x52, + 0x50, 0x94, 0x15, 0x17, 0x05, 0xc0, 0x8a, 0xcb, 0xf2, 0x71, 0x78, 0xff, 0x45, 0x8f, 0x43, 0xb4, + 0x13, 0x23, 0x51, 0x5a, 0x20, 0xa0, 0xd5, 0x4f, 0x60, 0x6e, 0xb4, 0x4b, 0x4c, 0x40, 0xd5, 0x1e, + 0x0d, 0xa8, 0xde, 0x79, 0x21, 0x8d, 0x18, 0x91, 0x81, 0x12, 0x3d, 0x20, 0x48, 0x62, 0xe0, 0xf1, + 0x28, 0x03, 0xbf, 0xf1, 0xa2, 0x52, 0x91, 0x79, 0xf8, 0x2e, 0x94, 0xc7, 0x8f, 0x73, 0x0c, 0x07, + 0xbb, 0xa3, 0x1c, 0xfc, 0xfa, 0x8b, 0x59, 0x0d, 0x69, 0xfe, 0x7b, 0x33, 0xf9, 0x54, 0x39, 0xad, + 0x5e, 0x86, 0x4b, 0xf7, 0xe9, 0x13, 0x8e, 0xfb, 0x38, 0x30, 0xcc, 0x28, 0xd5, 0xa9, 0xfe, 0xb3, + 0x02, 0x4b, 0xe3, 0x18, 0x6e, 0x42, 0x0d, 0xc8, 0xf7, 0x39, 0x8c, 0xab, 0xcb, 0x37, 0x12, 0xd8, + 0x8a, 0x27, 0x52, 0x17, 0x00, 0x59, 0x3d, 0x42, 0xb2, 0xd5, 0xdf, 0x84, 0xd2, 0x48, 0x87, 0x18, + 0xc9, 0xbc, 0x31, 0x2a, 0x19, 0x39, 0xba, 0x1a, 0x06, 0x96, 0x5d, 0xe7, 0x2f, 0x53, 0xc2, 0x89, + 0xa5, 0xa0, 0xee, 0x07, 0x29, 0xb8, 0xb8, 0xe5, 0x1c, 0x0e, 0xf1, 0x10, 0xd3, 0x6b, 0x51, 0xdc, + 0x6d, 0xbf, 0xca, 0xb4, 0x0d, 0xbb, 0x37, 0x43, 0x3f, 0x8e, 0x35, 0xd0, 0x6f, 0x49, 0x2e, 0x17, + 0xad, 0x56, 0x34, 0x37, 0x4e, 0x4f, 0x6a, 0x39, 0xca, 0x15, 0x9d, 0xf3, 0xf6, 0x54, 0x73, 0xf2, + 0x71, 0x91, 0xdf, 0x76, 0x03, 0x16, 0xfc, 0x03, 0x6b, 0xa0, 0xfb, 0xfb, 0xee, 0xd0, 0x36, 0x75, + 0xc6, 0x01, 0x4f, 0xf8, 0x10, 0x44, 0x9b, 0xc2, 0x1f, 0x12, 0xb0, 0xfa, 0xe7, 0x29, 0x58, 0x1c, + 0x95, 0x0a, 0xdf, 0xef, 0x87, 0x91, 0x37, 0xc3, 0xb6, 0xfb, 0xad, 0xa4, 0x3a, 0x49, 0x0c, 0x85, + 0xba, 0x78, 0x58, 0x10, 0xba, 0x41, 0x7f, 0xaf, 0x40, 0x8e, 0x03, 0x7f, 0xa5, 0x52, 0x7f, 0x67, + 0xcc, 0x19, 0x7d, 0x39, 0xa9, 0x5a, 0xe6, 0x19, 0x5d, 0x4c, 0x5d, 0x2d, 0xe1, 0x7a, 0x46, 0x51, + 0x51, 0x5a, 0x8a, 0x8a, 0xd4, 0x4b, 0x70, 0x71, 0x63, 0xdf, 0xf0, 0x82, 0x0d, 0xf6, 0x2c, 0x4a, + 0x9c, 0x98, 0x27, 0xb0, 0x38, 0x0a, 0xe6, 0xe2, 0x6b, 0x42, 0x8e, 0x3f, 0xa0, 0xe2, 0xe2, 0x53, + 0x25, 0x26, 0x02, 0xbf, 0x2e, 0x5e, 0x57, 0xd1, 0xc1, 0x6d, 0x56, 0xb7, 0x0e, 0xdf, 0x72, 0x30, + 0xdc, 0x8d, 0xf7, 0x60, 0x29, 0xfe, 0x91, 0x02, 0x9a, 0x85, 0xdc, 0xa3, 0x07, 0xef, 0x3d, 0x78, + 0xff, 0xc3, 0x07, 0xe5, 0x0b, 0xa4, 0xb1, 0xb1, 0xf3, 0xa8, 0xbd, 0xbb, 0xa5, 0x95, 0x15, 0x54, + 0x84, 0xfc, 0xe6, 0xfa, 0xee, 0x7a, 0x73, 0xbd, 0xbd, 0x55, 0x4e, 0xa1, 0x02, 0x64, 0x76, 0xd7, + 0x9b, 0x3b, 0x5b, 0xe5, 0xf4, 0x8d, 0xeb, 0x50, 0xa0, 0x19, 0xe6, 0xfb, 0xae, 0x89, 0x11, 0x40, + 0x76, 0x63, 0xa7, 0xb5, 0xf5, 0x60, 0xb7, 0x7c, 0x81, 0x7c, 0xef, 0x6c, 0xad, 0xb7, 0xb7, 0xda, + 0x65, 0x65, 0xed, 0xc7, 0x55, 0xc8, 0xac, 0x9b, 0x7d, 0xcb, 0x41, 0x01, 0x64, 0x68, 0x5d, 0x0f, + 0xbd, 0xf2, 0xfc, 0xca, 0x1f, 0x95, 0x44, 0x75, 0x75, 0xd2, 0x12, 0xa1, 0x5a, 0xf9, 0xfe, 0x3f, + 0xfd, 0xfb, 0x0f, 0x52, 0x08, 0x95, 0x1b, 0x3a, 0x7d, 0x49, 0xd7, 0x38, 0xba, 0xd9, 0xa0, 0xa5, + 0x42, 0xf4, 0xfb, 0x0a, 0x14, 0xc2, 0x37, 0x5e, 0xe8, 0xd5, 0x09, 0xde, 0x56, 0x85, 0xd3, 0xbf, + 0x36, 0x59, 0x67, 0xce, 0xc2, 0x35, 0xca, 0xc2, 0x12, 0x5a, 0x94, 0x58, 0x08, 0x9f, 0x8d, 0xa1, + 0x1f, 0x2b, 0x30, 0x3f, 0xf6, 0x78, 0x0b, 0xdd, 0x9c, 0xe6, 0xa1, 0x17, 0x63, 0x69, 0x6d, 0xfa, + 0xb7, 0x61, 0xea, 0x2b, 0x94, 0xb1, 0x97, 0x50, 0x2d, 0x8e, 0xb1, 0xc6, 0x27, 0xe2, 0xf3, 0x3b, + 0xe8, 0x2f, 0x14, 0x28, 0xca, 0xef, 0x77, 0x50, 0x7d, 0xe2, 0x87, 0x3e, 0x8c, 0xbb, 0xc6, 0x94, + 0x0f, 0x83, 0xd4, 0x37, 0x29, 0x6b, 0xaf, 0xa3, 0xfa, 0x73, 0x58, 0x6b, 0xd0, 0xfb, 0xde, 0x6f, + 0x7c, 0x42, 0x7f, 0x29, 0xa7, 0x10, 0x15, 0xa5, 0xd1, 0x6b, 0x13, 0xd6, 0xae, 0x19, 0x97, 0xd3, + 0x55, 0xba, 0xd5, 0xbb, 0x94, 0xc7, 0x37, 0xd1, 0xed, 0xe9, 0x78, 0x6c, 0xb0, 0x47, 0x0f, 0x3f, + 0x54, 0xa0, 0x34, 0x52, 0xe7, 0x47, 0x49, 0x42, 0x8a, 0x7b, 0x2a, 0x50, 0x7d, 0x7d, 0xf2, 0x01, + 0x9c, 0xe5, 0x15, 0xca, 0x72, 0x15, 0x55, 0x24, 0x96, 0x1d, 0xd7, 0x61, 0x0c, 0x52, 0x26, 0x9e, + 0x41, 0x96, 0x15, 0xa2, 0xd1, 0xea, 0x04, 0xb5, 0x6a, 0xc6, 0xc7, 0x57, 0x27, 0xae, 0x6a, 0xab, + 0x57, 0x28, 0x03, 0x17, 0xd1, 0x82, 0xc4, 0x00, 0x37, 0x85, 0xe4, 0x3c, 0x86, 0x45, 0xd2, 0xc4, + 0xf3, 0x38, 0x5e, 0xb6, 0x4d, 0x3c, 0x8f, 0x67, 0xeb, 0xae, 0xfc, 0x3c, 0xaa, 0x32, 0x0f, 0x43, + 0x8b, 0x6c, 0xd7, 0x1d, 0xe5, 0x06, 0xfa, 0x9e, 0x02, 0x85, 0xed, 0x89, 0xd8, 0xd8, 0x9e, 0x86, + 0x8d, 0x33, 0x55, 0xc9, 0x58, 0x51, 0x30, 0x36, 0xd0, 0xef, 0x40, 0x8e, 0xd7, 0x18, 0x51, 0x92, + 0x6c, 0x47, 0x6b, 0x98, 0xd5, 0x1b, 0x93, 0x74, 0xe5, 0x93, 0x57, 0xe9, 0xe4, 0x8b, 0x08, 0x49, + 0x93, 0xf3, 0x5a, 0x26, 0xfa, 0x5d, 0x05, 0xf2, 0xa2, 0x7c, 0x80, 0x6e, 0x4c, 0x54, 0x63, 0x60, + 0x0c, 0xbc, 0x3a, 0x45, 0x3d, 0x42, 0xbd, 0x4a, 0x39, 0xb8, 0x84, 0x2e, 0x4a, 0x1c, 0xf8, 0x62, + 0xd6, 0x67, 0x90, 0x65, 0xa5, 0x88, 0x44, 0x2d, 0x1c, 0x29, 0x5f, 0x24, 0x6a, 0xe1, 0x58, 0x5d, + 0x23, 0x4e, 0xf4, 0xfb, 0x6c, 0x3e, 0xb2, 0x78, 0x91, 0xac, 0x4f, 0x5c, 0xfc, 0x58, 0x61, 0x24, + 0x71, 0xf1, 0xe3, 0xd5, 0x8a, 0xd8, 0xc5, 0x8b, 0x62, 0x08, 0x1a, 0xc0, 0xcc, 0x3d, 0xb7, 0xe3, + 0xa3, 0xaf, 0x3c, 0x37, 0x11, 0xcf, 0x66, 0x7e, 0x65, 0xc2, 0x84, 0xbd, 0x7a, 0x99, 0xce, 0xba, + 0x80, 0xe6, 0xa5, 0x59, 0xbf, 0x4d, 0x66, 0x22, 0x47, 0x2f, 0xcc, 0x35, 0x26, 0xea, 0xfc, 0x78, + 0x9e, 0x33, 0x51, 0xe7, 0xcf, 0xa4, 0x2f, 0x63, 0xaf, 0xc2, 0x30, 0x85, 0x49, 0xd9, 0x08, 0xf3, + 0x2f, 0x89, 0x6c, 0x8c, 0x27, 0x8d, 0x12, 0xd9, 0x38, 0x93, 0x37, 0x8a, 0x65, 0x83, 0xe6, 0x8d, + 0x06, 0x64, 0xe2, 0x63, 0xc8, 0x50, 0xef, 0x25, 0xd1, 0x1d, 0x91, 0xeb, 0xf1, 0x89, 0xee, 0xc8, + 0x48, 0xa9, 0x5d, 0x6c, 0xbc, 0x2a, 0xbb, 0x23, 0xb4, 0xe8, 0x79, 0x47, 0xb9, 0xf1, 0xba, 0x82, + 0x9e, 0x42, 0x51, 0x2e, 0x0a, 0x27, 0xde, 0xb3, 0x31, 0x55, 0xf2, 0xea, 0x1b, 0xe7, 0xaa, 0x36, + 0xab, 0x17, 0xd0, 0xef, 0x29, 0x80, 0xce, 0x76, 0x40, 0xb7, 0xa7, 0xa4, 0xf7, 0x82, 0x5c, 0xfc, + 0xa9, 0x02, 0x79, 0x91, 0x46, 0x4c, 0x3c, 0x7c, 0x63, 0x89, 0xe5, 0xc4, 0xc3, 0x37, 0x9e, 0x97, + 0x54, 0xdf, 0xa1, 0x7b, 0xf0, 0xd6, 0xc8, 0xe1, 0xa3, 0xe1, 0x8c, 0xed, 0xf6, 0x9e, 0xac, 0xa0, + 0xe5, 0x18, 0x70, 0xe3, 0x13, 0x11, 0x44, 0x7d, 0x87, 0x38, 0x6c, 0xe5, 0xf1, 0x48, 0x18, 0xad, + 0x4d, 0x15, 0x36, 0x33, 0xa6, 0x6f, 0x9d, 0x23, 0xd4, 0x56, 0xbf, 0x4c, 0x99, 0x5f, 0x46, 0xd7, + 0xc6, 0x9c, 0x0e, 0xdd, 0x94, 0xd9, 0xf9, 0x91, 0x02, 0x0b, 0xeb, 0xb6, 0x3d, 0x1a, 0x19, 0xa3, + 0xd7, 0xa7, 0x08, 0xa2, 0x19, 0x8b, 0x37, 0xa7, 0x0e, 0xbb, 0xd5, 0x97, 0x28, 0x83, 0x57, 0xd1, + 0x15, 0x89, 0x41, 0x16, 0x29, 0x8b, 0xd8, 0x1b, 0x7d, 0xaa, 0x40, 0x51, 0x0e, 0x64, 0x12, 0xd5, + 0x3c, 0x26, 0x10, 0x4a, 0x74, 0x27, 0xe3, 0x22, 0x24, 0xb5, 0x46, 0x99, 0xba, 0x82, 0x2e, 0xcb, + 0xd7, 0x1d, 0xe9, 0xc8, 0xc3, 0x1f, 0xe2, 0x8d, 0x15, 0xe5, 0xc0, 0x32, 0x91, 0xa5, 0x98, 0xc8, + 0x3e, 0x91, 0xa5, 0xb8, 0x88, 0x55, 0xbd, 0x4e, 0x59, 0xfa, 0x92, 0x2a, 0xbb, 0x62, 0x98, 0x75, + 0xd4, 0xa9, 0xb2, 0xdd, 0x51, 0x6e, 0x34, 0x6f, 0x7c, 0xf6, 0x6f, 0xcb, 0x17, 0x3e, 0x3b, 0x5d, + 0x56, 0x7e, 0x76, 0xba, 0xac, 0xfc, 0xfc, 0x74, 0x59, 0xf9, 0xd7, 0xd3, 0x65, 0xe5, 0xd3, 0xcf, + 0x97, 0x2f, 0xfc, 0xec, 0xf3, 0xe5, 0x0b, 0x3f, 0xff, 0x7c, 0xf9, 0xc2, 0x93, 0xbc, 0x98, 0xa5, + 0x93, 0xa5, 0x49, 0xbe, 0x5b, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0xbc, 0x3c, 0x60, 0xd7, 0x42, + 0x34, 0x00, 0x00, } diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index 5cde50b9c3ad..ac28896a89d8 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -12,7 +12,7 @@ syntax = "proto3"; package cockroach.server.serverpb; option go_package = "serverpb"; -import "config/zonepb/zone.proto"; +import "config/zone.proto"; import "jobs/jobspb/jobs.proto"; import "server/serverpb/status.proto"; import "storage/engine/enginepb/mvcc.proto"; @@ -76,7 +76,7 @@ message DatabaseDetailsResponse { // the 'target_id' field of events. int64 descriptor_id = 3 [(gogoproto.customname) = "DescriptorID"]; // The zone configuration in effect for this database. - cockroach.config.zonepb.ZoneConfig zone_config = 4 [(gogoproto.nullable) = false]; + cockroach.config.ZoneConfig zone_config = 4 [(gogoproto.nullable) = false]; // The level at which this object's zone configuration is set. ZoneConfigurationLevel zone_config_level = 5; } @@ -162,7 +162,7 @@ message TableDetailsResponse { // executed. string create_table_statement = 5; // The zone configuration in effect for this table. - cockroach.config.zonepb.ZoneConfig zone_config = 6 [(gogoproto.nullable) = false]; + cockroach.config.ZoneConfig zone_config = 6 [(gogoproto.nullable) = false]; // The level at which this object's zone configuration is set. ZoneConfigurationLevel zone_config_level = 7; // descriptor_id is an identifier used to uniquely identify this table. @@ -540,7 +540,7 @@ message DataDistributionResponse { // "PARTITION north_america OF TABLE users". string target = 1; - config.zonepb.ZoneConfig config = 2 [(gogoproto.nullable) = false]; + config.ZoneConfig config = 2 [(gogoproto.nullable) = false]; reserved 3; diff --git a/pkg/server/testing_knobs.go b/pkg/server/testing_knobs.go index c8c308b8a60e..f50817fb369f 100644 --- a/pkg/server/testing_knobs.go +++ b/pkg/server/testing_knobs.go @@ -13,7 +13,7 @@ package server import ( "net" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/rpc" ) @@ -23,9 +23,9 @@ type TestingKnobs struct { // automatic version upgrade mechanism. DisableAutomaticVersionUpgrade int32 // accessed atomically // DefaultZoneConfigOverride, if set, overrides the default zone config defined in `pkg/config/zone.go` - DefaultZoneConfigOverride *zonepb.ZoneConfig + DefaultZoneConfigOverride *config.ZoneConfig // DefaultSystemZoneConfigOverride, if set, overrides the default system zone config defined in `pkg/config/zone.go` - DefaultSystemZoneConfigOverride *zonepb.ZoneConfig + DefaultSystemZoneConfigOverride *config.ZoneConfig // PauseAfterGettingRPCAddress, if non-nil, instructs the server to wait until // the channel is closed after getting an RPC serving address. PauseAfterGettingRPCAddress chan struct{} diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 9ad3f45226bd..18019d85788f 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -24,7 +24,6 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" @@ -36,7 +35,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sqlmigrations" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/engine" @@ -46,8 +44,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/errors" "github.com/gogo/protobuf/proto" + "github.com/pkg/errors" ) const ( @@ -391,7 +389,7 @@ func (ts *TestServer) ExpectedInitialRangeCount() (int, error) { // ExpectedInitialRangeCount returns the expected number of ranges that should // be on the server after bootstrap. func ExpectedInitialRangeCount( - db *client.DB, defaultZoneConfig *zonepb.ZoneConfig, defaultSystemZoneConfig *zonepb.ZoneConfig, + db *client.DB, defaultZoneConfig *config.ZoneConfig, defaultSystemZoneConfig *config.ZoneConfig, ) (int, error) { descriptorIDs, err := sqlmigrations.ExpectedDescriptorIDs(context.Background(), db, defaultZoneConfig, defaultSystemZoneConfig) if err != nil { @@ -808,39 +806,6 @@ func (ts *TestServer) GCSystemLog( return ts.gcSystemLog(ctx, table, timestampLowerBound, timestampUpperBound) } -// ForceTableGC sends a GCRequest for the ranges corresponding to a table. -func (ts *TestServer) ForceTableGC( - ctx context.Context, database, table string, timestamp hlc.Timestamp, -) error { - tableIDQuery := ` - SELECT tables.id FROM system.namespace tables - JOIN system.namespace dbs ON dbs.id = tables."parentID" - WHERE dbs.name = $1 AND tables.name = $2 - ` - row, err := ts.internalExecutor.QueryRow( - ctx, "resolve-table-id", nil /* txn */, tableIDQuery, database, table) - if err != nil { - return err - } - if row == nil { - return errors.Errorf("table not found") - } - if len(row) != 1 { - return errors.AssertionFailedf("expected 1 column from internal query") - } - tableID := uint32(*row[0].(*tree.DInt)) - tblKey := roachpb.Key(keys.MakeTablePrefix(tableID)) - gcr := roachpb.GCRequest{ - RequestHeader: roachpb.RequestHeader{ - Key: tblKey, - EndKey: tblKey.PrefixEnd(), - }, - Threshold: timestamp, - } - _, pErr := client.SendWrapped(ctx, ts.distSender, &gcr) - return pErr.GoError() -} - type testServerFactoryImpl struct{} // TestServerFactory can be passed to serverutils.InitTestServerFactory diff --git a/pkg/server/updates.go b/pkg/server/updates.go index b94d463294e2..df3f12262d43 100644 --- a/pkg/server/updates.go +++ b/pkg/server/updates.go @@ -28,7 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/diagnosticspb" @@ -370,10 +370,10 @@ func (s *Server) getReportingInfo( ); err != nil { log.Warning(ctx, err) } else { - info.ZoneConfigs = make(map[int64]zonepb.ZoneConfig) + info.ZoneConfigs = make(map[int64]config.ZoneConfig) for _, row := range datums { id := int64(tree.MustBeDInt(row[0])) - var zone zonepb.ZoneConfig + var zone config.ZoneConfig if bytes, ok := row[1].(*tree.DBytes); !ok { continue } else { @@ -382,7 +382,7 @@ func (s *Server) getReportingInfo( continue } } - var anonymizedZone zonepb.ZoneConfig + var anonymizedZone config.ZoneConfig anonymizeZoneConfig(&anonymizedZone, zone, secret) info.ZoneConfigs[id] = anonymizedZone } @@ -392,7 +392,7 @@ func (s *Server) getReportingInfo( return &info } -func anonymizeZoneConfig(dst *zonepb.ZoneConfig, src zonepb.ZoneConfig, secret string) { +func anonymizeZoneConfig(dst *config.ZoneConfig, src config.ZoneConfig, secret string) { if src.RangeMinBytes != nil { dst.RangeMinBytes = proto.Int64(*src.RangeMinBytes) } @@ -400,15 +400,15 @@ func anonymizeZoneConfig(dst *zonepb.ZoneConfig, src zonepb.ZoneConfig, secret s dst.RangeMaxBytes = proto.Int64(*src.RangeMaxBytes) } if src.GC != nil { - dst.GC = &zonepb.GCPolicy{TTLSeconds: src.GC.TTLSeconds} + dst.GC = &config.GCPolicy{TTLSeconds: src.GC.TTLSeconds} } if src.NumReplicas != nil { dst.NumReplicas = proto.Int32(*src.NumReplicas) } - dst.Constraints = make([]zonepb.Constraints, len(src.Constraints)) + dst.Constraints = make([]config.Constraints, len(src.Constraints)) for i := range src.Constraints { dst.Constraints[i].NumReplicas = src.Constraints[i].NumReplicas - dst.Constraints[i].Constraints = make([]zonepb.Constraint, len(src.Constraints[i].Constraints)) + dst.Constraints[i].Constraints = make([]config.Constraint, len(src.Constraints[i].Constraints)) for j := range src.Constraints[i].Constraints { dst.Constraints[i].Constraints[j].Type = src.Constraints[i].Constraints[j].Type if key := src.Constraints[i].Constraints[j].Key; key != "" { @@ -419,9 +419,9 @@ func anonymizeZoneConfig(dst *zonepb.ZoneConfig, src zonepb.ZoneConfig, secret s } } } - dst.LeasePreferences = make([]zonepb.LeasePreference, len(src.LeasePreferences)) + dst.LeasePreferences = make([]config.LeasePreference, len(src.LeasePreferences)) for i := range src.LeasePreferences { - dst.LeasePreferences[i].Constraints = make([]zonepb.Constraint, len(src.LeasePreferences[i].Constraints)) + dst.LeasePreferences[i].Constraints = make([]config.Constraint, len(src.LeasePreferences[i].Constraints)) for j := range src.LeasePreferences[i].Constraints { dst.LeasePreferences[i].Constraints[j].Type = src.LeasePreferences[i].Constraints[j].Type if key := src.LeasePreferences[i].Constraints[j].Key; key != "" { @@ -432,7 +432,7 @@ func anonymizeZoneConfig(dst *zonepb.ZoneConfig, src zonepb.ZoneConfig, secret s } } } - dst.Subzones = make([]zonepb.Subzone, len(src.Subzones)) + dst.Subzones = make([]config.Subzone, len(src.Subzones)) for i := range src.Subzones { dst.Subzones[i].IndexID = src.Subzones[i].IndexID dst.Subzones[i].PartitionName = sql.HashForReporting(secret, src.Subzones[i].PartitionName) diff --git a/pkg/server/updates_test.go b/pkg/server/updates_test.go index b6c158cd16f6..85e0defa093d 100644 --- a/pkg/server/updates_test.go +++ b/pkg/server/updates_test.go @@ -26,7 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/diagnosticspb" @@ -779,11 +779,11 @@ func TestReportUsage(t *testing.T) { if a, e := zone.GC.TTLSeconds, int32(1); a != e { t.Errorf("expected zone %d GC.TTLSeconds = %d; got %d", id, e, a) } - if a, e := zone.Constraints, []zonepb.Constraints{ + if a, e := zone.Constraints, []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: hashedZone, Value: hashedElemName, Type: zonepb.Constraint_REQUIRED}, - {Value: hashedElemName, Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: hashedZone, Value: hashedElemName, Type: config.Constraint_REQUIRED}, + {Value: hashedElemName, Type: config.Constraint_REQUIRED}, }, }, }; !reflect.DeepEqual(a, e) { @@ -791,30 +791,30 @@ func TestReportUsage(t *testing.T) { } } if id == keys.SystemDatabaseID { - if a, e := zone.Constraints, []zonepb.Constraints{ + if a, e := zone.Constraints, []config.Constraints{ { NumReplicas: 1, - Constraints: []zonepb.Constraint{{Value: hashedElemName, Type: zonepb.Constraint_REQUIRED}}, + Constraints: []config.Constraint{{Value: hashedElemName, Type: config.Constraint_REQUIRED}}, }, { NumReplicas: 2, - Constraints: []zonepb.Constraint{ - {Key: hashedZone, Value: hashedElemName, Type: zonepb.Constraint_REQUIRED}, - {Value: hashedElemName, Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: hashedZone, Value: hashedElemName, Type: config.Constraint_REQUIRED}, + {Value: hashedElemName, Type: config.Constraint_REQUIRED}, }, }, }; !reflect.DeepEqual(a, e) { t.Errorf("expected zone %d Constraints = %+v; got %+v", id, e, a) } - if a, e := zone.LeasePreferences, []zonepb.LeasePreference{ + if a, e := zone.LeasePreferences, []config.LeasePreference{ { - Constraints: []zonepb.Constraint{ - {Key: hashedZone, Value: hashedElemName, Type: zonepb.Constraint_REQUIRED}, - {Value: hashedElemName, Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: hashedZone, Value: hashedElemName, Type: config.Constraint_REQUIRED}, + {Value: hashedElemName, Type: config.Constraint_REQUIRED}, }, }, { - Constraints: []zonepb.Constraint{{Value: hashedElemName, Type: zonepb.Constraint_REQUIRED}}, + Constraints: []config.Constraint{{Value: hashedElemName, Type: config.Constraint_REQUIRED}}, }, }; !reflect.DeepEqual(a, e) { t.Errorf("expected zone %d LeasePreferences = %+v; got %+v", id, e, a) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 753c00a07d38..1ad9f5ad8105 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/keys" @@ -1210,14 +1210,14 @@ CREATE TABLE crdb_internal.create_statements ( } else { for _, row := range zoneConstraintRows { tableName := string(tree.MustBeDString(row[0])) - var zoneConfig zonepb.ZoneConfig + var zoneConfig config.ZoneConfig yamlString := string(tree.MustBeDString(row[1])) err := yaml.UnmarshalStrict([]byte(yamlString), &zoneConfig) if err != nil { return err } // If all constraints are default, then don't show anything. - if !zoneConfig.Equal(zonepb.ZoneConfig{}) { + if !zoneConfig.Equal(config.ZoneConfig{}) { sqlString := string(tree.MustBeDString(row[2])) zoneConfigStmts[tableName] = append(zoneConfigStmts[tableName], sqlString) } @@ -2139,7 +2139,7 @@ CREATE TABLE crdb_internal.zones ( id := uint32(tree.MustBeDInt(r[0])) var zoneSpecifier *tree.ZoneSpecifier - zs, err := zonepb.ZoneSpecifierFromID(id, resolveID) + zs, err := config.ZoneSpecifierFromID(id, resolveID) if err != nil { // We can have valid zoneSpecifiers whose table/database has been // deleted because zoneSpecifiers are collected asynchronously. @@ -2151,7 +2151,7 @@ CREATE TABLE crdb_internal.zones ( } configBytes := []byte(*r[1].(*tree.DBytes)) - var configProto zonepb.ZoneConfig + var configProto config.ZoneConfig if err := protoutil.Unmarshal(configBytes, &configProto); err != nil { return err } diff --git a/pkg/sql/create_stats_test.go b/pkg/sql/create_stats_test.go index a05de0f485f4..fbe6c0b0ebc4 100644 --- a/pkg/sql/create_stats_test.go +++ b/pkg/sql/create_stats_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -97,12 +98,9 @@ func TestStatsWithLowTTL(t *testing.T) { return } // Force a table GC of values older than 1 second. - if err := s.ForceTableGC( - context.Background(), "test", "t", s.Clock().Now().Add(-int64(1*time.Second), 0), - ); err != nil { - goroutineErr = err - return - } + sqlutils.ForceTableGC( + t, s.DistSenderI().(*kv.DistSender), db2, + "test", "t", s.Clock().Now().Add(-int64(1*time.Second), 0)) time.Sleep(10 * time.Millisecond) } }() diff --git a/pkg/sql/database_test.go b/pkg/sql/database_test.go index 1bb73f4ff97a..6cd196102939 100644 --- a/pkg/sql/database_test.go +++ b/pkg/sql/database_test.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -59,7 +58,7 @@ func TestDatabaseAccessors(t *testing.T) { return err } - databaseCache := newDatabaseCache(config.NewSystemConfig(zonepb.DefaultZoneConfigRef())) + databaseCache := newDatabaseCache(config.NewSystemConfig(config.DefaultZoneConfigRef())) _, err := databaseCache.getDatabaseDescByID(ctx, txn, sqlbase.SystemDB.ID) return err }); err != nil { diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index 214ec1036437..91139cc44d91 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -22,7 +22,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/kv" @@ -782,7 +782,7 @@ func TestPartitionSpans(t *testing.T) { testStopper := stop.NewStopper() defer testStopper.Stop(context.TODO()) mockGossip := gossip.NewTest(roachpb.NodeID(1), nil /* rpcContext */, nil, /* grpcServer */ - testStopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + testStopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) var nodeDescs []*roachpb.NodeDescriptor for i := 1; i <= 10; i++ { nodeID := roachpb.NodeID(i) @@ -972,7 +972,7 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { testStopper := stop.NewStopper() defer testStopper.Stop(context.TODO()) mockGossip := gossip.NewTest(roachpb.NodeID(1), nil /* rpcContext */, nil, /* grpcServer */ - testStopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + testStopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) var nodeDescs []*roachpb.NodeDescriptor for i := 1; i <= 2; i++ { nodeID := roachpb.NodeID(i) @@ -1061,7 +1061,7 @@ func TestPartitionSpansSkipsNodesNotInGossip(t *testing.T) { defer stopper.Stop(context.TODO()) mockGossip := gossip.NewTest(roachpb.NodeID(1), nil /* rpcContext */, nil, /* grpcServer */ - stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) var nodeDescs []*roachpb.NodeDescriptor for i := 1; i <= 2; i++ { nodeID := roachpb.NodeID(i) @@ -1151,7 +1151,7 @@ func TestCheckNodeHealth(t *testing.T) { const nodeID = roachpb.NodeID(5) mockGossip := gossip.NewTest(nodeID, nil /* rpcContext */, nil, /* grpcServer */ - stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) desc := &roachpb.NodeDescriptor{ NodeID: nodeID, diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index 8e310a41b15a..24e6d36d2671 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -19,7 +19,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -45,7 +45,7 @@ import ( // Returns an error if a zone config for the specified table or // database ID doesn't match the expected parameter. If expected // is nil, then we verify no zone config exists. -func zoneExists(sqlDB *gosql.DB, expected *zonepb.ZoneConfig, id sqlbase.ID) error { +func zoneExists(sqlDB *gosql.DB, expected *config.ZoneConfig, id sqlbase.ID) error { rows, err := sqlDB.Query(`SELECT * FROM system.zones WHERE id = $1`, id) if err != nil { return err @@ -64,7 +64,7 @@ func zoneExists(sqlDB *gosql.DB, expected *zonepb.ZoneConfig, id sqlbase.ID) err if storedID != id { return errors.Errorf("e = %d, v = %d", id, storedID) } - var cfg zonepb.ZoneConfig + var cfg config.ZoneConfig if err := protoutil.Unmarshal(val, &cfg); err != nil { return err } @@ -88,8 +88,8 @@ func descExists(sqlDB *gosql.DB, exists bool, id sqlbase.ID) error { return nil } -func addImmediateGCZoneConfig(sqlDB *gosql.DB, id sqlbase.ID) (zonepb.ZoneConfig, error) { - cfg := zonepb.DefaultZoneConfig() +func addImmediateGCZoneConfig(sqlDB *gosql.DB, id sqlbase.ID) (config.ZoneConfig, error) { + cfg := config.DefaultZoneConfig() cfg.GC.TTLSeconds = 0 buf, err := protoutil.Marshal(&cfg) if err != nil { @@ -99,8 +99,8 @@ func addImmediateGCZoneConfig(sqlDB *gosql.DB, id sqlbase.ID) (zonepb.ZoneConfig return cfg, err } -func addDefaultZoneConfig(sqlDB *gosql.DB, id sqlbase.ID) (zonepb.ZoneConfig, error) { - cfg := zonepb.DefaultZoneConfig() +func addDefaultZoneConfig(sqlDB *gosql.DB, id sqlbase.ID) (config.ZoneConfig, error) { + cfg := config.DefaultZoneConfig() buf, err := protoutil.Marshal(&cfg) if err != nil { return cfg, err @@ -164,7 +164,7 @@ INSERT INTO t.kv VALUES ('c', 'e'), ('a', 'c'), ('b', 'd'); tbDesc := desc.Table(ts) // Add a zone config for both the table and database. - cfg := zonepb.DefaultZoneConfig() + cfg := config.DefaultZoneConfig() buf, err := protoutil.Marshal(&cfg) if err != nil { t.Fatal(err) @@ -402,7 +402,7 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a'); tests.CheckKeyCount(t, kvDB, tableSpan, 0) tests.CheckKeyCount(t, kvDB, table2Span, 6) - def := zonepb.DefaultZoneConfig() + def := config.DefaultZoneConfig() if err := zoneExists(sqlDB, &def, dbDesc.ID); err != nil { t.Fatal(err) } @@ -622,8 +622,8 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { // binary to do this properly.) Dropping the index will thus require // regenerating the zone config's SubzoneSpans, which will fail with a "CCL // required" error. - zoneConfig := zonepb.ZoneConfig{ - Subzones: []zonepb.Subzone{ + zoneConfig := config.ZoneConfig{ + Subzones: []config.Subzone{ {IndexID: uint32(tableDesc.PrimaryIndex.ID), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, {IndexID: uint32(indexDesc.ID), Config: s.(*server.TestServer).Cfg.DefaultZoneConfig}, }, diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index a197a7a41811..5a6d54ea6e4d 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -481,7 +480,7 @@ type nodeStatusGenerator interface { type ExecutorConfig struct { Settings *cluster.Settings NodeInfo - DefaultZoneConfig *zonepb.ZoneConfig + DefaultZoneConfig *config.ZoneConfig Locality roachpb.Locality AmbientCtx log.AmbientContext DB *client.DB diff --git a/pkg/sql/logictest/parallel_test.go b/pkg/sql/logictest/parallel_test.go index 2c525a7a1f7f..5c2c0ebd1dce 100644 --- a/pkg/sql/logictest/parallel_test.go +++ b/pkg/sql/logictest/parallel_test.go @@ -29,7 +29,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" @@ -199,7 +199,7 @@ func (t *parallelTest) setup(spec *parTestSpec) { if testing.Verbose() || log.V(1) { log.Infof(t.ctx, "Setting range split size: %d", spec.RangeSplitSize) } - zoneCfg := zonepb.DefaultZoneConfig() + zoneCfg := config.DefaultZoneConfig() zoneCfg.RangeMaxBytes = proto.Int64(int64(spec.RangeSplitSize)) zoneCfg.RangeMinBytes = proto.Int64(*zoneCfg.RangeMaxBytes / 2) buf, err := protoutil.Marshal(&zoneCfg) diff --git a/pkg/sql/opt/testutils/testcat/create_table.go b/pkg/sql/opt/testutils/testcat/create_table.go index c25131721478..0dec17f2dea5 100644 --- a/pkg/sql/opt/testutils/testcat/create_table.go +++ b/pkg/sql/opt/testutils/testcat/create_table.go @@ -14,7 +14,7 @@ import ( "fmt" "strings" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -384,7 +384,7 @@ func (tt *Table) addIndex(def *tree.IndexTableDef, typ indexType) *Index { IdxName: tt.makeIndexName(def.Name, typ), Unique: typ != nonUniqueIndex, Inverted: def.Inverted, - IdxZone: &zonepb.ZoneConfig{}, + IdxZone: &config.ZoneConfig{}, table: tt, partitionBy: def.PartitionBy, } diff --git a/pkg/sql/opt/testutils/testcat/set_zone_config.go b/pkg/sql/opt/testutils/testcat/set_zone_config.go index 986e59cbb859..fedd6e17ec82 100644 --- a/pkg/sql/opt/testutils/testcat/set_zone_config.go +++ b/pkg/sql/opt/testutils/testcat/set_zone_config.go @@ -13,14 +13,14 @@ package testcat import ( "fmt" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "gopkg.in/yaml.v2" ) // SetZoneConfig is a partial implementation of the ALTER TABLE ... CONFIGURE // ZONE USING statement. -func (tc *Catalog) SetZoneConfig(stmt *tree.SetZoneConfig) *zonepb.ZoneConfig { +func (tc *Catalog) SetZoneConfig(stmt *tree.SetZoneConfig) *config.ZoneConfig { // Update the table name to include catalog and schema if not provided. tabName := stmt.TableOrIndex.Table tc.qualifyTableName(&tabName) @@ -43,12 +43,12 @@ func (tc *Catalog) SetZoneConfig(stmt *tree.SetZoneConfig) *zonepb.ZoneConfig { // makeZoneConfig constructs a ZoneConfig from options provided to the CONFIGURE // ZONE USING statement. -func makeZoneConfig(options tree.KVOptions) *zonepb.ZoneConfig { - zone := &zonepb.ZoneConfig{} +func makeZoneConfig(options tree.KVOptions) *config.ZoneConfig { + zone := &config.ZoneConfig{} for i := range options { switch options[i].Key { case "constraints": - constraintsList := &zonepb.ConstraintsList{} + constraintsList := &config.ConstraintsList{} value := options[i].Value.(*tree.StrVal).RawString() if err := yaml.UnmarshalStrict([]byte(value), constraintsList); err != nil { panic(err) diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index 31302b14414f..342d3f3d4e89 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -16,7 +16,7 @@ import ( "sort" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" @@ -710,7 +710,7 @@ type Index struct { // IdxZone is the zone associated with the index. This may be inherited from // the parent table, database, or even the default zone. - IdxZone *zonepb.ZoneConfig + IdxZone *config.ZoneConfig // Ordinal is the ordinal of this index in the table. ordinal int diff --git a/pkg/sql/opt/xform/coster_test.go b/pkg/sql/opt/xform/coster_test.go index 5bb38376efd7..51bfa401616b 100644 --- a/pkg/sql/opt/xform/coster_test.go +++ b/pkg/sql/opt/xform/coster_test.go @@ -14,7 +14,7 @@ import ( "math" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "gopkg.in/yaml.v2" @@ -68,7 +68,7 @@ func TestLocalityMatchScore(t *testing.T) { } for _, tc := range testCases { - zone := &zonepb.ZoneConfig{} + zone := &config.ZoneConfig{} var locality roachpb.Locality if err := locality.Set(tc.locality); err != nil { @@ -76,7 +76,7 @@ func TestLocalityMatchScore(t *testing.T) { } if tc.constraints != "" { - constraintsList := &zonepb.ConstraintsList{} + constraintsList := &config.ConstraintsList{} if err := yaml.UnmarshalStrict([]byte(tc.constraints), constraintsList); err != nil { t.Fatal(err) } diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index ce887ce83542..12f8a1040a2c 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -16,7 +16,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" @@ -367,7 +366,7 @@ func (oc *optCatalog) dataSourceForTable( return ds, nil } -var emptyZoneConfig = &zonepb.ZoneConfig{} +var emptyZoneConfig = &config.ZoneConfig{} // getZoneConfig returns the ZoneConfig data structure for the given table. // ZoneConfigs are stored in protobuf binary format in the SystemConfig, which @@ -375,7 +374,7 @@ var emptyZoneConfig = &zonepb.ZoneConfig{} // somewhat stale, since it's taken from the gossiped SystemConfig. func (oc *optCatalog) getZoneConfig( desc *sqlbase.ImmutableTableDescriptor, -) (*zonepb.ZoneConfig, error) { +) (*config.ZoneConfig, error) { // Lookup table's zone if system config is available (it may not be as node // is starting up and before it's received the gossiped config). If it is // not available, use an empty config that has no zone constraints. @@ -495,7 +494,7 @@ type optTable struct { // stats are the inlined wrappers for table statistics. stats []optTableStat - zone *zonepb.ZoneConfig + zone *config.ZoneConfig // family is the inlined wrapper for the table's primary family. The primary // family is the first family explicitly specified by the user. If no families @@ -519,7 +518,7 @@ type optTable struct { var _ cat.Table = &optTable{} func newOptTable( - desc *sqlbase.ImmutableTableDescriptor, stats []*stats.TableStatistic, tblZone *zonepb.ZoneConfig, + desc *sqlbase.ImmutableTableDescriptor, stats []*stats.TableStatistic, tblZone *config.ZoneConfig, ) (*optTable, error) { ot := &optTable{ desc: desc, @@ -620,7 +619,7 @@ func (ot *optTable) ID() cat.StableID { // isStale checks if the optTable object needs to be refreshed because the stats // or zone config have changed. False positives are ok. -func (ot *optTable) isStale(tableStats []*stats.TableStatistic, zone *zonepb.ZoneConfig) bool { +func (ot *optTable) isStale(tableStats []*stats.TableStatistic, zone *config.ZoneConfig) bool { // Fast check to verify that the statistics haven't changed: we check the // length and the address of the underlying array. This is not a perfect // check (in principle, the stats could have left the cache and then gotten @@ -663,7 +662,7 @@ func (ot *optTable) Equals(other cat.Object) bool { // Verify that indexes are in same zones. For performance, skip deep equality // check if it's the same as the previous index (common case). - var prevLeftZone, prevRightZone *zonepb.ZoneConfig + var prevLeftZone, prevRightZone *config.ZoneConfig for i := range ot.indexes { leftZone := ot.indexes[i].zone rightZone := otherTable.indexes[i].zone @@ -811,7 +810,7 @@ func (ot *optTable) lookupColumnOrdinal(colID sqlbase.ColumnID) (int, error) { type optIndex struct { tab *optTable desc *sqlbase.IndexDescriptor - zone *zonepb.ZoneConfig + zone *config.ZoneConfig // storedCols is the set of non-PK columns if this is the primary index, // otherwise it is desc.StoreColumnIDs. @@ -828,7 +827,7 @@ var _ cat.Index = &optIndex{} // init can be used instead of newOptIndex when we have a pre-allocated instance // (e.g. as part of a bigger struct). func (oi *optIndex) init( - tab *optTable, indexOrdinal int, desc *sqlbase.IndexDescriptor, zone *zonepb.ZoneConfig, + tab *optTable, indexOrdinal int, desc *sqlbase.IndexDescriptor, zone *config.ZoneConfig, ) { oi.tab = tab oi.desc = desc diff --git a/pkg/sql/partition_test.go b/pkg/sql/partition_test.go index 27767caed98d..db005c1c7c1d 100644 --- a/pkg/sql/partition_test.go +++ b/pkg/sql/partition_test.go @@ -14,7 +14,7 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -82,8 +82,8 @@ func TestRemovePartitioningOSS(t *testing.T) { // Hack in partition zone configs. This also requires a CCL binary to do // properly. - zoneConfig := zonepb.ZoneConfig{ - Subzones: []zonepb.Subzone{ + zoneConfig := config.ZoneConfig{ + Subzones: []config.Subzone{ { IndexID: uint32(tableDesc.PrimaryIndex.ID), PartitionName: "p1", diff --git a/pkg/sql/partition_utils.go b/pkg/sql/partition_utils.go index 09db75e3751d..db7fe13dd8ab 100644 --- a/pkg/sql/partition_utils.go +++ b/pkg/sql/partition_utils.go @@ -14,7 +14,7 @@ import ( "bytes" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -26,7 +26,7 @@ import ( // GenerateSubzoneSpans constructs from a TableDescriptor the entries mapping // zone config spans to subzones for use in the SubzoneSpans field of -// zonepb.ZoneConfig. SubzoneSpans controls which splits are created, so only +// config.ZoneConfig. SubzoneSpans controls which splits are created, so only // the spans corresponding to entries in subzones are returned. // // Zone configs target indexes and partitions via `subzones`, which are attached @@ -71,9 +71,9 @@ func GenerateSubzoneSpans( st *cluster.Settings, clusterID uuid.UUID, tableDesc *sqlbase.TableDescriptor, - subzones []zonepb.Subzone, + subzones []config.Subzone, hasNewSubzones bool, -) ([]zonepb.SubzoneSpan, error) { +) ([]config.SubzoneSpan, error) { // Removing zone configs does not require a valid license. if hasNewSubzones { org := ClusterOrganization.Get(&st.SV) @@ -105,7 +105,7 @@ func GenerateSubzoneSpans( // perspective) it's safe to append them all together. indexCovering = append(indexCovering, covering.Range{ Start: idxSpan.Key, End: idxSpan.EndKey, - Payload: zonepb.Subzone{IndexID: uint32(idxDesc.ID)}, + Payload: config.Subzone{IndexID: uint32(idxDesc.ID)}, }) } @@ -136,18 +136,18 @@ func GenerateSubzoneSpans( // checked in PartitionDescriptor validation. sharedPrefix := keys.MakeTablePrefix(uint32(tableDesc.ID)) - var subzoneSpans []zonepb.SubzoneSpan + var subzoneSpans []config.SubzoneSpan for _, r := range ranges { payloads := r.Payload.([]interface{}) if len(payloads) == 0 { continue } - subzoneSpan := zonepb.SubzoneSpan{ + subzoneSpan := config.SubzoneSpan{ Key: bytes.TrimPrefix(r.Start, sharedPrefix), EndKey: bytes.TrimPrefix(r.End, sharedPrefix), } var ok bool - if subzone := payloads[0].(zonepb.Subzone); len(subzone.PartitionName) > 0 { + if subzone := payloads[0].(config.Subzone); len(subzone.PartitionName) > 0 { subzoneSpan.SubzoneIndex, ok = subzoneIndexByPartition[subzone.PartitionName] } else { subzoneSpan.SubzoneIndex, ok = subzoneIndexByIndexID[sqlbase.IndexID(subzone.IndexID)] @@ -166,7 +166,7 @@ func GenerateSubzoneSpans( // indexCoveringsForPartitioning returns span coverings representing the // partitions in partDesc (including subpartitions). They are sorted with // highest precedence first and the interval.Range payloads are each a -// `zonepb.Subzone` with the PartitionName set. +// `config.Subzone` with the PartitionName set. func indexCoveringsForPartitioning( a *sqlbase.DatumAlloc, tableDesc *sqlbase.TableDescriptor, @@ -201,7 +201,7 @@ func indexCoveringsForPartitioning( if _, ok := relevantPartitions[p.Name]; ok { listCoverings[len(t.Datums)] = append(listCoverings[len(t.Datums)], covering.Range{ Start: keyPrefix, End: roachpb.Key(keyPrefix).PrefixEnd(), - Payload: zonepb.Subzone{PartitionName: p.Name}, + Payload: config.Subzone{PartitionName: p.Name}, }) } newPrefixDatums := append(prefixDatums, t.Datums...) @@ -238,7 +238,7 @@ func indexCoveringsForPartitioning( if _, ok := relevantPartitions[p.Name]; ok { coverings = append(coverings, covering.Covering{{ Start: fromKey, End: toKey, - Payload: zonepb.Subzone{PartitionName: p.Name}, + Payload: config.Subzone{PartitionName: p.Name}, }}) } } diff --git a/pkg/sql/physicalplan/replicaoracle/oracle_test.go b/pkg/sql/physicalplan/replicaoracle/oracle_test.go index 27296efab1b6..eb77b9099202 100644 --- a/pkg/sql/physicalplan/replicaoracle/oracle_test.go +++ b/pkg/sql/physicalplan/replicaoracle/oracle_test.go @@ -17,7 +17,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -97,7 +97,7 @@ func makeGossip(t *testing.T, stopper *stop.Stopper) (*gossip.Gossip, *hlc.Clock server := rpc.NewServer(rpcContext) const nodeID = 1 - g := gossip.NewTest(nodeID, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + g := gossip.NewTest(nodeID, rpcContext, server, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) if err := g.SetNodeDescriptor(newNodeDesc(nodeID)); err != nil { t.Fatal(err) } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index cb000a21c446..a2b19d5b9bbc 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -23,7 +23,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -3008,7 +3008,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT, pi DECIMAL DEFAULT (DECIMAL '3.14 tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") // Add a zone config. - cfg := zonepb.DefaultZoneConfig() + cfg := config.DefaultZoneConfig() buf, err := protoutil.Marshal(&cfg) if err != nil { t.Fatal(err) @@ -3118,7 +3118,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT, pi DECIMAL REFERENCES t.pi (d) DE tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") // Add a zone config. - var cfg zonepb.ZoneConfig + var cfg config.ZoneConfig cfg, err := addImmediateGCZoneConfig(sqlDB, tableDesc.ID) if err != nil { t.Fatal(err) diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 7be900fbd5e9..cc020c0ede0a 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -17,7 +17,6 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -52,19 +51,19 @@ type setZoneConfigNode struct { // supportedZoneConfigOptions indicates how to translate SQL variable // assignments in ALTER CONFIGURE ZONE to assignments to the member -// fields of zonepb.ZoneConfig. +// fields of config.ZoneConfig. var supportedZoneConfigOptions = map[tree.Name]struct { requiredType *types.T - setter func(*zonepb.ZoneConfig, tree.Datum) + setter func(*config.ZoneConfig, tree.Datum) }{ - "range_min_bytes": {types.Int, func(c *zonepb.ZoneConfig, d tree.Datum) { c.RangeMinBytes = proto.Int64(int64(tree.MustBeDInt(d))) }}, - "range_max_bytes": {types.Int, func(c *zonepb.ZoneConfig, d tree.Datum) { c.RangeMaxBytes = proto.Int64(int64(tree.MustBeDInt(d))) }}, - "num_replicas": {types.Int, func(c *zonepb.ZoneConfig, d tree.Datum) { c.NumReplicas = proto.Int32(int32(tree.MustBeDInt(d))) }}, - "gc.ttlseconds": {types.Int, func(c *zonepb.ZoneConfig, d tree.Datum) { - c.GC = &zonepb.GCPolicy{TTLSeconds: int32(tree.MustBeDInt(d))} + "range_min_bytes": {types.Int, func(c *config.ZoneConfig, d tree.Datum) { c.RangeMinBytes = proto.Int64(int64(tree.MustBeDInt(d))) }}, + "range_max_bytes": {types.Int, func(c *config.ZoneConfig, d tree.Datum) { c.RangeMaxBytes = proto.Int64(int64(tree.MustBeDInt(d))) }}, + "num_replicas": {types.Int, func(c *config.ZoneConfig, d tree.Datum) { c.NumReplicas = proto.Int32(int32(tree.MustBeDInt(d))) }}, + "gc.ttlseconds": {types.Int, func(c *config.ZoneConfig, d tree.Datum) { + c.GC = &config.GCPolicy{TTLSeconds: int32(tree.MustBeDInt(d))} }}, - "constraints": {types.String, func(c *zonepb.ZoneConfig, d tree.Datum) { - constraintsList := zonepb.ConstraintsList{ + "constraints": {types.String, func(c *config.ZoneConfig, d tree.Datum) { + constraintsList := config.ConstraintsList{ Constraints: c.Constraints, Inherited: c.InheritedConstraints, } @@ -72,7 +71,7 @@ var supportedZoneConfigOptions = map[tree.Name]struct { c.Constraints = constraintsList.Constraints c.InheritedConstraints = false }}, - "lease_preferences": {types.String, func(c *zonepb.ZoneConfig, d tree.Datum) { + "lease_preferences": {types.String, func(c *config.ZoneConfig, d tree.Datum) { loadYAML(&c.LeasePreferences, string(tree.MustBeDString(d))) c.InheritedLeasePreferences = false }}, @@ -203,7 +202,7 @@ type setZoneConfigRun struct { func (n *setZoneConfigNode) startExec(params runParams) error { var yamlConfig string - var setters []func(c *zonepb.ZoneConfig) + var setters []func(c *config.ZoneConfig) deleteZone := false // Evaluate the configuration input. @@ -262,7 +261,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { "unsupported NULL value for %q", tree.ErrString(name)) } setter := supportedZoneConfigOptions[*name].setter - setters = append(setters, func(c *zonepb.ZoneConfig) { setter(c, datum) }) + setters = append(setters, func(c *config.ZoneConfig) { setter(c, datum) }) if optionStr.Len() > 0 { optionStr.WriteString(", ") } @@ -350,17 +349,17 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // No zone was found. Possibly a SubzonePlaceholder depending on the index. if partialZone == nil { - partialZone = zonepb.NewZoneConfig() + partialZone = config.NewZoneConfig() if index != nil { subzonePlaceholder = true } } - var partialSubzone *zonepb.Subzone + var partialSubzone *config.Subzone if index != nil { partialSubzone = partialZone.GetSubzoneExact(uint32(index.ID), partition) if partialSubzone == nil { - partialSubzone = &zonepb.Subzone{Config: *zonepb.NewZoneConfig()} + partialSubzone = &config.Subzone{Config: *config.NewZoneConfig()} } } @@ -383,8 +382,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // the target ID is not a database object, i.e. one of the system // ranges (liveness, meta, etc.), and did not have a zone config // already. - completeZone = protoutil.Clone( - params.extendedEvalCtx.ExecCfg.DefaultZoneConfig).(*zonepb.ZoneConfig) + completeZone = protoutil.Clone(params.extendedEvalCtx.ExecCfg.DefaultZoneConfig).(*config.ZoneConfig) } else if err != nil { return err } @@ -404,7 +402,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // If we are operating on a zone, get all fields that the zone would // inherit from its parent. We do this by using an empty zoneConfig // and completing at the level of the current zone. - zoneInheritedFields := zonepb.ZoneConfig{} + zoneInheritedFields := config.ZoneConfig{} if err := completeZoneConfig(&zoneInheritedFields, uint32(targetID), getKey); err != nil { return err } @@ -423,7 +421,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { } else { // In the case of updating a partition, we need try inheriting fields // from the subzone's index, and inherit the remainder from the zone. - subzoneInheritedFields := zonepb.ZoneConfig{} + subzoneInheritedFields := config.ZoneConfig{} if indexSubzone := completeZone.GetSubzone(uint32(index.ID), ""); indexSubzone != nil { subzoneInheritedFields.InheritFromParent(&indexSubzone.Config) } @@ -474,10 +472,9 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // ALTER RANGE default USING DEFAULT sets the default to the in // memory default value. if n.setDefault && keys.RootNamespaceID == uint32(targetID) { - finalZone = *protoutil.Clone( - params.extendedEvalCtx.ExecCfg.DefaultZoneConfig).(*zonepb.ZoneConfig) + finalZone = *protoutil.Clone(params.extendedEvalCtx.ExecCfg.DefaultZoneConfig).(*config.ZoneConfig) } else if n.setDefault { - finalZone = *zonepb.NewZoneConfig() + finalZone = *config.NewZoneConfig() } // Load settings from YAML. If there was no YAML (e.g. because the // query specified CONFIGURE ZONE USING), the YAML string will be @@ -562,9 +559,9 @@ func (n *setZoneConfigNode) startExec(params runParams) error { if err != nil { return err } else if completeZone == nil { - completeZone = zonepb.NewZoneConfig() + completeZone = config.NewZoneConfig() } - completeZone.SetSubzone(zonepb.Subzone{ + completeZone.SetSubzone(config.Subzone{ IndexID: uint32(index.ID), PartitionName: partition, Config: newZone, @@ -576,7 +573,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { partialZone.DeleteTableConfig() } - partialZone.SetSubzone(zonepb.Subzone{ + partialZone.SetSubzone(config.Subzone{ IndexID: uint32(index.ID), PartitionName: partition, Config: finalZone, @@ -669,7 +666,7 @@ type nodeGetter func(context.Context, *serverpb.NodesRequest) (*serverpb.NodesRe // constraint. For example, constraints [+region=us-east1,+region=us-east2] // will be rejected. Additionally, invalid constraints such as // [+region=us-east1, -region=us-east1] will also be rejected. -func validateNoRepeatKeysInZone(zone *zonepb.ZoneConfig) error { +func validateNoRepeatKeysInZone(zone *config.ZoneConfig) error { for _, constraints := range zone.Constraints { // Because we expect to have a small number of constraints, a nested // loop is probably better than allocating a map. @@ -683,16 +680,16 @@ func validateNoRepeatKeysInZone(zone *zonepb.ZoneConfig) error { "incompatible zone constraints: %q and %q", curr, other) } } else { - if curr.Type == zonepb.Constraint_REQUIRED { - if other.Type == zonepb.Constraint_REQUIRED && other.Key == curr.Key || - other.Type == zonepb.Constraint_PROHIBITED && other.Key == curr.Key && other.Value == curr.Value { + if curr.Type == config.Constraint_REQUIRED { + if other.Type == config.Constraint_REQUIRED && other.Key == curr.Key || + other.Type == config.Constraint_PROHIBITED && other.Key == curr.Key && other.Value == curr.Value { return pgerror.Newf(pgcode.CheckViolation, "incompatible zone constraints: %q and %q", curr, other) } - } else if curr.Type == zonepb.Constraint_PROHIBITED { + } else if curr.Type == config.Constraint_PROHIBITED { // If we have a -k=v pair, verify that there are not any // +k=v pairs in the constraints. - if other.Type == zonepb.Constraint_REQUIRED && other.Key == curr.Key && other.Value == curr.Value { + if other.Type == config.Constraint_REQUIRED && other.Key == curr.Key && other.Value == curr.Value { return pgerror.Newf(pgcode.CheckViolation, "incompatible zone constraints: %q and %q", curr, other) } @@ -716,7 +713,7 @@ func validateNoRepeatKeysInZone(zone *zonepb.ZoneConfig) error { // the cluster. If you had to first add one of the nodes before creating the // constraints, data could be replicated there that shouldn't be. func validateZoneAttrsAndLocalities( - ctx context.Context, getNodes nodeGetter, zone *zonepb.ZoneConfig, + ctx context.Context, getNodes nodeGetter, zone *config.ZoneConfig, ) error { if len(zone.Constraints) == 0 && len(zone.LeasePreferences) == 0 { return nil @@ -730,8 +727,8 @@ func validateZoneAttrsAndLocalities( } // Accumulate a unique list of constraints to validate. - toValidate := make([]zonepb.Constraint, 0) - addToValidate := func(c zonepb.Constraint) { + toValidate := make([]config.Constraint, 0) + addToValidate := func(c config.Constraint) { var alreadyInList bool for _, val := range toValidate { if c == val { @@ -757,18 +754,18 @@ func validateZoneAttrsAndLocalities( // Check that each constraint matches some store somewhere in the cluster. for _, constraint := range toValidate { // We skip validation for negative constraints. See the function-level comment. - if constraint.Type == zonepb.Constraint_PROHIBITED { + if constraint.Type == config.Constraint_PROHIBITED { continue } var found bool node: for _, node := range nodes.Nodes { for _, store := range node.StoreStatuses { - // We could alternatively use zonepb.StoreMatchesConstraint here to + // We could alternatively use config.StoreMatchesConstraint here to // catch typos in prohibited constraints as well, but as noted in the // function-level comment that could break very reasonable use cases for // prohibited constraints. - if zonepb.StoreSatisfiesConstraint(store.Desc, constraint) { + if config.StoreSatisfiesConstraint(store.Desc, constraint) { found = true break node } @@ -789,7 +786,7 @@ func writeZoneConfig( txn *client.Txn, targetID sqlbase.ID, table *sqlbase.TableDescriptor, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, execCfg *ExecutorConfig, hasNewSubzones bool, ) (numAffected int, err error) { @@ -824,7 +821,7 @@ func writeZoneConfig( // zone config exists for the given ID, it returns nil. func getZoneConfigRaw( ctx context.Context, txn *client.Txn, id sqlbase.ID, -) (*zonepb.ZoneConfig, error) { +) (*config.ZoneConfig, error) { kv, err := txn.Get(ctx, config.MakeZoneKey(uint32(id))) if err != nil { return nil, err @@ -832,7 +829,7 @@ func getZoneConfigRaw( if kv.Value == nil { return nil, nil } - var zone zonepb.ZoneConfig + var zone config.ZoneConfig if err := kv.ValueProto(&zone); err != nil { return nil, err } @@ -855,7 +852,7 @@ func removeIndexZoneConfigs( if err != nil { return err } else if zone == nil { - zone = zonepb.NewZoneConfig() + zone = config.NewZoneConfig() } for _, indexDesc := range indexDescs { diff --git a/pkg/sql/set_zone_config_test.go b/pkg/sql/set_zone_config_test.go index 7419f7a1bb4a..9406eedac73c 100644 --- a/pkg/sql/set_zone_config_test.go +++ b/pkg/sql/set_zone_config_test.go @@ -14,7 +14,7 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" @@ -44,7 +44,7 @@ func TestValidateNoRepeatKeysInZone(t *testing.T) { } for _, tc := range testCases { - var zone zonepb.ZoneConfig + var zone config.ZoneConfig err := yaml.UnmarshalStrict([]byte(tc.constraint), &zone) if err != nil { t.Fatal(err) @@ -170,7 +170,7 @@ func TestValidateZoneAttrsAndLocalities(t *testing.T) { {`constraints: ["-ssd"]`, expectSuccess, getNodes}, {`constraints: ["-fake"]`, expectSuccess, getNodes}, } { - var zone zonepb.ZoneConfig + var zone config.ZoneConfig err := yaml.UnmarshalStrict([]byte(tc.cfg), &zone) if err != nil && tc.expectErr == expectSuccess { t.Fatalf("#%d: expected success for %q; got %v", i, tc.cfg, err) diff --git a/pkg/sql/show_trace_replica_test.go b/pkg/sql/show_trace_replica_test.go index 38d7ff48ce46..dc3598086ac4 100644 --- a/pkg/sql/show_trace_replica_test.go +++ b/pkg/sql/show_trace_replica_test.go @@ -17,7 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -35,7 +35,7 @@ func TestShowTraceReplica(t *testing.T) { const numNodes = 4 - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() zoneConfig.NumReplicas = proto.Int32(1) ctx := context.Background() diff --git a/pkg/sql/show_zone_config.go b/pkg/sql/show_zone_config.go index f5c271cc5afa..a3ef7eaa3f90 100644 --- a/pkg/sql/show_zone_config.go +++ b/pkg/sql/show_zone_config.go @@ -15,7 +15,7 @@ import ( "context" "strings" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -156,8 +156,8 @@ func getShowZoneConfigRow( } // zoneConfigToSQL pretty prints a zone configuration as a SQL string. -func zoneConfigToSQL(zs *tree.ZoneSpecifier, zone *zonepb.ZoneConfig) (string, error) { - constraints, err := yamlMarshalFlow(zonepb.ConstraintsList{ +func zoneConfigToSQL(zs *tree.ZoneSpecifier, zone *config.ZoneConfig) (string, error) { + constraints, err := yamlMarshalFlow(config.ConstraintsList{ Constraints: zone.Constraints, Inherited: zone.InheritedConstraints}) if err != nil { @@ -221,8 +221,8 @@ func generateZoneConfigIntrospectionValues( zoneID tree.Datum, subZoneID tree.Datum, zs *tree.ZoneSpecifier, - zone *zonepb.ZoneConfig, - fullZoneConfig *zonepb.ZoneConfig, + zone *config.ZoneConfig, + fullZoneConfig *config.ZoneConfig, ) error { // Populate the ID column. values[zoneIDCol] = zoneID @@ -336,12 +336,12 @@ func yamlMarshalFlow(v interface{}) (string, error) { // TODO(benesch): Teach GetZoneConfig to return the specifier of the zone it // finds without impacting performance. func ascendZoneSpecifier( - zs tree.ZoneSpecifier, resolvedID, actualID uint32, actualSubzone *zonepb.Subzone, + zs tree.ZoneSpecifier, resolvedID, actualID uint32, actualSubzone *config.Subzone, ) tree.ZoneSpecifier { if actualID == keys.RootNamespaceID { // We had to traverse to the top of the hierarchy, so we're showing the // default zone config. - zs.NamedZone = zonepb.DefaultZoneName + zs.NamedZone = config.DefaultZoneName zs.Database = "" zs.TableOrIndex = tree.TableIndexName{} // Since the default zone has no partition, we can erase the diff --git a/pkg/sql/sqlbase/keys.go b/pkg/sql/sqlbase/keys.go index 5214ea2b4b1e..e33225708904 100644 --- a/pkg/sql/sqlbase/keys.go +++ b/pkg/sql/sqlbase/keys.go @@ -34,12 +34,15 @@ func MakeNameMetadataKey(parentID ID, name string) roachpb.Key { // MakeAllDescsMetadataKey returns the key for all descriptors. func MakeAllDescsMetadataKey() roachpb.Key { - return keys.DescMetadataPrefix() + k := keys.MakeTablePrefix(uint32(DescriptorTable.ID)) + return encoding.EncodeUvarintAscending(k, uint64(DescriptorTable.PrimaryIndex.ID)) } // MakeDescMetadataKey returns the key for the descriptor. func MakeDescMetadataKey(descID ID) roachpb.Key { - return keys.DescMetadataKey(uint32(descID)) + k := MakeAllDescsMetadataKey() + k = encoding.EncodeUvarintAscending(k, uint64(descID)) + return keys.MakeFamilyKey(k, uint32(DescriptorTable.Columns[1].ID)) } // IndexKeyValDirs returns the corresponding encoding.Directions for all the diff --git a/pkg/sql/sqlbase/metadata.go b/pkg/sql/sqlbase/metadata.go index 109627def03a..f0483f460d36 100644 --- a/pkg/sql/sqlbase/metadata.go +++ b/pkg/sql/sqlbase/metadata.go @@ -15,7 +15,7 @@ import ( "fmt" "sort" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -81,7 +81,7 @@ type metadataDescriptor struct { // MakeMetadataSchema constructs a new MetadataSchema value which constructs // the "system" database. func MakeMetadataSchema( - defaultZoneConfig *zonepb.ZoneConfig, defaultSystemZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, defaultSystemZoneConfig *config.ZoneConfig, ) MetadataSchema { ms := MetadataSchema{} addSystemDatabaseToSchema(&ms, defaultZoneConfig, defaultSystemZoneConfig) diff --git a/pkg/sql/sqlbase/system.go b/pkg/sql/sqlbase/system.go index 9e8d6acb8af9..7fecd1b15455 100644 --- a/pkg/sql/sqlbase/system.go +++ b/pkg/sql/sqlbase/system.go @@ -14,7 +14,7 @@ import ( "fmt" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" @@ -23,6 +23,35 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) +func init() { + // We use a hook to avoid a dependency on the sqlbase package. We + // should probably move keys/protos elsewhere. + config.SplitAtIDHook = SplitAtIDHook +} + +// SplitAtIDHook determines whether a specific descriptor ID +// should be considered for a split at all. If it is a database +// or a view table descriptor, it should not be considered. +func SplitAtIDHook(id uint32, cfg *config.SystemConfig) bool { + descVal := cfg.GetDesc(MakeDescMetadataKey(ID(id))) + if descVal == nil { + return false + } + var desc Descriptor + if err := descVal.GetProto(&desc); err != nil { + return false + } + if dbDesc := desc.GetDatabase(); dbDesc != nil { + return false + } + if tableDesc := desc.Table(descVal.Timestamp); tableDesc != nil { + if viewStr := tableDesc.GetViewQuery(); viewStr != "" { + return false + } + } + return true +} + // sql CREATE commands and full schema for each system table. // These strings are *not* used at runtime, but are checked by the // `TestSystemTableLiterals` test that compares the table generated by @@ -307,15 +336,12 @@ var ( Version: 1, Columns: []ColumnDescriptor{ {Name: "id", ID: 1, Type: *types.Int}, - {Name: "descriptor", ID: keys.DescriptorTableDescriptorColID, Type: *types.Bytes, Nullable: true}, + {Name: "descriptor", ID: 2, Type: *types.Bytes, Nullable: true}, }, NextColumnID: 3, Families: []ColumnFamilyDescriptor{ - // The id of the first col fam is hardcoded in keys.MakeDescMetadataKey(). {Name: "primary", ID: 0, ColumnNames: []string{"id"}, ColumnIDs: singleID1}, - {Name: "fam_2_descriptor", ID: keys.DescriptorTableDescriptorColFamID, - ColumnNames: []string{"descriptor"}, - ColumnIDs: []ColumnID{keys.DescriptorTableDescriptorColID}, DefaultColumnID: keys.DescriptorTableDescriptorColID}, + {Name: "fam_2_descriptor", ID: 2, ColumnNames: []string{"descriptor"}, ColumnIDs: []ColumnID{2}, DefaultColumnID: 2}, }, PrimaryIndex: pk("id"), NextFamilyID: 3, @@ -1027,13 +1053,13 @@ var ( ) // Create a kv pair for the zone config for the given key and config value. -func createZoneConfigKV(keyID int, zoneConfig *zonepb.ZoneConfig) roachpb.KeyValue { +func createZoneConfigKV(keyID int, zoneConfig *config.ZoneConfig) roachpb.KeyValue { value := roachpb.Value{} if err := value.SetProto(zoneConfig); err != nil { panic(fmt.Sprintf("could not marshal ZoneConfig for ID: %d: %s", keyID, err)) } return roachpb.KeyValue{ - Key: keys.ZoneKey(uint32(keyID)), + Key: config.MakeZoneKey(uint32(keyID)), Value: value, } } @@ -1079,8 +1105,8 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) { // System database, its tables and zone configurations. func addSystemDatabaseToSchema( target *MetadataSchema, - defaultZoneConfig *zonepb.ZoneConfig, - defaultSystemZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, + defaultSystemZoneConfig *config.ZoneConfig, ) { addSystemDescriptorsToSchema(target) @@ -1093,9 +1119,9 @@ func addSystemDatabaseToSchema( target.otherKV = append(target.otherKV, createZoneConfigKV(keys.RootNamespaceID, defaultZoneConfig)) systemZoneConf := defaultSystemZoneConfig - metaRangeZoneConf := protoutil.Clone(defaultSystemZoneConfig).(*zonepb.ZoneConfig) - jobsZoneConf := protoutil.Clone(defaultSystemZoneConfig).(*zonepb.ZoneConfig) - livenessZoneConf := protoutil.Clone(defaultSystemZoneConfig).(*zonepb.ZoneConfig) + metaRangeZoneConf := protoutil.Clone(defaultSystemZoneConfig).(*config.ZoneConfig) + jobsZoneConf := protoutil.Clone(defaultSystemZoneConfig).(*config.ZoneConfig) + livenessZoneConf := protoutil.Clone(defaultSystemZoneConfig).(*config.ZoneConfig) // .meta zone config entry with a shorter GC time. metaRangeZoneConf.GC.TTLSeconds = 60 * 60 // 1h @@ -1106,11 +1132,11 @@ func addSystemDatabaseToSchema( target.otherKV = append(target.otherKV, createZoneConfigKV(keys.JobsTableID, jobsZoneConf)) // Some reporting tables have shorter GC times. - replicationConstraintStatsZoneConf := &zonepb.ZoneConfig{ - GC: &zonepb.GCPolicy{TTLSeconds: int32(ReplicationConstraintStatsTableTTL.Seconds())}, + replicationConstraintStatsZoneConf := &config.ZoneConfig{ + GC: &config.GCPolicy{TTLSeconds: int32(ReplicationConstraintStatsTableTTL.Seconds())}, } - replicationStatsZoneConf := &zonepb.ZoneConfig{ - GC: &zonepb.GCPolicy{TTLSeconds: int32(ReplicationStatsTableTTL.Seconds())}, + replicationStatsZoneConf := &config.ZoneConfig{ + GC: &config.GCPolicy{TTLSeconds: int32(ReplicationStatsTableTTL.Seconds())}, } // Liveness zone config entry with a shorter GC time. diff --git a/pkg/sql/sqlbase/table.go b/pkg/sql/sqlbase/table.go index c6f6bfc17ec0..2400464bb97a 100644 --- a/pkg/sql/sqlbase/table.go +++ b/pkg/sql/sqlbase/table.go @@ -14,9 +14,7 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -491,44 +489,3 @@ func ConditionalGetTableDescFromTxn( } return existingKV.Value, nil } - -// SplitKeysForTable computes the split keys for a given table descriptor, -// taking into account all its partitions and their zone configs. -// The descriptor is taken as a raw Value, as the config package needs to invoke -// this. If the descriptor represents indeed a table, at least one split point -// is returned (the start of the table). If the descriptor that's passed in is -// not a table (i.e. it's a database or a view), then nil is returned. -// -// zone is the zone config for the table. Can be nil if no zone config has been -// configured. -// -// The split keys are returned sorted. -// An error is returned iff a descVal is not a descriptor. -func SplitKeysForTable(descVal *roachpb.Value, zone *zonepb.ZoneConfig) ([]roachpb.RKey, error) { - var desc Descriptor - if err := descVal.GetProto(&desc); err != nil { - return nil, errors.AssertionFailedf("failed to decode descriptor") - } - - table := desc.Table(descVal.Timestamp) - if table == nil { - // Databases don't require splits. - return nil, nil - } - if viewStr := table.GetViewQuery(); viewStr != "" { - // Views don't require splits. - return nil, nil - } - tableKey := roachpb.RKey(keys.MakeTablePrefix(uint32(table.ID))) - if zone == nil { - return []roachpb.RKey{tableKey}, nil - } - subzoneSplits := zone.SubzoneSplits() - splits := make([]roachpb.RKey, len(subzoneSplits)+1) - splits[0] = tableKey - for i, s := range subzoneSplits { - // Prepend the table prefix to the subzone splits. - splits[i+1] = append(append([]byte(nil), tableKey...), s...) - } - return splits, nil -} diff --git a/pkg/sql/sqlbase/table_test.go b/pkg/sql/sqlbase/table_test.go index b3c5fe76d139..f8a53a97aaa4 100644 --- a/pkg/sql/sqlbase/table_test.go +++ b/pkg/sql/sqlbase/table_test.go @@ -21,7 +21,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -31,14 +30,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/timeofday" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" "github.com/pkg/errors" - "github.com/stretchr/testify/require" ) type indexKeyTest struct { @@ -1509,55 +1506,3 @@ func TestDecodeTableValue(t *testing.T) { }) } } - -func TestSplitKeysForTable(t *testing.T) { - var val roachpb.Value - err := val.SetProto(&Descriptor{ - Union: &Descriptor_Table{ - Table: &TableDescriptor{ - // Fill in the descriptor just enough for the test to work. - ID: ID(50), - }, - }, - }) - require.NoError(t, err) - // We need to set some timestamp on this proto, otherwise unwrapping the - // descriptor fatals. - val.Timestamp = hlc.Timestamp{WallTime: 123} - - indexKey := encoding.EncodeUvarintAscending(nil, 1) - // Remove any excess capacity, assuring that all the following encodes will - // create new slices. - indexKey = indexKey[:len(indexKey):len(indexKey)] - k1 := encoding.EncodeUvarintAscending(indexKey, 10) - k2 := encoding.EncodeUvarintAscending(indexKey, 20) - k3 := encoding.EncodeUvarintAscending(indexKey, 30) - k4 := encoding.EncodeUvarintAscending(indexKey, 40) - zone := zonepb.ZoneConfig{ - SubzoneSpans: []zonepb.SubzoneSpan{{ - Key: k1, - EndKey: k2, - SubzoneIndex: 0, - }, { - Key: k3, - EndKey: k4, - SubzoneIndex: 1, - }}, - } - splits, err := SplitKeysForTable(&val, &zone) - require.NoError(t, err) - - tableKey := encoding.EncodeUvarintAscending(nil, 50) - tableAndIndexKey := EncodeTableIDIndexID(nil, 50, 1) - // Remove any excess capacity, assuring that all the following encodes will - // create new slices. - tableAndIndexKey = tableAndIndexKey[:len(tableAndIndexKey):len(tableAndIndexKey)] - exp := []roachpb.RKey{ - tableKey, - encoding.EncodeUvarintAscending(tableAndIndexKey, 10), - encoding.EncodeUvarintAscending(tableAndIndexKey, 20), - encoding.EncodeUvarintAscending(tableAndIndexKey, 30), - encoding.EncodeUvarintAscending(tableAndIndexKey, 40), - } - require.Equal(t, exp, splits) -} diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 68b4fa269451..0355dd257730 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -15,7 +15,7 @@ import ( "strings" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" @@ -32,7 +32,7 @@ func TestInitialKeys(t *testing.T) { const keysPerDesc = 2 const nonDescKeys = 9 - ms := sqlbase.MakeMetadataSchema(zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + ms := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) kv, _ /* splits */ := ms.GetInitialValues() expected := nonDescKeys + keysPerDesc*ms.SystemDescriptorCount() if actual := len(kv); actual != expected { diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 0ccebafc5b94..4d9972af2bb3 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -15,7 +15,6 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -46,8 +45,8 @@ var errNoZoneConfigApplies = errors.New("no zone config applies") // ignored, and the default that would apply if it did not exist is returned instead. func getZoneConfig( id uint32, getKey func(roachpb.Key) (*roachpb.Value, error), getInheritedDefault bool, -) (uint32, *zonepb.ZoneConfig, uint32, *zonepb.ZoneConfig, error) { - var placeholder *zonepb.ZoneConfig +) (uint32, *config.ZoneConfig, uint32, *config.ZoneConfig, error) { + var placeholder *config.ZoneConfig var placeholderID uint32 if !getInheritedDefault { // Look in the zones table. @@ -55,7 +54,7 @@ func getZoneConfig( return 0, nil, 0, nil, err } else if zoneVal != nil { // We found a matching entry. - var zone zonepb.ZoneConfig + var zone config.ZoneConfig if err := zoneVal.GetProto(&zone); err != nil { return 0, nil, 0, nil, err } @@ -109,7 +108,7 @@ func getZoneConfig( // NOTE: This will not work for subzones. To complete subzones, find a complete // parent zone (index or table) and apply InheritFromParent to it. func completeZoneConfig( - cfg *zonepb.ZoneConfig, id uint32, getKey func(roachpb.Key) (*roachpb.Value, error), + cfg *config.ZoneConfig, id uint32, getKey func(roachpb.Key) (*roachpb.Value, error), ) error { if cfg.IsComplete() { return nil @@ -150,7 +149,7 @@ func completeZoneConfig( // cached. func ZoneConfigHook( cfg *config.SystemConfig, id uint32, -) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) { +) (*config.ZoneConfig, *config.ZoneConfig, bool, error) { getKey := func(key roachpb.Key) (*roachpb.Value, error) { return cfg.GetValue(key), nil } @@ -176,7 +175,7 @@ func GetZoneConfigInTxn( index *sqlbase.IndexDescriptor, partition string, getInheritedDefault bool, -) (uint32, *zonepb.ZoneConfig, *zonepb.Subzone, error) { +) (uint32, *config.ZoneConfig, *config.Subzone, error) { getKey := func(key roachpb.Key) (*roachpb.Value, error) { kv, err := txn.Get(ctx, key) if err != nil { @@ -192,7 +191,7 @@ func GetZoneConfigInTxn( if err = completeZoneConfig(zone, zoneID, getKey); err != nil { return 0, nil, nil, err } - var subzone *zonepb.Subzone + var subzone *config.Subzone if index != nil { if placeholder != nil { if subzone = placeholder.GetSubzone(uint32(index.ID), partition); subzone != nil { @@ -261,7 +260,7 @@ func (p *planner) resolveTableForZone( // responsibility to do this using e.g .resolveTableForZone(). func resolveZone(ctx context.Context, txn *client.Txn, zs *tree.ZoneSpecifier) (sqlbase.ID, error) { errMissingKey := errors.New("missing key") - id, err := zonepb.ResolveZoneSpecifier(zs, + id, err := config.ResolveZoneSpecifier(zs, func(parentID uint32, name string) (uint32, error) { kv, err := txn.Get(ctx, sqlbase.MakeNameMetadataKey(sqlbase.ID(parentID), name)) if err != nil { @@ -342,7 +341,7 @@ func deleteRemovedPartitionZoneConfigs( if err != nil { return err } else if zone == nil { - zone = zonepb.NewZoneConfig() + zone = config.NewZoneConfig() } for _, n := range removedNames { zone.DeleteSubzone(uint32(idxDesc.ID), n) diff --git a/pkg/sql/zone_config_test.go b/pkg/sql/zone_config_test.go index a8e3ec5e238f..97344a0dcd26 100644 --- a/pkg/sql/zone_config_test.go +++ b/pkg/sql/zone_config_test.go @@ -15,7 +15,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/server" @@ -87,11 +86,11 @@ func waitForConfigChange(t testing.TB, s *server.TestServer) *config.SystemConfi func TestGetZoneConfig(t *testing.T) { defer leaktest.AfterTest(t)() params, _ := tests.CreateTestServerParams() - defaultZoneConfig := zonepb.DefaultSystemZoneConfig() + defaultZoneConfig := config.DefaultSystemZoneConfig() defaultZoneConfig.NumReplicas = proto.Int32(1) defaultZoneConfig.RangeMinBytes = proto.Int64(1 << 20) defaultZoneConfig.RangeMaxBytes = proto.Int64(1 << 21) - defaultZoneConfig.GC = &zonepb.GCPolicy{TTLSeconds: 60} + defaultZoneConfig.GC = &config.GCPolicy{TTLSeconds: 60} require.NoError(t, defaultZoneConfig.Validate()) params.Knobs.Server = &server.TestingKnobs{ DefaultZoneConfigOverride: &defaultZoneConfig, @@ -111,7 +110,7 @@ func TestGetZoneConfig(t *testing.T) { keySuffix []byte partitionName string - zoneCfg zonepb.ZoneConfig + zoneCfg config.ZoneConfig } verifyZoneConfigs := func(testCases []testCase) { cfg := forceNewConfig(t, s) @@ -235,28 +234,28 @@ func TestGetZoneConfig(t *testing.T) { db1Cfg := defaultZoneConfig db1Cfg.NumReplicas = proto.Int32(1) - db1Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db1"}}}} + db1Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db1"}}}} tb11Cfg := defaultZoneConfig tb11Cfg.NumReplicas = proto.Int32(1) - tb11Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db1.tb1"}}}} + tb11Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db1.tb1"}}}} p211Cfg := defaultZoneConfig p211Cfg.NumReplicas = proto.Int32(1) - p211Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p1"}}}} + p211Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1.p1"}}}} p212Cfg := defaultZoneConfig p212Cfg.NumReplicas = proto.Int32(1) - p212Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p2"}}}} + p212Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1.p2"}}}} tb21Cfg := defaultZoneConfig tb21Cfg.NumReplicas = proto.Int32(1) - tb21Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} - tb21Cfg.Subzones = []zonepb.Subzone{ + tb21Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1"}}}} + tb21Cfg.Subzones = []config.Subzone{ {PartitionName: "p0", Config: p211Cfg}, {PartitionName: "p1", Config: p212Cfg}, } - tb21Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ + tb21Cfg.SubzoneSpans = []config.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}}, {SubzoneIndex: 1, Key: []byte{3}, EndKey: []byte{5}}, {SubzoneIndex: 0, Key: []byte{6}}, @@ -264,17 +263,17 @@ func TestGetZoneConfig(t *testing.T) { p221Cfg := defaultZoneConfig p221Cfg.NumReplicas = proto.Int32(1) - p221Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb2.p1"}}}} + p221Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb2.p1"}}}} // Subzone Placeholder - tb22Cfg := *zonepb.NewZoneConfig() + tb22Cfg := *config.NewZoneConfig() tb22Cfg.NumReplicas = proto.Int32(0) - tb22Cfg.Subzones = []zonepb.Subzone{{PartitionName: "p0", Config: p221Cfg}} - tb22Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ + tb22Cfg.Subzones = []config.Subzone{{PartitionName: "p0", Config: p221Cfg}} + tb22Cfg.SubzoneSpans = []config.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}, EndKey: []byte{255}}, } - for objID, objZone := range map[uint32]zonepb.ZoneConfig{ + for objID, objZone := range map[uint32]config.ZoneConfig{ db1: db1Cfg, tb11: tb11Cfg, tb21: tb21Cfg, @@ -323,11 +322,11 @@ func TestCascadingZoneConfig(t *testing.T) { defer leaktest.AfterTest(t)() params, _ := tests.CreateTestServerParams() - defaultZoneConfig := zonepb.DefaultZoneConfig() + defaultZoneConfig := config.DefaultZoneConfig() defaultZoneConfig.NumReplicas = proto.Int32(1) defaultZoneConfig.RangeMinBytes = proto.Int64(1 << 20) defaultZoneConfig.RangeMaxBytes = proto.Int64(1 << 21) - defaultZoneConfig.GC = &zonepb.GCPolicy{TTLSeconds: 60} + defaultZoneConfig.GC = &config.GCPolicy{TTLSeconds: 60} require.NoError(t, defaultZoneConfig.Validate()) params.Knobs.Server = &server.TestingKnobs{ DefaultZoneConfigOverride: &defaultZoneConfig, @@ -347,7 +346,7 @@ func TestCascadingZoneConfig(t *testing.T) { keySuffix []byte partitionName string - zoneCfg zonepb.ZoneConfig + zoneCfg config.ZoneConfig } verifyZoneConfigs := func(testCases []testCase) { cfg := forceNewConfig(t, s) @@ -468,50 +467,50 @@ func TestCascadingZoneConfig(t *testing.T) { // tb2: no zone config // p1: true [1, 255) - inherits replciation factor from default - db1Cfg := *zonepb.NewZoneConfig() + db1Cfg := *config.NewZoneConfig() db1Cfg.NumReplicas = proto.Int32(5) - db1Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db1"}}}} + db1Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db1"}}}} db1Cfg.InheritedConstraints = false // Expected complete config expectedDb1Cfg := defaultZoneConfig expectedDb1Cfg.NumReplicas = proto.Int32(5) - expectedDb1Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db1"}}}} + expectedDb1Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db1"}}}} - tb11Cfg := *zonepb.NewZoneConfig() - tb11Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db1.tb1"}}}} + tb11Cfg := *config.NewZoneConfig() + tb11Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db1.tb1"}}}} tb11Cfg.InheritedConstraints = false // Expected complete config expectedTb11Cfg := expectedDb1Cfg - expectedTb11Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db1.tb1"}}}} + expectedTb11Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db1.tb1"}}}} - p211Cfg := *zonepb.NewZoneConfig() + p211Cfg := *config.NewZoneConfig() p211Cfg.NumReplicas = proto.Int32(1) - p211Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p1"}}}} + p211Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1.p1"}}}} p211Cfg.InheritedConstraints = false // Expected complete config expectedP211Cfg := defaultZoneConfig expectedP211Cfg.NumReplicas = proto.Int32(1) - expectedP211Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p1"}}}} + expectedP211Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1.p1"}}}} - p212Cfg := *zonepb.NewZoneConfig() - p212Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p2"}}}} + p212Cfg := *config.NewZoneConfig() + p212Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1.p2"}}}} p212Cfg.InheritedConstraints = false // Expected complete config expectedP212Cfg := defaultZoneConfig - expectedP212Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p2"}}}} + expectedP212Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1.p2"}}}} - tb21Cfg := *zonepb.NewZoneConfig() - tb21Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} + tb21Cfg := *config.NewZoneConfig() + tb21Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1"}}}} tb21Cfg.InheritedConstraints = false - tb21Cfg.Subzones = []zonepb.Subzone{ + tb21Cfg.Subzones = []config.Subzone{ {PartitionName: "p0", Config: p211Cfg}, {PartitionName: "p1", Config: p212Cfg}, } - tb21Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ + tb21Cfg.SubzoneSpans = []config.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}}, {SubzoneIndex: 1, Key: []byte{3}, EndKey: []byte{5}}, {SubzoneIndex: 0, Key: []byte{6}}, @@ -519,34 +518,34 @@ func TestCascadingZoneConfig(t *testing.T) { // Expected complete config expectedTb21Cfg := defaultZoneConfig - expectedTb21Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} - expectedTb21Cfg.Subzones = []zonepb.Subzone{ + expectedTb21Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb1"}}}} + expectedTb21Cfg.Subzones = []config.Subzone{ {PartitionName: "p0", Config: p211Cfg}, {PartitionName: "p1", Config: p212Cfg}, } - expectedTb21Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ + expectedTb21Cfg.SubzoneSpans = []config.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}}, {SubzoneIndex: 1, Key: []byte{3}, EndKey: []byte{5}}, {SubzoneIndex: 0, Key: []byte{6}}, } - p221Cfg := *zonepb.NewZoneConfig() - p221Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb2.p1"}}}} + p221Cfg := *config.NewZoneConfig() + p221Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb2.p1"}}}} p221Cfg.InheritedConstraints = false // Expected complete config expectedP221Cfg := defaultZoneConfig - expectedP221Cfg.Constraints = []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "db2.tb2.p1"}}}} + expectedP221Cfg.Constraints = []config.Constraints{{Constraints: []config.Constraint{{Value: "db2.tb2.p1"}}}} // Subzone Placeholder - tb22Cfg := *zonepb.NewZoneConfig() + tb22Cfg := *config.NewZoneConfig() tb22Cfg.NumReplicas = proto.Int32(0) - tb22Cfg.Subzones = []zonepb.Subzone{{PartitionName: "p0", Config: p221Cfg}} - tb22Cfg.SubzoneSpans = []zonepb.SubzoneSpan{ + tb22Cfg.Subzones = []config.Subzone{{PartitionName: "p0", Config: p221Cfg}} + tb22Cfg.SubzoneSpans = []config.SubzoneSpan{ {SubzoneIndex: 0, Key: []byte{1}, EndKey: []byte{255}}, } - for objID, objZone := range map[uint32]zonepb.ZoneConfig{ + for objID, objZone := range map[uint32]config.ZoneConfig{ db1: db1Cfg, tb11: tb11Cfg, tb21: tb21Cfg, diff --git a/pkg/sql/zone_test.go b/pkg/sql/zone_test.go index 8d561989b96b..3b454ccdc869 100644 --- a/pkg/sql/zone_test.go +++ b/pkg/sql/zone_test.go @@ -15,7 +15,7 @@ import ( "fmt" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql/lex" @@ -40,9 +40,9 @@ func TestValidSetShowZones(t *testing.T) { yamlDefault := fmt.Sprintf("gc: {ttlseconds: %d}", s.(*server.TestServer).Cfg.DefaultZoneConfig.GC.TTLSeconds) yamlOverride := "gc: {ttlseconds: 42}" zoneOverride := s.(*server.TestServer).Cfg.DefaultZoneConfig - zoneOverride.GC = &zonepb.GCPolicy{TTLSeconds: 42} - partialZoneOverride := *zonepb.NewZoneConfig() - partialZoneOverride.GC = &zonepb.GCPolicy{TTLSeconds: 42} + zoneOverride.GC = &config.GCPolicy{TTLSeconds: 42} + partialZoneOverride := *config.NewZoneConfig() + partialZoneOverride.GC = &config.GCPolicy{TTLSeconds: 42} defaultRow := sqlutils.ZoneRow{ ID: keys.RootNamespaceID, diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go index 9f1fc86a415d..0daa9e04e0cd 100644 --- a/pkg/sqlmigrations/migrations.go +++ b/pkg/sqlmigrations/migrations.go @@ -17,7 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -353,8 +353,8 @@ func NewManager( func ExpectedDescriptorIDs( ctx context.Context, db db, - defaultZoneConfig *zonepb.ZoneConfig, - defaultSystemZoneConfig *zonepb.ZoneConfig, + defaultZoneConfig *config.ZoneConfig, + defaultSystemZoneConfig *config.ZoneConfig, ) (sqlbase.IDs, error) { completedMigrations, err := getCompletedMigrations(ctx, db) if err != nil { diff --git a/pkg/sqlmigrations/migrations_test.go b/pkg/sqlmigrations/migrations_test.go index efabdc5c9184..0353c17e6b94 100644 --- a/pkg/sqlmigrations/migrations_test.go +++ b/pkg/sqlmigrations/migrations_test.go @@ -619,7 +619,7 @@ func TestExpectedInitialRangeCount(t *testing.T) { if err := rows.Scan(&rangeID, &startKey, &endKey); err != nil { return err } - if sysCfg.NeedsSplit(ctx, startKey, endKey) { + if sysCfg.NeedsSplit(startKey, endKey) { return fmt.Errorf("range %d needs split", rangeID) } nranges++ diff --git a/pkg/storage/allocator.go b/pkg/storage/allocator.go index ff1b4b90f6e8..b8a64b6650a3 100644 --- a/pkg/storage/allocator.go +++ b/pkg/storage/allocator.go @@ -19,7 +19,7 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -142,7 +142,7 @@ const ( // can be retried quickly as soon as new stores come online, or additional // space frees up. type allocatorError struct { - constraints []zonepb.Constraints + constraints []config.Constraints existingReplicas int aliveStores int throttledStores int @@ -300,7 +300,7 @@ func GetNeededReplicas(zoneConfigReplicaCount int32, clusterNodes int) int { // supplied range, as governed by the supplied zone configuration. It // returns the required action that should be taken and a priority. func (a *Allocator) ComputeAction( - ctx context.Context, zone *zonepb.ZoneConfig, desc *roachpb.RangeDescriptor, + ctx context.Context, zone *config.ZoneConfig, desc *roachpb.RangeDescriptor, ) (AllocatorAction, float64) { if a.storePool == nil { // Do nothing if storePool is nil for some unittests. @@ -360,7 +360,7 @@ func (a *Allocator) ComputeAction( func (a *Allocator) computeAction( ctx context.Context, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, rangeID roachpb.RangeID, voterReplicas []roachpb.ReplicaDescriptor, ) (AllocatorAction, float64) { @@ -470,7 +470,7 @@ type decisionDetails struct { // TODO(tbg): AllocateReplacement? func (a *Allocator) AllocateTarget( ctx context.Context, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, rangeID roachpb.RangeID, existingReplicas []roachpb.ReplicaDescriptor, ) (*roachpb.StoreDescriptor, string, error) { @@ -501,7 +501,7 @@ func (a *Allocator) AllocateTarget( func (a *Allocator) allocateTargetFromList( ctx context.Context, sl StoreList, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, candidateReplicas []roachpb.ReplicaDescriptor, options scorerOptions, ) (*roachpb.StoreDescriptor, string) { @@ -528,7 +528,7 @@ func (a *Allocator) allocateTargetFromList( func (a Allocator) simulateRemoveTarget( ctx context.Context, targetStore roachpb.StoreID, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, candidates []roachpb.ReplicaDescriptor, existingReplicas []roachpb.ReplicaDescriptor, rangeUsageInfo RangeUsageInfo, @@ -553,7 +553,7 @@ func (a Allocator) simulateRemoveTarget( // replicas. func (a Allocator) RemoveTarget( ctx context.Context, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, candidates []roachpb.ReplicaDescriptor, existingReplicas []roachpb.ReplicaDescriptor, ) (roachpb.ReplicaDescriptor, string, error) { @@ -623,7 +623,7 @@ func (a Allocator) RemoveTarget( // opportunity was found). func (a Allocator) RebalanceTarget( ctx context.Context, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, raftStatus *raft.Status, rangeID roachpb.RangeID, existingReplicas []roachpb.ReplicaDescriptor, @@ -775,7 +775,7 @@ func (a *Allocator) scorerOptions() scorerOptions { // unless asked to do otherwise by the checkTransferLeaseSource parameter. func (a *Allocator) TransferLeaseTarget( ctx context.Context, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, existing []roachpb.ReplicaDescriptor, leaseStoreID roachpb.StoreID, rangeID roachpb.RangeID, @@ -922,7 +922,7 @@ func (a *Allocator) TransferLeaseTarget( // attributes. func (a *Allocator) ShouldTransferLease( ctx context.Context, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, existing []roachpb.ReplicaDescriptor, leaseStoreID roachpb.StoreID, rangeID roachpb.RangeID, @@ -1227,7 +1227,7 @@ func (a Allocator) shouldTransferLeaseWithoutStats( } func (a Allocator) preferredLeaseholders( - zone *zonepb.ZoneConfig, existing []roachpb.ReplicaDescriptor, + zone *config.ZoneConfig, existing []roachpb.ReplicaDescriptor, ) []roachpb.ReplicaDescriptor { // Go one preference at a time. As soon as we've found replicas that match a // preference, we don't need to look at the later preferences, because diff --git a/pkg/storage/allocator_scorer.go b/pkg/storage/allocator_scorer.go index 2dbd250e323d..e618e9a26ba3 100644 --- a/pkg/storage/allocator_scorer.go +++ b/pkg/storage/allocator_scorer.go @@ -18,7 +18,7 @@ import ( "sort" "strconv" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/storage/constraint" @@ -1008,7 +1008,7 @@ func containsStore(stores []roachpb.StoreID, target roachpb.StoreID) bool { // constraintsCheck returns true iff the provided store would be a valid in a // range with the provided constraints. -func constraintsCheck(store roachpb.StoreDescriptor, constraints []zonepb.Constraints) bool { +func constraintsCheck(store roachpb.StoreDescriptor, constraints []config.Constraints) bool { if len(constraints) == 0 { return true } diff --git a/pkg/storage/allocator_scorer_test.go b/pkg/storage/allocator_scorer_test.go index 2b39e53f1c72..eb71659b8012 100644 --- a/pkg/storage/allocator_scorer_test.go +++ b/pkg/storage/allocator_scorer_test.go @@ -19,7 +19,7 @@ import ( "sort" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/constraint" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -537,15 +537,15 @@ func TestConstraintsCheck(t *testing.T) { testCases := []struct { name string - constraints []zonepb.Constraints + constraints []config.Constraints expected map[roachpb.StoreID]bool }{ { name: "required constraint", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, }, }, @@ -556,10 +556,10 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "required locality constraints", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "us", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "us", Type: config.Constraint_REQUIRED}, }, }, }, @@ -572,10 +572,10 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "prohibited constraints", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_PROHIBITED}, }, }, }, @@ -587,10 +587,10 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "prohibited locality constraints", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "us", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "us", Type: config.Constraint_PROHIBITED}, }, }, }, @@ -600,12 +600,12 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "positive constraints are ignored", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, - {Value: "b", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, - {Value: "c", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_DEPRECATED_POSITIVE}, + {Value: "b", Type: config.Constraint_DEPRECATED_POSITIVE}, + {Value: "c", Type: config.Constraint_DEPRECATED_POSITIVE}, }, }, }, @@ -619,10 +619,10 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "positive locality constraints are ignored", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "eur", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "eur", Type: config.Constraint_DEPRECATED_POSITIVE}, }, }, }, @@ -636,10 +636,10 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "NumReplicas doesn't affect constraint checking", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "eur", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "eur", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -650,16 +650,16 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "multiple per-replica constraints are respected", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "eur", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "eur", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -691,7 +691,7 @@ func TestAllocateConstraintsCheck(t *testing.T) { testCases := []struct { name string - constraints []zonepb.Constraints + constraints []config.Constraints zoneNumReplicas int32 existing []roachpb.StoreID expectedValid map[roachpb.StoreID]bool @@ -699,10 +699,10 @@ func TestAllocateConstraintsCheck(t *testing.T) { }{ { name: "prohibited constraint", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_PROHIBITED}, }, }, }, @@ -716,10 +716,10 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "required constraint", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, }, }, @@ -732,10 +732,10 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "required constraint with NumReplicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 3, }, @@ -752,16 +752,16 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -782,16 +782,16 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas and existing replicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -807,16 +807,16 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas and not enough existing replicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 2, }, @@ -835,16 +835,16 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas and sum(NumReplicas) < zone.NumReplicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -867,16 +867,16 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with sum(NumReplicas) < zone.NumReplicas and not enough existing replicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 2, }, @@ -899,7 +899,7 @@ func TestAllocateConstraintsCheck(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - zone := &zonepb.ZoneConfig{ + zone := &config.ZoneConfig{ Constraints: tc.constraints, NumReplicas: proto.Int32(tc.zoneNumReplicas), } @@ -928,16 +928,16 @@ func TestRemoveConstraintsCheck(t *testing.T) { } testCases := []struct { name string - constraints []zonepb.Constraints + constraints []config.Constraints zoneNumReplicas int32 expected map[roachpb.StoreID]expected }{ { name: "prohibited constraint", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_PROHIBITED}, }, }, }, @@ -950,10 +950,10 @@ func TestRemoveConstraintsCheck(t *testing.T) { }, { name: "required constraint", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, }, }, @@ -966,10 +966,10 @@ func TestRemoveConstraintsCheck(t *testing.T) { }, { name: "required constraint with NumReplicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 2, }, @@ -983,16 +983,16 @@ func TestRemoveConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -1005,10 +1005,10 @@ func TestRemoveConstraintsCheck(t *testing.T) { }, { name: "required constraint with NumReplicas and sum(NumReplicas) < zone.NumReplicas", - constraints: []zonepb.Constraints{ + constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 2, }, @@ -1032,7 +1032,7 @@ func TestRemoveConstraintsCheck(t *testing.T) { StoreID: storeID, }) } - zone := &zonepb.ZoneConfig{ + zone := &config.ZoneConfig{ Constraints: tc.constraints, NumReplicas: proto.Int32(tc.zoneNumReplicas), } diff --git a/pkg/storage/allocator_test.go b/pkg/storage/allocator_test.go index 370223706268..58a326f359cc 100644 --- a/pkg/storage/allocator_test.go +++ b/pkg/storage/allocator_test.go @@ -23,7 +23,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -51,22 +51,22 @@ import ( const firstRangeID = roachpb.RangeID(1) -var simpleZoneConfig = zonepb.ZoneConfig{ +var simpleZoneConfig = config.ZoneConfig{ NumReplicas: proto.Int32(1), - Constraints: []zonepb.Constraints{ + Constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, - {Value: "ssd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_REQUIRED}, + {Value: "ssd", Type: config.Constraint_REQUIRED}, }, }, }, } -var multiDCConfig = zonepb.ZoneConfig{ +var multiDCConfig = config.ZoneConfig{ NumReplicas: proto.Int32(2), - Constraints: []zonepb.Constraints{ - {Constraints: []zonepb.Constraint{{Value: "ssd", Type: zonepb.Constraint_REQUIRED}}}, + Constraints: []config.Constraints{ + {Constraints: []config.Constraint{{Value: "ssd", Type: config.Constraint_REQUIRED}}}, }, } @@ -482,13 +482,13 @@ func TestAllocatorExistingReplica(t *testing.T) { gossiputil.NewStoreGossiper(g).GossipStores(sameDCStores, t) result, _, err := a.AllocateTarget( context.Background(), - &zonepb.ZoneConfig{ + &config.ZoneConfig{ NumReplicas: proto.Int32(0), - Constraints: []zonepb.Constraints{ + Constraints: []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, - {Value: "hdd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "a", Type: config.Constraint_REQUIRED}, + {Value: "hdd", Type: config.Constraint_REQUIRED}, }, }, }, @@ -595,7 +595,7 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { { result, _, err := a.AllocateTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), firstRangeID, tc.existing, ) @@ -609,7 +609,7 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { var rangeUsageInfo RangeUsageInfo target, _, details, ok := a.RebalanceTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, /* raftStatus */ firstRangeID, tc.existing, @@ -683,7 +683,7 @@ func TestAllocatorRebalance(t *testing.T) { var rangeUsageInfo RangeUsageInfo target, _, _, ok := a.RebalanceTarget( ctx, - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, firstRangeID, []roachpb.ReplicaDescriptor{{NodeID: 3, StoreID: 3}}, @@ -834,7 +834,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { for i := 0; i < 10; i++ { result, _, details, ok := a.RebalanceTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), status, firstRangeID, replicas, @@ -855,7 +855,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { for i := 0; i < 10; i++ { target, _, details, ok := a.RebalanceTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), status, firstRangeID, replicas, @@ -873,7 +873,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { for i := 0; i < 10; i++ { target, origin, details, ok := a.RebalanceTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), status, firstRangeID, replicas, @@ -953,7 +953,7 @@ func TestAllocatorRebalanceDeadNodes(t *testing.T) { var rangeUsageInfo RangeUsageInfo target, _, _, ok := a.RebalanceTarget( ctx, - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, firstRangeID, c.existing, @@ -1148,7 +1148,7 @@ func TestAllocatorRebalanceByCount(t *testing.T) { var rangeUsageInfo RangeUsageInfo result, _, _, ok := a.RebalanceTarget( ctx, - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, firstRangeID, []roachpb.ReplicaDescriptor{{StoreID: stores[0].StoreID}}, @@ -1221,7 +1221,7 @@ func TestAllocatorTransferLeaseTarget(t *testing.T) { t.Run("", func(t *testing.T) { target := a.TransferLeaseTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), c.existing, c.leaseholder, 0, @@ -1297,7 +1297,7 @@ func TestAllocatorTransferLeaseTargetDraining(t *testing.T) { t.Run("", func(t *testing.T) { target := a.TransferLeaseTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), c.existing, c.leaseholder, 0, @@ -1432,7 +1432,7 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { var rangeUsageInfo RangeUsageInfo result, _, details, ok := a.RebalanceTarget( ctx, - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, /* raftStatus */ firstRangeID, tc.existing, @@ -1503,7 +1503,7 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { var rangeUsageInfo RangeUsageInfo result, _, details, ok := a.RebalanceTarget( ctx, - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, /* raftStatus */ firstRangeID, tc.existing, @@ -1569,7 +1569,7 @@ func TestAllocatorTransferLeaseTargetMultiStore(t *testing.T) { t.Run("", func(t *testing.T) { target := a.TransferLeaseTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), existing, c.leaseholder, 0, @@ -1626,7 +1626,7 @@ func TestAllocatorShouldTransferLease(t *testing.T) { t.Run("", func(t *testing.T) { result := a.ShouldTransferLease( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), c.existing, c.leaseholder, 0, @@ -1688,7 +1688,7 @@ func TestAllocatorShouldTransferLeaseDraining(t *testing.T) { t.Run("", func(t *testing.T) { result := a.ShouldTransferLease( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), c.existing, c.leaseholder, 0, @@ -1728,39 +1728,39 @@ func TestAllocatorLeasePreferences(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(stores, t) - preferDC1 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "1", Type: zonepb.Constraint_REQUIRED}}}, + preferDC1 := []config.LeasePreference{ + {Constraints: []config.Constraint{{Key: "dc", Value: "1", Type: config.Constraint_REQUIRED}}}, } - preferDC4Then3Then2 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "4", Type: zonepb.Constraint_REQUIRED}}}, - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "3", Type: zonepb.Constraint_REQUIRED}}}, - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "2", Type: zonepb.Constraint_REQUIRED}}}, + preferDC4Then3Then2 := []config.LeasePreference{ + {Constraints: []config.Constraint{{Key: "dc", Value: "4", Type: config.Constraint_REQUIRED}}}, + {Constraints: []config.Constraint{{Key: "dc", Value: "3", Type: config.Constraint_REQUIRED}}}, + {Constraints: []config.Constraint{{Key: "dc", Value: "2", Type: config.Constraint_REQUIRED}}}, } - preferN2ThenS3 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Value: "n2", Type: zonepb.Constraint_REQUIRED}}}, - {Constraints: []zonepb.Constraint{{Value: "s3", Type: zonepb.Constraint_REQUIRED}}}, + preferN2ThenS3 := []config.LeasePreference{ + {Constraints: []config.Constraint{{Value: "n2", Type: config.Constraint_REQUIRED}}}, + {Constraints: []config.Constraint{{Value: "s3", Type: config.Constraint_REQUIRED}}}, } - preferNotS1ThenNotN2 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Value: "s1", Type: zonepb.Constraint_PROHIBITED}}}, - {Constraints: []zonepb.Constraint{{Value: "n2", Type: zonepb.Constraint_PROHIBITED}}}, + preferNotS1ThenNotN2 := []config.LeasePreference{ + {Constraints: []config.Constraint{{Value: "s1", Type: config.Constraint_PROHIBITED}}}, + {Constraints: []config.Constraint{{Value: "n2", Type: config.Constraint_PROHIBITED}}}, } - preferNotS1AndNotN2 := []zonepb.LeasePreference{ + preferNotS1AndNotN2 := []config.LeasePreference{ { - Constraints: []zonepb.Constraint{ - {Value: "s1", Type: zonepb.Constraint_PROHIBITED}, - {Value: "n2", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []config.Constraint{ + {Value: "s1", Type: config.Constraint_PROHIBITED}, + {Value: "n2", Type: config.Constraint_PROHIBITED}, }, }, } - preferMatchesNothing := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "5", Type: zonepb.Constraint_REQUIRED}}}, - {Constraints: []zonepb.Constraint{{Value: "n6", Type: zonepb.Constraint_REQUIRED}}}, + preferMatchesNothing := []config.LeasePreference{ + {Constraints: []config.Constraint{{Key: "dc", Value: "5", Type: config.Constraint_REQUIRED}}}, + {Constraints: []config.Constraint{{Value: "n6", Type: config.Constraint_REQUIRED}}}, } testCases := []struct { leaseholder roachpb.StoreID existing []roachpb.ReplicaDescriptor - preferences []zonepb.LeasePreference + preferences []config.LeasePreference expectedCheckTrue roachpb.StoreID /* checkTransferLeaseSource = true */ expectedCheckFalse roachpb.StoreID /* checkTransferLeaseSource = false */ }{ @@ -1815,7 +1815,7 @@ func TestAllocatorLeasePreferences(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { - zone := &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), LeasePreferences: c.preferences} + zone := &config.ZoneConfig{NumReplicas: proto.Int32(0), LeasePreferences: c.preferences} result := a.ShouldTransferLease( context.Background(), zone, @@ -1896,17 +1896,17 @@ func TestAllocatorLeasePreferencesMultipleStoresPerLocality(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(stores, t) - preferEast := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "region", Value: "us-east1", Type: zonepb.Constraint_REQUIRED}}}, + preferEast := []config.LeasePreference{ + {Constraints: []config.Constraint{{Key: "region", Value: "us-east1", Type: config.Constraint_REQUIRED}}}, } - preferNotEast := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "region", Value: "us-east1", Type: zonepb.Constraint_PROHIBITED}}}, + preferNotEast := []config.LeasePreference{ + {Constraints: []config.Constraint{{Key: "region", Value: "us-east1", Type: config.Constraint_PROHIBITED}}}, } testCases := []struct { leaseholder roachpb.StoreID existing []roachpb.ReplicaDescriptor - preferences []zonepb.LeasePreference + preferences []config.LeasePreference expectedCheckTrue roachpb.StoreID /* checkTransferLeaseSource = true */ expectedCheckFalse roachpb.StoreID /* checkTransferLeaseSource = false */ }{ @@ -1924,7 +1924,7 @@ func TestAllocatorLeasePreferencesMultipleStoresPerLocality(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { - zone := &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), LeasePreferences: c.preferences} + zone := &config.ZoneConfig{NumReplicas: proto.Int32(0), LeasePreferences: c.preferences} target := a.TransferLeaseTarget( context.Background(), zone, @@ -2006,7 +2006,7 @@ func TestAllocatorRemoveTargetLocality(t *testing.T) { } targetRepl, details, err := a.RemoveTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), existingRepls, existingRepls, ) @@ -2089,7 +2089,7 @@ func TestAllocatorAllocateTargetLocality(t *testing.T) { } targetStore, details, err := a.AllocateTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), firstRangeID, existingRepls, ) @@ -2211,7 +2211,7 @@ func TestAllocatorRebalanceTargetLocality(t *testing.T) { var rangeUsageInfo RangeUsageInfo target, _, details, ok := a.RebalanceTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, firstRangeID, existingRepls, @@ -2236,114 +2236,114 @@ func TestAllocatorRebalanceTargetLocality(t *testing.T) { } var ( - threeSpecificLocalities = []zonepb.Constraints{ + threeSpecificLocalities = []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "c", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "c", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - twoAndOneLocalities = []zonepb.Constraints{ + twoAndOneLocalities = []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 2, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - threeInOneLocality = []zonepb.Constraints{ + threeInOneLocality = []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 3, }, } - twoAndOneNodeAttrs = []zonepb.Constraints{ + twoAndOneNodeAttrs = []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "ssd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "ssd", Type: config.Constraint_REQUIRED}, }, NumReplicas: 2, }, { - Constraints: []zonepb.Constraint{ - {Value: "hdd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "hdd", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - twoAndOneStoreAttrs = []zonepb.Constraints{ + twoAndOneStoreAttrs = []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "odd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "odd", Type: config.Constraint_REQUIRED}, }, NumReplicas: 2, }, { - Constraints: []zonepb.Constraint{ - {Value: "even", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "even", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - mixLocalityAndAttrs = []zonepb.Constraints{ + mixLocalityAndAttrs = []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, - {Value: "ssd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "a", Type: config.Constraint_REQUIRED}, + {Value: "ssd", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "b", Type: zonepb.Constraint_REQUIRED}, - {Value: "odd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "b", Type: config.Constraint_REQUIRED}, + {Value: "odd", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "even", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "even", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - twoSpecificLocalities = []zonepb.Constraints{ + twoSpecificLocalities = []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "a", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Key: "datacenter", Value: "b", Type: config.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -2363,7 +2363,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { // stores from multiDiversityDCStores would be the best addition to the range // purely on the basis of constraint satisfaction and locality diversity. testCases := []struct { - constraints []zonepb.Constraints + constraints []config.Constraints existing []roachpb.StoreID expected []roachpb.StoreID }{ @@ -2557,7 +2557,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { StoreID: storeID, } } - zone := &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: tc.constraints} + zone := &config.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: tc.constraints} analyzed := constraint.AnalyzeConstraints( context.Background(), a.storePool.getStoreDescriptor, existingRepls, zone) candidates := allocateCandidates( @@ -2601,7 +2601,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { // stores would be best to remove if we had to remove one purely on the basis // of constraint-matching and locality diversity. testCases := []struct { - constraints []zonepb.Constraints + constraints []config.Constraints existing []roachpb.StoreID expected []roachpb.StoreID }{ @@ -2780,7 +2780,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { StoreID: storeID, } } - zone := &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: tc.constraints} + zone := &config.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: tc.constraints} analyzed := constraint.AnalyzeConstraints( context.Background(), a.storePool.getStoreDescriptor, existingRepls, zone) candidates := removeCandidates( @@ -2828,7 +2828,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { candidates []roachpb.StoreID } testCases := []struct { - constraints []zonepb.Constraints + constraints []config.Constraints zoneNumReplicas int32 existing []roachpb.StoreID expected []rebalanceStoreIDs @@ -3571,7 +3571,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { } } var rangeUsageInfo RangeUsageInfo - zone := &zonepb.ZoneConfig{ + zone := &config.ZoneConfig{ Constraints: tc.constraints, NumReplicas: proto.Int32(tc.zoneNumReplicas), } @@ -3782,7 +3782,7 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) { }) target := a.TransferLeaseTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), existing, c.leaseholder, 0, @@ -3980,7 +3980,7 @@ func TestAllocatorRemoveTarget(t *testing.T) { for i := 0; i < 10; i++ { targetRepl, _, err := a.RemoveTarget( ctx, - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), replicas, replicas, ) @@ -4000,15 +4000,15 @@ func TestAllocatorComputeAction(t *testing.T) { // Each test case should describe a repair situation which has a lower // priority than the previous test case. testCases := []struct { - zone zonepb.ZoneConfig + zone config.ZoneConfig desc roachpb.RangeDescriptor expectedAction AllocatorAction }{ // Need three replicas, have three, one is on a dead store. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4035,9 +4035,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need five replicas, one is on a dead store. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(5), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4074,9 +4074,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have two. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4098,9 +4098,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need five replicas, have four, one is on a dead store. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(5), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4132,9 +4132,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need five replicas, have four. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(5), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4166,9 +4166,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have four, one is on a dead store. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4200,9 +4200,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need five replicas, have six, one is on a dead store. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(5), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4244,9 +4244,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have five, one is on a dead store. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4283,9 +4283,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have four. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4317,9 +4317,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have five. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4358,9 +4358,9 @@ func TestAllocatorComputeAction(t *testing.T) { // be a noop because there aren't enough live replicas for // a quorum. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4387,9 +4387,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have three, none of the replicas in the store pool. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4416,9 +4416,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have three. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), - Constraints: []zonepb.Constraints{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, + Constraints: []config.Constraints{{Constraints: []config.Constraint{{Value: "us-east", Type: config.Constraint_DEPRECATED_POSITIVE}}}}, RangeMinBytes: proto.Int64(0), RangeMaxBytes: proto.Int64(64000), }, @@ -4477,7 +4477,7 @@ func TestAllocatorComputeAction(t *testing.T) { func TestAllocatorComputeActionRemoveDead(t *testing.T) { defer leaktest.AfterTest(t)() - zone := zonepb.ZoneConfig{ + zone := config.ZoneConfig{ NumReplicas: proto.Int32(3), } threeReplDesc := roachpb.RangeDescriptor{ @@ -4570,7 +4570,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { defer leaktest.AfterTest(t)() testCases := []struct { - zone zonepb.ZoneConfig + zone config.ZoneConfig desc roachpb.RangeDescriptor expectedAction AllocatorAction live []roachpb.StoreID @@ -4582,7 +4582,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { // replace it (nor add a new replica) since there isn't a live target, // but that's still the action being emitted. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), }, desc: roachpb.RangeDescriptor{ @@ -4612,7 +4612,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { // Has three replicas, one is in decommissioning status, and one is on a // dead node. Replacing the dead replica is more important. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), }, desc: roachpb.RangeDescriptor{ @@ -4642,7 +4642,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { // Needs three replicas, has four, where one is decommissioning and one is // dead. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), }, desc: roachpb.RangeDescriptor{ @@ -4677,7 +4677,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { // Needs three replicas, has four, where one is decommissioning and one is // decommissioned. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), }, desc: roachpb.RangeDescriptor{ @@ -4712,7 +4712,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { }, // Needs three replicas, has three, all decommissioning { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), }, desc: roachpb.RangeDescriptor{ @@ -4741,7 +4741,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { }, // Needs 3. Has 1 live, 3 decommissioning. { - zone: zonepb.ZoneConfig{ + zone: config.ZoneConfig{ NumReplicas: proto.Int32(3), }, desc: roachpb.RangeDescriptor{ @@ -4792,7 +4792,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { func TestAllocatorRemoveLearner(t *testing.T) { defer leaktest.AfterTest(t)() - zone := zonepb.ZoneConfig{ + zone := config.ZoneConfig{ NumReplicas: proto.Int32(3), } learnerType := roachpb.LEARNER @@ -5006,7 +5006,7 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { ctx := context.Background() defer stopper.Stop(ctx) - zone := &zonepb.ZoneConfig{ + zone := &config.ZoneConfig{ NumReplicas: proto.Int32(5), } @@ -5108,7 +5108,7 @@ func TestAllocatorComputeActionNoStorePool(t *testing.T) { defer leaktest.AfterTest(t)() a := MakeAllocator(nil /* storePool */, nil /* rpcContext */) - action, priority := a.ComputeAction(context.Background(), &zonepb.ZoneConfig{NumReplicas: proto.Int32(0)}, nil) + action, priority := a.ComputeAction(context.Background(), &config.ZoneConfig{NumReplicas: proto.Int32(0)}, nil) if action != AllocatorNoop { t.Errorf("expected AllocatorNoop, but got %v", action) } @@ -5122,14 +5122,14 @@ func TestAllocatorComputeActionNoStorePool(t *testing.T) { func TestAllocatorError(t *testing.T) { defer leaktest.AfterTest(t)() - constraint := []zonepb.Constraints{ - {Constraints: []zonepb.Constraint{{Value: "one", Type: zonepb.Constraint_REQUIRED}}}, + constraint := []config.Constraints{ + {Constraints: []config.Constraint{{Value: "one", Type: config.Constraint_REQUIRED}}}, } - constraints := []zonepb.Constraints{ + constraints := []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Value: "one", Type: zonepb.Constraint_REQUIRED}, - {Value: "two", Type: zonepb.Constraint_REQUIRED}, + Constraints: []config.Constraint{ + {Value: "one", Type: config.Constraint_REQUIRED}, + {Value: "two", Type: config.Constraint_REQUIRED}, }, }, } @@ -5472,43 +5472,43 @@ func TestAllocatorRebalanceAway(t *testing.T) { {StoreID: stores[2].StoreID}, } testCases := []struct { - constraint zonepb.Constraint + constraint config.Constraint expected *roachpb.StoreID }{ { - constraint: zonepb.Constraint{Key: "datacenter", Value: "us", Type: zonepb.Constraint_REQUIRED}, + constraint: config.Constraint{Key: "datacenter", Value: "us", Type: config.Constraint_REQUIRED}, expected: &stores[3].StoreID, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "eur", Type: zonepb.Constraint_PROHIBITED}, + constraint: config.Constraint{Key: "datacenter", Value: "eur", Type: config.Constraint_PROHIBITED}, expected: &stores[3].StoreID, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "eur", Type: zonepb.Constraint_REQUIRED}, + constraint: config.Constraint{Key: "datacenter", Value: "eur", Type: config.Constraint_REQUIRED}, expected: &stores[4].StoreID, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "us", Type: zonepb.Constraint_PROHIBITED}, + constraint: config.Constraint{Key: "datacenter", Value: "us", Type: config.Constraint_PROHIBITED}, expected: &stores[4].StoreID, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "other", Type: zonepb.Constraint_REQUIRED}, + constraint: config.Constraint{Key: "datacenter", Value: "other", Type: config.Constraint_REQUIRED}, expected: nil, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "other", Type: zonepb.Constraint_PROHIBITED}, + constraint: config.Constraint{Key: "datacenter", Value: "other", Type: config.Constraint_PROHIBITED}, expected: nil, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "other", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, + constraint: config.Constraint{Key: "datacenter", Value: "other", Type: config.Constraint_DEPRECATED_POSITIVE}, expected: nil, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "us", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, + constraint: config.Constraint{Key: "datacenter", Value: "us", Type: config.Constraint_DEPRECATED_POSITIVE}, expected: nil, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "eur", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, + constraint: config.Constraint{Key: "datacenter", Value: "eur", Type: config.Constraint_DEPRECATED_POSITIVE}, expected: nil, }, } @@ -5520,8 +5520,8 @@ func TestAllocatorRebalanceAway(t *testing.T) { for _, tc := range testCases { t.Run(tc.constraint.String(), func(t *testing.T) { - constraints := zonepb.Constraints{ - Constraints: []zonepb.Constraint{ + constraints := config.Constraints{ + Constraints: []config.Constraint{ tc.constraint, }, } @@ -5529,7 +5529,7 @@ func TestAllocatorRebalanceAway(t *testing.T) { var rangeUsageInfo RangeUsageInfo actual, _, _, ok := a.RebalanceTarget( ctx, - &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: []zonepb.Constraints{constraints}}, + &config.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: []config.Constraints{constraints}}, nil, firstRangeID, existingReplicas, @@ -5608,7 +5608,7 @@ func TestAllocatorFullDisks(t *testing.T) { &st.Version, ) server := rpc.NewServer(rpcContext) // never started - g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) TimeUntilStoreDead.Override(&st.SV, TestTimeUntilStoreDeadOff) @@ -5695,7 +5695,7 @@ func TestAllocatorFullDisks(t *testing.T) { var rangeUsageInfo RangeUsageInfo target, _, details, ok := alloc.RebalanceTarget( ctx, - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, firstRangeID, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, @@ -5751,7 +5751,7 @@ func Example_rebalancing() { &st.Version, ) server := rpc.NewServer(rpcContext) // never started - g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) TimeUntilStoreDead.Override(&st.SV, TestTimeUntilStoreDeadOff) @@ -5824,7 +5824,7 @@ func Example_rebalancing() { var rangeUsageInfo RangeUsageInfo target, _, details, ok := alloc.RebalanceTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + config.EmptyCompleteZoneConfig(), nil, firstRangeID, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, diff --git a/pkg/storage/client_merge_test.go b/pkg/storage/client_merge_test.go index c38cddccb275..60c7b9cab850 100644 --- a/pkg/storage/client_merge_test.go +++ b/pkg/storage/client_merge_test.go @@ -26,7 +26,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" @@ -3234,7 +3234,7 @@ func TestMergeQueue(t *testing.T) { // setThresholds simulates a zone config update that updates the ranges' // minimum and maximum sizes. - setZones := func(zone zonepb.ZoneConfig) { + setZones := func(zone config.ZoneConfig) { lhs().SetZoneConfig(&zone) rhs().SetZoneConfig(&zone) } @@ -3290,7 +3290,7 @@ func TestMergeQueue(t *testing.T) { t.Run("lhs-undersize", func(t *testing.T) { reset(t) - zone := protoutil.Clone(storeCfg.DefaultZoneConfig).(*zonepb.ZoneConfig) + zone := protoutil.Clone(storeCfg.DefaultZoneConfig).(*config.ZoneConfig) *zone.RangeMinBytes *= 2 lhs().SetZoneConfig(zone) store.MustForceMergeScanAndProcess() @@ -3302,7 +3302,7 @@ func TestMergeQueue(t *testing.T) { // The ranges are individually beneath the minimum size threshold, but // together they'll exceed the maximum size threshold. - zone := protoutil.Clone(storeCfg.DefaultZoneConfig).(*zonepb.ZoneConfig) + zone := protoutil.Clone(storeCfg.DefaultZoneConfig).(*config.ZoneConfig) zone.RangeMinBytes = proto.Int64(lhs().GetMVCCStats().Total() + 1) zone.RangeMaxBytes = proto.Int64(lhs().GetMVCCStats().Total()*2 - 1) setZones(*zone) diff --git a/pkg/storage/client_replica_test.go b/pkg/storage/client_replica_test.go index 21d31d98dc9a..1312ea01d91c 100644 --- a/pkg/storage/client_replica_test.go +++ b/pkg/storage/client_replica_test.go @@ -23,7 +23,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -1711,8 +1711,8 @@ func TestSystemZoneConfigs(t *testing.T) { } expectedUserRanges := 1 expectedSystemRanges -= expectedUserRanges - systemNumReplicas := int(*zonepb.DefaultSystemZoneConfig().NumReplicas) - userNumReplicas := int(*zonepb.DefaultZoneConfig().NumReplicas) + systemNumReplicas := int(*config.DefaultSystemZoneConfig().NumReplicas) + userNumReplicas := int(*config.DefaultZoneConfig().NumReplicas) expectedReplicas := expectedSystemRanges*systemNumReplicas + expectedUserRanges*userNumReplicas log.Infof(ctx, "TestSystemZoneConfig: expecting %d system ranges and %d user ranges", expectedSystemRanges, expectedUserRanges) diff --git a/pkg/storage/client_split_test.go b/pkg/storage/client_split_test.go index 2f6d3805eb87..eb90b3a6bf2d 100644 --- a/pkg/storage/client_split_test.go +++ b/pkg/storage/client_split_test.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" @@ -1006,7 +1005,7 @@ func TestStoreZoneUpdateAndRangeSplit(t *testing.T) { const maxBytes = 1 << 16 // Set max bytes. descID := uint32(keys.MinUserDescID) - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(maxBytes) config.TestingSetZoneConfig(descID, zoneConfig) @@ -1068,7 +1067,7 @@ func TestStoreRangeSplitWithMaxBytesUpdate(t *testing.T) { // Set max bytes. const maxBytes = 1 << 16 descID := uint32(keys.MinUserDescID) - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(maxBytes) config.TestingSetZoneConfig(descID, zoneConfig) @@ -1269,7 +1268,7 @@ func TestStoreRangeSystemSplits(t *testing.T) { userTableMax := keys.MinUserDescID + 4 var exceptions map[int]struct{} - schema := sqlbase.MakeMetadataSchema(zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + schema := sqlbase.MakeMetadataSchema(config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) // Write table descriptors for the tables in the metadata schema as well as // five dummy user tables. This does two things: // - descriptor IDs are used to determine split keys @@ -1290,17 +1289,9 @@ func TestStoreRangeSystemSplits(t *testing.T) { } } for i := keys.MinUserDescID; i <= userTableMax; i++ { - id := sqlbase.ID(i) - key := sqlbase.MakeDescMetadataKey(id) - if err := txn.Put(ctx, key, - &sqlbase.Descriptor{ - Union: &sqlbase.Descriptor_Table{ - Table: &sqlbase.TableDescriptor{ - // Fill in the descriptor just enough for the test to work. - ID: id, - }, - }, - }); err != nil { + // We don't care about the value, just the key. + key := sqlbase.MakeDescMetadataKey(sqlbase.ID(i)) + if err := txn.Put(ctx, key, &sqlbase.TableDescriptor{}); err != nil { return err } } @@ -1362,18 +1353,9 @@ func TestStoreRangeSystemSplits(t *testing.T) { return err } // This time, only write the last table descriptor. Splits only occur for - // the descriptor we add. - id := sqlbase.ID(userTableMax) - k := sqlbase.MakeDescMetadataKey(id) - return txn.Put(ctx, k, - &sqlbase.Descriptor{ - Union: &sqlbase.Descriptor_Table{ - Table: &sqlbase.TableDescriptor{ - // Fill in the descriptor just enough for the test to work. - ID: id, - }, - }, - }) + // the descriptor we add. We don't care about the value, just the key. + k := sqlbase.MakeDescMetadataKey(sqlbase.ID(userTableMax)) + return txn.Put(ctx, k, &sqlbase.TableDescriptor{}) }); err != nil { t.Fatal(err) } @@ -2330,7 +2312,7 @@ func TestStoreRangeGossipOnSplits(t *testing.T) { // Avoid excessive logging on under-replicated ranges due to our many splits. config.TestingSetupZoneConfigHook(stopper) - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() zoneConfig.NumReplicas = proto.Int32(1) config.TestingSetZoneConfig(0, zoneConfig) @@ -2533,11 +2515,11 @@ func TestUnsplittableRange(t *testing.T) { splitQueuePurgatoryChan := make(chan time.Time, 1) cfg := storage.TestStoreConfig(hlc.NewClock(manual.UnixNano, time.Nanosecond)) cfg.DefaultZoneConfig.RangeMaxBytes = proto.Int64(maxBytes) - cfg.DefaultZoneConfig.GC = &zonepb.GCPolicy{ + cfg.DefaultZoneConfig.GC = &config.GCPolicy{ TTLSeconds: int32(ttl.Seconds()), } cfg.DefaultSystemZoneConfig.RangeMaxBytes = proto.Int64(maxBytes) - cfg.DefaultSystemZoneConfig.GC = &zonepb.GCPolicy{ + cfg.DefaultSystemZoneConfig.GC = &config.GCPolicy{ TTLSeconds: int32(ttl.Seconds()), } cfg.TestingKnobs.SplitQueuePurgatoryChan = splitQueuePurgatoryChan diff --git a/pkg/storage/client_test.go b/pkg/storage/client_test.go index fb232fc46939..1f1a0803ca95 100644 --- a/pkg/storage/client_test.go +++ b/pkg/storage/client_test.go @@ -33,7 +33,6 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" "github.com/cockroachdb/cockroach/pkg/internal/client" @@ -834,7 +833,7 @@ func (m *multiTestContext) addStore(idx int) { grpcServer, m.transportStopper, metric.NewRegistry(), - zonepb.DefaultZoneConfigRef(), + config.DefaultZoneConfigRef(), ) nodeID := roachpb.NodeID(idx + 1) diff --git a/pkg/storage/constraint/analyzer.go b/pkg/storage/constraint/analyzer.go index d1c64ea77158..8a6445e26616 100644 --- a/pkg/storage/constraint/analyzer.go +++ b/pkg/storage/constraint/analyzer.go @@ -13,13 +13,13 @@ package constraint import ( "context" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" ) // AnalyzedConstraints comment !!! type AnalyzedConstraints struct { - Constraints []zonepb.Constraints + Constraints []config.Constraints // True if the per-replica constraints don't fully cover all the desired // replicas in the range (sum(constraints.NumReplicas) < zone.NumReplicas). // In such cases, we allow replicas that don't match any of the per-replica @@ -42,7 +42,7 @@ func AnalyzeConstraints( ctx context.Context, getStoreDescFn func(roachpb.StoreID) (roachpb.StoreDescriptor, bool), existing []roachpb.ReplicaDescriptor, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, ) AnalyzedConstraints { result := AnalyzedConstraints{ Constraints: zone.Constraints, @@ -77,12 +77,12 @@ func AnalyzeConstraints( // SubConstraintsCheck checks a store against a single set of constraints (out // of the possibly numerous sets that apply to a range), returning true iff the // store matches the constraints. -func SubConstraintsCheck(store roachpb.StoreDescriptor, constraints []zonepb.Constraint) bool { +func SubConstraintsCheck(store roachpb.StoreDescriptor, constraints []config.Constraint) bool { for _, constraint := range constraints { // StoreSatisfiesConstraint returns whether a store matches the given constraint. - hasConstraint := zonepb.StoreMatchesConstraint(store, constraint) - if (constraint.Type == zonepb.Constraint_REQUIRED && !hasConstraint) || - (constraint.Type == zonepb.Constraint_PROHIBITED && hasConstraint) { + hasConstraint := config.StoreMatchesConstraint(store, constraint) + if (constraint.Type == config.Constraint_REQUIRED && !hasConstraint) || + (constraint.Type == config.Constraint_PROHIBITED && hasConstraint) { return false } } diff --git a/pkg/storage/engine/gc.go b/pkg/storage/engine/gc.go index d359bde1ddfb..526861cd50f2 100644 --- a/pkg/storage/engine/gc.go +++ b/pkg/storage/engine/gc.go @@ -13,7 +13,7 @@ package engine import ( "sort" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -22,12 +22,12 @@ import ( // versions and maximum age. type GarbageCollector struct { Threshold hlc.Timestamp - policy zonepb.GCPolicy + policy config.GCPolicy } // MakeGarbageCollector allocates and returns a new GC, with expiration // computed based on current time and policy.TTLSeconds. -func MakeGarbageCollector(now hlc.Timestamp, policy zonepb.GCPolicy) GarbageCollector { +func MakeGarbageCollector(now hlc.Timestamp, policy config.GCPolicy) GarbageCollector { ttlNanos := int64(policy.TTLSeconds) * 1E9 return GarbageCollector{ Threshold: hlc.Timestamp{WallTime: now.WallTime - ttlNanos}, diff --git a/pkg/storage/engine/gc_test.go b/pkg/storage/engine/gc_test.go index 0386cca017c7..fd946d5784a3 100644 --- a/pkg/storage/engine/gc_test.go +++ b/pkg/storage/engine/gc_test.go @@ -13,7 +13,7 @@ package engine import ( "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -41,8 +41,8 @@ var ( // different sorts of MVCC keys. func TestGarbageCollectorFilter(t *testing.T) { defer leaktest.AfterTest(t)() - gcA := MakeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, zonepb.GCPolicy{TTLSeconds: 1}) - gcB := MakeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, zonepb.GCPolicy{TTLSeconds: 2}) + gcA := MakeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, config.GCPolicy{TTLSeconds: 1}) + gcB := MakeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, config.GCPolicy{TTLSeconds: 2}) n := []byte("data") d := []byte(nil) testData := []struct { diff --git a/pkg/storage/gc_queue.go b/pkg/storage/gc_queue.go index 3aeb0824ea70..1501341559b7 100644 --- a/pkg/storage/gc_queue.go +++ b/pkg/storage/gc_queue.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -599,7 +598,7 @@ type GCInfo struct { // Now is the timestamp used for age computations. Now hlc.Timestamp // Policy is the policy used for this garbage collection cycle. - Policy zonepb.GCPolicy + Policy config.GCPolicy // Stats about the userspace key-values considered, namely the number of // keys with GC'able data, the number of "old" intents and the number of // associated distinct transactions. @@ -680,7 +679,7 @@ func RunGC( desc *roachpb.RangeDescriptor, snap engine.Reader, now hlc.Timestamp, - policy zonepb.GCPolicy, + policy config.GCPolicy, gcer GCer, cleanupIntentsFn cleanupIntentsFunc, cleanupTxnIntentsAsyncFn cleanupTxnIntentsAsyncFunc, diff --git a/pkg/storage/merge_queue.go b/pkg/storage/merge_queue.go index 93178f0da94f..454589acbcc4 100644 --- a/pkg/storage/merge_queue.go +++ b/pkg/storage/merge_queue.go @@ -154,7 +154,7 @@ func (mq *mergeQueue) shouldQueue( return false, 0 } - if sysCfg.NeedsSplit(ctx, desc.StartKey, desc.EndKey.Next()) { + if sysCfg.NeedsSplit(desc.StartKey, desc.EndKey.Next()) { // This range would need to be split if it extended just one key further. // There is thus no possible right-hand neighbor that it could be merged // with. @@ -266,7 +266,7 @@ func (mq *mergeQueue) process( // in a situation where we keep merging ranges that would be split soon after // by a small increase in load. loadBasedSplitPossible := lhsRepl.SplitByLoadQPSThreshold() < 2*mergedQPS - if ok, _ := shouldSplitRange(ctx, mergedDesc, mergedStats, lhsRepl.GetMaxBytes(), sysCfg); ok || loadBasedSplitPossible { + if ok, _ := shouldSplitRange(mergedDesc, mergedStats, lhsRepl.GetMaxBytes(), sysCfg); ok || loadBasedSplitPossible { log.VEventf(ctx, 2, "skipping merge to avoid thrashing: merged range %s may split "+ "(estimated size, estimated QPS: %d, %v)", diff --git a/pkg/storage/merge_queue_test.go b/pkg/storage/merge_queue_test.go index 818d63a11140..e7e6e15ae81e 100644 --- a/pkg/storage/merge_queue_test.go +++ b/pkg/storage/merge_queue_test.go @@ -15,7 +15,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -43,8 +42,8 @@ func TestMergeQueueShouldQueue(t *testing.T) { return keys.MakeTablePrefix(keys.MaxReservedDescID + i) } - config.TestingSetZoneConfig(keys.MaxReservedDescID+1, *zonepb.NewZoneConfig()) - config.TestingSetZoneConfig(keys.MaxReservedDescID+2, *zonepb.NewZoneConfig()) + config.TestingSetZoneConfig(keys.MaxReservedDescID+1, *config.NewZoneConfig()) + config.TestingSetZoneConfig(keys.MaxReservedDescID+2, *config.NewZoneConfig()) // Disable merges for table ID 4. if err := testCtx.gossip.AddInfo( @@ -176,7 +175,7 @@ func TestMergeQueueShouldQueue(t *testing.T) { repl := &Replica{} repl.mu.state.Desc = &roachpb.RangeDescriptor{StartKey: tc.startKey, EndKey: tc.endKey} repl.mu.state.Stats = &enginepb.MVCCStats{KeyBytes: tc.bytes} - zoneConfig := zonepb.DefaultZoneConfigRef() + zoneConfig := config.DefaultZoneConfigRef() zoneConfig.RangeMinBytes = proto.Int64(tc.minBytes) repl.SetZoneConfig(zoneConfig) shouldQ, priority := mq.shouldQueue(ctx, hlc.Timestamp{}, repl, config.NewSystemConfig(zoneConfig)) diff --git a/pkg/storage/node_liveness_test.go b/pkg/storage/node_liveness_test.go index 7639e4ee2693..e97462e35a64 100644 --- a/pkg/storage/node_liveness_test.go +++ b/pkg/storage/node_liveness_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -807,7 +806,7 @@ func TestNodeLivenessStatusMap(t *testing.T) { // Allow for inserting zone configs without having to go through (or // duplicate the logic from) the CLI. config.TestingSetupZoneConfigHook(tc.Stopper()) - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() // Force just one replica per range to ensure that we can shut down // nodes without endangering the liveness range. zoneConfig.NumReplicas = proto.Int32(1) diff --git a/pkg/storage/queue.go b/pkg/storage/queue.go index 7cc8563b2c96..afcf7e24efec 100644 --- a/pkg/storage/queue.go +++ b/pkg/storage/queue.go @@ -569,7 +569,7 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. repl.maybeInitializeRaftGroup(ctx) } - if cfg != nil && bq.requiresSplit(ctx, cfg, repl) { + if cfg != nil && bq.requiresSplit(cfg, repl) { // Range needs to be split due to zone configs, but queue does // not accept unsplit ranges. if log.V(1) { @@ -602,14 +602,12 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. } } -func (bq *baseQueue) requiresSplit( - ctx context.Context, cfg *config.SystemConfig, repl replicaInQueue, -) bool { +func (bq *baseQueue) requiresSplit(cfg *config.SystemConfig, repl replicaInQueue) bool { if bq.acceptsUnsplitRanges { return false } desc := repl.Desc() - return cfg.NeedsSplit(ctx, desc.StartKey, desc.EndKey) + return cfg.NeedsSplit(desc.StartKey, desc.EndKey) } // addInternal adds the replica the queue with specified priority. If @@ -843,7 +841,7 @@ func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) er } } - if cfg != nil && bq.requiresSplit(ctx, cfg, repl) { + if cfg != nil && bq.requiresSplit(cfg, repl) { // Range needs to be split due to zone configs, but queue does // not accept unsplit ranges. log.VEventf(ctx, 3, "split needed; skipping") diff --git a/pkg/storage/queue_concurrency_test.go b/pkg/storage/queue_concurrency_test.go index c736fa83418f..0e82cfb9382b 100644 --- a/pkg/storage/queue_concurrency_test.go +++ b/pkg/storage/queue_concurrency_test.go @@ -19,7 +19,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/storagepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -66,7 +65,7 @@ func TestBaseQueueConcurrent(t *testing.T) { cfg: StoreConfig{ Clock: hlc.NewClock(hlc.UnixNano, time.Second), AmbientCtx: log.AmbientContext{Tracer: tracing.NewTracer()}, - DefaultZoneConfig: zonepb.DefaultZoneConfigRef(), + DefaultZoneConfig: config.DefaultZoneConfigRef(), }, } diff --git a/pkg/storage/queue_test.go b/pkg/storage/queue_test.go index 0dd433f4953e..c1f8f20fe7f2 100644 --- a/pkg/storage/queue_test.go +++ b/pkg/storage/queue_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -513,7 +512,7 @@ func TestNeedsSystemConfig(t *testing.T) { tc.store.cfg.AmbientCtx, &base.Config{Insecure: true}, tc.store.cfg.Clock, stopper, &cluster.MakeTestingClusterSettings().Version) emptyGossip := gossip.NewTest( - tc.gossip.NodeID.Get(), rpcContext, rpc.NewServer(rpcContext), stopper, tc.store.Registry(), zonepb.DefaultZoneConfigRef()) + tc.gossip.NodeID.Get(), rpcContext, rpc.NewServer(rpcContext), stopper, tc.store.Registry(), config.DefaultZoneConfigRef()) bqNeedsSysCfg := makeTestBaseQueue("test", testQueue, tc.store, emptyGossip, queueConfig{ needsSystemConfig: true, acceptsUnsplitRanges: true, @@ -628,11 +627,11 @@ func TestAcceptsUnsplitRanges(t *testing.T) { return nil }) neverSplitsDesc := neverSplits.Desc() - if sysCfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { + if sysCfg.NeedsSplit(neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { t.Fatal("System config says range needs to be split") } willSplitDesc := willSplit.Desc() - if sysCfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) { + if sysCfg.NeedsSplit(willSplitDesc.StartKey, willSplitDesc.EndKey) { t.Fatal("System config says range needs to be split") } @@ -658,19 +657,18 @@ func TestAcceptsUnsplitRanges(t *testing.T) { // Now add a user object, it will trigger a split. // The range willSplit starts at the beginning of the user data range, // which means keys.MaxReservedDescID+1. - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(1 << 20) config.TestingSetZoneConfig(keys.MaxReservedDescID+2, zoneConfig) // Check our config. neverSplitsDesc = neverSplits.Desc() - if sysCfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { + if sysCfg.NeedsSplit(neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { t.Fatal("System config says range needs to be split") } willSplitDesc = willSplit.Desc() - if !sysCfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) { - t.Fatalf("System config says range %s does not need to be split", - willSplitDesc.RSpan()) + if !sysCfg.NeedsSplit(willSplitDesc.StartKey, willSplitDesc.EndKey) { + t.Fatal("System config says range does not need to be split") } bq.maybeAdd(ctx, neverSplits, hlc.Timestamp{}) diff --git a/pkg/storage/raft_transport_test.go b/pkg/storage/raft_transport_test.go index 7cf1a04678f9..afebbbf58fe0 100644 --- a/pkg/storage/raft_transport_test.go +++ b/pkg/storage/raft_transport_test.go @@ -18,7 +18,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -128,7 +128,7 @@ func newRaftTransportTestContext(t testing.TB) *raftTransportTestContext { server := rpc.NewServer(rttc.nodeRPCContext) // never started rttc.gossip = gossip.NewTest( - 1, rttc.nodeRPCContext, server, rttc.stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef(), + 1, rttc.nodeRPCContext, server, rttc.stopper, metric.NewRegistry(), config.DefaultZoneConfigRef(), ) return rttc diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index fff89e4d334d..07c7f6f5a562 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -20,7 +20,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -311,7 +311,7 @@ type Replica struct { // used, if they eventually apply. minLeaseProposedTS hlc.Timestamp // A pointer to the zone config for this replica. - zone *zonepb.ZoneConfig + zone *config.ZoneConfig // proposalBuf buffers Raft commands as they are passed to the Raft // replication subsystem. The buffer is populated by requests after // evaluation and is consumed by the Raft processing thread. Once @@ -658,7 +658,7 @@ func (r *Replica) GetMaxBytes() int64 { } // SetZoneConfig sets the replica's zone config. -func (r *Replica) SetZoneConfig(zone *zonepb.ZoneConfig) { +func (r *Replica) SetZoneConfig(zone *config.ZoneConfig) { r.mu.Lock() defer r.mu.Unlock() r.mu.zone = zone @@ -683,7 +683,7 @@ func (r *Replica) isDestroyedRLocked() (DestroyReason, error) { // DescAndZone returns the authoritative range descriptor as well // as the zone config for the replica. -func (r *Replica) DescAndZone() (*roachpb.RangeDescriptor, *zonepb.ZoneConfig) { +func (r *Replica) DescAndZone() (*roachpb.RangeDescriptor, *config.ZoneConfig) { r.mu.RLock() defer r.mu.RUnlock() return r.mu.state.Desc, r.mu.zone diff --git a/pkg/storage/replica_metrics.go b/pkg/storage/replica_metrics.go index 21d21452d61d..5b7cb165ba85 100644 --- a/pkg/storage/replica_metrics.go +++ b/pkg/storage/replica_metrics.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/storagepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -89,7 +89,7 @@ func calcReplicaMetrics( _ context.Context, _ hlc.Timestamp, raftCfg *base.RaftConfig, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, livenessMap IsLiveMap, clusterNodes int, desc *roachpb.RangeDescriptor, diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 8f10ea94a98f..bb7f1ecd87fe 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" @@ -8428,7 +8427,7 @@ func TestReplicaMetrics(t *testing.T) { for i, c := range testCases { t.Run("", func(t *testing.T) { - zoneConfig := protoutil.Clone(cfg.DefaultZoneConfig).(*zonepb.ZoneConfig) + zoneConfig := protoutil.Clone(cfg.DefaultZoneConfig).(*config.ZoneConfig) zoneConfig.NumReplicas = proto.Int32(c.replicas) // Alternate between quiescent and non-quiescent replicas to test the diff --git a/pkg/storage/replicate_queue.go b/pkg/storage/replicate_queue.go index bfe00d3fb888..f2f39b254c1b 100644 --- a/pkg/storage/replicate_queue.go +++ b/pkg/storage/replicate_queue.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -558,7 +557,7 @@ func (rq *replicateQueue) addOrReplace( func (rq *replicateQueue) findRemoveTarget( ctx context.Context, repl interface { - DescAndZone() (*roachpb.RangeDescriptor, *zonepb.ZoneConfig) + DescAndZone() (*roachpb.RangeDescriptor, *config.ZoneConfig) LastReplicaAdded() (roachpb.ReplicaID, time.Time) RaftStatus() *raft.Status }, @@ -929,7 +928,7 @@ func (rq *replicateQueue) findTargetAndTransferLease( ctx context.Context, repl *Replica, desc *roachpb.RangeDescriptor, - zone *zonepb.ZoneConfig, + zone *config.ZoneConfig, opts transferLeaseOptions, ) (bool, error) { // Learner replicas aren't allowed to become the leaseholder or raft leader, diff --git a/pkg/storage/replicate_queue_test.go b/pkg/storage/replicate_queue_test.go index 0d8fda93d247..ac9c643c0523 100644 --- a/pkg/storage/replicate_queue_test.go +++ b/pkg/storage/replicate_queue_test.go @@ -21,7 +21,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -115,7 +115,7 @@ func testReplicateQueueRebalanceInner(t *testing.T, atomic bool) { return counts } - initialRanges, err := server.ExpectedInitialRangeCount(tc.Servers[0].DB(), zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + initialRanges, err := server.ExpectedInitialRangeCount(tc.Servers[0].DB(), config.DefaultZoneConfigRef(), config.DefaultSystemZoneConfigRef()) if err != nil { t.Fatal(err) } @@ -380,7 +380,7 @@ func TestLargeUnsplittableRangeReplicate(t *testing.T) { // Create a cluster with really small ranges. const rangeMaxSize = base.MinRangeMaxBytes - zcfg := zonepb.DefaultZoneConfig() + zcfg := config.DefaultZoneConfig() zcfg.RangeMinBytes = proto.Int64(rangeMaxSize / 2) zcfg.RangeMaxBytes = proto.Int64(rangeMaxSize) tc := testcluster.StartTestCluster(t, 5, diff --git a/pkg/storage/reports/constraint_stats_report.go b/pkg/storage/reports/constraint_stats_report.go index ccaf20d76308..4c1f8f09554d 100644 --- a/pkg/storage/reports/constraint_stats_report.go +++ b/pkg/storage/reports/constraint_stats_report.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -122,7 +121,7 @@ func (k ConstraintStatusKey) Less(other ConstraintStatusKey) bool { // MakeConstraintRepr creates a canonical string representation for a // constraint. The constraint is identified by the group it belongs to and the // index within the group. -func MakeConstraintRepr(constraintGroup zonepb.Constraints, constraintIdx int) ConstraintRepr { +func MakeConstraintRepr(constraintGroup config.Constraints, constraintIdx int) ConstraintRepr { cstr := constraintGroup.Constraints[constraintIdx].String() if constraintGroup.NumReplicas == 0 { return ConstraintRepr(cstr) @@ -163,7 +162,7 @@ func (r *replicationConstraintStatsReportSaver) EnsureEntry( } func (r *replicationConstraintStatsReportSaver) ensureEntries( - key ZoneKey, zone *zonepb.ZoneConfig, + key ZoneKey, zone *config.ZoneConfig, ) { for _, group := range zone.Constraints { for i := range group.Constraints { @@ -385,7 +384,7 @@ type constraintConformanceVisitor struct { // This state can be reused when a range is covered by the same zone config as // the previous one. Reusing it speeds up the report generation. prevZoneKey ZoneKey - prevConstraints []zonepb.Constraints + prevConstraints []config.Constraints } var _ rangeVisitor = &constraintConformanceVisitor{} @@ -447,10 +446,10 @@ func (v *constraintConformanceVisitor) visitNewZone( }() // Find the applicable constraints, which may be inherited. - var constraints []zonepb.Constraints + var constraints []config.Constraints var zKey ZoneKey _, err := visitZones(ctx, r, v.cfg, ignoreSubzonePlaceholders, - func(_ context.Context, zone *zonepb.ZoneConfig, key ZoneKey) bool { + func(_ context.Context, zone *config.ZoneConfig, key ZoneKey) bool { if zone.Constraints == nil { return false } @@ -476,7 +475,7 @@ func (v *constraintConformanceVisitor) visitSameZone( } func (v *constraintConformanceVisitor) countRange( - ctx context.Context, r *roachpb.RangeDescriptor, key ZoneKey, constraints []zonepb.Constraints, + ctx context.Context, r *roachpb.RangeDescriptor, key ZoneKey, constraints []config.Constraints, ) { storeDescs := v.storeResolver(r) violated := processRange(ctx, storeDescs, constraints) diff --git a/pkg/storage/reports/constraint_stats_report_test.go b/pkg/storage/reports/constraint_stats_report_test.go index 1d11d721f259..bdaae08d3395 100644 --- a/pkg/storage/reports/constraint_stats_report_test.go +++ b/pkg/storage/reports/constraint_stats_report_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -257,13 +256,13 @@ type zone struct { constraints string } -func (z zone) toZoneConfig() zonepb.ZoneConfig { - cfg := zonepb.NewZoneConfig() +func (z zone) toZoneConfig() config.ZoneConfig { + cfg := config.NewZoneConfig() if z.replicas != 0 { cfg.NumReplicas = proto.Int32(z.replicas) } if z.constraints != "" { - var constraintsList zonepb.ConstraintsList + var constraintsList config.ConstraintsList if err := yaml.UnmarshalStrict([]byte(z.constraints), &constraintsList); err != nil { panic(err) } @@ -788,11 +787,11 @@ func compileTestCase(tc baseReportTestCase) (compiledTestCase, error) { }, nil } -func generateTableZone(t table, tableDesc sqlbase.TableDescriptor) (*zonepb.ZoneConfig, error) { +func generateTableZone(t table, tableDesc sqlbase.TableDescriptor) (*config.ZoneConfig, error) { // Create the table's zone config. - var tableZone *zonepb.ZoneConfig + var tableZone *config.ZoneConfig if t.zone != nil { - tableZone = new(zonepb.ZoneConfig) + tableZone = new(config.ZoneConfig) *tableZone = t.zone.toZoneConfig() } // Add subzones for the PK partitions. @@ -889,7 +888,7 @@ func makeTableDesc(t table, tableID int, dbID int) (sqlbase.TableDescriptor, err // // parent: Can be nil if the parent table doesn't have a zone of its own. In that // case, if any subzones are created, a placeholder zone will also be created and returned. -func addIndexSubzones(idx index, parent *zonepb.ZoneConfig, idxID int) *zonepb.ZoneConfig { +func addIndexSubzones(idx index, parent *config.ZoneConfig, idxID int) *config.ZoneConfig { res := parent ensureParent := func() { @@ -897,13 +896,13 @@ func addIndexSubzones(idx index, parent *zonepb.ZoneConfig, idxID int) *zonepb.Z return } // Create a placeholder zone config. - res = zonepb.NewZoneConfig() + res = config.NewZoneConfig() res.DeleteTableConfig() } if idx.zone != nil { ensureParent() - res.SetSubzone(zonepb.Subzone{ + res.SetSubzone(config.Subzone{ IndexID: uint32(idxID), PartitionName: "", Config: idx.zone.toZoneConfig(), @@ -915,7 +914,7 @@ func addIndexSubzones(idx index, parent *zonepb.ZoneConfig, idxID int) *zonepb.Z continue } ensureParent() - res.SetSubzone(zonepb.Subzone{ + res.SetSubzone(config.Subzone{ IndexID: uint32(idxID), PartitionName: p.name, Config: p.zone.toZoneConfig(), @@ -927,7 +926,7 @@ func addIndexSubzones(idx index, parent *zonepb.ZoneConfig, idxID int) *zonepb.Z // systemConfigBuilder build a system config. Clients will call some setters and then call build(). type systemConfigBuilder struct { kv []roachpb.KeyValue - defaultZoneConfig *zonepb.ZoneConfig + defaultZoneConfig *config.ZoneConfig zoneToObject map[ZoneKey]string } @@ -948,12 +947,12 @@ func (b *systemConfigBuilder) addZoneToObjectMapping(k ZoneKey, object string) e return nil } -func (b *systemConfigBuilder) setDefaultZoneConfig(cfg zonepb.ZoneConfig) error { +func (b *systemConfigBuilder) setDefaultZoneConfig(cfg config.ZoneConfig) error { b.defaultZoneConfig = &cfg return b.addZoneInner("default", keys.RootNamespaceID, cfg) } -func (b *systemConfigBuilder) addZoneInner(objectName string, id int, cfg zonepb.ZoneConfig) error { +func (b *systemConfigBuilder) addZoneInner(objectName string, id int, cfg config.ZoneConfig) error { k := config.MakeZoneKey(uint32(id)) var v roachpb.Value if err := v.SetProto(&cfg); err != nil { @@ -963,11 +962,11 @@ func (b *systemConfigBuilder) addZoneInner(objectName string, id int, cfg zonepb return b.addZoneToObjectMapping(MakeZoneKey(uint32(id), NoSubzone), objectName) } -func (b *systemConfigBuilder) addDatabaseZone(name string, id int, cfg zonepb.ZoneConfig) error { +func (b *systemConfigBuilder) addDatabaseZone(name string, id int, cfg config.ZoneConfig) error { return b.addZoneInner(name, id, cfg) } -func (b *systemConfigBuilder) addTableZone(t sqlbase.TableDescriptor, cfg zonepb.ZoneConfig) error { +func (b *systemConfigBuilder) addTableZone(t sqlbase.TableDescriptor, cfg config.ZoneConfig) error { if err := b.addZoneInner(t.Name, int(t.ID), cfg); err != nil { return err } diff --git a/pkg/storage/reports/critical_localities_report.go b/pkg/storage/reports/critical_localities_report.go index 1e5c9518a983..a8ab8604a60b 100644 --- a/pkg/storage/reports/critical_localities_report.go +++ b/pkg/storage/reports/critical_localities_report.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -265,7 +264,7 @@ func (r *replicationCriticalLocalitiesReportSaver) upsertLocality( // criticalLocalitiesVisitor is a visitor that, when passed to visitRanges(), builds // a LocalityReport. type criticalLocalitiesVisitor struct { - localityConstraints []zonepb.Constraints + localityConstraints []config.Constraints cfg *config.SystemConfig storeResolver StoreResolver nodeChecker nodeChecker @@ -283,7 +282,7 @@ var _ rangeVisitor = &criticalLocalitiesVisitor{} func makeLocalityStatsVisitor( ctx context.Context, - localityConstraints []zonepb.Constraints, + localityConstraints []config.Constraints, cfg *config.SystemConfig, storeResolver StoreResolver, nodeChecker nodeChecker, @@ -322,7 +321,7 @@ func (v *criticalLocalitiesVisitor) visitNewZone( // Get the zone. var zKey ZoneKey found, err := visitZones(ctx, r, v.cfg, ignoreSubzonePlaceholders, - func(_ context.Context, zone *zonepb.ZoneConfig, key ZoneKey) bool { + func(_ context.Context, zone *config.ZoneConfig, key ZoneKey) bool { if !zoneChangesReplication(zone) { return false } @@ -373,7 +372,7 @@ func processLocalityForRange( r *roachpb.RangeDescriptor, zoneKey ZoneKey, rep *replicationCriticalLocalitiesReportSaver, - c *zonepb.Constraints, + c *config.Constraints, cfg *config.SystemConfig, nodeChecker nodeChecker, storeDescs []roachpb.StoreDescriptor, @@ -406,7 +405,7 @@ func processLocalityForRange( storeHasConstraint := true for _, constraint := range c.Constraints { // For required constraints - consider unavailable nodes as not matching. - if !zonepb.StoreMatchesConstraint(storeDesc, constraint) { + if !config.StoreMatchesConstraint(storeDesc, constraint) { storeHasConstraint = false break } diff --git a/pkg/storage/reports/replication_stats_report.go b/pkg/storage/reports/replication_stats_report.go index 2817747b0d34..0db53c6070ea 100644 --- a/pkg/storage/reports/replication_stats_report.go +++ b/pkg/storage/reports/replication_stats_report.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -334,7 +333,7 @@ func (v *replicationStatsVisitor) reset(ctx context.Context) { } } -func (v *replicationStatsVisitor) ensureEntries(key ZoneKey, zone *zonepb.ZoneConfig) { +func (v *replicationStatsVisitor) ensureEntries(key ZoneKey, zone *config.ZoneConfig) { if zoneChangesReplication(zone) { v.report.EnsureEntry(key) } @@ -352,7 +351,7 @@ func (v *replicationStatsVisitor) visitNewZone( v.visitErr = retErr != nil }() var zKey ZoneKey - var zConfig *zonepb.ZoneConfig + var zConfig *config.ZoneConfig var numReplicas int // Figure out the zone config for whose report the current range is to be @@ -361,7 +360,7 @@ func (v *replicationStatsVisitor) visitNewZone( // factor this zone is configured with; the replication factor might be // inherited from a higher-level zone config. found, err := visitZones(ctx, r, v.cfg, ignoreSubzonePlaceholders, - func(_ context.Context, zone *zonepb.ZoneConfig, key ZoneKey) bool { + func(_ context.Context, zone *config.ZoneConfig, key ZoneKey) bool { if zConfig == nil { if !zoneChangesReplication(zone) { return false @@ -428,7 +427,7 @@ func (v *replicationStatsVisitor) countRange( // This is used to determine which zone's report a range counts towards for the // replication_stats and the critical_localities reports : it'll count towards // the lowest ancestor for which this method returns true. -func zoneChangesReplication(zone *zonepb.ZoneConfig) bool { +func zoneChangesReplication(zone *config.ZoneConfig) bool { return (zone.NumReplicas != nil && *zone.NumReplicas != 0) || zone.Constraints != nil } diff --git a/pkg/storage/reports/reporter.go b/pkg/storage/reports/reporter.go index fb561ca5e49d..eaccb9976097 100644 --- a/pkg/storage/reports/reporter.go +++ b/pkg/storage/reports/reporter.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -52,7 +51,7 @@ type Reporter struct { // Contains the list of the stores of the current node localStores *storage.Stores // Constraints constructed from the locality information - localityConstraints []zonepb.Constraints + localityConstraints []config.Constraints // The store that is the current meta 1 leaseholder meta1LeaseHolder *storage.Store // Latest zone config @@ -271,19 +270,19 @@ func (stats *Reporter) updateLatestConfig() { } func (stats *Reporter) updateLocalityConstraints() error { - localityConstraintsByName := make(map[string]zonepb.Constraints, 16) + localityConstraintsByName := make(map[string]config.Constraints, 16) for _, sd := range stats.storePool.GetStores() { - c := zonepb.Constraints{ - Constraints: make([]zonepb.Constraint, 0), + c := config.Constraints{ + Constraints: make([]config.Constraint, 0), } for _, t := range sd.Node.Locality.Tiers { c.Constraints = append( c.Constraints, - zonepb.Constraint{Type: zonepb.Constraint_REQUIRED, Key: t.Key, Value: t.Value}) + config.Constraint{Type: config.Constraint_REQUIRED, Key: t.Key, Value: t.Value}) localityConstraintsByName[c.String()] = c } } - stats.localityConstraints = make([]zonepb.Constraints, 0, len(localityConstraintsByName)) + stats.localityConstraints = make([]config.Constraints, 0, len(localityConstraintsByName)) for _, c := range localityConstraintsByName { stats.localityConstraints = append(stats.localityConstraints, c) } @@ -321,7 +320,7 @@ type zoneResolver struct { // curRootZone is the lowest zone convering the previously resolved range // that's not a subzone. // This is used to compute the subzone for a range. - curRootZone *zonepb.ZoneConfig + curRootZone *config.ZoneConfig // curZoneKey is the zone key for the previously resolved range. curZoneKey ZoneKey } @@ -339,7 +338,7 @@ func (c *zoneResolver) resolveRange( // setZone remembers the passed-in info as the reference for further // checkSameZone() calls. // Clients should generally use the higher-level updateZone(). -func (c *zoneResolver) setZone(objectID uint32, key ZoneKey, rootZone *zonepb.ZoneConfig) { +func (c *zoneResolver) setZone(objectID uint32, key ZoneKey, rootZone *config.ZoneConfig) { c.init = true c.curObjectID = objectID c.curRootZone = rootZone @@ -354,14 +353,14 @@ func (c *zoneResolver) updateZone( objectID, _ := config.DecodeKeyIntoZoneIDAndSuffix(rd.StartKey) first := true var zoneKey ZoneKey - var rootZone *zonepb.ZoneConfig + var rootZone *config.ZoneConfig // We're going to walk the zone hierarchy looking for two things: // 1) The lowest zone containing rd. We'll use the subzone ID for it. // 2) The lowest zone containing rd that's not a subzone. // visitZones() walks the zone hierarchy from the bottom upwards. found, err := visitZones( ctx, rd, cfg, includeSubzonePlaceholders, - func(_ context.Context, zone *zonepb.ZoneConfig, key ZoneKey) bool { + func(_ context.Context, zone *config.ZoneConfig, key ZoneKey) bool { if first { first = false zoneKey = key @@ -424,7 +423,7 @@ func visitZones( rng *roachpb.RangeDescriptor, cfg *config.SystemConfig, opt visitOpt, - visitor func(context.Context, *zonepb.ZoneConfig, ZoneKey) bool, + visitor func(context.Context, *config.ZoneConfig, ZoneKey) bool, ) (bool, error) { id, keySuffix := config.DecodeKeyIntoZoneIDAndSuffix(rng.StartKey) zone, err := getZoneByID(id, cfg) @@ -464,7 +463,7 @@ func visitAncestors( ctx context.Context, id uint32, cfg *config.SystemConfig, - visitor func(context.Context, *zonepb.ZoneConfig, ZoneKey) bool, + visitor func(context.Context, *config.ZoneConfig, ZoneKey) bool, ) (bool, error) { // Check to see if it's a table. If so, inherit from the database. // For all other cases, inherit from the default. @@ -502,7 +501,7 @@ func visitAncestors( func visitDefaultZone( ctx context.Context, cfg *config.SystemConfig, - visitor func(context.Context, *zonepb.ZoneConfig, ZoneKey) bool, + visitor func(context.Context, *config.ZoneConfig, ZoneKey) bool, ) bool { zone, err := getZoneByID(keys.RootNamespaceID, cfg) if err != nil { @@ -515,12 +514,12 @@ func visitDefaultZone( } // getZoneByID returns a zone given its id. Inheritance does not apply. -func getZoneByID(id uint32, cfg *config.SystemConfig) (*zonepb.ZoneConfig, error) { +func getZoneByID(id uint32, cfg *config.SystemConfig) (*config.ZoneConfig, error) { zoneVal := cfg.GetValue(config.MakeZoneKey(id)) if zoneVal == nil { return nil, nil } - zone := new(zonepb.ZoneConfig) + zone := new(config.ZoneConfig) if err := zoneVal.GetProto(zone); err != nil { return nil, err } @@ -530,7 +529,7 @@ func getZoneByID(id uint32, cfg *config.SystemConfig) (*zonepb.ZoneConfig, error // processRange returns the list of constraints violated by a range. The range // is represented by the descriptors of the replicas' stores. func processRange( - ctx context.Context, storeDescs []roachpb.StoreDescriptor, constraintGroups []zonepb.Constraints, + ctx context.Context, storeDescs []roachpb.StoreDescriptor, constraintGroups []config.Constraints, ) []ConstraintRepr { var res []ConstraintRepr // Evaluate all zone constraints for the stores (i.e. replicas) of the given range. @@ -551,7 +550,7 @@ func processRange( // constraintSatisfied checks that a range (represented by its replicas' stores) // satisfies a constraint. func constraintSatisfied( - c zonepb.Constraint, replicasRequiredToMatch int, storeDescs []roachpb.StoreDescriptor, + c config.Constraint, replicasRequiredToMatch int, storeDescs []roachpb.StoreDescriptor, ) bool { passCount := 0 for _, storeDesc := range storeDescs { @@ -562,11 +561,11 @@ func constraintSatisfied( } storeMatches := true - match := zonepb.StoreMatchesConstraint(storeDesc, c) - if c.Type == zonepb.Constraint_REQUIRED && !match { + match := config.StoreMatchesConstraint(storeDesc, c) + if c.Type == config.Constraint_REQUIRED && !match { storeMatches = false } - if c.Type == zonepb.Constraint_PROHIBITED && match { + if c.Type == config.Constraint_PROHIBITED && match { storeMatches = false } diff --git a/pkg/storage/reports/reporter_test.go b/pkg/storage/reports/reporter_test.go index 3d4bea1e29eb..37785d950615 100644 --- a/pkg/storage/reports/reporter_test.go +++ b/pkg/storage/reports/reporter_test.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -155,14 +154,14 @@ func TestCriticalLocalitiesReportIntegration(t *testing.T) { // Collect all the zones that exist at cluster bootstrap. systemZoneIDs := make([]int, 0, 10) - systemZones := make([]zonepb.ZoneConfig, 0, 10) + systemZones := make([]config.ZoneConfig, 0, 10) { rows, err := db.Query("select id, config from system.zones") require.NoError(t, err) for rows.Next() { var zoneID int var buf []byte - cfg := zonepb.ZoneConfig{} + cfg := config.ZoneConfig{} require.NoError(t, rows.Scan(&zoneID, &buf)) require.NoError(t, protoutil.Unmarshal(buf, &cfg)) systemZoneIDs = append(systemZoneIDs, zoneID) @@ -485,7 +484,7 @@ func TestZoneChecker(t *testing.T) { type tc struct { split string newZone bool - newRootZoneCfg *zonepb.ZoneConfig + newRootZoneCfg *config.ZoneConfig newZoneKey ZoneKey } // NB: IDs need to be beyond MaxSystemConfigDescID, otherwise special logic diff --git a/pkg/storage/split_queue.go b/pkg/storage/split_queue.go index 4d39213d8e37..eea79ba5248f 100644 --- a/pkg/storage/split_queue.go +++ b/pkg/storage/split_queue.go @@ -90,13 +90,9 @@ func newSplitQueue(store *Store, db *client.DB, gossip *gossip.Gossip) *splitQue } func shouldSplitRange( - ctx context.Context, - desc *roachpb.RangeDescriptor, - ms enginepb.MVCCStats, - maxBytes int64, - sysCfg *config.SystemConfig, + desc *roachpb.RangeDescriptor, ms enginepb.MVCCStats, maxBytes int64, sysCfg *config.SystemConfig, ) (shouldQ bool, priority float64) { - if sysCfg.NeedsSplit(ctx, desc.StartKey, desc.EndKey) { + if sysCfg.NeedsSplit(desc.StartKey, desc.EndKey) { // Set priority to 1 in the event the range is split by zone configs. priority = 1 shouldQ = true @@ -119,7 +115,7 @@ func shouldSplitRange( func (sq *splitQueue) shouldQueue( ctx context.Context, now hlc.Timestamp, repl *Replica, sysCfg *config.SystemConfig, ) (shouldQ bool, priority float64) { - shouldQ, priority = shouldSplitRange(ctx, repl.Desc(), repl.GetMVCCStats(), + shouldQ, priority = shouldSplitRange(repl.Desc(), repl.GetMVCCStats(), repl.GetMaxBytes(), sysCfg) if !shouldQ && repl.SplitByLoadEnabled() { @@ -163,7 +159,7 @@ func (sq *splitQueue) processAttempt( ) error { desc := r.Desc() // First handle the case of splitting due to zone config maps. - if splitKey := sysCfg.ComputeSplitKey(ctx, desc.StartKey, desc.EndKey); splitKey != nil { + if splitKey := sysCfg.ComputeSplitKey(desc.StartKey, desc.EndKey); splitKey != nil { if _, err := r.adminSplitWithDescriptor( ctx, roachpb.AdminSplitRequest{ diff --git a/pkg/storage/split_queue_test.go b/pkg/storage/split_queue_test.go index da0e8a1c4b10..11018106304e 100644 --- a/pkg/storage/split_queue_test.go +++ b/pkg/storage/split_queue_test.go @@ -16,7 +16,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" @@ -30,14 +29,13 @@ import ( func TestSplitQueueShouldQueue(t *testing.T) { defer leaktest.AfterTest(t)() tc := testContext{} - ctx := context.Background() stopper := stop.NewStopper() - defer stopper.Stop(ctx) + defer stopper.Stop(context.TODO()) tc.Start(t, stopper) // Set zone configs. - config.TestingSetZoneConfig(2000, zonepb.ZoneConfig{RangeMaxBytes: proto.Int64(32 << 20)}) - config.TestingSetZoneConfig(2002, zonepb.ZoneConfig{RangeMaxBytes: proto.Int64(32 << 20)}) + config.TestingSetZoneConfig(2000, config.ZoneConfig{RangeMaxBytes: proto.Int64(32 << 20)}) + config.TestingSetZoneConfig(2002, config.ZoneConfig{RangeMaxBytes: proto.Int64(32 << 20)}) testCases := []struct { start, end roachpb.RKey @@ -86,15 +84,14 @@ func TestSplitQueueShouldQueue(t *testing.T) { repl.mu.Lock() repl.mu.state.Stats = &enginepb.MVCCStats{KeyBytes: test.bytes} repl.mu.Unlock() - zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig := config.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(test.maxBytes) repl.SetZoneConfig(&zoneConfig) // Testing using shouldSplitRange instead of shouldQueue to avoid using the splitFinder // This tests the merge queue behavior too as a result. For splitFinder tests, // see split/split_test.go. - shouldQ, priority := shouldSplitRange( - ctx, repl.Desc(), repl.GetMVCCStats(), repl.GetMaxBytes(), cfg) + shouldQ, priority := shouldSplitRange(repl.Desc(), repl.GetMVCCStats(), repl.GetMaxBytes(), cfg) if shouldQ != test.shouldQ { t.Errorf("%d: should queue expected %t; got %t", i, test.shouldQ, shouldQ) } diff --git a/pkg/storage/store.go b/pkg/storage/store.go index d7df334e46da..b01e7b491f49 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" @@ -149,8 +148,8 @@ func TestStoreConfig(clock *hlc.Clock) StoreConfig { } st := cluster.MakeTestingClusterSettings() sc := StoreConfig{ - DefaultZoneConfig: zonepb.DefaultZoneConfigRef(), - DefaultSystemZoneConfig: zonepb.DefaultSystemZoneConfigRef(), + DefaultZoneConfig: config.DefaultZoneConfigRef(), + DefaultSystemZoneConfig: config.DefaultSystemZoneConfigRef(), Settings: st, AmbientCtx: log.AmbientContext{Tracer: st.Tracer}, Clock: clock, @@ -595,8 +594,8 @@ type StoreConfig struct { AmbientCtx log.AmbientContext base.RaftConfig - DefaultZoneConfig *zonepb.ZoneConfig - DefaultSystemZoneConfig *zonepb.ZoneConfig + DefaultZoneConfig *config.ZoneConfig + DefaultSystemZoneConfig *config.ZoneConfig Settings *cluster.Settings Clock *hlc.Clock DB *client.DB diff --git a/pkg/storage/store_pool.go b/pkg/storage/store_pool.go index e32e0fd1cb10..039aad2acdb2 100644 --- a/pkg/storage/store_pool.go +++ b/pkg/storage/store_pool.go @@ -17,7 +17,7 @@ import ( "sort" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -561,7 +561,7 @@ func (sl StoreList) String() string { // filter takes a store list and filters it using the passed in constraints. It // maintains the original order of the passed in store list. -func (sl StoreList) filter(constraints []zonepb.Constraints) StoreList { +func (sl StoreList) filter(constraints []config.Constraints) StoreList { if len(constraints) == 0 { return sl } diff --git a/pkg/storage/store_pool_test.go b/pkg/storage/store_pool_test.go index f0285760891f..5c04148cdda0 100644 --- a/pkg/storage/store_pool_test.go +++ b/pkg/storage/store_pool_test.go @@ -20,7 +20,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -103,7 +103,7 @@ func createTestStorePool( log.AmbientContext{Tracer: st.Tracer}, &base.Config{Insecure: true}, clock, stopper, &st.Version) server := rpc.NewServer(rpcContext) // never started - g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) + g := gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), config.DefaultZoneConfigRef()) mnl := newMockNodeLiveness(defaultNodeStatus) TimeUntilStoreDead.Override(&st.SV, timeUntilStoreDeadValue) @@ -148,7 +148,7 @@ func TestStorePoolGossipUpdate(t *testing.T) { // verifyStoreList ensures that the returned list of stores is correct. func verifyStoreList( sp *StorePool, - constraints []zonepb.Constraints, + constraints []config.Constraints, storeIDs roachpb.StoreIDSlice, // optional rangeID roachpb.RangeID, filter storeFilter, @@ -197,11 +197,11 @@ func TestStorePoolGetStoreList(t *testing.T) { storagepb.NodeLivenessStatus_DEAD) defer stopper.Stop(context.TODO()) sg := gossiputil.NewStoreGossiper(g) - constraints := []zonepb.Constraints{ + constraints := []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Type: zonepb.Constraint_REQUIRED, Value: "ssd"}, - {Type: zonepb.Constraint_REQUIRED, Value: "dc"}, + Constraints: []config.Constraint{ + {Type: config.Constraint_REQUIRED, Value: "ssd"}, + {Type: config.Constraint_REQUIRED, Value: "dc"}, }, }, } @@ -354,13 +354,13 @@ func TestStorePoolGetStoreList(t *testing.T) { func TestStoreListFilter(t *testing.T) { defer leaktest.AfterTest(t)() - constraints := []zonepb.Constraints{ + constraints := []config.Constraints{ { - Constraints: []zonepb.Constraint{ - {Type: zonepb.Constraint_REQUIRED, Key: "region", Value: "us-west"}, - {Type: zonepb.Constraint_REQUIRED, Value: "MustMatch"}, - {Type: zonepb.Constraint_DEPRECATED_POSITIVE, Value: "MatchingOptional"}, - {Type: zonepb.Constraint_PROHIBITED, Value: "MustNotMatch"}, + Constraints: []config.Constraint{ + {Type: config.Constraint_REQUIRED, Key: "region", Value: "us-west"}, + {Type: config.Constraint_REQUIRED, Value: "MustMatch"}, + {Type: config.Constraint_DEPRECATED_POSITIVE, Value: "MatchingOptional"}, + {Type: config.Constraint_PROHIBITED, Value: "MustNotMatch"}, }, }, } diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 5f5391098e9f..bf0d8859a8e0 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" @@ -1498,8 +1497,8 @@ func TestStoreSetRangesMaxBytes(t *testing.T) { } // Set zone configs. - config.TestingSetZoneConfig(baseID, zonepb.ZoneConfig{RangeMaxBytes: proto.Int64(1 << 20)}) - config.TestingSetZoneConfig(baseID+2, zonepb.ZoneConfig{RangeMaxBytes: proto.Int64(2 << 20)}) + config.TestingSetZoneConfig(baseID, config.ZoneConfig{RangeMaxBytes: proto.Int64(1 << 20)}) + config.TestingSetZoneConfig(baseID+2, config.ZoneConfig{RangeMaxBytes: proto.Int64(2 << 20)}) // Despite faking the zone configs, we still need to have a system config // entry so that the store picks up the new zone configs. This new system diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index 3d4b02531d28..0b84b3e2759f 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -116,7 +115,7 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto cfg.RPCContext.NodeID.Set(ambient.AnnotateCtx(context.Background()), nodeID) c := &cfg.RPCContext.ClusterID server := rpc.NewServer(cfg.RPCContext) // never started - ltc.Gossip = gossip.New(ambient, c, nc, cfg.RPCContext, server, ltc.Stopper, metric.NewRegistry(), roachpb.Locality{}, zonepb.DefaultZoneConfigRef()) + ltc.Gossip = gossip.New(ambient, c, nc, cfg.RPCContext, server, ltc.Stopper, metric.NewRegistry(), roachpb.Locality{}, config.DefaultZoneConfigRef()) ltc.Eng = engine.NewInMem(roachpb.Attributes{}, 50<<20) ltc.Stopper.AddCloser(ltc.Eng) diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index b99a3fb3de38..1f8fb6f84d23 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -18,7 +18,6 @@ package serverutils import ( - "context" gosql "database/sql" "net/http" "net/url" @@ -157,9 +156,6 @@ type TestServerInterface interface { // assuming no additional information is added outside of the normal bootstrap // process. ExpectedInitialRangeCount() (int, error) - - // ForceTableGC sends a GCRequest for the ranges corresponding to a table. - ForceTableGC(ctx context.Context, database, table string, timestamp hlc.Timestamp) error } // TestServerFactory encompasses the actual implementation of the shim diff --git a/pkg/testutils/sqlutils/gc.go b/pkg/testutils/sqlutils/gc.go new file mode 100644 index 000000000000..ef06e8571346 --- /dev/null +++ b/pkg/testutils/sqlutils/gc.go @@ -0,0 +1,47 @@ +// Copyright 2019 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 sqlutils + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" +) + +// ForceTableGC sends a GCRequest for the ranges corresponding to a table. +func ForceTableGC( + t testing.TB, + distSender *kv.DistSender, + db DBHandle, + database, table string, + timestamp hlc.Timestamp, +) { + t.Helper() + tblID := QueryTableID(t, db, database, table) + tblKey := roachpb.Key(keys.MakeTablePrefix(tblID)) + gcr := roachpb.GCRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: tblKey, + EndKey: tblKey.PrefixEnd(), + }, + Threshold: timestamp, + } + if _, err := client.SendWrapped( + context.Background(), distSender, &gcr, + ); err != nil { + t.Error(err) + } +} diff --git a/pkg/testutils/sqlutils/zone.go b/pkg/testutils/sqlutils/zone.go index 93497e60c1b0..8540871accd1 100644 --- a/pkg/testutils/sqlutils/zone.go +++ b/pkg/testutils/sqlutils/zone.go @@ -15,7 +15,7 @@ import ( "fmt" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) @@ -23,7 +23,7 @@ import ( // ZoneRow represents a row returned by SHOW ZONE CONFIGURATION. type ZoneRow struct { ID uint32 - Config zonepb.ZoneConfig + Config config.ZoneConfig } func (row ZoneRow) sqlRowString() ([]string, error) { @@ -42,7 +42,7 @@ func RemoveAllZoneConfigs(t testing.TB, sqlDB *SQLRunner) { t.Helper() for _, row := range sqlDB.QueryStr(t, "SHOW ALL ZONE CONFIGURATIONS") { target := row[0] - if target == fmt.Sprintf("RANGE %s", zonepb.DefaultZoneName) { + if target == fmt.Sprintf("RANGE %s", config.DefaultZoneName) { // The default zone cannot be removed. continue } diff --git a/pkg/util/protoutil/clone_test.go b/pkg/util/protoutil/clone_test.go index a17ee364c0ce..ea4d2045bb4d 100644 --- a/pkg/util/protoutil/clone_test.go +++ b/pkg/util/protoutil/clone_test.go @@ -16,7 +16,7 @@ import ( "strings" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -40,7 +40,7 @@ func TestCloneProto(t *testing.T) { // Cloneable types. This includes all types for which a // protoutil.Clone call exists in the codebase as of 2016-11-21. - {&zonepb.ZoneConfig{}, false}, + {&config.ZoneConfig{}, false}, {&gossip.Info{}, false}, {&gossip.BootstrapInfo{}, false}, {&sqlbase.IndexDescriptor{}, false},