From 7be718ae59479add9e7803f7ad96e2116c06db4f Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 28 May 2020 14:27:43 -0400 Subject: [PATCH 1/5] config: remove outdated TODO for Marc We don't return nil values from MVCCScan. Also, a bit of harmless cleanup. --- pkg/config/system.go | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/pkg/config/system.go b/pkg/config/system.go index a17a9cc3ff5e..39b7958714da 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -152,10 +152,8 @@ func (s *SystemConfig) GetValue(key roachpb.Key) *roachpb.Value { // get searches the kv list for 'key' and returns its roachpb.KeyValue // if found. func (s *SystemConfig) get(key roachpb.Key) *roachpb.KeyValue { - if index, found := s.GetIndex(key); found { - // TODO(marc): I'm pretty sure a Value returned by MVCCScan can - // never be nil. Should check. - return &s.Values[index] + if i, ok := s.GetIndex(key); ok { + return &s.Values[i] } return nil } @@ -163,13 +161,13 @@ func (s *SystemConfig) get(key roachpb.Key) *roachpb.KeyValue { // GetIndex searches the kv list for 'key' and returns its index if found. func (s *SystemConfig) GetIndex(key roachpb.Key) (int, bool) { l := len(s.Values) - index := sort.Search(l, func(i int) bool { - return bytes.Compare(s.Values[i].Key, key) >= 0 + i := sort.Search(l, func(i int) bool { + return key.Compare(s.Values[i].Key) <= 0 }) - if index == l || !key.Equal(s.Values[index].Key) { + if i == l || !key.Equal(s.Values[i].Key) { return 0, false } - return index, true + return i, true } // GetLargestObjectID returns the largest object ID found in the config which is @@ -207,7 +205,7 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { return uint32(id), nil } - // Maximum specified: need to search the descriptor table. Binary search + // Maximum specified: need to search the descriptor table. Binary search // through all descriptor table values to find the first descriptor with ID // >= maxID. searchSlice := s.Values[lowIndex:highIndex] From 01856bb78886f66810b7665b713b52ec85a58216 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 28 May 2020 16:03:34 -0400 Subject: [PATCH 2/5] config: more minor cleanups Mostly cleaning up comments. Noticed while re-familiarizing myself with this package. --- pkg/config/system.go | 10 +++++----- pkg/config/zonepb/zone.pb.go | 26 +++++++++++++------------- pkg/config/zonepb/zone.proto | 9 ++++++--- pkg/sql/zone_config.go | 22 +++++++++++++++------- 4 files changed, 39 insertions(+), 28 deletions(-) diff --git a/pkg/config/system.go b/pkg/config/system.go index 39b7958714da..78ee997034ab 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -516,11 +516,11 @@ func (s *SystemConfig) NeedsSplit(startKey, endKey roachpb.RKey) bool { // 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 ID isn't found in the cache. -func (s *SystemConfig) shouldSplit(ID uint32) bool { +func (s *SystemConfig) shouldSplit(id uint32) bool { // Check the cache. { s.mu.RLock() - shouldSplit, ok := s.mu.shouldSplitCache[ID] + shouldSplit, ok := s.mu.shouldSplitCache[id] s.mu.RUnlock() if ok { return shouldSplit @@ -528,17 +528,17 @@ func (s *SystemConfig) shouldSplit(ID uint32) bool { } var shouldSplit bool - if ID < keys.MinUserDescID { + if id < keys.MinUserDescID { // The ID might be one of the reserved IDs that refer to ranges but not any // actual descriptors. shouldSplit = true } else { - desc := s.GetDesc(keys.TODOSQLCodec.DescMetadataKey(ID)) + desc := s.GetDesc(keys.TODOSQLCodec.DescMetadataKey(id)) shouldSplit = desc != nil && sqlbase.ShouldSplitAtDesc(desc) } // Populate the cache. s.mu.Lock() - s.mu.shouldSplitCache[ID] = shouldSplit + s.mu.shouldSplitCache[id] = shouldSplit s.mu.Unlock() return shouldSplit } diff --git a/pkg/config/zonepb/zone.pb.go b/pkg/config/zonepb/zone.pb.go index d9d49f181a7c..8990d9488f34 100644 --- a/pkg/config/zonepb/zone.pb.go +++ b/pkg/config/zonepb/zone.pb.go @@ -64,7 +64,7 @@ func (x *Constraint_Type) UnmarshalJSON(data []byte) error { return nil } func (Constraint_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_zone_84b39e9c66d84dc0, []int{1, 0} + return fileDescriptor_zone_cd8e37a638b916ef, []int{1, 0} } // GCPolicy defines garbage collection policies which apply to MVCC @@ -76,7 +76,7 @@ func (Constraint_Type) EnumDescriptor() ([]byte, []int) { type GCPolicy struct { // TTLSeconds specifies the maximum age of a value before it's // garbage collected. Only older versions of values are garbage - // collected. Specifying <=0 mean older versions are never GC'd. + // collected. Specifying <= 0 mean older versions are never GC'd. TTLSeconds int32 `protobuf:"varint,1,opt,name=ttl_seconds,json=ttlSeconds" json:"ttl_seconds"` } @@ -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_84b39e9c66d84dc0, []int{0} + return fileDescriptor_zone_cd8e37a638b916ef, []int{0} } func (m *GCPolicy) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -109,7 +109,7 @@ func (m *GCPolicy) XXX_DiscardUnknown() { var xxx_messageInfo_GCPolicy proto.InternalMessageInfo -// Constraint constrains the stores a replica can be stored on. +// Constraint constrains the stores that 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"` // Key is only set if this is a constraint on locality. @@ -121,7 +121,7 @@ type Constraint struct { func (m *Constraint) Reset() { *m = Constraint{} } func (*Constraint) ProtoMessage() {} func (*Constraint) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_84b39e9c66d84dc0, []int{1} + return fileDescriptor_zone_cd8e37a638b916ef, []int{1} } func (m *Constraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -163,7 +163,7 @@ type ConstraintsConjunction struct { func (m *ConstraintsConjunction) Reset() { *m = ConstraintsConjunction{} } func (*ConstraintsConjunction) ProtoMessage() {} func (*ConstraintsConjunction) Descriptor() ([]byte, []int) { - return fileDescriptor_zone_84b39e9c66d84dc0, []int{2} + return fileDescriptor_zone_cd8e37a638b916ef, []int{2} } func (m *ConstraintsConjunction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -198,7 +198,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_84b39e9c66d84dc0, []int{3} + return fileDescriptor_zone_cd8e37a638b916ef, []int{3} } func (m *LeasePreference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -225,7 +225,7 @@ var xxx_messageInfo_LeasePreference proto.InternalMessageInfo // ZoneConfig holds configuration that applies to one or more ranges. // -// Note: when adding.removing fields here, be sure to update +// Note: when adding/removing fields here, be sure to update // supportedZoneConfigOptions in the sql package, to synchronize // with the code for ALTER ... CONFIGURE ZONE. type ZoneConfig struct { @@ -276,7 +276,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_84b39e9c66d84dc0, []int{4} + return fileDescriptor_zone_cd8e37a638b916ef, []int{4} } func (m *ZoneConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -317,7 +317,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_84b39e9c66d84dc0, []int{5} + return fileDescriptor_zone_cd8e37a638b916ef, []int{5} } func (m *Subzone) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -363,7 +363,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_84b39e9c66d84dc0, []int{6} + return fileDescriptor_zone_cd8e37a638b916ef, []int{6} } func (m *SubzoneSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2381,9 +2381,9 @@ var ( ErrIntOverflowZone = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("config/zonepb/zone.proto", fileDescriptor_zone_84b39e9c66d84dc0) } +func init() { proto.RegisterFile("config/zonepb/zone.proto", fileDescriptor_zone_cd8e37a638b916ef) } -var fileDescriptor_zone_84b39e9c66d84dc0 = []byte{ +var fileDescriptor_zone_cd8e37a638b916ef = []byte{ // 873 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0x3d, 0x73, 0xe3, 0x44, 0x18, 0xf6, 0xda, 0x8e, 0xad, 0x7b, 0xed, 0x24, 0x66, 0xef, 0x48, 0x44, 0x18, 0x2c, 0x23, 0x60, diff --git a/pkg/config/zonepb/zone.proto b/pkg/config/zonepb/zone.proto index 747c8d1f7ce6..93200a9c9684 100644 --- a/pkg/config/zonepb/zone.proto +++ b/pkg/config/zonepb/zone.proto @@ -26,11 +26,11 @@ message GCPolicy { // TTLSeconds specifies the maximum age of a value before it's // garbage collected. Only older versions of values are garbage - // collected. Specifying <=0 mean older versions are never GC'd. + // collected. Specifying <= 0 mean older versions are never GC'd. optional int32 ttl_seconds = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "TTLSeconds"]; } -// Constraint constrains the stores a replica can be stored on. +// Constraint constrains the stores that a replica can be stored on. message Constraint { option (gogoproto.equal) = true; option (gogoproto.goproto_stringer) = false; @@ -82,7 +82,7 @@ message LeasePreference { // ZoneConfig holds configuration that applies to one or more ranges. // -// Note: when adding.removing fields here, be sure to update +// Note: when adding/removing fields here, be sure to update // supportedZoneConfigOptions in the sql package, to synchronize // with the code for ALTER ... CONFIGURE ZONE. message ZoneConfig { @@ -92,11 +92,14 @@ message ZoneConfig { reserved 1; optional int64 range_min_bytes = 2 [(gogoproto.moretags) = "yaml:\"range_min_bytes\""]; optional int64 range_max_bytes = 3 [(gogoproto.moretags) = "yaml:\"range_max_bytes\""]; + // If GC policy is not set, uses the next highest, non-null policy // in the zone config hierarchy, up to the default policy if necessary. optional GCPolicy gc = 4 [(gogoproto.customname) = "GC"]; + // NumReplicas specifies the desired number of replicas optional int32 num_replicas = 5 [(gogoproto.moretags) = "yaml:\"num_replicas\""]; + // Constraints constrains which stores the replicas can be stored on. The // order in which the constraints are stored is arbitrary and may change. // https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20160706_expressive_zone_config.md#constraint-system diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 6abee22e79a4..3239ae641312 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -49,7 +49,7 @@ var zoneConfigCodec = keys.TODOSQLCodec // // This function must be kept in sync with ascendZoneSpecifier. // -// if getInheritedDefault is true, the direct zone configuration, if it exists, is +// If getInheritedDefault is true, the direct zone configuration, if it exists, is // 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, @@ -151,10 +151,13 @@ func completeZoneConfig( return nil } -// ZoneConfigHook returns the zone config for the object with id using the -// cached system config. If keySuffix is within a subzone, the subzone's config -// is returned instead. The bool is set to true when the value returned is -// cached. +// ZoneConfigHook returns the zone config and optional placeholder config for +// the object with id using the cached system config. The returned boolean is +// set to true when the zone config returned can be cached. +// +// ZoneConfigHook is a pure function whose only inputs are a system config and +// an object ID. It does not make any external KV calls to look up additional +// state. func ZoneConfigHook( cfg *config.SystemConfig, id uint32, ) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) { @@ -174,8 +177,13 @@ func ZoneConfigHook( return zone, placeholder, true, nil } -// GetZoneConfigInTxn looks up the zone and subzone for the specified -// object ID, index, and partition. +// GetZoneConfigInTxn looks up the zone and subzone for the specified object ID, +// index, and partition. See the documentation on getZoneConfig for information +// about the getInheritedDefault parameter. +// +// Unlike ZoneConfigHook, GetZoneConfigInTxn does not used a cached system +// config. Instead, it uses the provided txn to make transactionally consistent +// KV lookups. func GetZoneConfigInTxn( ctx context.Context, txn *kv.Txn, From a102b1415faa459d84bf7569269f5259665b9a99 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 1 Jun 2020 22:39:03 -0400 Subject: [PATCH 3/5] config: fix GetLargestObjectID for pseudo object IDs This was not behaving correctly, and was breaking things when a pseudo table had the largest ID below MaxReservedDescID. --- pkg/config/system.go | 36 +++++++++---- pkg/config/system_test.go | 52 ++++++++++++++----- pkg/keys/sql.go | 4 +- .../reports/constraint_stats_report.go | 4 +- .../reports/replication_stats_report.go | 5 +- 5 files changed, 72 insertions(+), 29 deletions(-) diff --git a/pkg/config/system.go b/pkg/config/system.go index 78ee997034ab..a891a4b43e8c 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -171,9 +171,10 @@ func (s *SystemConfig) GetIndex(key roachpb.Key) (int, bool) { } // GetLargestObjectID returns the largest object ID found in the config which is -// less than or equal to maxID. If maxID is 0, returns the largest ID in the -// config. -func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { +// less than or equal to maxID. The objects in the config are augmented with the +// provided pseudo IDs. If maxID is 0, returns the largest ID in the config +// (again, augmented by the pseudo IDs). +func (s *SystemConfig) GetLargestObjectID(maxID uint32, pseudoIDs []uint32) (uint32, error) { testingLock.Lock() hook := testingLargestIDHook testingLock.Unlock() @@ -190,28 +191,38 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { lowIndex := sort.Search(len(s.Values), func(i int) bool { return bytes.Compare(s.Values[i].Key, lowBound) >= 0 }) - if highIndex == lowIndex { return 0, fmt.Errorf("descriptor table not found in system config of %d values", len(s.Values)) } + // Determine the largest pseudo table ID equal to or below maxID. + maxPseudoID := uint32(0) + for _, id := range pseudoIDs { + if id > maxPseudoID && (maxID == 0 || id <= maxID) { + maxPseudoID = id + } + } + // No maximum specified; maximum ID is the last entry in the descriptor - // table. + // table or the largest pseudo ID, whichever is larger. if maxID == 0 { id, err := keys.TODOSQLCodec.DecodeDescMetadataID(s.Values[highIndex-1].Key) if err != nil { return 0, err } - return uint32(id), nil + if id < maxPseudoID { + id = maxPseudoID + } + return id, nil } // Maximum specified: need to search the descriptor table. Binary search // through all descriptor table values to find the first descriptor with ID - // >= maxID. + // >= maxID and pick either it or maxPseudoID, whichever is larger. searchSlice := s.Values[lowIndex:highIndex] var err error maxIdx := sort.Search(len(searchSlice), func(i int) bool { - var id uint64 + var id uint32 id, err = keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[i].Key) if err != nil { return false @@ -243,7 +254,10 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { if err != nil { return 0, err } - return uint32(id), nil + if id < maxPseudoID { + id = maxPseudoID + } + return id, nil } // GetZoneConfigForKey looks up the zone config for the object (table @@ -487,7 +501,7 @@ func (s *SystemConfig) ComputeSplitKey(startKey, endKey roachpb.RKey) (rr roachp // If the startKey falls within the non-system reserved range, compute those // keys first. if startID <= keys.MaxReservedDescID { - endID, err := s.GetLargestObjectID(keys.MaxReservedDescID) + endID, err := s.GetLargestObjectID(keys.MaxReservedDescID, keys.PseudoTableIDs) if err != nil { log.Errorf(context.TODO(), "unable to determine largest reserved object ID from system config: %s", err) return nil @@ -499,7 +513,7 @@ func (s *SystemConfig) ComputeSplitKey(startKey, endKey roachpb.RKey) (rr roachp } // Find the split key in the user space. - endID, err := s.GetLargestObjectID(0) + endID, err := s.GetLargestObjectID(0, keys.PseudoTableIDs) if err != nil { log.Errorf(context.TODO(), "unable to determine largest object ID from system config: %s", err) return nil diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index 925acc76f53a..2339760a22e1 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -126,28 +126,29 @@ func TestGetLargestID(t *testing.T) { defer leaktest.AfterTest(t)() type testCase struct { - values []roachpb.KeyValue - largest uint32 - maxID uint32 - errStr string + values []roachpb.KeyValue + largest uint32 + maxID uint32 + pseudoIDs []uint32 + errStr string } testCases := []testCase{ // No data. - {nil, 0, 0, "descriptor table not found"}, + {nil, 0, 0, nil, "descriptor table not found"}, // Some data, but not from the system span. - {[]roachpb.KeyValue{plainKV("a", "b")}, 0, 0, "descriptor table not found"}, + {[]roachpb.KeyValue{plainKV("a", "b")}, 0, 0, nil, "descriptor table not found"}, // Some real data, but no descriptors. {[]roachpb.KeyValue{ sqlKV(keys.NamespaceTableID, 1, 1), sqlKV(keys.NamespaceTableID, 1, 2), sqlKV(keys.UsersTableID, 1, 3), - }, 0, 0, "descriptor table not found"}, + }, 0, 0, nil, "descriptor table not found"}, // Single correct descriptor entry. - {[]roachpb.KeyValue{sqlKV(keys.DescriptorTableID, 1, 1)}, 1, 0, ""}, + {[]roachpb.KeyValue{sqlKV(keys.DescriptorTableID, 1, 1)}, 1, 0, nil, ""}, // Surrounded by other data. {[]roachpb.KeyValue{ @@ -155,7 +156,7 @@ func TestGetLargestID(t *testing.T) { sqlKV(keys.NamespaceTableID, 1, 30), sqlKV(keys.DescriptorTableID, 1, 8), sqlKV(keys.ZonesTableID, 1, 40), - }, 8, 0, ""}, + }, 8, 0, nil, ""}, // Descriptors with holes. Index ID does not matter. {[]roachpb.KeyValue{ @@ -163,7 +164,7 @@ func TestGetLargestID(t *testing.T) { sqlKV(keys.DescriptorTableID, 2, 5), sqlKV(keys.DescriptorTableID, 3, 8), sqlKV(keys.DescriptorTableID, 4, 12), - }, 12, 0, ""}, + }, 12, 0, nil, ""}, // Real SQL layout. func() testCase { @@ -171,7 +172,8 @@ func TestGetLargestID(t *testing.T) { descIDs := ms.DescriptorIDs() maxDescID := descIDs[len(descIDs)-1] kvs, _ /* splits */ := ms.GetInitialValues() - return testCase{kvs, uint32(maxDescID), 0, ""} + pseudoIDs := keys.PseudoTableIDs + return testCase{kvs, uint32(maxDescID), 0, pseudoIDs, ""} }(), // Test non-zero max. @@ -180,7 +182,7 @@ func TestGetLargestID(t *testing.T) { sqlKV(keys.DescriptorTableID, 2, 5), sqlKV(keys.DescriptorTableID, 3, 8), sqlKV(keys.DescriptorTableID, 4, 12), - }, 8, 8, ""}, + }, 8, 8, nil, ""}, // Test non-zero max. {[]roachpb.KeyValue{ @@ -188,13 +190,35 @@ func TestGetLargestID(t *testing.T) { sqlKV(keys.DescriptorTableID, 2, 5), sqlKV(keys.DescriptorTableID, 3, 8), sqlKV(keys.DescriptorTableID, 4, 12), - }, 5, 7, ""}, + }, 5, 7, nil, ""}, + + // Test pseudo ID (MetaRangesID = 16), exact. + {[]roachpb.KeyValue{ + sqlKV(keys.DescriptorTableID, 1, 1), + sqlKV(keys.DescriptorTableID, 4, 12), + sqlKV(keys.DescriptorTableID, 4, 19), + sqlKV(keys.DescriptorTableID, 4, 22), + }, 16, 16, []uint32{16, 17, 18}, ""}, + + // Test pseudo ID (TimeseriesRangesID = 18), above. + {[]roachpb.KeyValue{ + sqlKV(keys.DescriptorTableID, 1, 1), + sqlKV(keys.DescriptorTableID, 4, 12), + sqlKV(keys.DescriptorTableID, 4, 21), + sqlKV(keys.DescriptorTableID, 4, 22), + }, 18, 20, []uint32{16, 17, 18}, ""}, + + // Test pseudo ID (TimeseriesRangesID = 18), largest. + {[]roachpb.KeyValue{ + sqlKV(keys.DescriptorTableID, 1, 1), + sqlKV(keys.DescriptorTableID, 4, 12), + }, 18, 0, []uint32{16, 17, 18}, ""}, } cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) for tcNum, tc := range testCases { cfg.Values = tc.values - ret, err := cfg.GetLargestObjectID(tc.maxID) + ret, err := cfg.GetLargestObjectID(tc.maxID, tc.pseudoIDs) if !testutils.IsError(err, tc.errStr) { t.Errorf("#%d: expected err=%q, got %v", tcNum, tc.errStr, err) continue diff --git a/pkg/keys/sql.go b/pkg/keys/sql.go index 25388e800f2a..9451c294422e 100644 --- a/pkg/keys/sql.go +++ b/pkg/keys/sql.go @@ -217,7 +217,7 @@ func (d sqlDecoder) DecodeIndexPrefix(key roachpb.Key) ([]byte, uint32, uint32, } // DecodeDescMetadataID decodes a descriptor ID from a descriptor metadata key. -func (d sqlDecoder) DecodeDescMetadataID(key roachpb.Key) (uint64, error) { +func (d sqlDecoder) DecodeDescMetadataID(key roachpb.Key) (uint32, error) { // Extract table and index ID from key. remaining, tableID, _, err := d.DecodeIndexPrefix(key) if err != nil { @@ -231,5 +231,5 @@ func (d sqlDecoder) DecodeDescMetadataID(key roachpb.Key) (uint64, error) { if err != nil { return 0, err } - return id, nil + return uint32(id), nil } diff --git a/pkg/kv/kvserver/reports/constraint_stats_report.go b/pkg/kv/kvserver/reports/constraint_stats_report.go index fd26dd6ca612..4a111bbe85ad 100644 --- a/pkg/kv/kvserver/reports/constraint_stats_report.go +++ b/pkg/kv/kvserver/reports/constraint_stats_report.go @@ -19,6 +19,7 @@ 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/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -389,7 +390,8 @@ func (v *constraintConformanceVisitor) reset(ctx context.Context) { // zones that have constraints. Otherwise, just iterating through the ranges // wouldn't create entries for constraints that aren't violated, and // definitely not for zones that don't apply to any ranges. - maxObjectID, err := v.cfg.GetLargestObjectID(0 /* maxID - return the largest ID in the config */) + maxObjectID, err := v.cfg.GetLargestObjectID( + 0 /* maxID - return the largest ID in the config */, keys.PseudoTableIDs) if err != nil { log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) } diff --git a/pkg/kv/kvserver/reports/replication_stats_report.go b/pkg/kv/kvserver/reports/replication_stats_report.go index 03bb639e5af9..c22686118db8 100644 --- a/pkg/kv/kvserver/reports/replication_stats_report.go +++ b/pkg/kv/kvserver/reports/replication_stats_report.go @@ -17,6 +17,7 @@ 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/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -298,7 +299,9 @@ func (v *replicationStatsVisitor) reset(ctx context.Context) { // Iterate through all the zone configs to create report entries for all the // zones that have constraints. Otherwise, just iterating through the ranges // wouldn't create entries for zones that don't apply to any ranges. - maxObjectID, err := v.cfg.GetLargestObjectID(0 /* maxID - return the largest ID in the config */) + maxObjectID, err := v.cfg.GetLargestObjectID( + 0 /* maxID - return the largest ID in the config */, keys.PseudoTableIDs, + ) if err != nil { log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) } From d8ae78a4e9d06dfcce341b6aeaa8664f36bf9ad9 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 3 Jun 2020 18:08:29 -0400 Subject: [PATCH 4/5] config: properly handle decoding errors in GetLargestObjectID This was wrong before. Adds testing. --- pkg/config/system.go | 4 ++-- pkg/config/system_test.go | 14 ++++++++++++++ pkg/keys/sql.go | 4 ++++ 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/pkg/config/system.go b/pkg/config/system.go index a891a4b43e8c..892e90ff4116 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -222,11 +222,11 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32, pseudoIDs []uint32) (uin searchSlice := s.Values[lowIndex:highIndex] var err error maxIdx := sort.Search(len(searchSlice), func(i int) bool { - var id uint32 - id, err = keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[i].Key) if err != nil { return false } + var id uint32 + id, err = keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[i].Key) return uint32(id) >= maxID }) if err != nil { diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index 2339760a22e1..fd7957ea5144 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -11,6 +11,7 @@ package config_test import ( + "math" "sort" "testing" @@ -147,6 +148,19 @@ func TestGetLargestID(t *testing.T) { sqlKV(keys.UsersTableID, 1, 3), }, 0, 0, nil, "descriptor table not found"}, + // Decoding error, unbounded max. + {[]roachpb.KeyValue{ + sqlKV(keys.DescriptorTableID, 1, 1), + sqlKV(keys.DescriptorTableID, 1, math.MaxUint64), + }, 0, 0, nil, "descriptor ID 18446744073709551615 exceeds uint32 bounds"}, + + // Decoding error, bounded max. + {[]roachpb.KeyValue{ + sqlKV(keys.DescriptorTableID, 1, 1), + sqlKV(keys.DescriptorTableID, 1, math.MaxUint64), + sqlKV(keys.DescriptorTableID, 2, 1), + }, 0, 5, nil, "descriptor ID 18446744073709551615 exceeds uint32 bounds"}, + // Single correct descriptor entry. {[]roachpb.KeyValue{sqlKV(keys.DescriptorTableID, 1, 1)}, 1, 0, nil, ""}, diff --git a/pkg/keys/sql.go b/pkg/keys/sql.go index 9451c294422e..003a12823eb3 100644 --- a/pkg/keys/sql.go +++ b/pkg/keys/sql.go @@ -12,6 +12,7 @@ package keys import ( "bytes" + "math" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -231,5 +232,8 @@ func (d sqlDecoder) DecodeDescMetadataID(key roachpb.Key) (uint32, error) { if err != nil { return 0, err } + if id > math.MaxUint32 { + return 0, errors.Errorf("descriptor ID %d exceeds uint32 bounds", id) + } return uint32(id), nil } From 695318206ddc56bfe2542e8e3d19672ccde6ad1b Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 1 Jun 2020 23:52:24 -0400 Subject: [PATCH 5/5] config: introduce pseudo "tenants" zone Fixes #49318. Fixes #49445. Progress towards #48123. Informs #48774. This commit introduces a new pseudo object ID in the system tenant's namespace called "tenants". Like "liveness" and "timeseries" before it, the pseudo object allows zone configurations to be applied to pseudo-objects that do not live directly in the system tenant's SQL keyspace. In this case, the "tenants" zone allows zone configurations to be set by the system tenant and applied to all other tenants in the system. There may come a time when we want secondary tenants to have more control over their zone configurations, but that will require a much larger change to the zone config structure and UX as a whole. While making this change, we rationalize the rest of zone configuration handling and how it relates to multi-tenancy. Now that secondary tenant ranges have a zone config to call their own, we can make sense of calls from KV into the zone configuration infrastructure. We gate off calls that don't make sense for secondary tenants and clean up hooks in SQL that handle zone config manipulation. All of this works towards a good cause - we eliminate the remaining uses of `keys.TODOSQLCodec` from `pkg/sql/...` and `pkg/config/...`, bringing us a big step closer towards being able to remove the placeholder and close #48123. This work also reveals that in order to address #48774, we need to be able to determine split points from the SystemConfig. This makes it very difficult to split on secondary tenant object (e.g. table) boundaries. However, it makes it straightforward to split on secondary tenant keysapce boundaries. This is already what we were thinking (see #47907), so out both convenience and desire, I expect that we'll follow this up with a PR that splits Ranges only at secondary tenant boundaries - placing the overhead of an otherwise empty tenant at only a single Range and a few KBs of data. --- pkg/ccl/backupccl/backup_test.go | 2 +- pkg/ccl/importccl/exportcsv_test.go | 2 +- pkg/ccl/partitionccl/partition_test.go | 2 +- pkg/cli/testdata/zip/partial1 | 6 +- pkg/cli/testdata/zip/partial1_excluded | 6 +- pkg/cli/testdata/zip/partial2 | 6 +- pkg/cli/testdata/zip/testzip | 3 +- pkg/config/keys.go | 21 +- pkg/config/keys_test.go | 6 +- pkg/config/system.go | 189 ++++++++++++------ pkg/config/system_test.go | 44 ++-- pkg/config/testutil.go | 8 +- pkg/config/zonepb/zone.go | 2 + pkg/keys/constants.go | 45 +++-- pkg/kv/kvserver/client_split_test.go | 7 +- .../reports/constraint_stats_report.go | 4 +- .../reports/constraint_stats_report_test.go | 6 +- .../reports/critical_localities_report.go | 2 +- .../reports/replication_stats_report.go | 4 +- pkg/kv/kvserver/reports/reporter.go | 16 +- pkg/kv/kvserver/reports/reporter_test.go | 6 +- pkg/kv/kvserver/reports/zone_key.go | 5 +- pkg/kv/kvserver/store_test.go | 8 +- pkg/server/admin.go | 6 +- pkg/server/admin_test.go | 2 +- pkg/server/testserver.go | 3 + pkg/sql/crdb_internal.go | 20 +- pkg/sql/drop_database.go | 6 +- pkg/sql/drop_index.go | 45 +++-- pkg/sql/drop_table.go | 9 - pkg/sql/gcjob/descriptor_utils.go | 31 +-- pkg/sql/gcjob/gc_job.go | 2 +- pkg/sql/gcjob/refresh_statuses.go | 11 +- pkg/sql/logictest/testdata/logic_test/ranges | 6 +- pkg/sql/logictest/testdata/logic_test/tenant | 27 +++ .../testdata/logic_test/tenant_unsupported | 16 +- .../opt/exec/execbuilder/testdata/autocommit | 120 +++++------ pkg/sql/opt/exec/execbuilder/testdata/delete | 6 +- pkg/sql/opt/exec/execbuilder/testdata/upsert | 6 +- pkg/sql/opt_catalog.go | 2 +- pkg/sql/pgwire/testdata/pgtest/notice | 2 +- pkg/sql/schema_changer.go | 2 +- pkg/sql/set_zone_config.go | 26 ++- pkg/sql/show_zone_config.go | 14 +- pkg/sql/sqlbase/system.go | 1 + pkg/sql/tests/testdata/initial_keys | 9 +- pkg/sql/truncate.go | 42 ++-- pkg/sql/zone_config.go | 50 ++--- pkg/sql/zone_config_test.go | 10 +- 49 files changed, 538 insertions(+), 336 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 6f890645d1bc..6920ba7d7137 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -1228,7 +1228,7 @@ func TestBackupRestoreControlJob(t *testing.T) { if err != nil { t.Fatal(err) } - last := uint32(v.ValueInt()) + last := config.SystemTenantObjectID(v.ValueInt()) zoneConfig := zonepb.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(5000) config.TestingSetZoneConfig(last+1, zoneConfig) diff --git a/pkg/ccl/importccl/exportcsv_test.go b/pkg/ccl/importccl/exportcsv_test.go index 56e4a5a12a44..8ebcc9033ed7 100644 --- a/pkg/ccl/importccl/exportcsv_test.go +++ b/pkg/ccl/importccl/exportcsv_test.go @@ -55,7 +55,7 @@ func setupExportableBank(t *testing.T, nodes, rows int) (*sqlutils.SQLRunner, st if err != nil { t.Fatal(err) } - last := uint32(v.ValueInt()) + last := config.SystemTenantObjectID(v.ValueInt()) zoneConfig := zonepb.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(5000) config.TestingSetZoneConfig(last+1, zoneConfig) diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index 3ac15fd35819..84ca48091c0f 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -1406,7 +1406,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (y) // Get the zone config corresponding to the table. table := sqlbase.GetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "t") - kv, err := kvDB.Get(ctx, config.MakeZoneKey(uint32(table.ID))) + kv, err := kvDB.Get(ctx, config.MakeZoneKey(config.SystemTenantObjectID(table.ID))) if err != nil { t.Fatal(err) } diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index 07aaae5a8554..82878e5cc6ee 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -54,7 +54,7 @@ requesting heap files for node 1... writing: debug/nodes/1/heapprof.err.txt requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.txt ^- resulted in ... requesting log file ... -requesting ranges... 33 found +requesting ranges... 34 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -88,6 +88,7 @@ writing: debug/nodes/1/ranges/30.json writing: debug/nodes/1/ranges/31.json writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json +writing: debug/nodes/1/ranges/34.json writing: debug/nodes/2/status.json using SQL connection URL for node 2: postgresql://... retrieving SQL data for crdb_internal.feature_usage... writing: debug/nodes/2/crdb_internal.feature_usage.txt @@ -179,7 +180,7 @@ requesting heap files for node 3... writing: debug/nodes/3/heapprof.err.txt requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.txt ^- resulted in ... requesting log file ... -requesting ranges... 33 found +requesting ranges... 34 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -213,6 +214,7 @@ writing: debug/nodes/3/ranges/30.json writing: debug/nodes/3/ranges/31.json writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json +writing: debug/nodes/3/ranges/34.json requesting list of SQL databases... 3 found requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json 0 tables found diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index 30c143d9c2a2..713db5627633 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -54,7 +54,7 @@ requesting heap files for node 1... writing: debug/nodes/1/heapprof.err.txt requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.txt ^- resulted in ... requesting log file ... -requesting ranges... 33 found +requesting ranges... 34 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -88,6 +88,7 @@ writing: debug/nodes/1/ranges/30.json writing: debug/nodes/1/ranges/31.json writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json +writing: debug/nodes/1/ranges/34.json writing: debug/nodes/2.skipped writing: debug/nodes/3/status.json using SQL connection URL for node 3: postgresql://... @@ -116,7 +117,7 @@ requesting heap files for node 3... writing: debug/nodes/3/heapprof.err.txt requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.txt ^- resulted in ... requesting log file ... -requesting ranges... 33 found +requesting ranges... 34 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -150,6 +151,7 @@ writing: debug/nodes/3/ranges/30.json writing: debug/nodes/3/ranges/31.json writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json +writing: debug/nodes/3/ranges/34.json requesting list of SQL databases... 3 found requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json 0 tables found diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index f253e0b0abab..1267cd17ee51 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -54,7 +54,7 @@ requesting heap files for node 1... writing: debug/nodes/1/heapprof.err.txt requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.txt ^- resulted in ... requesting log file ... -requesting ranges... 33 found +requesting ranges... 34 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -88,6 +88,7 @@ writing: debug/nodes/1/ranges/30.json writing: debug/nodes/1/ranges/31.json writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json +writing: debug/nodes/1/ranges/34.json writing: debug/nodes/3/status.json using SQL connection URL for node 3: postgresql://... retrieving SQL data for crdb_internal.feature_usage... writing: debug/nodes/3/crdb_internal.feature_usage.txt @@ -115,7 +116,7 @@ requesting heap files for node 3... writing: debug/nodes/3/heapprof.err.txt requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.txt ^- resulted in ... requesting log file ... -requesting ranges... 33 found +requesting ranges... 34 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -149,6 +150,7 @@ writing: debug/nodes/3/ranges/30.json writing: debug/nodes/3/ranges/31.json writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json +writing: debug/nodes/3/ranges/34.json requesting list of SQL databases... 3 found requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json 0 tables found diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 40b58ebce6a8..f9c8a212fb17 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -52,7 +52,7 @@ requesting heap profile for node 1... writing: debug/nodes/1/heap.pprof requesting heap files for node 1... ? found requesting goroutine files for node 1... 0 found requesting log file ... -requesting ranges... 33 found +requesting ranges... 34 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -86,6 +86,7 @@ writing: debug/nodes/1/ranges/30.json writing: debug/nodes/1/ranges/31.json writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json +writing: debug/nodes/1/ranges/34.json requesting list of SQL databases... 3 found requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json 0 tables found diff --git a/pkg/config/keys.go b/pkg/config/keys.go index 0817b3f29936..903634a77217 100644 --- a/pkg/config/keys.go +++ b/pkg/config/keys.go @@ -17,19 +17,20 @@ import ( // MakeZoneKeyPrefix returns the key prefix for id's row in the system.zones // table. -func MakeZoneKeyPrefix(id uint32) roachpb.Key { - return keys.SystemSQLCodec.ZoneKeyPrefix(id) +func MakeZoneKeyPrefix(id SystemTenantObjectID) roachpb.Key { + return keys.SystemSQLCodec.ZoneKeyPrefix(uint32(id)) } // MakeZoneKey returns the key for id's entry in the system.zones table. -func MakeZoneKey(id uint32) roachpb.Key { - return keys.SystemSQLCodec.ZoneKey(id) +func MakeZoneKey(id SystemTenantObjectID) roachpb.Key { + return keys.SystemSQLCodec.ZoneKey(uint32(id)) } -// DecodeObjectID decodes the object ID from the front of key. It returns the -// decoded object ID, the remainder of the key, and whether the result is valid -// (i.e., whether the key was within the structured key space). -func DecodeObjectID(key roachpb.RKey) (uint32, []byte, bool) { - rem, id, err := keys.TODOSQLCodec.DecodeTablePrefix(key.AsRawKey()) - return id, rem, err == nil +// DecodeSystemTenantObjectID decodes the object ID for the system-tenant from +// the front of key. It returns the decoded object ID, the remainder of the key, +// and whether the result is valid (i.e., whether the key was within the system +// tenant's structured key space). +func DecodeSystemTenantObjectID(key roachpb.RKey) (SystemTenantObjectID, []byte, bool) { + rem, id, err := keys.SystemSQLCodec.DecodeTablePrefix(key.AsRawKey()) + return SystemTenantObjectID(id), rem, err == nil } diff --git a/pkg/config/keys_test.go b/pkg/config/keys_test.go index 3ce4741e0345..eac7175f28f9 100644 --- a/pkg/config/keys_test.go +++ b/pkg/config/keys_test.go @@ -21,14 +21,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -func TestDecodeObjectID(t *testing.T) { +func TestDecodeSystemTenantObjectID(t *testing.T) { defer leaktest.AfterTest(t)() testCases := []struct { key roachpb.RKey keySuffix []byte success bool - id uint32 + id config.SystemTenantObjectID }{ // Before the structured span. {roachpb.RKeyMin, nil, false, 0}, @@ -43,7 +43,7 @@ func TestDecodeObjectID(t *testing.T) { } for tcNum, tc := range testCases { - id, keySuffix, success := config.DecodeObjectID(tc.key) + id, keySuffix, success := config.DecodeSystemTenantObjectID(tc.key) if success != tc.success { t.Errorf("#%d: expected success=%t", tcNum, tc.success) continue diff --git a/pkg/config/system.go b/pkg/config/system.go index 892e90ff4116..bbffa493092e 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -24,19 +24,34 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) +// SystemTenantObjectID is an identifier for an object (e.g. database or table) +// in the system tenant. Each object in the system tenant is capable of being +// associated with a zone configuration, which describes how and where the +// object's data is stored in KV. Some objects in the system tenant also serve +// as Range split point boundaries, see ShouldSplitAtDesc. +// +// Identifiers for objects in secondary tenants are treated differently for the +// purposes of the system configuration. An individual object in a secondary +// tenant cannot be associated with a zone configuration. Instead, all secondary +// tenant data is associated with the "tenants" zone which is accessible only to +// the system tenant. Additionally, individual objects in secondary tenants do +// not serve as Range split boundaries. However, each tenant is guaranteed to be +// split off into its own range. +type SystemTenantObjectID uint32 + type zoneConfigHook func( - sysCfg *SystemConfig, objectID uint32, + sysCfg *SystemConfig, objectID SystemTenantObjectID, ) (zone *zonepb.ZoneConfig, placeholder *zonepb.ZoneConfig, cache bool, err error) var ( - // ZoneConfigHook is a function used to lookup a zone config given a table - // or database ID. + // ZoneConfigHook is a function used to lookup a zone config given a system + // tenant table or database ID. // This is also used by testing to simplify fake configs. ZoneConfigHook zoneConfigHook // testingLargestIDHook is a function used to bypass GetLargestObjectID // in tests. - testingLargestIDHook func(uint32) uint32 + testingLargestIDHook func(SystemTenantObjectID) SystemTenantObjectID ) type zoneEntry struct { @@ -68,8 +83,8 @@ type SystemConfig struct { DefaultZoneConfig *zonepb.ZoneConfig mu struct { syncutil.RWMutex - zoneCache map[uint32]zoneEntry - shouldSplitCache map[uint32]bool + zoneCache map[SystemTenantObjectID]zoneEntry + shouldSplitCache map[SystemTenantObjectID]bool } } @@ -77,8 +92,8 @@ type SystemConfig struct { func NewSystemConfig(defaultZoneConfig *zonepb.ZoneConfig) *SystemConfig { sc := &SystemConfig{} sc.DefaultZoneConfig = defaultZoneConfig - sc.mu.zoneCache = map[uint32]zoneEntry{} - sc.mu.shouldSplitCache = map[uint32]bool{} + sc.mu.zoneCache = map[SystemTenantObjectID]zoneEntry{} + sc.mu.shouldSplitCache = map[SystemTenantObjectID]bool{} return sc } @@ -105,22 +120,23 @@ func (s *SystemConfig) Equal(other *SystemConfigEntries) bool { return true } -// 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 { +// getSystemTenantDesc 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) getSystemTenantDesc(key roachpb.Key) *roachpb.Value { if getVal := s.GetValue(key); getVal != nil { return getVal } - id, err := keys.TODOSQLCodec.DecodeDescMetadataID(key) + id, err := keys.SystemSQLCodec.DecodeDescMetadataID(key) if err != nil { // No ID found for key. No roachpb.Value corresponds to this key. panic(err) } testingLock.Lock() - _, ok := testingZoneConfig[uint32(id)] + _, ok := testingZoneConfig[SystemTenantObjectID(id)] testingLock.Unlock() if ok { @@ -174,7 +190,9 @@ func (s *SystemConfig) GetIndex(key roachpb.Key) (int, bool) { // less than or equal to maxID. The objects in the config are augmented with the // provided pseudo IDs. If maxID is 0, returns the largest ID in the config // (again, augmented by the pseudo IDs). -func (s *SystemConfig) GetLargestObjectID(maxID uint32, pseudoIDs []uint32) (uint32, error) { +func (s *SystemConfig) GetLargestObjectID( + maxID SystemTenantObjectID, pseudoIDs []uint32, +) (SystemTenantObjectID, error) { testingLock.Lock() hook := testingLargestIDHook testingLock.Unlock() @@ -183,11 +201,11 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32, pseudoIDs []uint32) (uin } // Search for the descriptor table entries within the SystemConfig. - highBound := keys.TODOSQLCodec.TablePrefix(keys.DescriptorTableID + 1) + highBound := keys.SystemSQLCodec.TablePrefix(keys.DescriptorTableID + 1) highIndex := sort.Search(len(s.Values), func(i int) bool { return bytes.Compare(s.Values[i].Key, highBound) >= 0 }) - lowBound := keys.TODOSQLCodec.TablePrefix(keys.DescriptorTableID) + lowBound := keys.SystemSQLCodec.TablePrefix(keys.DescriptorTableID) lowIndex := sort.Search(len(s.Values), func(i int) bool { return bytes.Compare(s.Values[i].Key, lowBound) >= 0 }) @@ -196,24 +214,26 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32, pseudoIDs []uint32) (uin } // Determine the largest pseudo table ID equal to or below maxID. - maxPseudoID := uint32(0) + maxPseudoID := SystemTenantObjectID(0) for _, id := range pseudoIDs { - if id > maxPseudoID && (maxID == 0 || id <= maxID) { - maxPseudoID = id + objID := SystemTenantObjectID(id) + if objID > maxPseudoID && (maxID == 0 || objID <= maxID) { + maxPseudoID = objID } } // No maximum specified; maximum ID is the last entry in the descriptor // table or the largest pseudo ID, whichever is larger. if maxID == 0 { - id, err := keys.TODOSQLCodec.DecodeDescMetadataID(s.Values[highIndex-1].Key) + id, err := keys.SystemSQLCodec.DecodeDescMetadataID(s.Values[highIndex-1].Key) if err != nil { return 0, err } - if id < maxPseudoID { - id = maxPseudoID + objID := SystemTenantObjectID(id) + if objID < maxPseudoID { + objID = maxPseudoID } - return id, nil + return objID, nil } // Maximum specified: need to search the descriptor table. Binary search @@ -226,8 +246,8 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32, pseudoIDs []uint32) (uin return false } var id uint32 - id, err = keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[i].Key) - return uint32(id) >= maxID + id, err = keys.SystemSQLCodec.DecodeDescMetadataID(searchSlice[i].Key) + return SystemTenantObjectID(id) >= maxID }) if err != nil { return 0, err @@ -236,12 +256,12 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32, pseudoIDs []uint32) (uin // If we found an index within the list, maxIdx might point to a descriptor // with exactly maxID. if maxIdx < len(searchSlice) { - id, err := keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[maxIdx].Key) + id, err := keys.SystemSQLCodec.DecodeDescMetadataID(searchSlice[maxIdx].Key) if err != nil { return 0, err } - if uint32(id) == maxID { - return uint32(id), nil + if SystemTenantObjectID(id) == maxID { + return SystemTenantObjectID(id), nil } } @@ -250,14 +270,15 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32, pseudoIDs []uint32) (uin } // Return ID of the immediately preceding descriptor. - id, err := keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[maxIdx-1].Key) + id, err := keys.SystemSQLCodec.DecodeDescMetadataID(searchSlice[maxIdx-1].Key) if err != nil { return 0, err } - if id < maxPseudoID { - id = maxPseudoID + objID := SystemTenantObjectID(id) + if objID < maxPseudoID { + objID = maxPseudoID } - return id, nil + return objID, nil } // GetZoneConfigForKey looks up the zone config for the object (table @@ -268,12 +289,12 @@ func (s *SystemConfig) GetZoneConfigForKey(key roachpb.RKey) (*zonepb.ZoneConfig } // DecodeKeyIntoZoneIDAndSuffix figures out the zone that the key belongs to. -func DecodeKeyIntoZoneIDAndSuffix(key roachpb.RKey) (id uint32, keySuffix []byte) { - objectID, keySuffix, ok := DecodeObjectID(key) +func DecodeKeyIntoZoneIDAndSuffix(key roachpb.RKey) (id SystemTenantObjectID, keySuffix []byte) { + objectID, keySuffix, ok := DecodeSystemTenantObjectID(key) if !ok { // Not in the structured data namespace. objectID = keys.RootNamespaceID - } else if objectID <= keys.MaxSystemConfigDescID || isPseudoTableID(objectID) { + } else if objectID <= keys.MaxSystemConfigDescID || isPseudoTableID(uint32(objectID)) { // For now, you cannot set the zone config on gossiped tables. The only // way to set a zone config on these tables is to modify config for the // system database as a whole. This is largely because all the @@ -295,6 +316,8 @@ func DecodeKeyIntoZoneIDAndSuffix(key roachpb.RKey) (id uint32, keySuffix []byte } else { objectID = keys.SystemRangesID } + } else if bytes.HasPrefix(key, keys.TenantPrefix) { + objectID = keys.TenantsRangesID } return objectID, keySuffix } @@ -310,22 +333,31 @@ func isPseudoTableID(id uint32) bool { } // GetZoneConfigForObject returns the combined zone config for the given object -// identifier. +// identifier and SQL codec. // 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) { - entry, err := s.getZoneEntry(id) +func (s *SystemConfig) GetZoneConfigForObject( + codec keys.SQLCodec, id uint32, +) (*zonepb.ZoneConfig, error) { + var sysID SystemTenantObjectID + if codec.ForSystemTenant() { + sysID = SystemTenantObjectID(id) + } else { + sysID = keys.TenantsRangesID + } + entry, err := s.getZoneEntry(sysID) if err != nil { return nil, err } return entry.combined, nil } -// 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 -// them as an entry in the cache. -func (s *SystemConfig) getZoneEntry(id uint32) (zoneEntry, error) { +// getZoneEntry returns the zone entry for the given system-tenant +// 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 them as an +// entry in the cache. +func (s *SystemConfig) getZoneEntry(id SystemTenantObjectID) (zoneEntry, error) { s.mu.RLock() entry, ok := s.mu.zoneCache[id] s.mu.RUnlock() @@ -361,7 +393,7 @@ func (s *SystemConfig) getZoneEntry(id uint32) (zoneEntry, error) { } func (s *SystemConfig) getZoneConfigForKey( - id uint32, keySuffix []byte, + id SystemTenantObjectID, keySuffix []byte, ) (*zonepb.ZoneConfig, error) { entry, err := s.getZoneEntry(id) if err != nil { @@ -415,11 +447,14 @@ 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/), 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. +// Splits are required between user tables (i.e. /table/) in the system +// tenant, 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. +// +// Splits are also required between secondary tenants (i.e. /tenant/). +// However, splits are not required between the tables of secondary tenants. 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 @@ -442,9 +477,30 @@ func (s *SystemConfig) ComputeSplitKey(startKey, endKey roachpb.RKey) (rr roachp // [startKey, endKey) is somewhere greater than this split point. Continue. } - // 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 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. First, look for split keys within the system-tenant's keyspace. + if split := s.systemTenantTableBoundarySplitKey(startKey, endKey); split != nil { + return split + } + + // If the system tenant does not have any splits, look for split keys at the + // boundary of each secondary tenant. + return s.tenantBoundarySplitKey(startKey, endKey) +} + +func (s *SystemConfig) systemTenantTableBoundarySplitKey( + startKey, endKey roachpb.RKey, +) roachpb.RKey { + if bytes.HasPrefix(startKey, keys.TenantPrefix) { + // If the start key has a tenant prefix, don't try to find a split key + // between the system tenant's tables. The rest of this method will + // still work even without this short-circuiting, but we might as well + // avoid it. + return nil + } + + startID, _, ok := DecodeSystemTenantObjectID(startKey) if !ok || startID <= keys.MaxSystemConfigDescID { // The start key is either: // - not part of the structured data span @@ -459,14 +515,14 @@ func (s *SystemConfig) ComputeSplitKey(startKey, endKey roachpb.RKey) (rr roachp // 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 { + // findSplitKey returns the first possible split key between the given + // range of IDs. + findSplitKey := func(startID, endID SystemTenantObjectID) roachpb.RKey { // endID could be smaller than startID if we don't have user tables. for id := startID; id <= endID; id++ { - tableKey := roachpb.RKey(keys.TODOSQLCodec.TablePrefix(id)) + tableKey := roachpb.RKey(keys.SystemSQLCodec.TablePrefix(uint32(id))) // This logic is analogous to the well-commented static split logic above. - if startKey.Less(tableKey) && s.shouldSplit(id) { + if startKey.Less(tableKey) && s.shouldSplitOnSystemTenantObject(id) { if tableKey.Less(endKey) { return tableKey } @@ -512,7 +568,7 @@ func (s *SystemConfig) ComputeSplitKey(startKey, endKey roachpb.RKey) (rr roachp startID = keys.MaxReservedDescID + 1 } - // Find the split key in the user space. + // Find the split key in the system tenant's user space. endID, err := s.GetLargestObjectID(0, keys.PseudoTableIDs) if err != nil { log.Errorf(context.TODO(), "unable to determine largest object ID from system config: %s", err) @@ -521,16 +577,21 @@ func (s *SystemConfig) ComputeSplitKey(startKey, endKey roachpb.RKey) (rr roachp return findSplitKey(startID, endID) } +func (s *SystemConfig) tenantBoundarySplitKey(startKey, endKey roachpb.RKey) roachpb.RKey { + // TODO(nvanbenschoten): implement this logic. Tracked in #48774. + return nil +} + // NeedsSplit returns whether the range [startKey, endKey) needs a split due // 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 ID isn't found in the cache. -func (s *SystemConfig) shouldSplit(id uint32) bool { +// shouldSplitOnSystemTenantObject 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 ID isn't found in the cache. +func (s *SystemConfig) shouldSplitOnSystemTenantObject(id SystemTenantObjectID) bool { // Check the cache. { s.mu.RLock() @@ -547,7 +608,7 @@ func (s *SystemConfig) shouldSplit(id uint32) bool { // actual descriptors. shouldSplit = true } else { - desc := s.GetDesc(keys.TODOSQLCodec.DescMetadataKey(id)) + desc := s.getSystemTenantDesc(keys.SystemSQLCodec.DescMetadataKey(uint32(id))) shouldSplit = desc != nil && sqlbase.ShouldSplitAtDesc(desc) } // Populate the cache. diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index fd7957ea5144..6d69f5d56509 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -40,16 +40,24 @@ func tkey(tableID uint32, chunks ...string) []byte { return key } -func sqlKV(tableID uint32, indexID, descriptorID uint64) roachpb.KeyValue { +func tenantTkey(tenantID uint64, tableID uint32, chunks ...string) []byte { + key := keys.MakeSQLCodec(roachpb.MakeTenantID(tenantID)).TablePrefix(tableID) + for _, c := range chunks { + key = append(key, []byte(c)...) + } + return key +} + +func sqlKV(tableID uint32, indexID, descID uint64) roachpb.KeyValue { k := tkey(tableID) k = encoding.EncodeUvarintAscending(k, indexID) - k = encoding.EncodeUvarintAscending(k, descriptorID) + k = encoding.EncodeUvarintAscending(k, descID) k = encoding.EncodeUvarintAscending(k, 12345) // Column ID, but could be anything. return kv(k, nil) } -func descriptor(descriptorID uint64) roachpb.KeyValue { - k := sqlbase.MakeDescMetadataKey(keys.SystemSQLCodec, sqlbase.ID(descriptorID)) +func descriptor(descID uint64) roachpb.KeyValue { + k := sqlbase.MakeDescMetadataKey(keys.SystemSQLCodec, sqlbase.ID(descID)) v := sqlbase.WrapDescriptor(&sqlbase.TableDescriptor{}) kv := roachpb.KeyValue{Key: k} if err := kv.Value.SetProto(v); err != nil { @@ -58,9 +66,9 @@ func descriptor(descriptorID uint64) roachpb.KeyValue { return kv } -func zoneConfig(descriptorID uint32, spans ...zonepb.SubzoneSpan) roachpb.KeyValue { +func zoneConfig(descID config.SystemTenantObjectID, spans ...zonepb.SubzoneSpan) roachpb.KeyValue { kv := roachpb.KeyValue{ - Key: config.MakeZoneKey(descriptorID), + Key: config.MakeZoneKey(descID), } if err := kv.Value.SetProto(&zonepb.ZoneConfig{SubzoneSpans: spans}); err != nil { panic(err) @@ -128,8 +136,8 @@ func TestGetLargestID(t *testing.T) { type testCase struct { values []roachpb.KeyValue - largest uint32 - maxID uint32 + largest config.SystemTenantObjectID + maxID config.SystemTenantObjectID pseudoIDs []uint32 errStr string } @@ -184,10 +192,14 @@ func TestGetLargestID(t *testing.T) { func() testCase { ms := sqlbase.MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) descIDs := ms.DescriptorIDs() - maxDescID := descIDs[len(descIDs)-1] + maxDescID := config.SystemTenantObjectID(descIDs[len(descIDs)-1]) kvs, _ /* splits */ := ms.GetInitialValues() pseudoIDs := keys.PseudoTableIDs - return testCase{kvs, uint32(maxDescID), 0, pseudoIDs, ""} + const pseudoIDIsMax = true // NOTE: will change as new system objects are added + if pseudoIDIsMax { + maxDescID = config.SystemTenantObjectID(keys.MaxPseudoTableID) + } + return testCase{kvs, maxDescID, 0, pseudoIDs, ""} }(), // Test non-zero max. @@ -431,7 +443,7 @@ func TestGetZoneConfigForKey(t *testing.T) { testCases := []struct { key roachpb.RKey - expectedID uint32 + expectedID config.SystemTenantObjectID }{ {roachpb.RKeyMin, keys.MetaRangesID}, {roachpb.RKey(keys.Meta1Prefix), keys.MetaRangesID}, @@ -473,6 +485,12 @@ func TestGetZoneConfigForKey(t *testing.T) { {tkey(keys.MinUserDescID), keys.MinUserDescID}, {tkey(keys.MinUserDescID + 22), keys.MinUserDescID + 22}, {roachpb.RKeyMax, keys.RootNamespaceID}, + + // Secondary tenant tables should refer to the TenantsRangesID. + {tenantTkey(5, keys.MinUserDescID), keys.TenantsRangesID}, + {tenantTkey(5, keys.MinUserDescID+22), keys.TenantsRangesID}, + {tenantTkey(10, keys.MinUserDescID), keys.TenantsRangesID}, + {tenantTkey(10, keys.MinUserDescID+22), keys.TenantsRangesID}, } originalZoneConfigHook := config.ZoneConfigHook @@ -488,9 +506,9 @@ func TestGetZoneConfigForKey(t *testing.T) { Values: kvs, } for tcNum, tc := range testCases { - var objectID uint32 + var objectID config.SystemTenantObjectID config.ZoneConfigHook = func( - _ *config.SystemConfig, id uint32, + _ *config.SystemConfig, id config.SystemTenantObjectID, ) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) { objectID = id return &zonepb.ZoneConfig{}, nil, false, nil diff --git a/pkg/config/testutil.go b/pkg/config/testutil.go index 8c0ee7782c51..d96441e97aa2 100644 --- a/pkg/config/testutil.go +++ b/pkg/config/testutil.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) -type zoneConfigMap map[uint32]zonepb.ZoneConfig +type zoneConfigMap map[SystemTenantObjectID]zonepb.ZoneConfig var ( testingZoneConfig zoneConfigMap @@ -40,7 +40,7 @@ func TestingSetupZoneConfigHook(stopper *stop.Stopper) { testingZoneConfig = make(zoneConfigMap) testingPreviousHook = ZoneConfigHook ZoneConfigHook = testingZoneConfigHook - testingLargestIDHook = func(maxID uint32) (max uint32) { + testingLargestIDHook = func(maxID SystemTenantObjectID) (max SystemTenantObjectID) { testingLock.Lock() defer testingLock.Unlock() for id := range testingZoneConfig { @@ -70,14 +70,14 @@ 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 SystemTenantObjectID, zone zonepb.ZoneConfig) { testingLock.Lock() defer testingLock.Unlock() testingZoneConfig[id] = zone } func testingZoneConfigHook( - _ *SystemConfig, id uint32, + _ *SystemConfig, id SystemTenantObjectID, ) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) { testingLock.Lock() defer testingLock.Unlock() diff --git a/pkg/config/zonepb/zone.go b/pkg/config/zonepb/zone.go index 03c047d3a4f9..7c5536a89ed7 100644 --- a/pkg/config/zonepb/zone.go +++ b/pkg/config/zonepb/zone.go @@ -35,6 +35,7 @@ const ( MetaZoneName = "meta" SystemZoneName = "system" TimeseriesZoneName = "timeseries" + TenantsZoneName = "tenants" ) // NamedZones maps named zones to their pseudo-table ID that can be used to @@ -45,6 +46,7 @@ var NamedZones = map[string]uint32{ MetaZoneName: keys.MetaRangesID, SystemZoneName: keys.SystemRangesID, TimeseriesZoneName: keys.TimeseriesRangesID, + TenantsZoneName: keys.TenantsRangesID, } // NamedZonesByID is the inverse of NamedZones: it maps pseudo-table IDs to diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 0520b38dd7c8..7724808e693b 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -353,34 +353,31 @@ const ( RangeEventTableID = 13 UITableID = 14 JobsTableID = 15 - MetaRangesID = 16 - SystemRangesID = 17 - TimeseriesRangesID = 18 + MetaRangesID = 16 // pseudo + SystemRangesID = 17 // pseudo + TimeseriesRangesID = 18 // pseudo WebSessionsTableID = 19 TableStatisticsTableID = 20 LocationsTableID = 21 - LivenessRangesID = 22 + LivenessRangesID = 22 // pseudo RoleMembersTableID = 23 CommentsTableID = 24 ReplicationConstraintStatsTableID = 25 ReplicationCriticalLocalitiesTableID = 26 ReplicationStatsTableID = 27 ReportsMetaTableID = 28 - PublicSchemaID = 29 + PublicSchemaID = 29 // pseudo // New NamespaceTableID for cluster version >= 20.1 // Ensures that NamespaceTable does not get gossiped again - NamespaceTableID = 30 - - ProtectedTimestampsMetaTableID = 31 - ProtectedTimestampsRecordsTableID = 32 - - RoleOptionsTableID = 33 - + NamespaceTableID = 30 + ProtectedTimestampsMetaTableID = 31 + ProtectedTimestampsRecordsTableID = 32 + RoleOptionsTableID = 33 StatementBundleChunksTableID = 34 StatementDiagnosticsRequestsTableID = 35 StatementDiagnosticsTableID = 36 - - ScheduledJobsTableID = 37 + ScheduledJobsTableID = 37 + TenantsRangesID = 38 // pseudo // CommentType is type for system.comments DatabaseCommentType = 0 @@ -402,4 +399,22 @@ const ( // there's no table descriptor). They're grouped here because the cluster // bootstrap process needs to create splits for them; splits for the tables // happen separately. -var PseudoTableIDs = []uint32{MetaRangesID, SystemRangesID, TimeseriesRangesID, LivenessRangesID, PublicSchemaID} +var PseudoTableIDs = []uint32{ + MetaRangesID, + SystemRangesID, + TimeseriesRangesID, + LivenessRangesID, + PublicSchemaID, + TenantsRangesID, +} + +// MaxPseudoTableID is the largest ID in PseudoTableIDs. +var MaxPseudoTableID = func() uint32 { + var max uint32 + for _, id := range PseudoTableIDs { + if max < id { + max = id + } + } + return max +}() diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index fbdee0ab5023..a69c3709b579 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -1017,7 +1017,7 @@ func TestStoreZoneUpdateAndRangeSplit(t *testing.T) { descID := uint32(keys.MinUserDescID) zoneConfig := zonepb.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(maxBytes) - config.TestingSetZoneConfig(descID, zoneConfig) + config.TestingSetZoneConfig(config.SystemTenantObjectID(descID), zoneConfig) // Trigger gossip callback. if err := store.Gossip().AddInfoProto(gossip.KeySystemConfig, &config.SystemConfigEntries{}, 0); err != nil { @@ -1079,7 +1079,7 @@ func TestStoreRangeSplitWithMaxBytesUpdate(t *testing.T) { descID := uint32(keys.MinUserDescID) zoneConfig := zonepb.DefaultZoneConfig() zoneConfig.RangeMaxBytes = proto.Int64(maxBytes) - config.TestingSetZoneConfig(descID, zoneConfig) + config.TestingSetZoneConfig(config.SystemTenantObjectID(descID), zoneConfig) // Trigger gossip callback. if err := store.Gossip().AddInfoProto(gossip.KeySystemConfig, &config.SystemConfigEntries{}, 0); err != nil { @@ -1325,6 +1325,9 @@ func TestStoreRangeSystemSplits(t *testing.T) { } ids := schema.DescriptorIDs() maxID := uint32(ids[len(ids)-1]) + if maxPseudo := keys.MaxPseudoTableID; maxID < maxPseudo { + maxID = maxPseudo + } for i := uint32(keys.MaxSystemConfigDescID + 1); i <= maxID; i++ { expKeys = append(expKeys, testutils.MakeKey(keys.Meta2Prefix, keys.SystemSQLCodec.TablePrefix(i)), diff --git a/pkg/kv/kvserver/reports/constraint_stats_report.go b/pkg/kv/kvserver/reports/constraint_stats_report.go index 4a111bbe85ad..d885b3ee79f0 100644 --- a/pkg/kv/kvserver/reports/constraint_stats_report.go +++ b/pkg/kv/kvserver/reports/constraint_stats_report.go @@ -183,7 +183,7 @@ func (r *replicationConstraintStatsReportSaver) loadPreviousVersion( r.previousVersion = make(ConstraintReport, len(rows)) for _, row := range rows { key := ConstraintStatusKey{} - key.ZoneID = (uint32)(*row[0].(*tree.DInt)) + key.ZoneID = (config.SystemTenantObjectID)(*row[0].(*tree.DInt)) key.SubzoneID = base.SubzoneID((*row[1].(*tree.DInt))) key.ViolationType = (ConstraintType)(*row[2].(*tree.DString)) key.Constraint = (ConstraintRepr)(*row[3].(*tree.DString)) @@ -395,7 +395,7 @@ func (v *constraintConformanceVisitor) reset(ctx context.Context) { if err != nil { log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) } - for i := uint32(1); i <= maxObjectID; i++ { + for i := config.SystemTenantObjectID(1); i <= maxObjectID; i++ { zone, err := getZoneByID(i, v.cfg) if err != nil { log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) diff --git a/pkg/kv/kvserver/reports/constraint_stats_report_test.go b/pkg/kv/kvserver/reports/constraint_stats_report_test.go index 14b0349c5139..ebcd739c5d00 100644 --- a/pkg/kv/kvserver/reports/constraint_stats_report_test.go +++ b/pkg/kv/kvserver/reports/constraint_stats_report_test.go @@ -1011,13 +1011,13 @@ func (b *systemConfigBuilder) setDefaultZoneConfig(cfg zonepb.ZoneConfig) error } func (b *systemConfigBuilder) addZoneInner(objectName string, id int, cfg zonepb.ZoneConfig) error { - k := config.MakeZoneKey(uint32(id)) + k := config.MakeZoneKey(config.SystemTenantObjectID(id)) var v roachpb.Value if err := v.SetProto(&cfg); err != nil { panic(err) } b.kv = append(b.kv, roachpb.KeyValue{Key: k, Value: v}) - return b.addZoneToObjectMapping(MakeZoneKey(uint32(id), NoSubzone), objectName) + return b.addZoneToObjectMapping(MakeZoneKey(config.SystemTenantObjectID(id), NoSubzone), objectName) } func (b *systemConfigBuilder) addDatabaseZone(name string, id int, cfg zonepb.ZoneConfig) error { @@ -1044,7 +1044,7 @@ func (b *systemConfigBuilder) addTableZone(t sqlbase.TableDescriptor, cfg zonepb object = fmt.Sprintf("%s.%s", idx, subzone.PartitionName) } if err := b.addZoneToObjectMapping( - MakeZoneKey(uint32(t.ID), base.SubzoneIDFromIndex(i)), object, + MakeZoneKey(config.SystemTenantObjectID(t.ID), base.SubzoneIDFromIndex(i)), object, ); err != nil { return err } diff --git a/pkg/kv/kvserver/reports/critical_localities_report.go b/pkg/kv/kvserver/reports/critical_localities_report.go index 9ed1f635fd8d..41d1c90afa51 100644 --- a/pkg/kv/kvserver/reports/critical_localities_report.go +++ b/pkg/kv/kvserver/reports/critical_localities_report.go @@ -113,7 +113,7 @@ func (r *replicationCriticalLocalitiesReportSaver) loadPreviousVersion( r.previousVersion = make(LocalityReport, len(rows)) for _, row := range rows { key := localityKey{} - key.ZoneID = (uint32)(*row[0].(*tree.DInt)) + key.ZoneID = (config.SystemTenantObjectID)(*row[0].(*tree.DInt)) key.SubzoneID = base.SubzoneID(*row[1].(*tree.DInt)) key.locality = (LocalityRepr)(*row[2].(*tree.DString)) r.previousVersion[key] = localityStatus{(int32)(*row[3].(*tree.DInt))} diff --git a/pkg/kv/kvserver/reports/replication_stats_report.go b/pkg/kv/kvserver/reports/replication_stats_report.go index c22686118db8..29bff152e5c4 100644 --- a/pkg/kv/kvserver/reports/replication_stats_report.go +++ b/pkg/kv/kvserver/reports/replication_stats_report.go @@ -121,7 +121,7 @@ func (r *replicationStatsReportSaver) loadPreviousVersion( r.previousVersion = make(RangeReport, len(rows)) for _, row := range rows { key := ZoneKey{} - key.ZoneID = (uint32)(*row[0].(*tree.DInt)) + key.ZoneID = (config.SystemTenantObjectID)(*row[0].(*tree.DInt)) key.SubzoneID = base.SubzoneID(*row[1].(*tree.DInt)) r.previousVersion[key] = zoneRangeStatus{ (int32)(*row[2].(*tree.DInt)), @@ -305,7 +305,7 @@ func (v *replicationStatsVisitor) reset(ctx context.Context) { if err != nil { log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) } - for i := uint32(1); i <= maxObjectID; i++ { + for i := config.SystemTenantObjectID(1); i <= maxObjectID; i++ { zone, err := getZoneByID(i, v.cfg) if err != nil { log.Fatalf(ctx, "unexpected failure to compute max object id: %s", err) diff --git a/pkg/kv/kvserver/reports/reporter.go b/pkg/kv/kvserver/reports/reporter.go index 8042af826cc5..9a1267585805 100644 --- a/pkg/kv/kvserver/reports/reporter.go +++ b/pkg/kv/kvserver/reports/reporter.go @@ -280,7 +280,7 @@ type nodeChecker func(nodeID roachpb.NodeID) bool type zoneResolver struct { init bool // curObjectID is the object (i.e. usually table) of the configured range. - curObjectID uint32 + curObjectID config.SystemTenantObjectID // 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. @@ -302,7 +302,9 @@ 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 config.SystemTenantObjectID, key ZoneKey, rootZone *zonepb.ZoneConfig, +) { c.init = true c.curObjectID = objectID c.curRootZone = rootZone @@ -425,7 +427,7 @@ func visitZones( // corresponding to id. The zone corresponding to id itself is not visited. func visitAncestors( ctx context.Context, - id uint32, + id config.SystemTenantObjectID, cfg *config.SystemConfig, visitor func(context.Context, *zonepb.ZoneConfig, ZoneKey) bool, ) (bool, error) { @@ -449,12 +451,12 @@ func visitAncestors( } // If it's a table, the parent is a database. - zone, err := getZoneByID(uint32(tableDesc.ParentID), cfg) + zone, err := getZoneByID(config.SystemTenantObjectID(tableDesc.ParentID), cfg) if err != nil { return false, err } if zone != nil { - if visitor(ctx, zone, MakeZoneKey(uint32(tableDesc.ParentID), NoSubzone)) { + if visitor(ctx, zone, MakeZoneKey(config.SystemTenantObjectID(tableDesc.ParentID), NoSubzone)) { return true, nil } } @@ -478,7 +480,9 @@ 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 config.SystemTenantObjectID, cfg *config.SystemConfig, +) (*zonepb.ZoneConfig, error) { zoneVal := cfg.GetValue(config.MakeZoneKey(id)) if zoneVal == nil { return nil, nil diff --git a/pkg/kv/kvserver/reports/reporter_test.go b/pkg/kv/kvserver/reports/reporter_test.go index a00087cf8291..86902d52a20b 100644 --- a/pkg/kv/kvserver/reports/reporter_test.go +++ b/pkg/kv/kvserver/reports/reporter_test.go @@ -511,9 +511,9 @@ func TestZoneChecker(t *testing.T) { p2SubzoneIndex := 1 require.Equal(t, "p1", t1Zone.Subzones[p1SubzoneIndex].PartitionName) require.Equal(t, "p2", t1Zone.Subzones[p2SubzoneIndex].PartitionName) - t1ZoneKey := MakeZoneKey(uint32(t1ID), NoSubzone) - p1ZoneKey := MakeZoneKey(uint32(t1ID), base.SubzoneIDFromIndex(p1SubzoneIndex)) - p2ZoneKey := MakeZoneKey(uint32(t1ID), base.SubzoneIDFromIndex(p2SubzoneIndex)) + t1ZoneKey := MakeZoneKey(config.SystemTenantObjectID(t1ID), NoSubzone) + p1ZoneKey := MakeZoneKey(config.SystemTenantObjectID(t1ID), base.SubzoneIDFromIndex(p1SubzoneIndex)) + p2ZoneKey := MakeZoneKey(config.SystemTenantObjectID(t1ID), base.SubzoneIDFromIndex(p2SubzoneIndex)) ranges := []tc{ { diff --git a/pkg/kv/kvserver/reports/zone_key.go b/pkg/kv/kvserver/reports/zone_key.go index 84ffd1f79bce..ebf5e5865e8b 100644 --- a/pkg/kv/kvserver/reports/zone_key.go +++ b/pkg/kv/kvserver/reports/zone_key.go @@ -14,12 +14,13 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config" ) // ZoneKey is the index of the first level in the constraint conformance report. type ZoneKey struct { // ZoneID is the id of the zone this key is referencing. - ZoneID uint32 + ZoneID config.SystemTenantObjectID // SubzoneID identifies what subzone, if any, this key is referencing. The // zero value (also named NoSubzone) indicates that the key is referring to a // zone, not a subzone. @@ -34,7 +35,7 @@ const NoSubzone base.SubzoneID = 0 // // Use NoSubzone for subzoneID to indicate that the key references a zone, not a // subzone. -func MakeZoneKey(zoneID uint32, subzoneID base.SubzoneID) ZoneKey { +func MakeZoneKey(zoneID config.SystemTenantObjectID, subzoneID base.SubzoneID) ZoneKey { return ZoneKey{ ZoneID: zoneID, SubzoneID: subzoneID, diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index aa7e5152798b..3c4950a2684d 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -1475,8 +1475,12 @@ 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( + config.SystemTenantObjectID(baseID), zonepb.ZoneConfig{RangeMaxBytes: proto.Int64(1 << 20)}, + ) + config.TestingSetZoneConfig( + config.SystemTenantObjectID(baseID+2), zonepb.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/server/admin.go b/pkg/server/admin.go index 7d8d672d23d3..6cf16c16fbfe 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -78,6 +78,7 @@ func nonTableDescriptorRangeCount() int64 { keys.TimeseriesRangesID, keys.LivenessRangesID, keys.PublicSchemaID, + keys.TenantsRangesID, })) } @@ -699,9 +700,8 @@ func (s *adminServer) NonTableStats( } } - // There are four empty ranges for table descriptors 17, 18, 19, and 22 that - // aren't actually tables (a.k.a. MetaRangesID, SystemRangesID, - // TimeseriesRangesID, and LivenessRangesID in pkg/keys). + // There are six empty ranges for table descriptors 17, 18, 19, 22, 29, and + // 37 that aren't actually tables (a.k.a. the PseudoTableIDs in pkg/keys). // No data is ever really written to them since they don't have actual // tables. Some backend work could probably be done to eliminate these empty // ranges, but it may be more trouble than it's worth. In the meantime, diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index e1e6720a45e1..80d4a7374314 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -403,7 +403,7 @@ func TestAdminAPINonTableStats(t *testing.T) { NodeCount: 3, }, InternalUseStats: &serverpb.TableStatsResponse{ - RangeCount: 9, + RangeCount: 10, ReplicaCount: 12, NodeCount: 3, }, diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 6d690bd958a5..0d3e8ef2cb49 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -728,6 +728,9 @@ func ExpectedInitialRangeCount( maxSystemDescriptorID = descID } } + if maxSystemDescriptorID < sqlbase.ID(keys.MaxPseudoTableID) { + maxSystemDescriptorID = sqlbase.ID(keys.MaxPseudoTableID) + } systemTableSplits := int(maxSystemDescriptorID - keys.MaxSystemConfigDescID) // `n` splits create `n+1` ranges. diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 24dda5a757c5..dee77abbadbf 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -42,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -2379,6 +2381,10 @@ CREATE TABLE crdb_internal.zones ( ) `, populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { + if !p.ExecCfg().Codec.ForSystemTenant() { + return errorutil.UnsupportedWithMultiTenancy() + } + namespace, err := p.getAllNames(ctx) if err != nil { return err @@ -2429,7 +2435,7 @@ CREATE TABLE crdb_internal.zones ( // Inherit full information about this zone. fullZone := configProto - if err := completeZoneConfig(&fullZone, uint32(tree.MustBeDInt(r[0])), getKey); err != nil { + if err := completeZoneConfig(&fullZone, config.SystemTenantObjectID(tree.MustBeDInt(r[0])), getKey); err != nil { return err } @@ -2900,6 +2906,14 @@ func addPartitioningRows( colOffset int, addRow func(...tree.Datum) error, ) error { + // Secondary tenants cannot set zone configs on individual objects, so they + // have no ability to partition tables/indexes. + // NOTE: we assume the system tenant below by casting object IDs directly to + // config.SystemTenantObjectID. + if !p.ExecCfg().Codec.ForSystemTenant() { + return nil + } + tableID := tree.NewDInt(tree.DInt(table.ID)) indexID := tree.NewDInt(tree.DInt(index.ID)) numColumns := tree.NewDInt(tree.DInt(partitioning.NumColumns)) @@ -2943,7 +2957,7 @@ func addPartitioningRows( // Figure out which zone and subzone this partition should correspond to. zoneID, zone, subzone, err := GetZoneConfigInTxn( - ctx, p.txn, uint32(table.ID), index, l.Name, false /* getInheritedDefault */) + ctx, p.txn, config.SystemTenantObjectID(table.ID), index, l.Name, false /* getInheritedDefault */) if err != nil { return err } @@ -2999,7 +3013,7 @@ func addPartitioningRows( // Figure out which zone and subzone this partition should correspond to. zoneID, zone, subzone, err := GetZoneConfigInTxn( - ctx, p.txn, uint32(table.ID), index, r.Name, false /* getInheritedDefault */) + ctx, p.txn, config.SystemTenantObjectID(table.ID), index, r.Name, false /* getInheritedDefault */) if err != nil { return err } diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 61a3f420366f..d40e6fe2c52f 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -228,9 +228,9 @@ func (n *dropDatabaseNode) startExec(params runParams) error { } // No job was created because no tables were dropped, so zone config can be - // immediately removed. - if len(tableDescs) == 0 { - zoneKeyPrefix := config.MakeZoneKeyPrefix(uint32(n.dbDesc.ID)) + // immediately removed, if applicable. + if len(tableDescs) == 0 && params.ExecCfg().Codec.ForSystemTenant() { + zoneKeyPrefix := config.MakeZoneKeyPrefix(config.SystemTenantObjectID(n.dbDesc.ID)) if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { log.VEventf(ctx, 2, "DelRange %s", zoneKeyPrefix) } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 25dab4992bfd..25bd598b136f 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -16,6 +16,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" @@ -258,28 +259,32 @@ func (p *planner) dropIndexByName( ) } - // Check if requires CCL binary for eventual zone config removal. - _, zone, _, err := GetZoneConfigInTxn(ctx, p.txn, uint32(tableDesc.ID), nil, "", false) - if err != nil { - return err - } + // Check if requires CCL binary for eventual zone config removal. Only + // necessary for the system tenant, because secondary tenants do not have + // zone configs for individual objects. + if p.ExecCfg().Codec.ForSystemTenant() { + _, zone, _, err := GetZoneConfigInTxn(ctx, p.txn, config.SystemTenantObjectID(tableDesc.ID), nil, "", false) + if err != nil { + return err + } - for _, s := range zone.Subzones { - if s.IndexID != uint32(idx.ID) { - _, err = GenerateSubzoneSpans( - p.ExecCfg().Settings, - p.ExecCfg().ClusterID(), - p.ExecCfg().Codec, - tableDesc.TableDesc(), - zone.Subzones, - false, /* newSubzones */ - ) - if sqlbase.IsCCLRequiredError(err) { - return sqlbase.NewCCLRequiredError(fmt.Errorf("schema change requires a CCL binary "+ - "because table %q has at least one remaining index or partition with a zone config", - tableDesc.Name)) + for _, s := range zone.Subzones { + if s.IndexID != uint32(idx.ID) { + _, err = GenerateSubzoneSpans( + p.ExecCfg().Settings, + p.ExecCfg().ClusterID(), + p.ExecCfg().Codec, + tableDesc.TableDesc(), + zone.Subzones, + false, /* newSubzones */ + ) + if sqlbase.IsCCLRequiredError(err) { + return sqlbase.NewCCLRequiredError(fmt.Errorf("schema change requires a CCL binary "+ + "because table %q has at least one remaining index or partition with a zone config", + tableDesc.Name)) + } + break } - break } } diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index bad4b9f7953a..8bfac2c2f0d2 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -351,15 +351,6 @@ func (p *planner) initiateDropTable( // TODO(bram): If interleaved and ON DELETE CASCADE, we will be able to use // this faster mechanism. if tableDesc.IsTable() && !tableDesc.IsInterleaved() { - // Get the zone config applying to this table in order to - // ensure there is a GC TTL. - _, _, _, err := GetZoneConfigInTxn( - ctx, p.txn, uint32(tableDesc.ID), &sqlbase.IndexDescriptor{}, "", false, /* getInheritedDefault */ - ) - if err != nil { - return err - } - tableDesc.DropTime = timeutil.Now().UnixNano() } diff --git a/pkg/sql/gcjob/descriptor_utils.go b/pkg/sql/gcjob/descriptor_utils.go index d4d7bca37d1b..fe30290925cf 100644 --- a/pkg/sql/gcjob/descriptor_utils.go +++ b/pkg/sql/gcjob/descriptor_utils.go @@ -58,37 +58,44 @@ func dropTableDesc( ctx context.Context, db *kv.DB, codec keys.SQLCodec, tableDesc *sqlbase.TableDescriptor, ) error { log.Infof(ctx, "removing table descriptor for table %d", tableDesc.ID) - descKey := sqlbase.MakeDescMetadataKey(codec, tableDesc.ID) - zoneKeyPrefix := config.MakeZoneKeyPrefix(uint32(tableDesc.ID)) - return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { if err := txn.SetSystemConfigTrigger(); err != nil { return err } b := &kv.Batch{} + // Delete the descriptor. + descKey := sqlbase.MakeDescMetadataKey(codec, tableDesc.ID) b.Del(descKey) - // Delete the zone config entry for this table. - b.DelRange(zoneKeyPrefix, zoneKeyPrefix.PrefixEnd(), false /* returnKeys */) - + // Delete the zone config entry for this table, if necessary. + if codec.ForSystemTenant() { + zoneKeyPrefix := config.MakeZoneKeyPrefix(config.SystemTenantObjectID(tableDesc.ID)) + b.DelRange(zoneKeyPrefix, zoneKeyPrefix.PrefixEnd(), false /* returnKeys */) + } return txn.Run(ctx, b) }) } // deleteDatabaseZoneConfig removes the zone config for a given database ID. -func deleteDatabaseZoneConfig(ctx context.Context, db *kv.DB, databaseID sqlbase.ID) error { +func deleteDatabaseZoneConfig( + ctx context.Context, db *kv.DB, codec keys.SQLCodec, databaseID sqlbase.ID, +) error { + if databaseID == sqlbase.InvalidID { + return nil + } + if !codec.ForSystemTenant() { + // Secondary tenants do not have zone configs for individual objects. + return nil + } return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - b := &kv.Batch{} if err := txn.SetSystemConfigTrigger(); err != nil { return err } + b := &kv.Batch{} // Delete the zone config entry for the dropped database associated with the // job, if it exists. - if databaseID == sqlbase.InvalidID { - return nil - } - dbZoneKeyPrefix := config.MakeZoneKeyPrefix(uint32(databaseID)) + dbZoneKeyPrefix := config.MakeZoneKeyPrefix(config.SystemTenantObjectID(databaseID)) b.DelRange(dbZoneKeyPrefix, dbZoneKeyPrefix.PrefixEnd(), false /* returnKeys */) return txn.Run(ctx, b) }) diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index 2d17ae3b0901..4ba924e25dce 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -72,7 +72,7 @@ func performGC( // Drop database zone config when all the tables have been GCed. if details.ParentID != sqlbase.InvalidID && isDoneGC(progress) { - if err := deleteDatabaseZoneConfig(ctx, execCfg.DB, details.ParentID); err != nil { + if err := deleteDatabaseZoneConfig(ctx, execCfg.DB, execCfg.Codec, details.ParentID); err != nil { return false, errors.Wrap(err, "deleting database zone config") } } diff --git a/pkg/sql/gcjob/refresh_statuses.go b/pkg/sql/gcjob/refresh_statuses.go index bbfe416dd3b2..f7e4bdcfe309 100644 --- a/pkg/sql/gcjob/refresh_statuses.go +++ b/pkg/sql/gcjob/refresh_statuses.go @@ -92,15 +92,12 @@ func updateStatusForGCElements( return err } - zoneCfg, placeholder, _, err := sql.ZoneConfigHook(cfg, uint32(tableID)) + zoneCfg, err := cfg.GetZoneConfigForObject(execCfg.Codec, uint32(tableID)) if err != nil { log.Errorf(ctx, "zone config for desc: %d, err = %+v", tableID, err) return nil } tableTTL := getTableTTL(defTTL, zoneCfg) - if placeholder == nil { - placeholder = zoneCfg - } // Update the status of the table if the table was dropped. if table.Dropped() { @@ -113,7 +110,7 @@ func updateStatusForGCElements( } // Update the status of any indexes waiting for GC. - indexesExpired, deadline := updateIndexesStatus(ctx, execCfg, tableTTL, table, protectedtsCache, placeholder, indexDropTimes, progress) + indexesExpired, deadline := updateIndexesStatus(ctx, execCfg, tableTTL, table, protectedtsCache, zoneCfg, indexDropTimes, progress) if indexesExpired { expired = true } @@ -184,7 +181,7 @@ func updateIndexesStatus( tableTTL int32, table *sqlbase.TableDescriptor, protectedtsCache protectedts.Cache, - placeholder *zonepb.ZoneConfig, + zoneCfg *zonepb.ZoneConfig, indexDropTimes map[sqlbase.IndexID]int64, progress *jobspb.SchemaChangeGCProgress, ) (expired bool, soonestDeadline time.Time) { @@ -198,7 +195,7 @@ func updateIndexesStatus( sp := table.IndexSpan(execCfg.Codec, idxProgress.IndexID) - ttlSeconds := getIndexTTL(tableTTL, placeholder, idxProgress.IndexID) + ttlSeconds := getIndexTTL(tableTTL, zoneCfg, idxProgress.IndexID) deadlineNanos := indexDropTimes[idxProgress.IndexID] + int64(ttlSeconds)*time.Second.Nanoseconds() deadline := timeutil.Unix(0, deadlineNanos) diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index e4d459acadd3..2ced9c7bfcf9 100644 --- a/pkg/sql/logictest/testdata/logic_test/ranges +++ b/pkg/sql/logictest/testdata/logic_test/ranges @@ -307,7 +307,8 @@ start_key start_pretty end_key [170] /Table/34 [171] /Table/35 system statement_bundle_chunks · {1} 1 [171] /Table/35 [172] /Table/36 system statement_diagnostics_requests · {1} 1 [172] /Table/36 [173] /Table/37 system statement_diagnostics · {1} 1 -[173] /Table/37 [189 137] /Table/53/1 system scheduled_jobs · {1} 1 +[173] /Table/37 [174] /Table/38 system scheduled_jobs · {1} 1 +[174] /Table/38 [189 137] /Table/53/1 · · · {1} 1 [189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 @@ -363,7 +364,8 @@ start_key start_pretty end_key [170] /Table/34 [171] /Table/35 system statement_bundle_chunks · {1} 1 [171] /Table/35 [172] /Table/36 system statement_diagnostics_requests · {1} 1 [172] /Table/36 [173] /Table/37 system statement_diagnostics · {1} 1 -[173] /Table/37 [189 137] /Table/53/1 system scheduled_jobs · {1} 1 +[173] /Table/37 [174] /Table/38 system scheduled_jobs · {1} 1 +[174] /Table/38 [189 137] /Table/53/1 · · · {1} 1 [189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 diff --git a/pkg/sql/logictest/testdata/logic_test/tenant b/pkg/sql/logictest/testdata/logic_test/tenant index 90e280eefe8a..1e40e155251f 100644 --- a/pkg/sql/logictest/testdata/logic_test/tenant +++ b/pkg/sql/logictest/testdata/logic_test/tenant @@ -84,3 +84,30 @@ query T SHOW CLUSTER SETTING sql.defaults.vectorize ---- on + +# Manipulate the tenants zone configurations + +query TT +SHOW ZONE CONFIGURATION FOR RANGE tenants +---- +RANGE default ALTER RANGE default CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 3, + constraints = '[]', + lease_preferences = '[]' + +statement ok +ALTER RANGE tenants CONFIGURE ZONE USING gc.ttlseconds = 300 + +query TT +SHOW ZONE CONFIGURATION FOR RANGE tenants +---- +RANGE tenants ALTER RANGE tenants CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 300, + num_replicas = 3, + constraints = '[]', + lease_preferences = '[]' diff --git a/pkg/sql/logictest/testdata/logic_test/tenant_unsupported b/pkg/sql/logictest/testdata/logic_test/tenant_unsupported index e926626138f9..6ce443d096a9 100644 --- a/pkg/sql/logictest/testdata/logic_test/tenant_unsupported +++ b/pkg/sql/logictest/testdata/logic_test/tenant_unsupported @@ -31,11 +31,6 @@ SET experimental_enable_temp_tables = true statement ok CREATE TEMP TABLE users (id UUID, city STRING, CONSTRAINT "primary" PRIMARY KEY (id ASC, city ASC)) -# TODO(asubiotto): Uncomment this test once -# https://github.com/cockroachdb/cockroach/issues/49318 is fixed. -# statement error operation is unsupported -# ALTER TABLE kv CONFIGURE ZONE USING num_replicas = 123 - # Missing status server statement error operation is unsupported @@ -70,3 +65,14 @@ SELECT * FROM crdb_internal.kv_store_status statement error operation is unsupported SELECT * FROM crdb_internal.kv_node_status + +# Cannot manipulate zone configurations + +statement error operation is unsupported +SELECT * FROM crdb_internal.zones + +statement error operation is unsupported +SHOW ZONE CONFIGURATION FOR TABLE kv + +statement error operation is unsupported +ALTER TABLE kv CONFIGURE ZONE USING num_replicas = 123 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit b/pkg/sql/opt/exec/execbuilder/testdata/autocommit index ae51e4f558c7..cb6f42f4280b 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit @@ -41,7 +41,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 # Multi-row insert should auto-commit. query B @@ -62,7 +62,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -86,7 +86,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 2 CPut to (n1,s1):1 +dist sender send r34: sending batch 2 CPut to (n1,s1):1 statement ok ROLLBACK @@ -110,7 +110,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -132,8 +132,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 2 CPut to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 CPut to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -156,8 +156,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 2 CPut to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 CPut to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -192,7 +192,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Put, 1 EndTxn to (n1,s1):1 # Multi-row upsert should auto-commit. query B @@ -213,7 +213,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -237,7 +237,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 2 Put to (n1,s1):1 +dist sender send r34: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -261,7 +261,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -283,8 +283,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 2 Put to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 Put to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 # Upsert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -307,8 +307,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 2 Put to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 Put to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -343,8 +343,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -368,8 +368,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Put to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -393,8 +393,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -416,9 +416,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Put to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Put to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 # Update with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -441,9 +441,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Put to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Put to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -478,7 +478,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Multi-row delete should auto-commit. query B @@ -499,7 +499,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -523,7 +523,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 DelRng to (n1,s1):1 +dist sender send r34: sending batch 1 DelRng to (n1,s1):1 statement ok ROLLBACK @@ -547,8 +547,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Del, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Del, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -570,9 +570,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Del to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Del to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -595,9 +595,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Del to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Del to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 statement ok INSERT INTO ab VALUES (12, 0); @@ -646,9 +646,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 2 CPut, 2 InitPut to (n1,s1):1 -dist sender send r33: sending batch 2 Scan to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 CPut, 2 InitPut to (n1,s1):1 +dist sender send r34: sending batch 2 Scan to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -669,10 +669,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 1 Put, 1 CPut, 1 Del to (n1,s1):1 -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 1 Put, 1 CPut, 1 Del to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -694,10 +694,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 1 Del to (n1,s1):1 -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 1 Del to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 # Test with a single cascade, which should use autocommit. statement ok @@ -721,10 +721,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 1 Del to (n1,s1):1 -dist sender send r33: sending batch 1 Scan to (n1,s1):1 -dist sender send r33: sending batch 2 Del, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 1 Del to (n1,s1):1 +dist sender send r34: sending batch 1 Scan to (n1,s1):1 +dist sender send r34: sending batch 2 Del, 1 EndTxn to (n1,s1):1 # ----------------------- # Multiple mutation tests @@ -752,9 +752,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 2 CPut to (n1,s1):1 -dist sender send r33: sending batch 2 CPut to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 CPut to (n1,s1):1 +dist sender send r34: sending batch 2 CPut to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -777,6 +777,6 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r33: sending batch 2 CPut to (n1,s1):1 -dist sender send r33: sending batch 2 CPut to (n1,s1):1 -dist sender send r33: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 2 CPut to (n1,s1):1 +dist sender send r34: sending batch 2 CPut to (n1,s1):1 +dist sender send r34: sending batch 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/delete b/pkg/sql/opt/exec/execbuilder/testdata/delete index d7bad759a50a..a3599ad205c8 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/delete +++ b/pkg/sql/opt/exec/execbuilder/testdata/delete @@ -216,9 +216,9 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%DelRng%' ---- flow DelRange /Table/57/1 - /Table/57/2 -dist sender send r33: sending batch 1 DelRng to (n1,s1):1 +dist sender send r34: sending batch 1 DelRng to (n1,s1):1 flow DelRange /Table/57/1/601/0 - /Table/57/2 -dist sender send r33: sending batch 1 DelRng to (n1,s1):1 +dist sender send r34: sending batch 1 DelRng to (n1,s1):1 # Ensure that DelRange requests are autocommitted when DELETE FROM happens on a # chunk of fewer than 600 keys. @@ -234,7 +234,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%' ---- flow DelRange /Table/57/1/5 - /Table/57/1/5/# -dist sender send r33: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r34: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Test use of fast path when there are interleaved tables. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index 1e4a09ebac2c..408abf52699e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -255,7 +255,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] table reader Scan /Table/57/1/2{-/#} flow CPut /Table/57/1/2/0 -> /TUPLE/2:2:Int/3 flow InitPut /Table/57/2/3/0 -> /BYTES/0x8a -kv.DistSender: sending partial batch r33: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r34: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -269,7 +269,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] table reader Scan /Table/57/1/1{-/#} flow CPut /Table/57/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/57/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r33: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r34: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -286,7 +286,7 @@ table reader fetched: /kv/primary/2/v -> /3 flow Put /Table/57/1/2/0 -> /TUPLE/2:2:Int/2 flow Del /Table/57/2/3/0 flow CPut /Table/57/2/2/0 -> /BYTES/0x8a (expecting does not exist) -kv.DistSender: sending partial batch r33: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r34: sending batch 1 Put, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 0c7aa5b1ff40..656bbb465012 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -391,7 +391,7 @@ func (oc *optCatalog) getZoneConfig( if oc.cfg == nil || desc.IsVirtualTable() { return emptyZoneConfig, nil } - zone, err := oc.cfg.GetZoneConfigForObject(uint32(desc.ID)) + zone, err := oc.cfg.GetZoneConfigForObject(oc.codec(), uint32(desc.ID)) if err != nil { return nil, err } diff --git a/pkg/sql/pgwire/testdata/pgtest/notice b/pkg/sql/pgwire/testdata/pgtest/notice index 0f9627bfe588..8c5dd5cb42a1 100644 --- a/pkg/sql/pgwire/testdata/pgtest/notice +++ b/pkg/sql/pgwire/testdata/pgtest/notice @@ -55,7 +55,7 @@ Query {"String": "DROP INDEX t_x_idx"} until crdb_only CommandComplete ---- -{"Severity":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":500,"Routine":"dropIndexByName","UnknownFields":null} +{"Severity":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":505,"Routine":"dropIndexByName","UnknownFields":null} {"Type":"CommandComplete","CommandTag":"DROP INDEX"} until noncrdb_only diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 35aff3f07d14..9f668ddf5713 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -998,7 +998,7 @@ func (sc *SchemaChanger) maybeUpdateZoneConfigsForPKChange( table *sqlbase.TableDescriptor, swapInfo *sqlbase.PrimaryKeySwap, ) error { - zone, err := getZoneConfigRaw(ctx, txn, table.ID) + zone, err := getZoneConfigRaw(ctx, txn, execCfg.Codec, table.ID) if err != nil { return err } diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 92a16f4aad15..e16f85bf77d1 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" "github.com/gogo/protobuf/proto" @@ -101,6 +102,9 @@ func (p *planner) SetZoneConfig(ctx context.Context, n *tree.SetZoneConfig) (pla if err := checkPrivilegeForSetZoneConfig(ctx, p, n.ZoneSpecifier); err != nil { return nil, err } + if !p.ExecCfg().Codec.ForSystemTenant() { + return nil, errorutil.UnsupportedWithMultiTenancy() + } var yamlConfig tree.TypedExpr @@ -382,7 +386,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { } // Retrieve the partial zone configuration - partialZone, err := getZoneConfigRaw(params.ctx, params.p.txn, targetID) + partialZone, err := getZoneConfigRaw(params.ctx, params.p.txn, params.ExecCfg().Codec, targetID) if err != nil { return err } @@ -413,7 +417,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // These zones are only used for validations. The merged zone is will // not be written. _, completeZone, completeSubzone, err := GetZoneConfigInTxn(params.ctx, params.p.txn, - uint32(targetID), index, partition, n.setDefault) + config.SystemTenantObjectID(targetID), index, partition, n.setDefault) if errors.Is(err, errNoZoneConfigApplies) { // No zone config yet. @@ -444,7 +448,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // inherit from its parent. We do this by using an empty zoneConfig // and completing at the level of the current zone. zoneInheritedFields := zonepb.ZoneConfig{} - if err := completeZoneConfig(&zoneInheritedFields, uint32(targetID), getKey); err != nil { + if err := completeZoneConfig(&zoneInheritedFields, config.SystemTenantObjectID(targetID), getKey); err != nil { return err } partialZone.CopyFromZone(zoneInheritedFields, copyFromParentList) @@ -452,7 +456,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // If we are operating on a subZone, we need to inherit all remaining // unset fields in its parent zone, which is partialZone. zoneInheritedFields := *partialZone - if err := completeZoneConfig(&zoneInheritedFields, uint32(targetID), getKey); err != nil { + if err := completeZoneConfig(&zoneInheritedFields, config.SystemTenantObjectID(targetID), getKey); err != nil { return err } // In the case we have just an index, we should copy from the inherited @@ -602,7 +606,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // here to complete the missing fields. The reason is because we don't know // here if a zone is a placeholder or not. Can we do a GetConfigInTxn here? // And if it is a placeholder, we use getZoneConfigRaw to create one. - completeZone, err = getZoneConfigRaw(params.ctx, params.p.txn, targetID) + completeZone, err = getZoneConfigRaw(params.ctx, params.p.txn, params.ExecCfg().Codec, targetID) if err != nil { return err } else if completeZone == nil { @@ -866,8 +870,14 @@ func writeZoneConfig( // getZoneConfigRaw looks up the zone config with the given ID. Unlike // getZoneConfig, it does not attempt to ascend the zone config hierarchy. If no // zone config exists for the given ID, it returns nil. -func getZoneConfigRaw(ctx context.Context, txn *kv.Txn, id sqlbase.ID) (*zonepb.ZoneConfig, error) { - kv, err := txn.Get(ctx, config.MakeZoneKey(uint32(id))) +func getZoneConfigRaw( + ctx context.Context, txn *kv.Txn, codec keys.SQLCodec, id sqlbase.ID, +) (*zonepb.ZoneConfig, error) { + if !codec.ForSystemTenant() { + // Secondary tenants do not have zone configs for individual objects. + return nil, nil + } + kv, err := txn.Get(ctx, config.MakeZoneKey(config.SystemTenantObjectID(id))) if err != nil { return nil, err } @@ -899,7 +909,7 @@ func RemoveIndexZoneConfigs( return err } - zone, err := getZoneConfigRaw(ctx, txn, tableID) + zone, err := getZoneConfigRaw(ctx, txn, execCfg.Codec, tableID) if err != nil { return err } else if zone == nil { diff --git a/pkg/sql/show_zone_config.go b/pkg/sql/show_zone_config.go index ad0d1292b665..aa6d63af1c42 100644 --- a/pkg/sql/show_zone_config.go +++ b/pkg/sql/show_zone_config.go @@ -15,12 +15,14 @@ import ( "context" "strings" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" "gopkg.in/yaml.v2" @@ -61,6 +63,10 @@ const ( ) func (p *planner) ShowZoneConfig(ctx context.Context, n *tree.ShowZoneConfig) (planNode, error) { + if !p.ExecCfg().Codec.ForSystemTenant() { + return nil, errorutil.UnsupportedWithMultiTenancy() + } + return &delayedNode{ name: n.String(), columns: showZoneConfigColumns, @@ -125,7 +131,7 @@ func getShowZoneConfigRow( subZoneIdx := uint32(0) zoneID, zone, subzone, err := GetZoneConfigInTxn(ctx, p.txn, - uint32(targetID), index, partition, false /* getInheritedDefault */) + config.SystemTenantObjectID(targetID), index, partition, false /* getInheritedDefault */) if errors.Is(err, errNoZoneConfigApplies) { // TODO(benesch): This shouldn't be the caller's responsibility; // GetZoneConfigInTxn should just return the default zone config if no zone @@ -146,7 +152,7 @@ func getShowZoneConfigRow( // Determine the zone specifier for the zone config that actually applies // without performing another KV lookup. - zs := ascendZoneSpecifier(zoneSpecifier, uint32(targetID), zoneID, subzone) + zs := ascendZoneSpecifier(zoneSpecifier, config.SystemTenantObjectID(targetID), zoneID, subzone) // Ensure subzone configs don't infect the output of config_bytes. zone.Subzones = nil @@ -342,7 +348,9 @@ 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 config.SystemTenantObjectID, + actualSubzone *zonepb.Subzone, ) tree.ZoneSpecifier { if actualID == keys.RootNamespaceID { // We had to traverse to the top of the hierarchy, so we're showing the diff --git a/pkg/sql/sqlbase/system.go b/pkg/sql/sqlbase/system.go index bd9a69063489..b2fd8c90ebf6 100644 --- a/pkg/sql/sqlbase/system.go +++ b/pkg/sql/sqlbase/system.go @@ -1782,6 +1782,7 @@ func addSystemDatabaseToSchema( defaultSystemZoneConfig *zonepb.ZoneConfig, ) { addSystemDescriptorsToSchema(target) + // TODO(nvanbenschoten): only do this for the system tenant. Tracked in #48774. addSplitIDs(target) addZoneConfigKVsToSchema(target, defaultZoneConfig, defaultSystemZoneConfig) } diff --git a/pkg/sql/tests/testdata/initial_keys b/pkg/sql/tests/testdata/initial_keys index 48a40546f4a1..5185b9578eac 100644 --- a/pkg/sql/tests/testdata/initial_keys +++ b/pkg/sql/tests/testdata/initial_keys @@ -68,7 +68,7 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"users"/4/1 /NamespaceTable/30/1/1/29/"web_sessions"/4/1 /NamespaceTable/30/1/1/29/"zones"/4/1 -27 splits: +28 splits: /Table/11 /Table/12 /Table/13 @@ -96,6 +96,7 @@ initial-keys tenant=system /Table/35 /Table/36 /Table/37 + /Table/38 initial-keys tenant=5 ---- @@ -158,7 +159,7 @@ initial-keys tenant=5 /Tenant/5/NamespaceTable/30/1/1/29/"ui"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"users"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"web_sessions"/4/1 -27 splits: +28 splits: /Tenant/5/Table/11 /Tenant/5/Table/12 /Tenant/5/Table/13 @@ -186,6 +187,7 @@ initial-keys tenant=5 /Tenant/5/Table/35 /Tenant/5/Table/36 /Tenant/5/Table/37 + /Tenant/5/Table/38 initial-keys tenant=999 ---- @@ -248,7 +250,7 @@ initial-keys tenant=999 /Tenant/999/NamespaceTable/30/1/1/29/"ui"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"users"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"web_sessions"/4/1 -27 splits: +28 splits: /Tenant/999/Table/11 /Tenant/999/Table/12 /Tenant/999/Table/13 @@ -276,3 +278,4 @@ initial-keys tenant=999 /Tenant/999/Table/35 /Tenant/999/Table/36 /Tenant/999/Table/37 + /Tenant/999/Table/38 diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 0873935cd8cc..1f8c7b014175 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -191,7 +191,6 @@ func (p *planner) truncateTable( // structured.proto // // TODO(vivek): Fix properly along with #12123. - zoneKey := config.MakeZoneKey(uint32(tableDesc.ID)) key := sqlbase.MakeObjectNameKey( ctx, p.ExecCfg().Settings, newTableDesc.ParentID, @@ -272,24 +271,31 @@ func (p *planner) truncateTable( return err } - // Copy the zone config. - b := &kv.Batch{} - b.Get(zoneKey) - if err := p.txn.Run(ctx, b); err != nil { - return err - } - val := b.Results[0].Rows[0].Value - if val == nil { - return nil - } - zoneCfg, err := val.GetBytes() - if err != nil { - return err + // Copy the zone config, if this is for the system tenant. Secondary tenants + // do not have zone configs for individual objects. + if p.ExecCfg().Codec.ForSystemTenant() { + zoneKey := config.MakeZoneKey(config.SystemTenantObjectID(tableDesc.ID)) + b := &kv.Batch{} + b.Get(zoneKey) + if err := p.txn.Run(ctx, b); err != nil { + return err + } + val := b.Results[0].Rows[0].Value + if val == nil { + return nil + } + zoneCfg, err := val.GetBytes() + if err != nil { + return err + } + const insertZoneCfg = `INSERT INTO system.zones (id, config) VALUES ($1, $2)` + if _, err = p.ExtendedEvalContext().ExecCfg.InternalExecutor.Exec( + ctx, "insert-zone", p.txn, insertZoneCfg, newID, zoneCfg, + ); err != nil { + return err + } } - const insertZoneCfg = `INSERT INTO system.zones (id, config) VALUES ($1, $2)` - _, err = p.ExtendedEvalContext().ExecCfg.InternalExecutor.Exec( - ctx, "insert-zone", p.txn, insertZoneCfg, newID, zoneCfg) - return err + return nil } // For all the references from a table diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 3239ae641312..a7a4a1afa26b 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -30,17 +30,11 @@ import ( func init() { // TODO(marc): we use a hook to avoid a dependency on the sql package. We // should probably move keys/protos elsewhere. - config.ZoneConfigHook = ZoneConfigHook + config.ZoneConfigHook = zoneConfigHook } var errNoZoneConfigApplies = errors.New("no zone config applies") -// TODO(nvanbenschoten): determine how zone configurations fit into a -// multi-tenant cluster. Does each tenant have its own Zones table? Does KV have -// to make sure to look at the correct Zones according to the tenant prefix of -// its key range? See #48375. -var zoneConfigCodec = keys.TODOSQLCodec - // getZoneConfig recursively looks up entries in system.zones until an // entry that applies to the object with the specified id is // found. Returns the ID of the matching zone, its zone config, and an @@ -52,10 +46,18 @@ var zoneConfigCodec = keys.TODOSQLCodec // If getInheritedDefault is true, the direct zone configuration, if it exists, is // 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) { + id config.SystemTenantObjectID, + getKey func(roachpb.Key) (*roachpb.Value, error), + getInheritedDefault bool, +) ( + config.SystemTenantObjectID, + *zonepb.ZoneConfig, + config.SystemTenantObjectID, + *zonepb.ZoneConfig, + error, +) { var placeholder *zonepb.ZoneConfig - var placeholderID uint32 + var placeholderID config.SystemTenantObjectID if !getInheritedDefault { // Look in the zones table. if zoneVal, err := getKey(config.MakeZoneKey(id)); err != nil { @@ -79,7 +81,7 @@ func getZoneConfig( // No zone config for this ID. We need to figure out if it's a table, so we // look up its descriptor. - if descVal, err := getKey(sqlbase.MakeDescMetadataKey(zoneConfigCodec, sqlbase.ID(id))); err != nil { + if descVal, err := getKey(sqlbase.MakeDescMetadataKey(keys.SystemSQLCodec, sqlbase.ID(id))); err != nil { return 0, nil, 0, nil, err } else if descVal != nil { var desc sqlbase.Descriptor @@ -88,7 +90,7 @@ func getZoneConfig( } if tableDesc := desc.Table(descVal.Timestamp); tableDesc != nil { // This is a table descriptor. Look up its parent database zone config. - dbID, zone, _, _, err := getZoneConfig(uint32(tableDesc.ParentID), getKey, false /* getInheritedDefault */) + dbID, zone, _, _, err := getZoneConfig(config.SystemTenantObjectID(tableDesc.ParentID), getKey, false /* getInheritedDefault */) if err != nil { return 0, nil, 0, nil, err } @@ -116,14 +118,16 @@ 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 *zonepb.ZoneConfig, + id config.SystemTenantObjectID, + getKey func(roachpb.Key) (*roachpb.Value, error), ) error { if cfg.IsComplete() { return nil } // Check to see if its a table. If so, inherit from the database. // For all other cases, inherit from the default. - if descVal, err := getKey(sqlbase.MakeDescMetadataKey(zoneConfigCodec, sqlbase.ID(id))); err != nil { + if descVal, err := getKey(sqlbase.MakeDescMetadataKey(keys.SystemSQLCodec, sqlbase.ID(id))); err != nil { return err } else if descVal != nil { var desc sqlbase.Descriptor @@ -131,7 +135,7 @@ func completeZoneConfig( return err } if tableDesc := desc.Table(descVal.Timestamp); tableDesc != nil { - _, dbzone, _, _, err := getZoneConfig(uint32(tableDesc.ParentID), getKey, false /* getInheritedDefault */) + _, dbzone, _, _, err := getZoneConfig(config.SystemTenantObjectID(tableDesc.ParentID), getKey, false /* getInheritedDefault */) if err != nil { return err } @@ -151,15 +155,15 @@ func completeZoneConfig( return nil } -// ZoneConfigHook returns the zone config and optional placeholder config for +// zoneConfigHook returns the zone config and optional placeholder config for // the object with id using the cached system config. The returned boolean is // set to true when the zone config returned can be cached. // -// ZoneConfigHook is a pure function whose only inputs are a system config and +// zoneConfigHook is a pure function whose only inputs are a system config and // an object ID. It does not make any external KV calls to look up additional // state. -func ZoneConfigHook( - cfg *config.SystemConfig, id uint32, +func zoneConfigHook( + cfg *config.SystemConfig, id config.SystemTenantObjectID, ) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) { getKey := func(key roachpb.Key) (*roachpb.Value, error) { return cfg.GetValue(key), nil @@ -187,11 +191,11 @@ func ZoneConfigHook( func GetZoneConfigInTxn( ctx context.Context, txn *kv.Txn, - id uint32, + id config.SystemTenantObjectID, index *sqlbase.IndexDescriptor, partition string, getInheritedDefault bool, -) (uint32, *zonepb.ZoneConfig, *zonepb.Subzone, error) { +) (config.SystemTenantObjectID, *zonepb.ZoneConfig, *zonepb.Subzone, error) { getKey := func(key roachpb.Key) (*roachpb.Value, error) { kv, err := txn.Get(ctx, key) if err != nil { @@ -278,7 +282,7 @@ func resolveZone(ctx context.Context, txn *kv.Txn, zs *tree.ZoneSpecifier) (sqlb errMissingKey := errors.New("missing key") id, err := zonepb.ResolveZoneSpecifier(zs, func(parentID uint32, name string) (uint32, error) { - found, id, err := sqlbase.LookupPublicTableID(ctx, txn, zoneConfigCodec, sqlbase.ID(parentID), name) + found, id, err := sqlbase.LookupPublicTableID(ctx, txn, keys.SystemSQLCodec, sqlbase.ID(parentID), name) if err != nil { return 0, err } @@ -349,7 +353,7 @@ func deleteRemovedPartitionZoneConfigs( if len(removedNames) == 0 { return nil } - zone, err := getZoneConfigRaw(ctx, txn, tableDesc.ID) + zone, err := getZoneConfigRaw(ctx, txn, execCfg.Codec, tableDesc.ID) if err != nil { return err } else if zone == nil { diff --git a/pkg/sql/zone_config_test.go b/pkg/sql/zone_config_test.go index e26caca8823f..0a07908191bf 100644 --- a/pkg/sql/zone_config_test.go +++ b/pkg/sql/zone_config_test.go @@ -133,8 +133,9 @@ func TestGetZoneConfig(t *testing.T) { // Verify sql.GetZoneConfigInTxn. if err := s.DB().Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { - _, zoneCfg, subzone, err := sql.GetZoneConfigInTxn(ctx, txn, - tc.objectID, &sqlbase.IndexDescriptor{}, tc.partitionName, false) + _, zoneCfg, subzone, err := sql.GetZoneConfigInTxn( + ctx, txn, config.SystemTenantObjectID(tc.objectID), &sqlbase.IndexDescriptor{}, tc.partitionName, false, + ) if err != nil { return err } else if subzone != nil { @@ -369,8 +370,9 @@ func TestCascadingZoneConfig(t *testing.T) { // Verify sql.GetZoneConfigInTxn. if err := s.DB().Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { - _, zoneCfg, subzone, err := sql.GetZoneConfigInTxn(ctx, txn, - tc.objectID, &sqlbase.IndexDescriptor{}, tc.partitionName, false) + _, zoneCfg, subzone, err := sql.GetZoneConfigInTxn( + ctx, txn, config.SystemTenantObjectID(tc.objectID), &sqlbase.IndexDescriptor{}, tc.partitionName, false, + ) if err != nil { return err } else if subzone != nil {