diff --git a/c-deps/libroach/encoding.h b/c-deps/libroach/encoding.h index e1a597448570..ef89821d5778 100644 --- a/c-deps/libroach/encoding.h +++ b/c-deps/libroach/encoding.h @@ -121,6 +121,8 @@ WARN_UNUSED_RESULT bool IsInt(rocksdb::Slice* buf); // DecodeTablePrefix validates that the given key has a table prefix. On // completion, buf holds the remainder of the key (with the prefix removed) and // tbl stores the decoded descriptor ID of the table. +// +// TODO(nvanbenschoten): support tenant ID prefix. WARN_UNUSED_RESULT bool DecodeTablePrefix(rocksdb::Slice* buf, uint64_t* tbl); } // namespace cockroach diff --git a/c-deps/libroach/row_counter.h b/c-deps/libroach/row_counter.h index 48b71fa8b9d5..662f1c163116 100644 --- a/c-deps/libroach/row_counter.h +++ b/c-deps/libroach/row_counter.h @@ -23,6 +23,8 @@ const int MaxReservedDescID = 49; // RowCounter counts how many distinct rows appear in the KVs that is is shown // via `Count`. Note: the `DataSize` field of the BulkOpSummary is *not* // populated by this and should be set separately. +// +// TODO(nvanbenschoten): support tenant ID prefix. struct RowCounter { RowCounter(cockroach::roachpb::BulkOpSummary* summary) : summary(summary) {} bool Count(const rocksdb::Slice& key); diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index e686265b1e45..f4559f93c860 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -967,11 +967,11 @@ func TestBackupRestoreCheckpointing(t *testing.T) { if err != nil { return err } - low := keys.MakeTablePrefix(ip.backupTableID) - high := keys.MakeTablePrefix(ip.backupTableID + 1) + low := keys.SystemSQLCodec.TablePrefix(ip.backupTableID) + high := keys.SystemSQLCodec.TablePrefix(ip.backupTableID + 1) if bytes.Compare(highWaterMark, low) <= 0 || bytes.Compare(highWaterMark, high) >= 0 { return errors.Errorf("expected high-water mark %v to be between %v and %v", - highWaterMark, roachpb.Key(low), roachpb.Key(high)) + highWaterMark, low, high) } return nil } @@ -2467,7 +2467,7 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) { gcr := roachpb.GCRequest{ // Bogus span to make it a valid request. RequestHeader: roachpb.RequestHeader{ - Key: keys.MakeTablePrefix(keys.MinUserDescID), + Key: keys.SystemSQLCodec.TablePrefix(keys.MinUserDescID), EndKey: keys.MaxKey, }, Threshold: tc.Server(0).Clock().Now(), diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 0c4d67b5e5cc..c5e5f1e1d167 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -544,11 +544,11 @@ func rewriteBackupSpanKey(kr *storageccl.KeyRewriter, key roachpb.Key) (roachpb. // start of the table. That is, change a span start key from /Table/51/1 to // /Table/51. Otherwise a permanently empty span at /Table/51-/Table/51/1 // will be created. - if b, id, idx, err := sqlbase.DecodeTableIDIndexID(newKey); err != nil { + if b, id, idx, err := keys.TODOSQLCodec.DecodeIndexPrefix(newKey); err != nil { return nil, errors.NewAssertionErrorWithWrappedErrf(err, "could not rewrite span start key: %s", key) } else if idx == 1 && len(b) == 0 { - newKey = keys.MakeTablePrefix(uint32(id)) + newKey = keys.TODOSQLCodec.TablePrefix(id) } return newKey, nil } diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index f947a8f00481..76bd5de8b297 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -410,7 +409,7 @@ func getAllDescChanges( startTime, endTime hlc.Timestamp, priorIDs map[sqlbase.ID]sqlbase.ID, ) ([]BackupManifest_DescriptorRevision, error) { - startKey := roachpb.Key(keys.MakeTablePrefix(keys.DescriptorTableID)) + startKey := keys.TODOSQLCodec.TablePrefix(keys.DescriptorTableID) endKey := startKey.PrefixEnd() allRevs, err := storageccl.GetAllRevisions(ctx, db, startKey, endKey, startTime, endTime) @@ -421,7 +420,7 @@ func getAllDescChanges( var res []BackupManifest_DescriptorRevision for _, revs := range allRevs { - id, err := keys.DecodeDescMetadataID(revs.Key) + id, err := keys.TODOSQLCodec.DecodeDescMetadataID(revs.Key) if err != nil { return nil, err } @@ -445,7 +444,7 @@ func getAllDescChanges( } func allSQLDescriptors(ctx context.Context, txn *kv.Txn) ([]sqlbase.Descriptor, error) { - startKey := roachpb.Key(keys.MakeTablePrefix(keys.DescriptorTableID)) + startKey := keys.TODOSQLCodec.TablePrefix(keys.DescriptorTableID) endKey := startKey.PrefixEnd() rows, err := txn.Scan(ctx, startKey, endKey, 0) if err != nil { diff --git a/pkg/ccl/changefeedccl/changefeed.go b/pkg/ccl/changefeedccl/changefeed.go index b6f558dc6ae3..dd0ccc8df799 100644 --- a/pkg/ccl/changefeedccl/changefeed.go +++ b/pkg/ccl/changefeedccl/changefeed.go @@ -402,7 +402,7 @@ func makeSpansToProtect(targets jobspb.ChangefeedTargets) []roachpb.Span { // of table descriptors to version data. spansToProtect := make([]roachpb.Span, 0, len(targets)+1) addTablePrefix := func(id uint32) { - tablePrefix := roachpb.Key(keys.MakeTablePrefix(id)) + tablePrefix := keys.TODOSQLCodec.TablePrefix(id) spansToProtect = append(spansToProtect, roachpb.Span{ Key: tablePrefix, EndKey: tablePrefix.PrefixEnd(), diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 88cad8a807a8..aafb6562a697 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -790,7 +790,7 @@ func TestChangefeedSchemaChangeAllowBackfill(t *testing.T) { func fetchDescVersionModificationTime( t testing.TB, db *gosql.DB, f cdctest.TestFeedFactory, tableName string, version int, ) hlc.Timestamp { - tblKey := roachpb.Key(keys.MakeTablePrefix(keys.DescriptorTableID)) + tblKey := keys.SystemSQLCodec.TablePrefix(keys.DescriptorTableID) header := roachpb.RequestHeader{ Key: tblKey, EndKey: tblKey.PrefixEnd(), @@ -822,7 +822,7 @@ func fetchDescVersionModificationTime( continue } k := it.UnsafeKey() - remaining, _, _, err := sqlbase.DecodeTableIDIndexID(k.Key) + remaining, _, _, err := keys.SystemSQLCodec.DecodeIndexPrefix(k.Key) if err != nil { t.Fatal(err) } @@ -2204,8 +2204,8 @@ func TestChangefeedProtectedTimestamps(t *testing.T) { } mkCheckRecord = func(t *testing.T, tableID int) func(r *ptpb.Record) error { expectedKeys := map[string]struct{}{ - string(keys.MakeTablePrefix(uint32(tableID))): {}, - string(keys.MakeTablePrefix(keys.DescriptorTableID)): {}, + string(keys.SystemSQLCodec.TablePrefix(uint32(tableID))): {}, + string(keys.SystemSQLCodec.TablePrefix(keys.DescriptorTableID)): {}, } return func(ptr *ptpb.Record) error { if ptr == nil { diff --git a/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go b/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go index e72685664c0e..bde88e03c52d 100644 --- a/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go +++ b/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go @@ -40,7 +40,7 @@ func TestKVFeed(t *testing.T) { } kv := func(tableID uint32, k, v string, ts hlc.Timestamp) roachpb.KeyValue { vDatum := tree.DString(k) - key, err := sqlbase.EncodeTableKey(keys.MakeTablePrefix(tableID), &vDatum, encoding.Ascending) + key, err := sqlbase.EncodeTableKey(keys.SystemSQLCodec.TablePrefix(tableID), &vDatum, encoding.Ascending) if err != nil { panic(err) } @@ -356,7 +356,7 @@ var _ schemaFeed = (*rawTableFeed)(nil) func tableSpan(tableID uint32) roachpb.Span { return roachpb.Span{ - Key: keys.MakeTablePrefix(tableID), - EndKey: roachpb.Key(keys.MakeTablePrefix(tableID)).PrefixEnd(), + Key: keys.SystemSQLCodec.TablePrefix(tableID), + EndKey: keys.SystemSQLCodec.TablePrefix(tableID).PrefixEnd(), } } diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index 821df5fa3e34..f108e4ddf98e 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -11,6 +11,7 @@ package changefeedccl import ( "context" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -43,8 +44,12 @@ func (c *rowFetcherCache) TableDescForKey( ctx context.Context, key roachpb.Key, ts hlc.Timestamp, ) (*sqlbase.ImmutableTableDescriptor, error) { var tableDesc *sqlbase.ImmutableTableDescriptor + key, err := keys.TODOSQLCodec.StripTenantPrefix(key) + if err != nil { + return nil, err + } for skippedCols := 0; ; { - remaining, tableID, _, err := sqlbase.DecodeTableIDIndexID(key) + remaining, tableID, _, err := sqlbase.DecodePartialTableIDIndexID(key) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go index 57a5ba37a52f..2c0768aa5c61 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go +++ b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go @@ -452,7 +452,7 @@ func fetchTableDescriptorVersions( log.Infof(ctx, `fetching table descs (%s,%s]`, startTS, endTS) } start := timeutil.Now() - span := roachpb.Span{Key: keys.MakeTablePrefix(keys.DescriptorTableID)} + span := roachpb.Span{Key: keys.TODOSQLCodec.TablePrefix(keys.DescriptorTableID)} span.EndKey = span.Key.PrefixEnd() header := roachpb.Header{Timestamp: endTS} req := &roachpb.ExportRequest{ @@ -486,7 +486,7 @@ func fetchTableDescriptorVersions( return nil } k := it.UnsafeKey() - remaining, _, _, err := sqlbase.DecodeTableIDIndexID(k.Key) + remaining, _, _, err := keys.TODOSQLCodec.DecodeIndexPrefix(k.Key) if err != nil { return err } diff --git a/pkg/ccl/importccl/bench_test.go b/pkg/ccl/importccl/bench_test.go index 6540698c14a5..c8b9562b1589 100644 --- a/pkg/ccl/importccl/bench_test.go +++ b/pkg/ccl/importccl/bench_test.go @@ -61,7 +61,7 @@ func BenchmarkImportWorkload(b *testing.B) { t := tableSSTable{ meta: table, - span: roachpb.Span{Key: keys.MakeTablePrefix(uint32(tableID))}, + span: roachpb.Span{Key: keys.SystemSQLCodec.TablePrefix(uint32(tableID))}, sstData: sst, } t.span.EndKey = t.span.Key.PrefixEnd() diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go index a4c2de6bba65..6dae7fea78ce 100644 --- a/pkg/ccl/importccl/import_processor.go +++ b/pkg/ccl/importccl/import_processor.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/storagebase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -319,7 +320,7 @@ func ingestKvs( // number of L0 (and total) files, but with a lower memory usage. for kvBatch := range kvCh { for _, kv := range kvBatch.KVs { - _, _, indexID, indexErr := sqlbase.DecodeTableIDIndexID(kv.Key) + _, _, indexID, indexErr := keys.TODOSQLCodec.DecodeIndexPrefix(kv.Key) if indexErr != nil { return indexErr } diff --git a/pkg/ccl/storageccl/bench_test.go b/pkg/ccl/storageccl/bench_test.go index a88a97b098ac..e3914303525a 100644 --- a/pkg/ccl/storageccl/bench_test.go +++ b/pkg/ccl/storageccl/bench_test.go @@ -189,7 +189,7 @@ func BenchmarkImport(b *testing.B) { OldID: uint32(tableDesc.ID), NewDesc: newDescBytes, }) } - newStartKey := roachpb.Key(keys.MakeTablePrefix(uint32(id))) + newStartKey := keys.SystemSQLCodec.TablePrefix(uint32(id)) b.StartTimer() var files []roachpb.ImportRequest_File diff --git a/pkg/ccl/storageccl/key_rewriter.go b/pkg/ccl/storageccl/key_rewriter.go index 8df90829ab74..723c39d58939 100644 --- a/pkg/ccl/storageccl/key_rewriter.go +++ b/pkg/ccl/storageccl/key_rewriter.go @@ -11,6 +11,7 @@ package storageccl import ( "bytes" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -116,10 +117,7 @@ func MakeKeyRewriter(descs map[sqlbase.ID]*sqlbase.TableDescriptor) (*KeyRewrite // function, but it takes into account interleaved ancestors, which we don't // want here. func makeKeyRewriterPrefixIgnoringInterleaved(tableID sqlbase.ID, indexID sqlbase.IndexID) []byte { - var key []byte - key = encoding.EncodeUvarintAscending(key, uint64(tableID)) - key = encoding.EncodeUvarintAscending(key, uint64(indexID)) - return key + return keys.TODOSQLCodec.IndexPrefix(uint32(tableID), uint32(indexID)) } // RewriteKey modifies key (possibly in place), changing all table IDs to their @@ -142,7 +140,7 @@ func (kr *KeyRewriter) RewriteKey(key []byte, isFromSpan bool) ([]byte, bool, er // Fetch the original table ID for descriptor lookup. Ignore errors because // they will be caught later on if tableID isn't in descs or kr doesn't // perform a rewrite. - _, tableID, _ := encoding.DecodeUvarintAscending(key) + _, tableID, _ := keys.TODOSQLCodec.DecodeTablePrefix(key) // Rewrite the first table ID. key, ok := kr.prefixes.rewriteKey(key) if !ok { @@ -153,7 +151,7 @@ func (kr *KeyRewriter) RewriteKey(key []byte, isFromSpan bool) ([]byte, bool, er return nil, false, errors.Errorf("missing descriptor for table %d", tableID) } // Check if this key may have interleaved children. - k, _, indexID, err := sqlbase.DecodeTableIDIndexID(key) + k, _, indexID, err := keys.TODOSQLCodec.DecodeIndexPrefix(key) if err != nil { return nil, false, err } @@ -161,7 +159,7 @@ func (kr *KeyRewriter) RewriteKey(key []byte, isFromSpan bool) ([]byte, bool, er // If there isn't any more data, we are at some split boundary. return key, true, nil } - idx, err := desc.FindIndexByID(indexID) + idx, err := desc.FindIndexByID(sqlbase.IndexID(indexID)) if err != nil { return nil, false, err } diff --git a/pkg/ccl/utilccl/sampledataccl/bankdata_test.go b/pkg/ccl/utilccl/sampledataccl/bankdata_test.go index 4fa89c6822cb..e0583ce46376 100644 --- a/pkg/ccl/utilccl/sampledataccl/bankdata_test.go +++ b/pkg/ccl/utilccl/sampledataccl/bankdata_test.go @@ -73,7 +73,7 @@ func TestToBackup(t *testing.T) { t.Run("NextKeyValues", func(t *testing.T) { for _, requestedKVs := range []int{2, 3} { newTableID := sqlbase.ID(keys.MaxReservedDescID + requestedKVs) - newTablePrefix := roachpb.Key(keys.MakeTablePrefix(uint32(newTableID))) + newTablePrefix := keys.SystemSQLCodec.TablePrefix(uint32(newTableID)) keys := make(map[string]struct{}, rows) for { diff --git a/pkg/cmd/zerosum/main.go b/pkg/cmd/zerosum/main.go index b179c245f2dd..704f784140b1 100644 --- a/pkg/cmd/zerosum/main.go +++ b/pkg/cmd/zerosum/main.go @@ -216,7 +216,7 @@ func (z *zeroSum) monkey(tableID uint32, d time.Duration) { for { time.Sleep(time.Duration(rand.Float64() * float64(d))) - key := keys.MakeTablePrefix(tableID) + key := keys.SystemSQLCodec.TablePrefix(tableID) key = encoding.EncodeVarintAscending(key, int64(zipf.Uint64())) switch r.Intn(2) { diff --git a/pkg/config/keys.go b/pkg/config/keys.go index 347f10b4e61f..14e0f7c58345 100644 --- a/pkg/config/keys.go +++ b/pkg/config/keys.go @@ -13,32 +13,23 @@ package config import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/encoding" ) // MakeZoneKeyPrefix returns the key prefix for id's row in the system.zones // table. func MakeZoneKeyPrefix(id uint32) roachpb.Key { - return keys.ZoneKeyPrefix(id) + return keys.TODOSQLCodec.ZoneKeyPrefix(id) } // MakeZoneKey returns the key for id's entry in the system.zones table. func MakeZoneKey(id uint32) roachpb.Key { - return keys.ZoneKey(id) + return keys.TODOSQLCodec.ZoneKey(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) { - if key.Equal(roachpb.RKeyMax) { - return 0, nil, false - } - if encoding.PeekType(key) != encoding.Int { - // TODO(marc): this should eventually return SystemDatabaseID. - return 0, nil, false - } - // Consume first encoded int. - rem, id64, err := encoding.DecodeUvarintAscending(key) - return uint32(id64), rem, err == nil + rem, id, err := keys.TODOSQLCodec.DecodeTablePrefix(key.AsRawKey()) + return id, rem, err == nil } diff --git a/pkg/config/keys_test.go b/pkg/config/keys_test.go index 35b3d05a338e..3ce4741e0345 100644 --- a/pkg/config/keys_test.go +++ b/pkg/config/keys_test.go @@ -37,9 +37,9 @@ func TestDecodeObjectID(t *testing.T) { {roachpb.RKeyMax, nil, false, 0}, // Valid, even if there are things after the ID. - {testutils.MakeKey(keys.MakeTablePrefix(42), roachpb.RKey("\xff")), []byte{'\xff'}, true, 42}, - {keys.MakeTablePrefix(0), []byte{}, true, 0}, - {keys.MakeTablePrefix(999), []byte{}, true, 999}, + {testutils.MakeKey(keys.SystemSQLCodec.TablePrefix(42), roachpb.RKey("\xff")), []byte{'\xff'}, true, 42}, + {roachpb.RKey(keys.SystemSQLCodec.TablePrefix(0)), []byte{}, true, 0}, + {roachpb.RKey(keys.SystemSQLCodec.TablePrefix(999)), []byte{}, true, 999}, } for tcNum, tc := range testCases { diff --git a/pkg/config/system.go b/pkg/config/system.go index d28eaa8119c9..64ef3a26c7f9 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -113,7 +113,7 @@ func (s *SystemConfig) GetDesc(key roachpb.Key) *roachpb.Value { return getVal } - id, err := keys.DecodeDescMetadataID(key) + id, err := keys.TODOSQLCodec.DecodeDescMetadataID(key) if err != nil { // No ID found for key. No roachpb.Value corresponds to this key. panic(err) @@ -181,11 +181,11 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { } // Search for the descriptor table entries within the SystemConfig. - highBound := roachpb.Key(keys.MakeTablePrefix(keys.DescriptorTableID + 1)) + highBound := keys.TODOSQLCodec.TablePrefix(keys.DescriptorTableID + 1) highIndex := sort.Search(len(s.Values), func(i int) bool { return bytes.Compare(s.Values[i].Key, highBound) >= 0 }) - lowBound := roachpb.Key(keys.MakeTablePrefix(keys.DescriptorTableID)) + lowBound := keys.TODOSQLCodec.TablePrefix(keys.DescriptorTableID) lowIndex := sort.Search(len(s.Values), func(i int) bool { return bytes.Compare(s.Values[i].Key, lowBound) >= 0 }) @@ -197,7 +197,7 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { // No maximum specified; maximum ID is the last entry in the descriptor // table. if maxID == 0 { - id, err := keys.DecodeDescMetadataID(s.Values[highIndex-1].Key) + id, err := keys.TODOSQLCodec.DecodeDescMetadataID(s.Values[highIndex-1].Key) if err != nil { return 0, err } @@ -211,7 +211,7 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { var err error maxIdx := sort.Search(len(searchSlice), func(i int) bool { var id uint64 - id, err = keys.DecodeDescMetadataID(searchSlice[i].Key) + id, err = keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[i].Key) if err != nil { return false } @@ -224,7 +224,7 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { // If we found an index within the list, maxIdx might point to a descriptor // with exactly maxID. if maxIdx < len(searchSlice) { - id, err := keys.DecodeDescMetadataID(searchSlice[maxIdx].Key) + id, err := keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[maxIdx].Key) if err != nil { return 0, err } @@ -238,7 +238,7 @@ func (s *SystemConfig) GetLargestObjectID(maxID uint32) (uint32, error) { } // Return ID of the immediately preceding descriptor. - id, err := keys.DecodeDescMetadataID(searchSlice[maxIdx-1].Key) + id, err := keys.TODOSQLCodec.DecodeDescMetadataID(searchSlice[maxIdx-1].Key) if err != nil { return 0, err } @@ -449,7 +449,7 @@ func (s *SystemConfig) ComputeSplitKey(startKey, endKey roachpb.RKey) (rr roachp findSplitKey := func(startID, endID uint32) roachpb.RKey { // endID could be smaller than startID if we don't have user tables. for id := startID; id <= endID; id++ { - tableKey := roachpb.RKey(keys.MakeTablePrefix(id)) + tableKey := roachpb.RKey(keys.TODOSQLCodec.TablePrefix(id)) // This logic is analogous to the well-commented static split logic above. if startKey.Less(tableKey) && s.shouldSplit(id) { if tableKey.Less(endKey) { @@ -532,7 +532,7 @@ func (s *SystemConfig) shouldSplit(ID uint32) bool { // actual descriptors. shouldSplit = true } else { - desc := s.GetDesc(keys.DescMetadataKey(ID)) + desc := s.GetDesc(keys.TODOSQLCodec.DescMetadataKey(ID)) shouldSplit = desc != nil && sqlbase.ShouldSplitAtID(ID, desc) } // Populate the cache. diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index 2b289fdef9a6..94927f2e4121 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -33,7 +33,7 @@ func plainKV(k, v string) roachpb.KeyValue { } func tkey(tableID uint32, chunks ...string) []byte { - key := keys.MakeTablePrefix(tableID) + key := keys.SystemSQLCodec.TablePrefix(tableID) for _, c := range chunks { key = append(key, []byte(c)...) } diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 8ebada4bdfc1..005cc89026db 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -29,6 +29,7 @@ const ( metaMaxByte = '\x04' systemPrefixByte = metaMaxByte systemMaxByte = '\x05' + tenantPrefixByte = '\xfe' ) // Constants for system-reserved keys in the KV map. @@ -262,26 +263,34 @@ var ( // 3. SQL keys // + // TODO(nvanbenschoten): Figure out what to do with all of these. At a + // minimum, prefix them all with "System". + // // TableDataMin is the start of the range of table data keys. - TableDataMin = roachpb.Key(MakeTablePrefix(0)) + TableDataMin = SystemSQLCodec.TablePrefix(0) // TableDataMin is the end of the range of table data keys. - TableDataMax = roachpb.Key(MakeTablePrefix(math.MaxUint32)) + TableDataMax = SystemSQLCodec.TablePrefix(math.MaxUint32).PrefixEnd() // // SystemConfigSplitKey is the key to split at immediately prior to the // system config span. NB: Split keys need to be valid column keys. // TODO(bdarnell): this should be either roachpb.Key or RKey, not []byte. SystemConfigSplitKey = []byte(TableDataMin) // SystemConfigTableDataMax is the end key of system config span. - SystemConfigTableDataMax = roachpb.Key(MakeTablePrefix(MaxSystemConfigDescID + 1)) + SystemConfigTableDataMax = SystemSQLCodec.TablePrefix(MaxSystemConfigDescID + 1) // // NamespaceTableMin is the start key of system.namespace, which is a system // table that does not reside in the same range as other system tables. - NamespaceTableMin = roachpb.Key(MakeTablePrefix(NamespaceTableID)) + NamespaceTableMin = SystemSQLCodec.TablePrefix(NamespaceTableID) // NamespaceTableMax is the end key of system.namespace. - NamespaceTableMax = roachpb.Key(MakeTablePrefix(NamespaceTableID + 1)) + NamespaceTableMax = SystemSQLCodec.TablePrefix(NamespaceTableID + 1) // // UserTableDataMin is the start key of user structured data. - UserTableDataMin = roachpb.Key(MakeTablePrefix(MinUserDescID)) + UserTableDataMin = SystemSQLCodec.TablePrefix(MinUserDescID) + + // tenantPrefix is the prefix for all non-system tenant keys. + tenantPrefix = roachpb.Key{tenantPrefixByte} + TenantTableDataMin = MakeTenantPrefix(roachpb.MinTenantID) + TenantTableDataMax = MakeTenantPrefix(roachpb.MaxTenantID).PrefixEnd() ) // Various IDs used by the structured data layer. @@ -374,6 +383,15 @@ const ( IndexCommentType = 3 ) +const ( + // SequenceIndexID is the ID of the single index on each special single-column, + // single-row sequence table. + SequenceIndexID = 1 + // SequenceColumnFamilyID is the ID of the column family on each special single-column, + // single-row sequence table. + SequenceColumnFamilyID = 0 +) + // PseudoTableIDs is the list of ids from above that are not real tables (i.e. // there's no table descriptor). They're grouped here because the cluster // bootstrap process needs to create splits for them; splits for the tables diff --git a/pkg/keys/doc.go b/pkg/keys/doc.go index 1cf8f097738d..e06198a7067d 100644 --- a/pkg/keys/doc.go +++ b/pkg/keys/doc.go @@ -21,26 +21,37 @@ // // This is the ten-thousand foot view of the keyspace: // -// +----------+ -// | (empty) | /Min -// | \x01... | /Local ----+ -// | | | -// | ... | | local keys -// | | | -// | | ----+ -// | \x02... | /Meta1 ----+ -// | \x03... | /Meta2 | -// | \x04... | /System | -// | | | -// | ... | | -// | | | global keys -// | \x89... | /Table/1 | -// | \x8a... | /Table/2 | -// | | | -// | ... | | -// | | | -// | \xff\xff | /Max ----+ -// +----------+ +// +------------------+ +// | (empty) | /Min +// | \x01... | /Local ---------------------+ +// | | | +// | ... | | local keys +// | | | +// | | ---------------------+ +// | | ---------------------+ +// | \x02... | /Meta1 ----+ | +// | \x03... | /Meta2 | | +// | \x04... | /System | | +// | | | system keys | +// | ... | | | +// | | ----+ | +// | \x89... | /Table/1 ----+ | +// | \x8a... | /Table/2 | | +// | | | system tenant | +// | ... | | | global keys +// | | ----+ | +// | \xfe\x8a\x89... | /Tenant/2/Table/1 ----+ | +// | \xfe\x8a\x8a... | /Tenant/2/Table/2 | | +// | | | tenant 2 | +// | ... | | | +// | | ----+ | +// | \xfe... | /Tenant/... ----+ | +// | \xfe... | | | +// | | | tenant ... | +// | ... | | | +// | | ----+ | +// | \xff\xff | /Max ---------------------+ +// +------------------+ // // When keys are pretty printed, the logical name to the right of the table is // shown instead of the raw byte sequence. @@ -205,7 +216,8 @@ var _ = [...]interface{}{ StoreIdentKey, // "iden" StoreLastUpKey, // "uptm" - // The global keyspace includes the meta{1,2}, system, and SQL keys. + // The global keyspace includes the meta{1,2}, system, system tenant SQL + // keys, and non-system tenant SQL keys. // // 1. Meta keys: This is where we store all key addressing data. MetaMin, @@ -229,12 +241,18 @@ var _ = [...]interface{}{ TimeseriesPrefix, // "tsd" SystemMax, - // 3. SQL keys: This is where we store all table data. + // 3. System tenant SQL keys: This is where we store all system-tenant + // table data. TableDataMin, NamespaceTableMin, UserTableDataMin, TableDataMax, + // 4. Non-system tenant SQL keys: This is where we store all non-system + // tenant table data. + TenantTableDataMin, + TenantTableDataMax, + MaxKey, } diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 46957185ff82..12a2be61d3d1 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -643,57 +643,12 @@ func MetaReverseScanBounds(key roachpb.RKey) (roachpb.RSpan, error) { return roachpb.RSpan{Key: start, EndKey: end}, nil } -// MakeTablePrefix returns the key prefix used for the table's data. -func MakeTablePrefix(tableID uint32) []byte { - return encoding.EncodeUvarintAscending(nil, uint64(tableID)) -} - -// DecodeTablePrefix validates that the given key has a table prefix, returning -// the remainder of the key (with the prefix removed) and the decoded descriptor -// ID of the table. -func DecodeTablePrefix(key roachpb.Key) ([]byte, uint64, error) { - if encoding.PeekType(key) != encoding.Int { - return key, 0, errors.Errorf("invalid key prefix: %q", key) - } - return encoding.DecodeUvarintAscending(key) -} - -// DescMetadataPrefix returns the key prefix for all descriptors. -func DescMetadataPrefix() []byte { - k := MakeTablePrefix(uint32(DescriptorTableID)) - return encoding.EncodeUvarintAscending(k, DescriptorTablePrimaryKeyIndexID) -} - -// DescMetadataKey returns the key for the descriptor. -func DescMetadataKey(descID uint32) roachpb.Key { - k := DescMetadataPrefix() - k = encoding.EncodeUvarintAscending(k, uint64(descID)) - return MakeFamilyKey(k, DescriptorTableDescriptorColFamID) -} - -// DecodeDescMetadataID decodes a descriptor ID from a descriptor metadata key. -func DecodeDescMetadataID(key roachpb.Key) (uint64, error) { - // Extract object ID from key. - // TODO(marc): move sql/keys.go to keys (or similar) and use a DecodeDescMetadataKey. - // We should also check proper encoding. - remaining, tableID, err := DecodeTablePrefix(key) - if err != nil { - return 0, err - } - if tableID != DescriptorTableID { - return 0, errors.Errorf("key is not a descriptor table entry: %v", key) - } - // DescriptorTable.PrimaryIndex.ID - remaining, _, err = encoding.DecodeUvarintAscending(remaining) - if err != nil { - return 0, err - } - // descID - _, id, err := encoding.DecodeUvarintAscending(remaining) - if err != nil { - return 0, err - } - return id, nil +// MakeTableIDIndexID returns the key for the table id and index id by appending +// to the passed key. The key must already contain a tenant id. +func MakeTableIDIndexID(key []byte, tableID uint32, indexID uint32) []byte { + key = encoding.EncodeUvarintAscending(key, uint64(tableID)) + key = encoding.EncodeUvarintAscending(key, uint64(indexID)) + return key } // MakeFamilyKey returns the key for the family in the given row by appending to @@ -711,22 +666,22 @@ func MakeFamilyKey(key []byte, famID uint32) []byte { return encoding.EncodeUvarintAscending(key, uint64(len(key)-size)) } -const ( - // SequenceIndexID is the ID of the single index on each special single-column, - // single-row sequence table. - SequenceIndexID = 1 - // SequenceColumnFamilyID is the ID of the column family on each special single-column, - // single-row sequence table. - SequenceColumnFamilyID = 0 -) +// DecodeTableIDIndexID decodes a table id followed by an index id from the +// provided key. The input key must already have its tenant id removed. +func DecodeTableIDIndexID(key []byte) ([]byte, uint32, uint32, error) { + var tableID uint64 + var indexID uint64 + var err error -// MakeSequenceKey returns the key used to store the value of a sequence. -func MakeSequenceKey(tableID uint32) []byte { - key := MakeTablePrefix(tableID) - key = encoding.EncodeUvarintAscending(key, SequenceIndexID) // Index id - key = encoding.EncodeUvarintAscending(key, 0) // Primary key value - key = encoding.EncodeUvarintAscending(key, SequenceColumnFamilyID) // Column family - return key + key, tableID, err = encoding.DecodeUvarintAscending(key) + if err != nil { + return nil, 0, 0, err + } + key, indexID, err = encoding.DecodeUvarintAscending(key) + if err != nil { + return nil, 0, 0, err + } + return key, uint32(tableID), uint32(indexID), nil } // GetRowPrefixLength returns the length of the row prefix of the key. A table @@ -734,6 +689,8 @@ func MakeSequenceKey(tableID uint32) []byte { // prefix of every key for the same row. (Any key with this maximal prefix is // also guaranteed to be part of the input key's row.) // For secondary index keys, the row prefix is defined as the entire key. +// +// TODO(nvanbenschoten): support tenant ID prefix. func GetRowPrefixLength(key roachpb.Key) (int, error) { n := len(key) if encoding.PeekType(key) != encoding.Int { @@ -928,16 +885,3 @@ func (b RangeIDPrefixBuf) RaftLogKey(logIndex uint64) roachpb.Key { func (b RangeIDPrefixBuf) RangeLastReplicaGCTimestampKey() roachpb.Key { return append(b.unreplicatedPrefix(), LocalRangeLastReplicaGCTimestampSuffix...) } - -// ZoneKeyPrefix returns the key prefix for id's row in the system.zones table. -func ZoneKeyPrefix(id uint32) roachpb.Key { - k := MakeTablePrefix(uint32(ZonesTableID)) - k = encoding.EncodeUvarintAscending(k, uint64(ZonesTablePrimaryIndexID)) - return encoding.EncodeUvarintAscending(k, uint64(id)) -} - -// ZoneKey returns the key for id's entry in the system.zones table. -func ZoneKey(id uint32) roachpb.Key { - k := ZoneKeyPrefix(id) - return MakeFamilyKey(k, uint32(ZonesTableConfigColumnID)) -} diff --git a/pkg/keys/keys_test.go b/pkg/keys/keys_test.go index 4b1e941daead..77868e81c593 100644 --- a/pkg/keys/keys_test.go +++ b/pkg/keys/keys_test.go @@ -12,6 +12,7 @@ package keys import ( "bytes" + "fmt" "math" "reflect" "testing" @@ -21,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/stretchr/testify/require" ) func TestStoreKeyEncodeDecode(t *testing.T) { @@ -236,7 +238,7 @@ func TestUserKey(t *testing.T) { } func TestSequenceKey(t *testing.T) { - actual := MakeSequenceKey(55) + actual := SystemSQLCodec.SequenceKey(55) expected := []byte("\xbf\x89\x88\x88") if !bytes.Equal(actual, expected) { t.Errorf("expected %q (len %d), got %q (len %d)", expected, len(expected), actual, len(actual)) @@ -543,6 +545,7 @@ func TestEnsureSafeSplitKey(t *testing.T) { {e(1, 2, 3, 1), e(1, 2)}, // /Table/1/2/3/1 -> /Table/1/2 {e(1, 2, 200, 2), e(1, 2)}, // /Table/1/2/200/2 -> /Table/1/2 {e(1, 2, 3, 4, 1), e(1, 2, 3)}, // /Table/1/2/3/4/1 -> /Table/1/2/3 + // TODO(nvanbenschoten): add test cases for tenant keys. } for i, d := range goodData { out, err := EnsureSafeSplitKey(d.in) @@ -579,7 +582,8 @@ func TestEnsureSafeSplitKey(t *testing.T) { {e(1, 2, 200)[:3], "insufficient bytes to decode uvarint value"}, // Exercises a former overflow bug. We decode a uint(18446744073709551610) which, if casted // to int carelessly, results in -6. - {encoding.EncodeVarintAscending(MakeTablePrefix(999), 322434), "malformed table key"}, + {encoding.EncodeVarintAscending(SystemSQLCodec.TablePrefix(999), 322434), "malformed table key"}, + // TODO(nvanbenschoten): add test cases for tenant keys. } for i, d := range errorData { _, err := EnsureSafeSplitKey(d.in) @@ -588,3 +592,38 @@ func TestEnsureSafeSplitKey(t *testing.T) { } } } + +func TestTenantPrefix(t *testing.T) { + tIDs := []roachpb.TenantID{ + roachpb.SystemTenantID, + roachpb.MakeTenantID(2), + roachpb.MakeTenantID(999), + roachpb.MakeTenantID(math.MaxUint64), + } + for _, tID := range tIDs { + t.Run(fmt.Sprintf("%v", tID), func(t *testing.T) { + // Encode tenant ID. + k := MakeTenantPrefix(tID) + + // The system tenant has no tenant prefix. + if tID == roachpb.SystemTenantID { + require.Len(t, k, 0) + } + + // Encode table prefix. + const tableID = 5 + k = encoding.EncodeUvarintAscending(k, tableID) + + // Decode tenant ID. + rem, retTID, err := DecodeTenantPrefix(k) + require.Equal(t, tID, retTID) + require.NoError(t, err) + + // Decode table prefix. + rem, retTableID, err := encoding.DecodeUvarintAscending(rem) + require.Len(t, rem, 0) + require.Equal(t, uint64(tableID), retTableID) + require.NoError(t, err) + }) + } +} diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index a87ea39d7d04..09c704e68cf1 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -122,7 +122,7 @@ var ( PSFunc: parseUnsupported, }, {Name: "/tsd", prefix: TimeseriesPrefix, - ppFunc: decodeTimeseriesKey, + ppFunc: timeseriesKeyPrint, PSFunc: parseUnsupported, }, }}, @@ -132,6 +132,9 @@ var ( {Name: "/Table", start: TableDataMin, end: TableDataMax, Entries: []DictEntry{ {Name: "", prefix: nil, ppFunc: decodeKeyPrint, PSFunc: tableKeyParse}, }}, + {Name: "/Tenant", start: TenantTableDataMin, end: TenantTableDataMax, Entries: []DictEntry{ + {Name: "", prefix: nil, ppFunc: tenantKeyPrint, PSFunc: tenantKeyParse}, + }}, } // keyofKeyDict means the key of suffix which is itself a key, @@ -231,9 +234,32 @@ func localStoreKeyParse(input string) (remainder string, output roachpb.Key) { return } +const strTable = "/Table/" const strSystemConfigSpan = "SystemConfigSpan" const strSystemConfigSpanStart = "Start" +func tenantKeyParse(input string) (remainder string, output roachpb.Key) { + input = mustShiftSlash(input) + slashPos := strings.Index(input, "/") + if slashPos < 0 { + slashPos = len(input) + } + remainder = input[slashPos:] // `/something/else` -> `/else` + tenantIDStr := input[:slashPos] + tenantID, err := strconv.ParseUint(tenantIDStr, 10, 64) + if err != nil { + panic(&ErrUglifyUnsupported{err}) + } + output = MakeTenantPrefix(roachpb.MakeTenantID(tenantID)) + if strings.HasPrefix(remainder, strTable) { + var indexKey roachpb.Key + remainder = remainder[len(strTable)-1:] + remainder, indexKey = tableKeyParse(remainder) + output = append(output, indexKey...) + } + return remainder, output +} + func tableKeyParse(input string) (remainder string, output roachpb.Key) { input = mustShiftSlash(input) slashPos := strings.Index(input, "/") @@ -253,13 +279,13 @@ func tableKeyParse(input string) (remainder string, output roachpb.Key) { if err != nil { panic(&ErrUglifyUnsupported{err}) } - output = roachpb.Key(MakeTablePrefix(uint32(tableID))) + output = encoding.EncodeUvarintAscending(nil /* key */, tableID) if remainder != "" { var indexKey roachpb.Key remainder, indexKey = tableIndexParse(remainder) output = append(output, indexKey...) } - return + return remainder, output } // tableIndexParse parses an index id out of the input and returns the remainder. @@ -278,7 +304,6 @@ func tableIndexParse(input string) (string, roachpb.Key) { if err != nil { panic(&ErrUglifyUnsupported{err}) } - output := encoding.EncodeUvarintAscending(nil /* key */, indexID) return remainder, output } @@ -527,10 +552,21 @@ func decodeKeyPrint(valDirs []encoding.Direction, key roachpb.Key) string { return encoding.PrettyPrintValue(valDirs, key, "/") } -func decodeTimeseriesKey(_ []encoding.Direction, key roachpb.Key) string { +func timeseriesKeyPrint(_ []encoding.Direction, key roachpb.Key) string { return PrettyPrintTimeseriesKey(key) } +func tenantKeyPrint(valDirs []encoding.Direction, key roachpb.Key) string { + key, tID, err := DecodeTenantPrefix(key) + if err != nil { + return fmt.Sprintf("/err:%v", err) + } + if len(key) == 0 { + return fmt.Sprintf("/%s", tID) + } + return fmt.Sprintf("/%s%s", tID, key.StringWithDirs(valDirs, 0)) +} + // prettyPrintInternal parse key with prefix in KeyDict. // For table keys, valDirs correspond to the encoding direction of each encoded // value in key. diff --git a/pkg/keys/printer_test.go b/pkg/keys/printer_test.go index bd1023422390..c17eed8cd722 100644 --- a/pkg/keys/printer_test.go +++ b/pkg/keys/printer_test.go @@ -29,6 +29,8 @@ import ( ) func TestPrettyPrint(t *testing.T) { + tenSysCodec := keys.SystemSQLCodec + ten5Codec := keys.MakeSQLCodec(roachpb.MakeTenantID(5)) tm, _ := time.Parse(time.RFC3339Nano, "2016-03-30T13:40:35.053725008Z") duration := duration.MakeDuration(1*time.Second.Nanoseconds(), 1, 1) durationAsc, _ := encoding.EncodeDurationAscending(nil, duration) @@ -72,10 +74,16 @@ func TestPrettyPrint(t *testing.T) { {keys.RaftLogKey(roachpb.RangeID(1000001), uint64(200001)), "/Local/RangeID/1000001/u/RaftLog/logIndex:200001", revertSupportUnknown}, {keys.RangeLastReplicaGCTimestampKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RangeLastReplicaGCTimestamp", revertSupportUnknown}, - {keys.MakeRangeKeyPrefix(roachpb.RKey(keys.MakeTablePrefix(42))), `/Local/Range/Table/42`, revertSupportUnknown}, - {keys.RangeDescriptorKey(roachpb.RKey(keys.MakeTablePrefix(42))), `/Local/Range/Table/42/RangeDescriptor`, revertSupportUnknown}, - {keys.TransactionKey(roachpb.Key(keys.MakeTablePrefix(42)), txnID), fmt.Sprintf(`/Local/Range/Table/42/Transaction/%q`, txnID), revertSupportUnknown}, - {keys.QueueLastProcessedKey(roachpb.RKey(keys.MakeTablePrefix(42)), "foo"), `/Local/Range/Table/42/QueueLastProcessed/"foo"`, revertSupportUnknown}, + {keys.MakeRangeKeyPrefix(roachpb.RKey(tenSysCodec.TablePrefix(42))), `/Local/Range/Table/42`, revertSupportUnknown}, + {keys.RangeDescriptorKey(roachpb.RKey(tenSysCodec.TablePrefix(42))), `/Local/Range/Table/42/RangeDescriptor`, revertSupportUnknown}, + {keys.TransactionKey(tenSysCodec.TablePrefix(42), txnID), fmt.Sprintf(`/Local/Range/Table/42/Transaction/%q`, txnID), revertSupportUnknown}, + {keys.QueueLastProcessedKey(roachpb.RKey(tenSysCodec.TablePrefix(42)), "foo"), `/Local/Range/Table/42/QueueLastProcessed/"foo"`, revertSupportUnknown}, + + {keys.MakeRangeKeyPrefix(roachpb.RKey(ten5Codec.TenantPrefix())), `/Local/Range/Tenant/5`, revertSupportUnknown}, + {keys.MakeRangeKeyPrefix(roachpb.RKey(ten5Codec.TablePrefix(42))), `/Local/Range/Tenant/5/Table/42`, revertSupportUnknown}, + {keys.RangeDescriptorKey(roachpb.RKey(ten5Codec.TablePrefix(42))), `/Local/Range/Tenant/5/Table/42/RangeDescriptor`, revertSupportUnknown}, + {keys.TransactionKey(ten5Codec.TablePrefix(42), txnID), fmt.Sprintf(`/Local/Range/Tenant/5/Table/42/Transaction/%q`, txnID), revertSupportUnknown}, + {keys.QueueLastProcessedKey(roachpb.RKey(ten5Codec.TablePrefix(42)), "foo"), `/Local/Range/Tenant/5/Table/42/QueueLastProcessed/"foo"`, revertSupportUnknown}, {keys.LocalMax, `/Meta1/""`, revertSupportUnknown}, // LocalMax == Meta1Prefix @@ -90,96 +98,177 @@ func TestPrettyPrint(t *testing.T) { {keys.SystemMax, "/System/Max", revertSupportUnknown}, // key of key - {keys.RangeMetaKey(roachpb.RKey(keys.MakeRangeKeyPrefix(keys.MakeTablePrefix(42)))).AsRawKey(), `/Meta2/Local/Range/Table/42`, revertSupportUnknown}, - {keys.RangeMetaKey(roachpb.RKey(makeKey(keys.MakeTablePrefix(42), roachpb.RKey("foo")))).AsRawKey(), `/Meta2/Table/42/"foo"`, revertSupportUnknown}, + {keys.RangeMetaKey(roachpb.RKey(keys.MakeRangeKeyPrefix(roachpb.RKey(tenSysCodec.TablePrefix(42))))).AsRawKey(), `/Meta2/Local/Range/Table/42`, revertSupportUnknown}, + {keys.RangeMetaKey(roachpb.RKey(makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey("foo")))).AsRawKey(), `/Meta2/Table/42/"foo"`, revertSupportUnknown}, {keys.RangeMetaKey(roachpb.RKey(makeKey(keys.Meta2Prefix, roachpb.Key("foo")))).AsRawKey(), `/Meta1/"foo"`, revertSupportUnknown}, // table {keys.SystemConfigSpan.Key, "/Table/SystemConfigSpan/Start", revertSupportUnknown}, {keys.UserTableDataMin, "/Table/50", revertMustSupport}, - {keys.MakeTablePrefix(111), "/Table/111", revertMustSupport}, - {makeKey(keys.MakeTablePrefix(42), encoding.EncodeUvarintAscending(nil, 1)), `/Table/42/1`, revertMustSupport}, - {makeKey(keys.MakeTablePrefix(42), roachpb.RKey("foo")), `/Table/42/"foo"`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {tenSysCodec.TablePrefix(111), "/Table/111", revertMustSupport}, + {makeKey(tenSysCodec.TablePrefix(42), encoding.EncodeUvarintAscending(nil, 1)), `/Table/42/1`, revertMustSupport}, + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey("foo")), `/Table/42/"foo"`, revertSupportUnknown}, + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeFloatAscending(nil, float64(233.221112)))), "/Table/42/233.221112", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeFloatDescending(nil, float64(-233.221112)))), "/Table/42/233.221112", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeFloatAscending(nil, math.Inf(1)))), "/Table/42/+Inf", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeFloatAscending(nil, math.NaN()))), "/Table/42/NaN", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeVarintAscending(nil, 1222)), roachpb.RKey(encoding.EncodeStringAscending(nil, "handsome man"))), `/Table/42/1222/"handsome man"`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeVarintAscending(nil, 1222))), `/Table/42/1222`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeVarintDescending(nil, 1222))), `/Table/42/-1223`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeBytesAscending(nil, []byte{1, 2, 8, 255}))), `/Table/42/"\x01\x02\b\xff"`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeBytesAscending(nil, []byte{1, 2, 8, 255})), roachpb.RKey("bar")), `/Table/42/"\x01\x02\b\xff"/"bar"`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeBytesDescending(nil, []byte{1, 2, 8, 255})), roachpb.RKey("bar")), `/Table/42/"\x01\x02\b\xff"/"bar"`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeNullAscending(nil))), "/Table/42/NULL", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeNullDescending(nil))), "/Table/42/NULL", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeNotNullAscending(nil))), "/Table/42/!NULL", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeNotNullDescending(nil))), "/Table/42/#", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeTimeAscending(nil, tm))), "/Table/42/2016-03-30T13:40:35.053725008Z", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeTimeDescending(nil, tm))), "/Table/42/1923-10-04T10:19:23.946274991Z", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeDecimalAscending(nil, apd.New(1234, -2)))), "/Table/42/12.34", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeDecimalDescending(nil, apd.New(1234, -2)))), "/Table/42/-12.34", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeBitArrayAscending(nil, bitArray))), "/Table/42/B00111010", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeBitArrayDescending(nil, bitArray))), "/Table/42/B00111010", revertSupportUnknown}, // Regression test for #31115. - {roachpb.Key(makeKey(keys.MakeTablePrefix(42), + {roachpb.Key(makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(encoding.EncodeBitArrayAscending(nil, bitarray.MakeZeroBitArray(64))), )).PrefixEnd(), "/Table/42/B0000000000000000000000000000000000000000000000000000000000000000/PrefixEnd", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(durationAsc)), "/Table/42/1 mon 1 day 00:00:01", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey(durationDesc)), "/Table/42/-2 mons -2 days +743:59:58.999999+999ns", revertSupportUnknown}, + // sequence + {tenSysCodec.SequenceKey(55), `/Table/55/1/0/0`, revertSupportUnknown}, + // tenant table + {ten5Codec.TenantPrefix(), "/Tenant/5", revertMustSupport}, + {ten5Codec.TablePrefix(0), "/Tenant/5/Table/SystemConfigSpan/Start", revertSupportUnknown}, + {ten5Codec.TablePrefix(keys.MinUserDescID), "/Tenant/5/Table/50", revertMustSupport}, + {ten5Codec.TablePrefix(111), "/Tenant/5/Table/111", revertMustSupport}, + {makeKey(ten5Codec.TablePrefix(42), encoding.EncodeUvarintAscending(nil, 1)), `/Tenant/5/Table/42/1`, revertMustSupport}, + {makeKey(ten5Codec.TablePrefix(42), roachpb.RKey("foo")), `/Tenant/5/Table/42/"foo"`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeFloatAscending(nil, float64(233.221112)))), + "/Tenant/5/Table/42/233.221112", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeFloatDescending(nil, float64(-233.221112)))), + "/Tenant/5/Table/42/233.221112", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeFloatAscending(nil, math.Inf(1)))), + "/Tenant/5/Table/42/+Inf", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeFloatAscending(nil, math.NaN()))), + "/Tenant/5/Table/42/NaN", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeVarintAscending(nil, 1222)), + roachpb.RKey(encoding.EncodeStringAscending(nil, "handsome man"))), + `/Tenant/5/Table/42/1222/"handsome man"`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeVarintAscending(nil, 1222))), + `/Tenant/5/Table/42/1222`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeVarintDescending(nil, 1222))), + `/Tenant/5/Table/42/-1223`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeBytesAscending(nil, []byte{1, 2, 8, 255}))), + `/Tenant/5/Table/42/"\x01\x02\b\xff"`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeBytesAscending(nil, []byte{1, 2, 8, 255})), + roachpb.RKey("bar")), `/Tenant/5/Table/42/"\x01\x02\b\xff"/"bar"`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeBytesDescending(nil, []byte{1, 2, 8, 255})), + roachpb.RKey("bar")), `/Tenant/5/Table/42/"\x01\x02\b\xff"/"bar"`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeNullAscending(nil))), "/Tenant/5/Table/42/NULL", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeNullDescending(nil))), "/Tenant/5/Table/42/NULL", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeNotNullAscending(nil))), "/Tenant/5/Table/42/!NULL", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeNotNullDescending(nil))), "/Tenant/5/Table/42/#", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeTimeAscending(nil, tm))), + "/Tenant/5/Table/42/2016-03-30T13:40:35.053725008Z", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeTimeDescending(nil, tm))), + "/Tenant/5/Table/42/1923-10-04T10:19:23.946274991Z", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeDecimalAscending(nil, apd.New(1234, -2)))), + "/Tenant/5/Table/42/12.34", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeDecimalDescending(nil, apd.New(1234, -2)))), + "/Tenant/5/Table/42/-12.34", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeBitArrayAscending(nil, bitArray))), + "/Tenant/5/Table/42/B00111010", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeBitArrayDescending(nil, bitArray))), + "/Tenant/5/Table/42/B00111010", revertSupportUnknown}, + // Regression test for #31115. + {roachpb.Key(makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(encoding.EncodeBitArrayAscending(nil, bitarray.MakeZeroBitArray(64))), + )).PrefixEnd(), + "/Tenant/5/Table/42/B0000000000000000000000000000000000000000000000000000000000000000/PrefixEnd", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(durationAsc)), + "/Tenant/5/Table/42/1 mon 1 day 00:00:01", revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), + roachpb.RKey(durationDesc)), + "/Tenant/5/Table/42/-2 mons -2 days +743:59:58.999999+999ns", revertSupportUnknown}, // sequence - {keys.MakeSequenceKey(55), `/Table/55/1/0/0`, revertSupportUnknown}, + {ten5Codec.SequenceKey(55), `/Tenant/5/Table/55/1/0/0`, revertSupportUnknown}, // others {makeKey([]byte("")), "/Min", revertSupportUnknown}, {keys.Meta1KeyMax, "/Meta1/Max", revertSupportUnknown}, {keys.Meta2KeyMax, "/Meta2/Max", revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), roachpb.RKey([]byte{0xf6})), `/Table/42/109/PrefixEnd`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), roachpb.RKey([]byte{0xf7})), `/Table/42/255/PrefixEnd`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), roachpb.RKey([]byte{0x12, 'a', 0x00, 0x02})), `/Table/42/"a"/PrefixEnd`, revertSupportUnknown}, - {makeKey(keys.MakeTablePrefix(42), roachpb.RKey([]byte{0x12, 'a', 0x00, 0x03})), `/Table/42/???`, revertSupportUnknown}, + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey([]byte{0xf6})), `/Table/42/109/PrefixEnd`, revertSupportUnknown}, + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey([]byte{0xf7})), `/Table/42/255/PrefixEnd`, revertSupportUnknown}, + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey([]byte{0x12, 'a', 0x00, 0x02})), `/Table/42/"a"/PrefixEnd`, revertSupportUnknown}, + {makeKey(tenSysCodec.TablePrefix(42), roachpb.RKey([]byte{0x12, 'a', 0x00, 0x03})), `/Table/42/???`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), roachpb.RKey([]byte{0xf6})), `/Tenant/5/Table/42/109/PrefixEnd`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), roachpb.RKey([]byte{0xf7})), `/Tenant/5/Table/42/255/PrefixEnd`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), roachpb.RKey([]byte{0x12, 'a', 0x00, 0x02})), `/Tenant/5/Table/42/"a"/PrefixEnd`, revertSupportUnknown}, + {makeKey(ten5Codec.TablePrefix(42), roachpb.RKey([]byte{0x12, 'a', 0x00, 0x03})), `/Tenant/5/Table/42/???`, revertSupportUnknown}, } for i, test := range testCases { t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { @@ -220,10 +309,13 @@ exp: %s } func TestPrettyPrintRange(t *testing.T) { + tenSysCodec := keys.SystemSQLCodec + ten5Codec := keys.MakeSQLCodec(roachpb.MakeTenantID(5)) key := makeKey([]byte("a")) key2 := makeKey([]byte("z")) - tableKey := makeKey(keys.MakeTablePrefix(61), encoding.EncodeVarintAscending(nil, 4)) - tableKey2 := makeKey(keys.MakeTablePrefix(61), encoding.EncodeVarintAscending(nil, 500)) + tableKey := makeKey(tenSysCodec.TablePrefix(61), encoding.EncodeVarintAscending(nil, 4)) + tableKey2 := makeKey(tenSysCodec.TablePrefix(61), encoding.EncodeVarintAscending(nil, 500)) + tenTableKey := makeKey(ten5Codec.TablePrefix(61), encoding.EncodeVarintAscending(nil, 999)) testCases := []struct { start, end roachpb.Key @@ -232,17 +324,27 @@ func TestPrettyPrintRange(t *testing.T) { }{ {key, nil, 20, "a"}, {tableKey, nil, 10, "/Table/61…"}, + {tenTableKey, nil, 20, "/Tenant/5/Table/61/…"}, {key, key2, 20, "{a-z}"}, {keys.MinKey, tableKey, 8, "/{M…-T…}"}, {keys.MinKey, tableKey, 15, "/{Min-Tabl…}"}, {keys.MinKey, tableKey, 20, "/{Min-Table/6…}"}, {keys.MinKey, tableKey, 25, "/{Min-Table/61/4}"}, + {keys.MinKey, tenTableKey, 8, "/{M…-T…}"}, + {keys.MinKey, tenTableKey, 15, "/{Min-Tena…}"}, + {keys.MinKey, tenTableKey, 20, "/{Min-Tenant/…}"}, + {keys.MinKey, tenTableKey, 25, "/{Min-Tenant/5/…}"}, + {keys.MinKey, tenTableKey, 30, "/{Min-Tenant/5/Tab…}"}, {tableKey, tableKey2, 8, "/Table/…"}, {tableKey, tableKey2, 15, "/Table/61/…"}, {tableKey, tableKey2, 20, "/Table/61/{4-500}"}, {tableKey, keys.MaxKey, 10, "/{Ta…-Max}"}, {tableKey, keys.MaxKey, 20, "/{Table/6…-Max}"}, {tableKey, keys.MaxKey, 25, "/{Table/61/4-Max}"}, + {tenTableKey, keys.MaxKey, 10, "/{Te…-Max}"}, + {tenTableKey, keys.MaxKey, 20, "/{Tenant/…-Max}"}, + {tenTableKey, keys.MaxKey, 25, "/{Tenant/5/…-Max}"}, + {tenTableKey, keys.MaxKey, 30, "/{Tenant/5/Tab…-Max}"}, } for i, tc := range testCases { diff --git a/pkg/keys/sql.go b/pkg/keys/sql.go new file mode 100644 index 000000000000..edfa0a0f4d44 --- /dev/null +++ b/pkg/keys/sql.go @@ -0,0 +1,202 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package keys + +import ( + "bytes" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/pkg/errors" +) + +// MakeTenantPrefix creates the key prefix associated with the specified tenant. +func MakeTenantPrefix(tenID roachpb.TenantID) roachpb.Key { + if tenID == roachpb.SystemTenantID { + return nil + } + return encoding.EncodeUvarintAscending(tenantPrefix, tenID.ToUint64()) +} + +// DecodeTenantPrefix determines the tenant ID from the key prefix, returning +// the remainder of the key (with the prefix removed) and the decoded tenant ID. +func DecodeTenantPrefix(key roachpb.Key) ([]byte, roachpb.TenantID, error) { + if len(key) == 0 { // key.Equal(roachpb.RKeyMin) + return nil, roachpb.SystemTenantID, nil + } + if key[0] != tenantPrefixByte { + return key, roachpb.SystemTenantID, nil + } + rem, tenID, err := encoding.DecodeUvarintAscending(key[1:]) + if err != nil { + return nil, roachpb.TenantID{}, err + } + return rem, roachpb.MakeTenantID(tenID), nil +} + +// SQLCodec provides methods for encoding SQL table keys bound to a given +// tenant. The generator also provides methods for efficiently decoding keys +// previously generated by it. The generated keys are safe to use indefinitely +// and the generator is safe to use concurrently. +type SQLCodec struct { + sqlEncoder + sqlDecoder +} + +// sqlEncoder implements the encoding logic for SQL keys. +// +// The type is expressed as a pointer to a slice instead of a slice directly so +// that its zero value is not usable. Any attempt to use the methods on the zero +// value of a sqlEncoder will panic. +type sqlEncoder struct { + buf *roachpb.Key +} + +// sqlEncoder implements the decoding logic for SQL keys. +// +// The type is expressed as a pointer to a slice instead of a slice directly so +// that its zero value is not usable. Any attempt to use the methods on the zero +// value of a sqlDecoder will panic. +type sqlDecoder struct { + buf *roachpb.Key +} + +// MakeSQLCodec creates a new SQLCodec suitable for manipulating SQL keys. +func MakeSQLCodec(tenID roachpb.TenantID) SQLCodec { + k := MakeTenantPrefix(tenID) + k = k[:len(k):len(k)] // bound capacity, avoid aliasing + return SQLCodec{ + sqlEncoder: sqlEncoder{&k}, + sqlDecoder: sqlDecoder{&k}, + } +} + +// SystemSQLCodec is a SQL key codec for the system tenant. +var SystemSQLCodec = MakeSQLCodec(roachpb.SystemTenantID) + +// TODOSQLCodec is a SQL key codec. It is equivalent to SystemSQLCodec, but +// should be used when it is unclear which tenant should be referenced by the +// surrounding context. +var TODOSQLCodec = MakeSQLCodec(roachpb.SystemTenantID) + +// TenantPrefix returns the key prefix used for the tenants's data. +func (e sqlEncoder) TenantPrefix() roachpb.Key { + return *e.buf +} + +// TablePrefix returns the key prefix used for the table's data. +func (e sqlEncoder) TablePrefix(tableID uint32) roachpb.Key { + k := e.TenantPrefix() + return encoding.EncodeUvarintAscending(k, uint64(tableID)) +} + +// IndexPrefix returns the key prefix used for the index's data. +func (e sqlEncoder) IndexPrefix(tableID, indexID uint32) roachpb.Key { + k := e.TablePrefix(tableID) + return encoding.EncodeUvarintAscending(k, uint64(indexID)) +} + +// DescMetadataPrefix returns the key prefix for all descriptors. +func (e sqlEncoder) DescMetadataPrefix() roachpb.Key { + return e.IndexPrefix(DescriptorTableID, DescriptorTablePrimaryKeyIndexID) +} + +// DescMetadataKey returns the key for the descriptor. +func (e sqlEncoder) DescMetadataKey(descID uint32) roachpb.Key { + k := e.DescMetadataPrefix() + k = encoding.EncodeUvarintAscending(k, uint64(descID)) + return MakeFamilyKey(k, DescriptorTableDescriptorColFamID) +} + +// SequenceKey returns the key used to store the value of a sequence. +func (e sqlEncoder) SequenceKey(tableID uint32) roachpb.Key { + k := e.IndexPrefix(tableID, SequenceIndexID) + k = encoding.EncodeUvarintAscending(k, 0) // Primary key value + k = MakeFamilyKey(k, SequenceColumnFamilyID) // Column family + return k +} + +// ZoneKeyPrefix returns the key prefix for id's row in the system.zones table. +func (e sqlEncoder) ZoneKeyPrefix(id uint32) roachpb.Key { + k := e.IndexPrefix(ZonesTableID, ZonesTablePrimaryIndexID) + return encoding.EncodeUvarintAscending(k, uint64(id)) +} + +// ZoneKey returns the key for id's entry in the system.zones table. +func (e sqlEncoder) ZoneKey(id uint32) roachpb.Key { + k := e.ZoneKeyPrefix(id) + return MakeFamilyKey(k, uint32(ZonesTableConfigColumnID)) +} + +// unexpected to avoid colliding with sqlEncoder.tenantPrefix. +func (d sqlDecoder) tenantPrefix() roachpb.Key { + return *d.buf +} + +// StripTenantPrefix validates that the given key has the proper tenant ID +// prefix, returning the remainder of the key with the prefix removed. The +// method returns an error if the key has a different tenant ID prefix than +// would be generated by the generator. +func (d sqlDecoder) StripTenantPrefix(key roachpb.Key) ([]byte, error) { + tenPrefix := d.tenantPrefix() + if !bytes.HasPrefix(key, tenPrefix) { + return nil, errors.Errorf("invalid tenant id prefix: %q", key) + } + return key[len(tenPrefix):], nil +} + +// DecodeTablePrefix validates that the given key has a table prefix, returning +// the remainder of the key (with the prefix removed) and the decoded descriptor +// ID of the table. +func (d sqlDecoder) DecodeTablePrefix(key roachpb.Key) ([]byte, uint32, error) { + key, err := d.StripTenantPrefix(key) + if err != nil { + return nil, 0, err + } + if encoding.PeekType(key) != encoding.Int { + return nil, 0, errors.Errorf("invalid key prefix: %q", key) + } + key, tableID, err := encoding.DecodeUvarintAscending(key) + return key, uint32(tableID), err +} + +// DecodeIndexPrefix validates that the given key has a table ID followed by an +// index ID, returning the remainder of the key (with the table and index prefix +// removed) and the decoded IDs of the table and index, respectively. +func (d sqlDecoder) DecodeIndexPrefix(key roachpb.Key) ([]byte, uint32, uint32, error) { + key, tableID, err := d.DecodeTablePrefix(key) + if err != nil { + return nil, 0, 0, err + } + if encoding.PeekType(key) != encoding.Int { + return nil, 0, 0, errors.Errorf("invalid key prefix: %q", key) + } + key, indexID, err := encoding.DecodeUvarintAscending(key) + return key, tableID, uint32(indexID), err +} + +// DecodeDescMetadataID decodes a descriptor ID from a descriptor metadata key. +func (d sqlDecoder) DecodeDescMetadataID(key roachpb.Key) (uint64, error) { + // Extract table and index ID from key. + remaining, tableID, _, err := d.DecodeIndexPrefix(key) + if err != nil { + return 0, err + } + if tableID != DescriptorTableID { + return 0, errors.Errorf("key is not a descriptor table entry: %v", key) + } + // Extract the descriptor ID. + _, id, err := encoding.DecodeUvarintAscending(remaining) + if err != nil { + return 0, err + } + return id, nil +} diff --git a/pkg/kv/bulk/sst_batcher_test.go b/pkg/kv/bulk/sst_batcher_test.go index bd0677890d26..d479b14c1eb9 100644 --- a/pkg/kv/bulk/sst_batcher_test.go +++ b/pkg/kv/bulk/sst_batcher_test.go @@ -39,7 +39,7 @@ import ( ) func makeIntTableKVs(numKeys, valueSize, maxRevisions int) []storage.MVCCKeyValue { - prefix := encoding.EncodeUvarintAscending(keys.MakeTablePrefix(uint32(100)), uint64(1)) + prefix := keys.SystemSQLCodec.IndexPrefix(100, 1) kvs := make([]storage.MVCCKeyValue, numKeys) r, _ := randutil.NewPseudoRand() @@ -146,7 +146,7 @@ func runTestImport(t *testing.T, batchSizeValue int64) { {{0, 3}, {4}}, } { t.Run(fmt.Sprintf("%d-%v", i, testCase), func(t *testing.T) { - prefix := encoding.EncodeUvarintAscending(keys.MakeTablePrefix(uint32(100+i)), uint64(1)) + prefix := keys.SystemSQLCodec.IndexPrefix(uint32(100+i), 1) key := func(i int) roachpb.Key { return encoding.EncodeStringAscending(append([]byte{}, prefix...), fmt.Sprintf("k%d", i)) } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index df974b8ce450..a1577950fb25 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -3394,7 +3394,7 @@ func TestConnectionClass(t *testing.T) { for _, key := range []roachpb.Key{ keys.Meta1Prefix, keys.NodeLivenessKey(1), - roachpb.Key(keys.MakeTablePrefix(1234)), // A non-system table + keys.SystemSQLCodec.TablePrefix(1234), // A non-system table } { t.Run(key.String(), func(t *testing.T) { var ba roachpb.BatchRequest diff --git a/pkg/kv/kvnemesis/generator.go b/pkg/kv/kvnemesis/generator.go index 3eae33f81eeb..8c07bfb44053 100644 --- a/pkg/kv/kvnemesis/generator.go +++ b/pkg/kv/kvnemesis/generator.go @@ -193,8 +193,8 @@ func NewDefaultConfig() GeneratorConfig { // by this Generator. func GeneratorDataSpan() roachpb.Span { return roachpb.Span{ - Key: roachpb.Key(keys.MakeTablePrefix(50)), - EndKey: roachpb.Key(keys.MakeTablePrefix(51)), + Key: keys.SystemSQLCodec.TablePrefix(50), + EndKey: keys.SystemSQLCodec.TablePrefix(51), } } diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index b810a86377d2..9dd19493252a 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -4492,7 +4492,7 @@ func TestDefaultConnectionDisruptionDoesNotInterfereWithSystemTraffic(t *testing } mtc.replicateRange(1, 1, 2) // Make a key that's in the user data space. - keyA := append(keys.MakeTablePrefix(100), 'a') + keyA := append(keys.SystemSQLCodec.TablePrefix(100), 'a') replica1 := mtc.stores[0].LookupReplica(roachpb.RKey(keyA)) mtc.replicateRange(replica1.RangeID, 1, 2) // Create a test function so that we can run the test both immediately after @@ -4847,7 +4847,7 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) { // Split off a non-system range so we don't have to account for node liveness // traffic. - scratchTableKey := keys.MakeTablePrefix(math.MaxUint32) + scratchTableKey := keys.SystemSQLCodec.TablePrefix(math.MaxUint32) // Put some data in the range so we'll have something to test for. keyA = append(append(roachpb.Key{}, scratchTableKey...), 'a') keyB = append(append(roachpb.Key{}, scratchTableKey...), 'b') diff --git a/pkg/kv/kvserver/client_replica_backpressure_test.go b/pkg/kv/kvserver/client_replica_backpressure_test.go index 12945d495fdc..c9b0dcdc124b 100644 --- a/pkg/kv/kvserver/client_replica_backpressure_test.go +++ b/pkg/kv/kvserver/client_replica_backpressure_test.go @@ -96,7 +96,7 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { var tableID int tdb.QueryRow(t, "SELECT table_id FROM crdb_internal.tables WHERE name = 'foo'").Scan(&tableID) require.NotEqual(t, 0, tableID) - tablePrefix = keys.MakeTablePrefix(uint32(tableID)) + tablePrefix = keys.SystemSQLCodec.TablePrefix(uint32(tableID)) tc.SplitRangeOrFatal(t, tablePrefix) require.NoError(t, tc.WaitForSplitAndInitialization(tablePrefix)) diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index ec92e64230f3..93826d7d9409 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -2978,7 +2978,7 @@ func TestStrictGCEnforcement(t *testing.T) { } tableID = getTableID() tenSecondsAgo hlc.Timestamp // written in setup - tableKey = roachpb.Key(keys.MakeTablePrefix(tableID)) + tableKey = keys.SystemSQLCodec.TablePrefix(tableID) tableSpan = roachpb.Span{Key: tableKey, EndKey: tableKey.PrefixEnd()} mkRecord = func() ptpb.Record { return ptpb.Record{ @@ -3110,7 +3110,7 @@ func TestStrictGCEnforcement(t *testing.T) { t.Run("system ranges are unaffected", func(t *testing.T) { setSystemGCTTL(t, 1) txn := mkStaleTxn() - descriptorTable := roachpb.Key(keys.MakeTablePrefix(keys.DescriptorTableID)) + descriptorTable := keys.SystemSQLCodec.TablePrefix(keys.DescriptorTableID) _, err := txn.Scan(ctx, descriptorTable, descriptorTable.PrefixEnd(), 1) require.NoError(t, err) }) diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 6402594bbd5b..503e28c98570 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -91,7 +91,7 @@ func TestStoreRangeSplitAtIllegalKeys(t *testing.T) { testutils.MakeKey(keys.Meta1Prefix, roachpb.RKeyMax), keys.Meta2KeyMax, testutils.MakeKey(keys.Meta2KeyMax, []byte("a")), - keys.MakeTablePrefix(10 /* system descriptor ID */), + keys.SystemSQLCodec.TablePrefix(10 /* system descriptor ID */), } { args := adminSplitArgs(key) _, pErr := kv.SendWrapped(context.Background(), store.TestSender(), args) @@ -304,7 +304,7 @@ func TestStoreRangeSplitInsideRow(t *testing.T) { // Manually create some the column keys corresponding to the table: // // CREATE TABLE t (id STRING PRIMARY KEY, col1 INT, col2 INT) - tableKey := keys.MakeTablePrefix(keys.MinUserDescID) + tableKey := roachpb.RKey(keys.SystemSQLCodec.TablePrefix(keys.MinUserDescID)) rowKey := roachpb.Key(encoding.EncodeVarintAscending(append([]byte(nil), tableKey...), 1)) rowKey = encoding.EncodeStringAscending(encoding.EncodeVarintAscending(rowKey, 1), "a") col1Key, err := keys.EnsureSafeSplitKey(keys.MakeFamilyKey(append([]byte(nil), rowKey...), 1)) @@ -703,13 +703,13 @@ func TestStoreRangeSplitStats(t *testing.T) { ctx := context.Background() // Split the range after the last table data key. - keyPrefix := keys.MakeTablePrefix(keys.MinUserDescID) + keyPrefix := keys.SystemSQLCodec.TablePrefix(keys.MinUserDescID) args := adminSplitArgs(keyPrefix) if _, pErr := kv.SendWrapped(ctx, store.TestSender(), args); pErr != nil { t.Fatal(pErr) } // Verify empty range has empty stats. - repl := store.LookupReplica(keyPrefix) + repl := store.LookupReplica(roachpb.RKey(keyPrefix)) // NOTE that this value is expected to change over time, depending on what // we store in the sys-local keyspace. Update it accordingly for this test. empty := enginepb.MVCCStats{LastUpdateNanos: manual.UnixNano()} @@ -836,7 +836,7 @@ func TestStoreEmptyRangeSnapshotSize(t *testing.T) { // Split the range after the last table data key to get a range that contains // no user data. - splitKey := keys.MakeTablePrefix(keys.MinUserDescID) + splitKey := keys.SystemSQLCodec.TablePrefix(keys.MinUserDescID) splitArgs := adminSplitArgs(splitKey) if _, err := kv.SendWrapped(ctx, mtc.distSenders[0], splitArgs); err != nil { t.Fatal(err) @@ -862,7 +862,7 @@ func TestStoreEmptyRangeSnapshotSize(t *testing.T) { // Replicate the newly-split range to trigger a snapshot request from store 0 // to store 1. - rangeID := mtc.stores[0].LookupReplica(splitKey).RangeID + rangeID := mtc.stores[0].LookupReplica(roachpb.RKey(splitKey)).RangeID mtc.replicateRange(rangeID, 1) // Verify that we saw at least one snapshot request, @@ -903,13 +903,13 @@ func TestStoreRangeSplitStatsWithMerges(t *testing.T) { ctx := context.Background() // Split the range after the last table data key. - keyPrefix := keys.MakeTablePrefix(keys.MinUserDescID) + keyPrefix := keys.SystemSQLCodec.TablePrefix(keys.MinUserDescID) args := adminSplitArgs(keyPrefix) if _, pErr := kv.SendWrapped(ctx, store.TestSender(), args); pErr != nil { t.Fatal(pErr) } // Verify empty range has empty stats. - repl := store.LookupReplica(keyPrefix) + repl := store.LookupReplica(roachpb.RKey(keyPrefix)) // NOTE that this value is expected to change over time, depending on what // we store in the sys-local keyspace. Update it accordingly for this test. empty := enginepb.MVCCStats{LastUpdateNanos: manual.UnixNano()} @@ -1025,7 +1025,7 @@ func TestStoreZoneUpdateAndRangeSplit(t *testing.T) { t.Fatal(err) } - tableBoundary := keys.MakeTablePrefix(descID) + tableBoundary := keys.SystemSQLCodec.TablePrefix(descID) { var repl *kvserver.Replica @@ -1033,7 +1033,7 @@ func TestStoreZoneUpdateAndRangeSplit(t *testing.T) { // Wait for the range to be split along table boundaries. expectedRSpan := roachpb.RSpan{Key: roachpb.RKey(tableBoundary), EndKey: roachpb.RKeyMax} testutils.SucceedsSoon(t, func() error { - repl = store.LookupReplica(tableBoundary) + repl = store.LookupReplica(roachpb.RKey(tableBoundary)) if actualRSpan := repl.Desc().RSpan(); !actualRSpan.Equal(expectedRSpan) { return errors.Errorf("expected range %s to span %s", repl, expectedRSpan) } @@ -1051,7 +1051,7 @@ func TestStoreZoneUpdateAndRangeSplit(t *testing.T) { // Verify that the range is in fact split. testutils.SucceedsSoon(t, func() error { - repl := store.LookupReplica(keys.MakeTablePrefix(descID + 1)) + repl := store.LookupReplica(roachpb.RKey(keys.SystemSQLCodec.TablePrefix(descID + 1))) rngDesc := repl.Desc() rngStart, rngEnd := rngDesc.StartKey, rngDesc.EndKey if rngStart.Equal(tableBoundary) || !rngEnd.Equal(roachpb.RKeyMax) { @@ -1089,7 +1089,7 @@ func TestStoreRangeSplitWithMaxBytesUpdate(t *testing.T) { // Verify that the range is split and the new range has the correct max bytes. testutils.SucceedsSoon(t, func() error { - newRng := store.LookupReplica(keys.MakeTablePrefix(descID)) + newRng := store.LookupReplica(roachpb.RKey(keys.SystemSQLCodec.TablePrefix(descID))) if newRng.RangeID == origRng.RangeID { return errors.Errorf("expected new range created by split") } @@ -1289,7 +1289,7 @@ func TestStoreRangeSystemSplits(t *testing.T) { if err := txn.SetSystemConfigTrigger(); err != nil { return err } - descTablePrefix := keys.MakeTablePrefix(keys.DescriptorTableID) + descTablePrefix := keys.SystemSQLCodec.TablePrefix(keys.DescriptorTableID) kvs, _ /* splits */ := schema.GetInitialValues(clusterversion.TestingClusterVersion) for _, kv := range kvs { if !bytes.HasPrefix(kv.Key, descTablePrefix) { @@ -1326,13 +1326,13 @@ func TestStoreRangeSystemSplits(t *testing.T) { maxID := uint32(ids[len(ids)-1]) for i := uint32(keys.MaxSystemConfigDescID + 1); i <= maxID; i++ { expKeys = append(expKeys, - testutils.MakeKey(keys.Meta2Prefix, keys.MakeTablePrefix(i)), + testutils.MakeKey(keys.Meta2Prefix, keys.SystemSQLCodec.TablePrefix(i)), ) } for i := keys.MinUserDescID; i <= userTableMax; i++ { if _, ok := exceptions[i]; !ok { expKeys = append(expKeys, - testutils.MakeKey(keys.Meta2Prefix, keys.MakeTablePrefix(uint32(i))), + testutils.MakeKey(keys.Meta2Prefix, keys.SystemSQLCodec.TablePrefix(uint32(i))), ) } } @@ -2416,7 +2416,7 @@ func TestUnsplittableRange(t *testing.T) { store := createTestStoreWithConfig(t, stopper, cfg) // Add a single large row to /Table/14. - tableKey := keys.MakeTablePrefix(keys.UITableID) + tableKey := roachpb.RKey(keys.SystemSQLCodec.TablePrefix(keys.UITableID)) row1Key := roachpb.Key(encoding.EncodeVarintAscending(append([]byte(nil), tableKey...), 1)) col1Key := keys.MakeFamilyKey(append([]byte(nil), row1Key...), 0) valueLen := 0.9 * maxBytes @@ -2722,12 +2722,12 @@ func TestRangeLookupAfterMeta2Split(t *testing.T) { // the user range [/Table/48-/Max) is stored on the right meta2 range, so the lookup // will require a scan that continues into the next meta2 range. const tableID = keys.MinUserDescID + 1 // 51 - splitReq := adminSplitArgs(keys.MakeTablePrefix(tableID - 3 /* 48 */)) + splitReq := adminSplitArgs(keys.SystemSQLCodec.TablePrefix(tableID - 3 /* 48 */)) if _, pErr := kv.SendWrapped(ctx, s.DB().NonTransactionalSender(), splitReq); pErr != nil { t.Fatal(pErr) } - metaKey := keys.RangeMetaKey(keys.MakeTablePrefix(tableID)).AsRawKey() + metaKey := keys.RangeMetaKey(roachpb.RKey(keys.SystemSQLCodec.TablePrefix(tableID))).AsRawKey() splitReq = adminSplitArgs(metaKey) if _, pErr := kv.SendWrapped(ctx, s.DB().NonTransactionalSender(), splitReq); pErr != nil { t.Fatal(pErr) @@ -2741,8 +2741,8 @@ func TestRangeLookupAfterMeta2Split(t *testing.T) { // Scan from [/Table/49-/Table/50) both forwards and backwards. // Either way, the resulting RangeLookup scan will be forced to // perform a continuation lookup. - scanStart := roachpb.Key(keys.MakeTablePrefix(tableID - 2)) // 49 - scanEnd := scanStart.PrefixEnd() // 50 + scanStart := keys.SystemSQLCodec.TablePrefix(tableID - 2) // 49 + scanEnd := scanStart.PrefixEnd() // 50 header := roachpb.RequestHeader{ Key: scanStart, EndKey: scanEnd, diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index d30967866698..c8831630a28e 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -289,8 +290,8 @@ func TestClosedTimestampCanServeAfterSplitAndMerges(t *testing.T) { t.Fatalf("failed to lookup ids: %+v", err) } // Split the table at key 2. - k, err := sqlbase.EncodeTableKey(sqlbase.EncodeTableIDIndexID(nil, tableID, 1), - tree.NewDInt(2), encoding.Ascending) + idxPrefix := keys.SystemSQLCodec.IndexPrefix(uint32(tableID), 1) + k, err := sqlbase.EncodeTableKey(idxPrefix, tree.NewDInt(2), encoding.Ascending) if err != nil { t.Fatalf("failed to encode key: %+v", err) } diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index c791fd37c434..0ab6d3d810a7 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -180,10 +180,10 @@ func (ds uniformDistSpec) dist(maxRows int, rng *rand.Rand) dataDistribution { } func (ds uniformDistSpec) desc() *roachpb.RangeDescriptor { - tablePrefix := keys.MakeTablePrefix(42) + tablePrefix := keys.SystemSQLCodec.TablePrefix(42) return &roachpb.RangeDescriptor{ - StartKey: tablePrefix, - EndKey: roachpb.RKey(roachpb.Key(tablePrefix).PrefixEnd()), + StartKey: roachpb.RKey(tablePrefix), + EndKey: roachpb.RKey(tablePrefix.PrefixEnd()), } } diff --git a/pkg/kv/kvserver/gc/gc_iterator_test.go b/pkg/kv/kvserver/gc/gc_iterator_test.go index ebf5f758d94c..fa1fd7cfb892 100644 --- a/pkg/kv/kvserver/gc/gc_iterator_test.go +++ b/pkg/kv/kvserver/gc/gc_iterator_test.go @@ -81,8 +81,8 @@ func TestGCIterator(t *testing.T) { } } vals := uniformValueDistribution(3, 5, 0, rand.New(rand.NewSource(1))) - tablePrefix := keys.MakeTablePrefix(42) - desc := roachpb.RangeDescriptor{StartKey: tablePrefix, EndKey: roachpb.RKey(roachpb.Key(tablePrefix).PrefixEnd())} + tablePrefix := keys.SystemSQLCodec.TablePrefix(42) + desc := roachpb.RangeDescriptor{StartKey: roachpb.RKey(tablePrefix), EndKey: roachpb.RKey(tablePrefix.PrefixEnd())} keyA := append(tablePrefix[0:len(tablePrefix):len(tablePrefix)], 'a') keyB := append(tablePrefix[0:len(tablePrefix):len(tablePrefix)], 'b') keyC := append(tablePrefix[0:len(tablePrefix):len(tablePrefix)], 'c') diff --git a/pkg/kv/kvserver/gossip_test.go b/pkg/kv/kvserver/gossip_test.go index 7368f3d42a2d..46bc52b38538 100644 --- a/pkg/kv/kvserver/gossip_test.go +++ b/pkg/kv/kvserver/gossip_test.go @@ -211,10 +211,10 @@ func TestGossipAfterAbortOfSystemConfigTransactionAfterFailureDueToIntents(t *te txB := db.NewTxn(ctx, "b") require.NoError(t, txA.SetSystemConfigTrigger()) - require.NoError(t, txA.Put(ctx, keys.DescMetadataKey(1000), "foo")) + require.NoError(t, txA.Put(ctx, keys.SystemSQLCodec.DescMetadataKey(1000), "foo")) require.NoError(t, txB.SetSystemConfigTrigger()) - require.NoError(t, txB.Put(ctx, keys.DescMetadataKey(2000), "bar")) + require.NoError(t, txB.Put(ctx, keys.SystemSQLCodec.DescMetadataKey(2000), "bar")) const someTime = 10 * time.Millisecond clearNotifictions := func(ch <-chan struct{}) { diff --git a/pkg/kv/kvserver/merge_queue_test.go b/pkg/kv/kvserver/merge_queue_test.go index fa7d7165b95a..bfcd078af319 100644 --- a/pkg/kv/kvserver/merge_queue_test.go +++ b/pkg/kv/kvserver/merge_queue_test.go @@ -39,7 +39,7 @@ func TestMergeQueueShouldQueue(t *testing.T) { storagebase.MergeQueueEnabled.Override(&testCtx.store.ClusterSettings().SV, true) tableKey := func(i uint32) []byte { - return keys.MakeTablePrefix(keys.MaxReservedDescID + i) + return keys.SystemSQLCodec.TablePrefix(keys.MaxReservedDescID + i) } config.TestingSetZoneConfig(keys.MaxReservedDescID+1, *zonepb.NewZoneConfig()) diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go index 80ec4feb058f..f126b0892f4f 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go @@ -420,8 +420,8 @@ func waitForAsOfAfter(t *testing.T, c protectedts.Cache, ts hlc.Timestamp) (asOf func tableSpan(tableID uint32) roachpb.Span { return roachpb.Span{ - Key: roachpb.Key(keys.MakeTablePrefix(tableID)), - EndKey: roachpb.Key(keys.MakeTablePrefix(tableID)).PrefixEnd(), + Key: keys.SystemSQLCodec.TablePrefix(tableID), + EndKey: keys.SystemSQLCodec.TablePrefix(tableID).PrefixEnd(), } } diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go index 7aece29babe4..31e866ca0178 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go @@ -384,8 +384,8 @@ func pickOneRecord(tCtx *testContext) uuid.UUID { func tableSpan(tableID uint32) roachpb.Span { return roachpb.Span{ - Key: keys.MakeTablePrefix(tableID), - EndKey: roachpb.Key(keys.MakeTablePrefix(tableID)).PrefixEnd(), + Key: keys.SystemSQLCodec.TablePrefix(tableID), + EndKey: keys.SystemSQLCodec.TablePrefix(tableID).PrefixEnd(), } } diff --git a/pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go b/pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go index f3a785aba3a2..79b8cbdef91f 100644 --- a/pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go +++ b/pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go @@ -68,7 +68,7 @@ func TestVerifier(t *testing.T) { db := kv.NewDB(s.DB().AmbientContext, tsf, s.Clock()) ptv := ptverifier.New(db, pts) makeTableSpan := func(tableID uint32) roachpb.Span { - k := roachpb.Key(keys.MakeTablePrefix(tableID)) + k := keys.SystemSQLCodec.TablePrefix(tableID) return roachpb.Span{Key: k, EndKey: k.PrefixEnd()} } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 28c7d9231dde..5e89ea4baf07 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1208,7 +1208,7 @@ func TestReplicaGossipConfigsOnLease(t *testing.T) { } // Write some arbitrary data in the system config span. - key := keys.MakeTablePrefix(keys.MaxSystemConfigDescID) + key := keys.SystemSQLCodec.TablePrefix(keys.MaxSystemConfigDescID) var val roachpb.Value val.SetInt(42) if err := storage.MVCCPut(context.Background(), tc.engine, nil, key, hlc.Timestamp{}, val, nil); err != nil { @@ -1480,7 +1480,7 @@ func TestReplicaNoGossipConfig(t *testing.T) { tc.Start(t, stopper) // Write some arbitrary data in the system span (up to, but not including MaxReservedID+1) - key := keys.MakeTablePrefix(keys.MaxReservedDescID) + key := keys.SystemSQLCodec.TablePrefix(keys.MaxReservedDescID) txn := newTransaction("test", key, 1 /* userPriority */, tc.Clock()) h := roachpb.Header{Txn: txn} @@ -1525,7 +1525,7 @@ func TestReplicaNoGossipFromNonLeader(t *testing.T) { tc.Start(t, stopper) // Write some arbitrary data in the system span (up to, but not including MaxReservedID+1) - key := keys.MakeTablePrefix(keys.MaxReservedDescID) + key := keys.SystemSQLCodec.TablePrefix(keys.MaxReservedDescID) txn := newTransaction("test", key, 1 /* userPriority */, tc.Clock()) req1 := putArgs(key, nil) @@ -12245,7 +12245,7 @@ func TestLaterReproposalsDoNotReuseContext(t *testing.T) { func TestReplicaTelemetryCounterForPushesDueToClosedTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - keyA := append(keys.MakeTablePrefix(math.MaxUint32), 'a') + keyA := append(keys.SystemSQLCodec.TablePrefix(math.MaxUint32), 'a') keyAA := append(keyA[:len(keyA):len(keyA)], 'a') rKeyA, err := keys.Addr(keyA) putReq := func(key roachpb.Key) *roachpb.PutRequest { diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index c7788cd54e4f..eedd5900f650 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -83,7 +83,7 @@ func testReplicateQueueRebalanceInner(t *testing.T, atomic bool) { trackedRanges := map[roachpb.RangeID]struct{}{} for i := 0; i < newRanges; i++ { tableID := keys.MinUserDescID + i - splitKey := keys.MakeTablePrefix(uint32(tableID)) + splitKey := keys.SystemSQLCodec.TablePrefix(uint32(tableID)) // Retry the splits on descriptor errors which are likely as the replicate // queue is already hard at work. testutils.SucceedsSoon(t, func() error { diff --git a/pkg/kv/kvserver/split/decider_test.go b/pkg/kv/kvserver/split/decider_test.go index f770d6faf363..494d8abc6755 100644 --- a/pkg/kv/kvserver/split/decider_test.go +++ b/pkg/kv/kvserver/split/decider_test.go @@ -182,7 +182,7 @@ func TestDeciderCallsEnsureSafeSplitKey(t *testing.T) { var d Decider Init(&d, intn, func() float64 { return 1.0 }) - baseKey := keys.MakeTablePrefix(51) + baseKey := keys.SystemSQLCodec.TablePrefix(51) for i := 0; i < 4; i++ { baseKey = encoding.EncodeUvarintAscending(baseKey, uint64(52+i)) } @@ -215,7 +215,7 @@ func TestDeciderIgnoresEnsureSafeSplitKeyOnError(t *testing.T) { var d Decider Init(&d, intn, func() float64 { return 1.0 }) - baseKey := keys.MakeTablePrefix(51) + baseKey := keys.SystemSQLCodec.TablePrefix(51) for i := 0; i < 4; i++ { baseKey = encoding.EncodeUvarintAscending(baseKey, uint64(52+i)) } diff --git a/pkg/kv/kvserver/split/finder_test.go b/pkg/kv/kvserver/split/finder_test.go index 02c5aba5d72d..98fb9a260167 100644 --- a/pkg/kv/kvserver/split/finder_test.go +++ b/pkg/kv/kvserver/split/finder_test.go @@ -39,7 +39,7 @@ func TestSplitFinderKey(t *testing.T) { noLoadReservoir := [splitKeySampleSize]sample{} for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: 0, right: 0, contained: 0, @@ -51,7 +51,7 @@ func TestSplitFinderKey(t *testing.T) { uniformReservoir := [splitKeySampleSize]sample{} for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: splitKeyMinCounter, right: splitKeyMinCounter, contained: 0, @@ -63,7 +63,7 @@ func TestSplitFinderKey(t *testing.T) { nonUniformReservoir := [splitKeySampleSize]sample{} for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: splitKeyMinCounter * i, right: splitKeyMinCounter * (splitKeySampleSize - i), contained: 0, @@ -75,7 +75,7 @@ func TestSplitFinderKey(t *testing.T) { singleHotKeyReservoir := [splitKeySampleSize]sample{} for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: 0, right: splitKeyMinCounter, contained: 0, @@ -87,7 +87,7 @@ func TestSplitFinderKey(t *testing.T) { multipleHotKeysReservoir := [splitKeySampleSize]sample{} for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: splitKeyMinCounter, right: splitKeyMinCounter, contained: 0, @@ -100,7 +100,7 @@ func TestSplitFinderKey(t *testing.T) { spanningReservoir := [splitKeySampleSize]sample{} for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: 0, right: 0, contained: splitKeyMinCounter, @@ -112,7 +112,7 @@ func TestSplitFinderKey(t *testing.T) { multipleSpanReservoir := [splitKeySampleSize]sample{} for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: splitKeyMinCounter, right: splitKeyMinCounter, contained: splitKeyMinCounter, @@ -120,7 +120,7 @@ func TestSplitFinderKey(t *testing.T) { multipleSpanReservoir[i] = tempSample } midSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + splitKeySampleSize/2)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + splitKeySampleSize/2)), left: splitKeyMinCounter, right: splitKeyMinCounter, contained: 0, @@ -136,17 +136,17 @@ func TestSplitFinderKey(t *testing.T) { // Test reservoir with no load should have no splits. {noLoadReservoir, nil}, // Test a uniform reservoir (Splits at the first key) - {uniformReservoir, keys.MakeTablePrefix(ReservoirKeyOffset)}, + {uniformReservoir, keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset)}, // Testing a non-uniform reservoir. - {nonUniformReservoir, keys.MakeTablePrefix(ReservoirKeyOffset + splitKeySampleSize/2)}, + {nonUniformReservoir, keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset + splitKeySampleSize/2)}, // Test a load heavy reservoir on a single hot key. Splitting can't help here. {singleHotKeyReservoir, nil}, // Test a load heavy reservoir on multiple hot keys. Splits between the hot keys. - {multipleHotKeysReservoir, keys.MakeTablePrefix(ReservoirKeyOffset + 1)}, + {multipleHotKeysReservoir, keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset + 1)}, // Test a spanning reservoir. Splitting will be bad here. Should avoid it. {spanningReservoir, nil}, // Test that splits happen between two heavy spans. - {multipleSpanReservoir, keys.MakeTablePrefix(ReservoirKeyOffset + splitKeySampleSize/2)}, + {multipleSpanReservoir, keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset + splitKeySampleSize/2)}, } for i, test := range testCases { @@ -184,8 +184,8 @@ func TestSplitFinderRecorder(t *testing.T) { // Test recording a key query before the reservoir is full. basicReservoir := [splitKeySampleSize]sample{} basicSpan := roachpb.Span{ - Key: keys.MakeTablePrefix(ReservoirKeyOffset), - EndKey: keys.MakeTablePrefix(ReservoirKeyOffset + 1), + Key: keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset), + EndKey: keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset + 1), } expectedBasicReservoir := [splitKeySampleSize]sample{} expectedBasicReservoir[0] = sample{ @@ -196,7 +196,7 @@ func TestSplitFinderRecorder(t *testing.T) { replacementReservoir := [splitKeySampleSize]sample{} for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: 0, right: 0, contained: 0, @@ -204,8 +204,8 @@ func TestSplitFinderRecorder(t *testing.T) { replacementReservoir[i] = tempSample } replacementSpan := roachpb.Span{ - Key: keys.MakeTablePrefix(ReservoirKeyOffset + splitKeySampleSize), - EndKey: keys.MakeTablePrefix(ReservoirKeyOffset + splitKeySampleSize + 1), + Key: keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset + splitKeySampleSize), + EndKey: keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset + splitKeySampleSize + 1), } expectedReplacementReservoir := replacementReservoir expectedReplacementReservoir[0] = sample{ @@ -215,13 +215,13 @@ func TestSplitFinderRecorder(t *testing.T) { // Test recording a key query after the reservoir is full without replacement. fullReservoir := replacementReservoir fullSpan := roachpb.Span{ - Key: keys.MakeTablePrefix(ReservoirKeyOffset), - EndKey: keys.MakeTablePrefix(ReservoirKeyOffset + 1), + Key: keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset), + EndKey: keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset + 1), } expectedFullReservoir := fullReservoir for i := 0; i < splitKeySampleSize; i++ { tempSample := sample{ - key: keys.MakeTablePrefix(uint32(ReservoirKeyOffset + i)), + key: keys.SystemSQLCodec.TablePrefix(uint32(ReservoirKeyOffset + i)), left: 1, right: 0, contained: 0, @@ -234,8 +234,8 @@ func TestSplitFinderRecorder(t *testing.T) { // Test recording a spanning query. spanningReservoir := replacementReservoir spanningSpan := roachpb.Span{ - Key: keys.MakeTablePrefix(ReservoirKeyOffset - 1), - EndKey: keys.MakeTablePrefix(ReservoirKeyOffset + splitKeySampleSize + 1), + Key: keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset - 1), + EndKey: keys.SystemSQLCodec.TablePrefix(ReservoirKeyOffset + splitKeySampleSize + 1), } expectedSpanningReservoir := spanningReservoir for i := 0; i < splitKeySampleSize; i++ { diff --git a/pkg/kv/kvserver/split_queue_test.go b/pkg/kv/kvserver/split_queue_test.go index 00fe068b30ba..2ac8f305bb1b 100644 --- a/pkg/kv/kvserver/split_queue_test.go +++ b/pkg/kv/kvserver/split_queue_test.go @@ -48,9 +48,9 @@ func TestSplitQueueShouldQueue(t *testing.T) { // No intersection, no bytes, no load. {roachpb.RKeyMin, roachpb.RKey(keys.MetaMax), 0, 64 << 20, false, 0}, // Intersection in zone, no bytes, no load. - {keys.MakeTablePrefix(2001), roachpb.RKeyMax, 0, 64 << 20, true, 1}, + {roachpb.RKey(keys.SystemSQLCodec.TablePrefix(2001)), roachpb.RKeyMax, 0, 64 << 20, true, 1}, // Already split at largest ID, no load. - {keys.MakeTablePrefix(2002), roachpb.RKeyMax, 0, 32 << 20, false, 0}, + {roachpb.RKey(keys.SystemSQLCodec.TablePrefix(2002)), roachpb.RKeyMax, 0, 32 << 20, false, 0}, // Multiple intersections, no bytes, no load. {roachpb.RKeyMin, roachpb.RKeyMax, 0, 64 << 20, true, 1}, // No intersection, max bytes, no load. @@ -62,9 +62,9 @@ func TestSplitQueueShouldQueue(t *testing.T) { // No intersection, max bytes * 4, no load, should not backpressure. {roachpb.RKeyMin, roachpb.RKey(keys.MetaMax), 64 << 22, 64 << 20, true, 4}, // Intersection, max bytes +1, no load. - {keys.MakeTablePrefix(2000), roachpb.RKeyMax, 32<<20 + 1, 32 << 20, true, 2}, + {roachpb.RKey(keys.SystemSQLCodec.TablePrefix(2000)), roachpb.RKeyMax, 32<<20 + 1, 32 << 20, true, 2}, // Split needed at table boundary, but no zone config, no load. - {keys.MakeTablePrefix(2001), roachpb.RKeyMax, 32<<20 + 1, 64 << 20, true, 1}, + {roachpb.RKey(keys.SystemSQLCodec.TablePrefix(2001)), roachpb.RKeyMax, 32<<20 + 1, 64 << 20, true, 1}, } cfg := tc.gossip.GetSystemConfig() diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index cdd3480527f4..1a5b9f86e456 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -1464,11 +1464,14 @@ func TestStoreSetRangesMaxBytes(t *testing.T) { }{ {store.LookupReplica(roachpb.RKeyMin), *store.cfg.DefaultZoneConfig.RangeMaxBytes}, - {splitTestRange(store, roachpb.RKeyMin, keys.MakeTablePrefix(baseID), t), + {splitTestRange( + store, roachpb.RKeyMin, roachpb.RKey(keys.SystemSQLCodec.TablePrefix(baseID)), t), 1 << 20}, - {splitTestRange(store, keys.MakeTablePrefix(baseID), keys.MakeTablePrefix(baseID+1), t), + {splitTestRange( + store, roachpb.RKey(keys.SystemSQLCodec.TablePrefix(baseID)), roachpb.RKey(keys.SystemSQLCodec.TablePrefix(baseID+1)), t), *store.cfg.DefaultZoneConfig.RangeMaxBytes}, - {splitTestRange(store, keys.MakeTablePrefix(baseID+1), keys.MakeTablePrefix(baseID+2), t), + {splitTestRange( + store, roachpb.RKey(keys.SystemSQLCodec.TablePrefix(baseID+1)), roachpb.RKey(keys.SystemSQLCodec.TablePrefix(baseID+2)), t), 2 << 20}, } @@ -3204,7 +3207,7 @@ func TestPreemptiveSnapshotsAreRemoved(t *testing.T) { defer stopper.Stop(ctx) config := TestStoreConfig(hlc.NewClock(hlc.UnixNano, base.DefaultMaxClockOffset)) s := createTestStoreWithoutStart(t, stopper, testStoreOpts{}, &config) - tablePrefix := roachpb.Key(keys.MakeTablePrefix(42)) + tablePrefix := keys.SystemSQLCodec.TablePrefix(42) tablePrefixEnd := tablePrefix.PrefixEnd() const rangeID = 42 diff --git a/pkg/roachpb/tenant.go b/pkg/roachpb/tenant.go new file mode 100644 index 000000000000..852abd45c1e7 --- /dev/null +++ b/pkg/roachpb/tenant.go @@ -0,0 +1,72 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package roachpb + +import ( + "math" + "strconv" +) + +// A TenantID is a unique ID associated with a tenant in a multi-tenant cluster. +// Each tenant is granted exclusive access to a portion of the keyspace and a +// collection of SQL tables in that keyspace which comprise a "logical" cluster. +// +// The type is intentionally opaque to require deliberate use. +type TenantID struct{ id uint64 } + +// SystemTenantID is the ID associated with the system's internal tenant in a +// multi-tenant cluster and the only tenant in a single-tenant cluster. +// +// The system tenant differs from all other tenants in four important ways: +// 1. the system tenant's keyspace is not prefixed with a tenant specifier. +// 2. the system tenant is created by default during cluster initialization. +// 3. the system tenant is always present and can never be destroyed. +// 4. the system tenant has the ability to create and destroy other tenants. +var SystemTenantID = MakeTenantID(1) + +// MinTenantID is the minimum ID of a (non-system) tenant in a multi-tenant +// cluster. +var MinTenantID = MakeTenantID(2) + +// MaxTenantID is the maximum ID of a (non-system) tenant in a multi-tenant +// cluster. +var MaxTenantID = MakeTenantID(math.MaxUint64) + +// MakeTenantID constructs a new TenantID from the provided uint64. +func MakeTenantID(id uint64) TenantID { + checkValid(id) + return TenantID{id} +} + +// ToUint64 returns the TenantID as a uint64. +func (t TenantID) ToUint64() uint64 { + checkValid(t.id) + return t.id +} + +// String implements the fmt.Stringer interface. +func (t TenantID) String() string { + switch t { + case TenantID{}: + return "invalid" + case SystemTenantID: + return "system" + default: + return strconv.FormatUint(t.id, 10) + } +} + +// Protects against zero value. +func checkValid(id uint64) { + if id == 0 { + panic("invalid tenant ID 0") + } +} diff --git a/pkg/roachpb/tenant_test.go b/pkg/roachpb/tenant_test.go new file mode 100644 index 000000000000..48723f14e791 --- /dev/null +++ b/pkg/roachpb/tenant_test.go @@ -0,0 +1,30 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package roachpb + +import ( + "math" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestTenantIDString(t *testing.T) { + for tID, expStr := range map[TenantID]string{ + {}: "invalid", + SystemTenantID: "system", + MakeTenantID(2): "2", + MakeTenantID(999): "999", + MakeTenantID(math.MaxUint64): "18446744073709551615", + } { + require.Equal(t, expStr, tID.String()) + } +} diff --git a/pkg/server/admin.go b/pkg/server/admin.go index d1f15ccc5fd4..c7e377df47af 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -627,8 +627,7 @@ func (s *adminServer) TableDetails( // NOTE: this doesn't make sense for interleaved (children) table. As of // 03/2018, callers around here use it anyway. func generateTableSpan(tableID sqlbase.ID) roachpb.Span { - tablePrefix := keys.MakeTablePrefix(uint32(tableID)) - tableStartKey := roachpb.Key(tablePrefix) + tableStartKey := keys.TODOSQLCodec.TablePrefix(uint32(tableID)) tableEndKey := tableStartKey.PrefixEnd() return roachpb.Span{Key: tableStartKey, EndKey: tableEndKey} } @@ -1764,11 +1763,11 @@ func (s *adminServer) DataDistribution( } // Used later when we're scanning Meta2 and only have IDs, not names. - tableInfosByTableID := map[uint64]serverpb.DataDistributionResponse_TableInfo{} + tableInfosByTableID := map[uint32]serverpb.DataDistributionResponse_TableInfo{} for _, row := range rows1 { tableName := (*string)(row[0].(*tree.DString)) - tableID := uint64(tree.MustBeDInt(row[1])) + tableID := uint32(tree.MustBeDInt(row[1])) dbName := (*string)(row[2].(*tree.DString)) // Look at whether it was dropped. @@ -1847,7 +1846,7 @@ func (s *adminServer) DataDistribution( return err } - _, tableID, err := keys.DecodeTablePrefix(rangeDesc.StartKey.AsRawKey()) + _, tableID, err := keys.TODOSQLCodec.DecodeTablePrefix(rangeDesc.StartKey.AsRawKey()) if err != nil { return err } diff --git a/pkg/server/settingsworker.go b/pkg/server/settingsworker.go index 14742aaa4592..f3e697c8e697 100644 --- a/pkg/server/settingsworker.go +++ b/pkg/server/settingsworker.go @@ -31,7 +31,7 @@ func (s *Server) refreshSettings() { tbl := &sqlbase.SettingsTable a := &sqlbase.DatumAlloc{} - settingsTablePrefix := keys.MakeTablePrefix(uint32(tbl.ID)) + settingsTablePrefix := keys.TODOSQLCodec.TablePrefix(uint32(tbl.ID)) colIdxMap := row.ColIDtoRowIndexFromCols(tbl.Columns) processKV := func(ctx context.Context, kv roachpb.KeyValue, u settings.Updater) error { diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 7d6fc44bec2b..8f5220b82d8d 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -1062,7 +1062,7 @@ func (ts *TestServer) ForceTableGC( return errors.AssertionFailedf("expected 1 column from internal query") } tableID := uint32(*row[0].(*tree.DInt)) - tblKey := roachpb.Key(keys.MakeTablePrefix(tableID)) + tblKey := keys.SystemSQLCodec.TablePrefix(tableID) gcr := roachpb.GCRequest{ RequestHeader: roachpb.RequestHeader{ Key: tblKey, diff --git a/pkg/server/updates.go b/pkg/server/updates.go index 99890a9c1d30..93b38dcefd7b 100644 --- a/pkg/server/updates.go +++ b/pkg/server/updates.go @@ -462,7 +462,7 @@ func (s *Server) ReportDiagnostics(ctx context.Context) { } func (s *Server) collectSchemaInfo(ctx context.Context) ([]sqlbase.TableDescriptor, error) { - startKey := roachpb.Key(keys.MakeTablePrefix(keys.DescriptorTableID)) + startKey := keys.TODOSQLCodec.TablePrefix(keys.DescriptorTableID) endKey := startKey.PrefixEnd() kvs, err := s.db.Scan(ctx, startKey, endKey, 0) if err != nil { diff --git a/pkg/sql/ambiguous_commit_test.go b/pkg/sql/ambiguous_commit_test.go index 2c989879abc0..1df384f122a1 100644 --- a/pkg/sql/ambiguous_commit_test.go +++ b/pkg/sql/ambiguous_commit_test.go @@ -67,7 +67,7 @@ func TestAmbiguousCommit(t *testing.T) { translateToRPCError := roachpb.NewError(errors.Errorf("%s: RPC error: success=%t", t.Name(), ambiguousSuccess)) maybeRPCError := func(req *roachpb.ConditionalPutRequest) *roachpb.Error { - tsk, ok := tableStartKey.Load().([]byte) + tsk, ok := tableStartKey.Load().(roachpb.Key) if !ok { return nil } @@ -156,10 +156,10 @@ func TestAmbiguousCommit(t *testing.T) { } tableID := sqlutils.QueryTableID(t, sqlDB, "test", "public", "t") - tableStartKey.Store(keys.MakeTablePrefix(tableID)) + tableStartKey.Store(keys.SystemSQLCodec.TablePrefix(tableID)) // Wait for new table to split & replication. - if err := tc.WaitForSplitAndInitialization(tableStartKey.Load().([]byte)); err != nil { + if err := tc.WaitForSplitAndInitialization(tableStartKey.Load().(roachpb.Key)); err != nil { t.Fatal(err) } diff --git a/pkg/sql/colencoding/key_encoding.go b/pkg/sql/colencoding/key_encoding.go index ad434838024b..28de30028528 100644 --- a/pkg/sql/colencoding/key_encoding.go +++ b/pkg/sql/colencoding/key_encoding.go @@ -27,11 +27,11 @@ import ( // DecodeIndexKeyToCols decodes an index key into the idx'th position of the // provided slices of colexec.Vecs. The input index key must already have its -// first table id / index id prefix removed. If matches is false, the key is -// from a different table, and the returned remainingKey indicates a +// tenant id and first table id / index id prefix removed. If matches is false, +// the key is from a different table, and the returned remainingKey indicates a // "seek prefix": the next key that might be part of the table being searched -// for. The input key will also be mutated if matches is false. -// See the analog in sqlbase/index_encoding.go. +// for. The input key will also be mutated if matches is false. See the analog +// in sqlbase/index_encoding.go. func DecodeIndexKeyToCols( vecs []coldata.Vec, idx int, @@ -53,7 +53,7 @@ func DecodeIndexKeyToCols( // Our input key had its first table id / index id chopped off, so // don't try to decode those for the first ancestor. if i != 0 { - key, decodedTableID, decodedIndexID, err = sqlbase.DecodeTableIDIndexID(key) + key, decodedTableID, decodedIndexID, err = sqlbase.DecodePartialTableIDIndexID(key) if err != nil { return nil, false, false, err } @@ -61,7 +61,7 @@ func DecodeIndexKeyToCols( // We don't match. Return a key with the table ID / index ID we're // searching for, so the caller knows what to seek to. curPos := len(origKey) - len(key) - key = sqlbase.EncodeTableIDIndexID(origKey[:curPos], ancestor.TableID, ancestor.IndexID) + key = sqlbase.EncodePartialTableIDIndexID(origKey[:curPos], ancestor.TableID, ancestor.IndexID) return key, false, false, nil } } @@ -90,7 +90,7 @@ func DecodeIndexKeyToCols( } } - key, decodedTableID, decodedIndexID, err = sqlbase.DecodeTableIDIndexID(key) + key, decodedTableID, decodedIndexID, err = sqlbase.DecodePartialTableIDIndexID(key) if err != nil { return nil, false, false, err } @@ -98,7 +98,7 @@ func DecodeIndexKeyToCols( // We don't match. Return a key with the table ID / index ID we're // searching for, so the caller knows what to seek to. curPos := len(origKey) - len(key) - key = sqlbase.EncodeTableIDIndexID(origKey[:curPos], desc.ID, index.ID) + key = sqlbase.EncodePartialTableIDIndexID(origKey[:curPos], desc.ID, index.ID) return key, false, false, nil } } diff --git a/pkg/sql/conn_executor_test.go b/pkg/sql/conn_executor_test.go index 51ac8d27eb17..ca89ae5dc6b3 100644 --- a/pkg/sql/conn_executor_test.go +++ b/pkg/sql/conn_executor_test.go @@ -481,7 +481,7 @@ func TestQueryProgress(t *testing.T) { var queryRunningAtomic, scannedBatchesAtomic int64 stalled, unblock := make(chan struct{}), make(chan struct{}) - tableKey := roachpb.Key(keys.MakeTablePrefix(keys.MinNonPredefinedUserDescID + 1)) + tableKey := keys.SystemSQLCodec.TablePrefix(keys.MinNonPredefinedUserDescID + 1) tableSpan := roachpb.Span{Key: tableKey, EndKey: tableKey.PrefixEnd()} // Install a store filter which, if queryRunningAtomic is 1, will count scan @@ -715,7 +715,7 @@ func TestErrorDuringPrepareInExplicitTransactionPropagates(t *testing.T) { } if req, ok := ba.GetArg(roachpb.Get); ok { get := req.(*roachpb.GetRequest) - _, tableID, err := keys.DecodeTablePrefix(get.Key) + _, tableID, err := keys.SystemSQLCodec.DecodeTablePrefix(get.Key) if err != nil || tableID != keys.NamespaceTableID { err = nil return nil diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 172566251226..8a72b93565a1 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2131,19 +2131,19 @@ CREATE TABLE crdb_internal.ranges_no_leases ( return nil, nil, err } // TODO(knz): maybe this could use internalLookupCtx. - dbNames := make(map[uint64]string) - tableNames := make(map[uint64]string) - indexNames := make(map[uint64]map[sqlbase.IndexID]string) - parents := make(map[uint64]uint64) + dbNames := make(map[uint32]string) + tableNames := make(map[uint32]string) + indexNames := make(map[uint32]map[uint32]string) + parents := make(map[uint32]uint32) for _, desc := range descs { - id := uint64(desc.GetID()) + id := uint32(desc.GetID()) switch desc := desc.(type) { case *sqlbase.TableDescriptor: - parents[id] = uint64(desc.ParentID) + parents[id] = uint32(desc.ParentID) tableNames[id] = desc.GetName() - indexNames[id] = make(map[sqlbase.IndexID]string) + indexNames[id] = make(map[uint32]string) for _, idx := range desc.Indexes { - indexNames[id][idx.ID] = idx.Name + indexNames[id][uint32(idx.ID)] = idx.Name } case *sqlbase.DatabaseDescriptor: dbNames[id] = desc.GetName() @@ -2214,16 +2214,16 @@ CREATE TABLE crdb_internal.ranges_no_leases ( } var dbName, tableName, indexName string - if _, id, err := keys.DecodeTablePrefix(desc.StartKey.AsRawKey()); err == nil { - parent := parents[id] + if _, tableID, err := keys.TODOSQLCodec.DecodeTablePrefix(desc.StartKey.AsRawKey()); err == nil { + parent := parents[tableID] if parent != 0 { - tableName = tableNames[id] + tableName = tableNames[tableID] dbName = dbNames[parent] - if _, _, idxID, err := sqlbase.DecodeTableIDIndexID(desc.StartKey.AsRawKey()); err == nil { - indexName = indexNames[id][idxID] + if _, _, idxID, err := keys.TODOSQLCodec.DecodeIndexPrefix(desc.StartKey.AsRawKey()); err == nil { + indexName = indexNames[tableID][idxID] } } else { - dbName = dbNames[id] + dbName = dbNames[tableID] } } diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go index 0868ad7a3329..5eb18121bbd1 100644 --- a/pkg/sql/create_sequence.go +++ b/pkg/sql/create_sequence.go @@ -126,7 +126,7 @@ func doCreateSequence( } // Initialize the sequence value. - seqValueKey := keys.MakeSequenceKey(uint32(id)) + seqValueKey := keys.TODOSQLCodec.SequenceKey(uint32(id)) b := &kv.Batch{} b.Inc(seqValueKey, desc.SequenceOpts.Start-desc.SequenceOpts.Increment) if err := params.p.txn.Run(params.ctx, b); err != nil { diff --git a/pkg/sql/create_test.go b/pkg/sql/create_test.go index 88b5ea48bc61..4ca0102bb7cc 100644 --- a/pkg/sql/create_test.go +++ b/pkg/sql/create_test.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -88,7 +87,7 @@ func TestDatabaseDescriptor(t *testing.T) { t.Fatalf("expected descriptor ID == %d, got %d", expectedCounter, actual) } - start := roachpb.Key(keys.MakeTablePrefix(uint32(keys.NamespaceTableID))) + start := keys.SystemSQLCodec.TablePrefix(uint32(keys.NamespaceTableID)) if kvs, err := kvDB.Scan(ctx, start, start.PrefixEnd(), 0 /* maxRows */); err != nil { t.Fatal(err) } else { diff --git a/pkg/sql/descriptor_mutation_test.go b/pkg/sql/descriptor_mutation_test.go index 0dfd5f3d2907..e41756333051 100644 --- a/pkg/sql/descriptor_mutation_test.go +++ b/pkg/sql/descriptor_mutation_test.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -54,8 +53,7 @@ func makeMutationTest( // in the table equals e. func (mt mutationTest) checkTableSize(e int) { // Check that there are no hidden values - tablePrefix := keys.MakeTablePrefix(uint32(mt.tableDesc.ID)) - tableStartKey := roachpb.Key(tablePrefix) + tableStartKey := keys.SystemSQLCodec.TablePrefix(uint32(mt.tableDesc.ID)) tableEndKey := tableStartKey.PrefixEnd() if kvs, err := mt.kvDB.Scan(context.TODO(), tableStartKey, tableEndKey, 0); err != nil { mt.Error(err) diff --git a/pkg/sql/distsql_plan_join_test.go b/pkg/sql/distsql_plan_join_test.go index d90f6c8fd38d..bc75d2d518f3 100644 --- a/pkg/sql/distsql_plan_join_test.go +++ b/pkg/sql/distsql_plan_join_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -103,7 +104,7 @@ var tableNames = map[string]bool{ // Format for any key: // ///.../#///.... func encodeTestKey(kvDB *kv.DB, keyStr string) (roachpb.Key, error) { - var key []byte + key := keys.SystemSQLCodec.TenantPrefix() tokens := strings.Split(keyStr, "/") for _, tok := range tokens { diff --git a/pkg/sql/gcjob/refresh_statuses.go b/pkg/sql/gcjob/refresh_statuses.go index 56fd21069029..10ad836c0c93 100644 --- a/pkg/sql/gcjob/refresh_statuses.go +++ b/pkg/sql/gcjob/refresh_statuses.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" - "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -272,8 +271,7 @@ func isProtected( func setupConfigWatcher( execCfg *sql.ExecutorConfig, ) (gossip.SystemConfigDeltaFilter, <-chan struct{}) { - k := keys.MakeTablePrefix(uint32(keys.ZonesTableID)) - k = encoding.EncodeUvarintAscending(k, uint64(keys.ZonesTablePrimaryIndexID)) + k := keys.TODOSQLCodec.IndexPrefix(uint32(keys.ZonesTableID), uint32(keys.ZonesTablePrimaryIndexID)) zoneCfgFilter := gossip.MakeSystemConfigDeltaFilter(k) gossipUpdateC := execCfg.Gossip.Deprecated(47150).RegisterSystemConfigChannel() return zoneCfgFilter, gossipUpdateC diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index c25f015407af..a4204df47389 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -88,7 +88,7 @@ func clearTableData( } log.Infof(ctx, "clearing data for table %d", table.ID) - tableKey := roachpb.RKey(keys.MakeTablePrefix(uint32(table.ID))) + tableKey := roachpb.RKey(keys.TODOSQLCodec.TablePrefix(uint32(table.ID))) tableSpan := roachpb.RSpan{Key: tableKey, EndKey: tableKey.PrefixEnd()} // ClearRange requests lays down RocksDB range deletion tombstones that have diff --git a/pkg/sql/lease.go b/pkg/sql/lease.go index dbe61a1bdffa..56f6cc56a5a0 100644 --- a/pkg/sql/lease.go +++ b/pkg/sql/lease.go @@ -1773,7 +1773,7 @@ func (m *LeaseManager) findTableState(tableID sqlbase.ID, create bool) *tableSta func (m *LeaseManager) RefreshLeases(s *stop.Stopper, db *kv.DB, g *gossip.Gossip) { ctx := context.TODO() s.RunWorker(ctx, func(ctx context.Context) { - descKeyPrefix := keys.MakeTablePrefix(uint32(sqlbase.DescriptorTable.ID)) + descKeyPrefix := keys.TODOSQLCodec.TablePrefix(uint32(sqlbase.DescriptorTable.ID)) cfgFilter := gossip.MakeSystemConfigDeltaFilter(descKeyPrefix) gossipUpdateC := g.RegisterSystemConfigChannel() for { diff --git a/pkg/sql/partition_utils.go b/pkg/sql/partition_utils.go index 09db75e3751d..aee63ca704b4 100644 --- a/pkg/sql/partition_utils.go +++ b/pkg/sql/partition_utils.go @@ -134,7 +134,7 @@ func GenerateSubzoneSpans( // NB: This assumes that none of the indexes are interleaved, which is // checked in PartitionDescriptor validation. - sharedPrefix := keys.MakeTablePrefix(uint32(tableDesc.ID)) + sharedPrefix := keys.TODOSQLCodec.TablePrefix(uint32(tableDesc.ID)) var subzoneSpans []zonepb.SubzoneSpan for _, r := range ranges { diff --git a/pkg/sql/scatter_test.go b/pkg/sql/scatter_test.go index fd24f20f8693..9b0feeeb87b4 100644 --- a/pkg/sql/scatter_test.go +++ b/pkg/sql/scatter_test.go @@ -135,7 +135,7 @@ func TestScatterResponse(t *testing.T) { } var expectedKey roachpb.Key if i == 0 { - expectedKey = keys.MakeTablePrefix(uint32(tableDesc.ID)) + expectedKey = keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.ID)) } else { var err error expectedKey, err = sqlbase.TestingMakePrimaryIndexKey(tableDesc, i*10) diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index ac7be635d280..3c53d056e794 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -45,7 +45,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" - "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -297,7 +296,7 @@ CREATE INDEX foo ON t.test (v) func getTableKeyCount(ctx context.Context, kvDB *kv.DB) (int, error) { tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") - tablePrefix := roachpb.Key(keys.MakeTablePrefix(uint32(tableDesc.ID))) + tablePrefix := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.ID)) tableEnd := tablePrefix.PrefixEnd() kvs, err := kvDB.Scan(ctx, tablePrefix, tableEnd, 0) return len(kvs), err @@ -721,7 +720,7 @@ CREATE UNIQUE INDEX vidx ON t.test (v); wg.Wait() // Ensure that the table data hasn't been deleted. - tablePrefix := roachpb.Key(keys.MakeTablePrefix(uint32(tableDesc.ID))) + tablePrefix := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.ID)) tableEnd := tablePrefix.PrefixEnd() if kvs, err := kvDB.Scan(ctx, tablePrefix, tableEnd, 0); err != nil { t.Fatal(err) @@ -1924,7 +1923,7 @@ CREATE TABLE t.test ( // values. This is done to make the table appear like it were // written in the past when cockroachdb used to write sentinel // values for each table row. - startKey := roachpb.Key(keys.MakeTablePrefix(uint32(tableDesc.ID))) + startKey := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.ID)) kvs, err := kvDB.Scan( ctx, startKey, @@ -3887,7 +3886,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT, pi DECIMAL DEFAULT (DECIMAL '3.14 } // Ensure that the table data hasn't been deleted. - tablePrefix := roachpb.Key(keys.MakeTablePrefix(uint32(tableDesc.ID))) + tablePrefix := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.ID)) tableEnd := tablePrefix.PrefixEnd() if kvs, err := kvDB.Scan(ctx, tablePrefix, tableEnd, 0); err != nil { t.Fatal(err) @@ -4034,7 +4033,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT, pi DECIMAL REFERENCES t.pi (d) DE } // Ensure that the table data has been deleted. - tablePrefix := roachpb.Key(keys.MakeTablePrefix(uint32(tableDesc.ID))) + tablePrefix := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.ID)) tableEnd := tablePrefix.PrefixEnd() if kvs, err := kvDB.Scan(ctx, tablePrefix, tableEnd, 0); err != nil { t.Fatal(err) @@ -4043,7 +4042,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT, pi DECIMAL REFERENCES t.pi (d) DE } fkTableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "pi") - tablePrefix = roachpb.Key(keys.MakeTablePrefix(uint32(fkTableDesc.ID))) + tablePrefix = keys.SystemSQLCodec.TablePrefix(uint32(fkTableDesc.ID)) tableEnd = tablePrefix.PrefixEnd() if kvs, err := kvDB.Scan(ctx, tablePrefix, tableEnd, 0); err != nil { t.Fatal(err) @@ -4150,7 +4149,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); // The new table is truncated. tableDesc = sqlbase.GetTableDescriptor(kvDB, "t", "test") - tablePrefix := roachpb.Key(keys.MakeTablePrefix(uint32(tableDesc.ID))) + tablePrefix := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.ID)) tableEnd := tablePrefix.PrefixEnd() if kvs, err := kvDB.Scan(context.TODO(), tablePrefix, tableEnd, 0); err != nil { t.Fatal(err) @@ -4915,8 +4914,7 @@ func TestIndexBackfillValidation(t *testing.T) { count := atomic.AddInt64(&backfillCount, 1) if count == 2 { // drop an index value before validation. - keyEnc := keys.MakeTablePrefix(uint32(tableDesc.ID)) - key := roachpb.Key(encoding.EncodeUvarintAscending(keyEnc, uint64(tableDesc.NextIndexID))) + key := keys.SystemSQLCodec.IndexPrefix(uint32(tableDesc.ID), uint32(tableDesc.NextIndexID)) kv, err := db.Scan(context.TODO(), key, key.PrefixEnd(), 1) if err != nil { t.Error(err) @@ -4985,8 +4983,7 @@ func TestInvertedIndexBackfillValidation(t *testing.T) { count := atomic.AddInt64(&backfillCount, 1) if count == 2 { // drop an index value before validation. - keyEnc := keys.MakeTablePrefix(uint32(tableDesc.ID)) - key := roachpb.Key(encoding.EncodeUvarintAscending(keyEnc, uint64(tableDesc.NextIndexID))) + key := keys.SystemSQLCodec.IndexPrefix(uint32(tableDesc.ID), uint32(tableDesc.NextIndexID)) kv, err := db.Scan(context.TODO(), key, key.PrefixEnd(), 1) if err != nil { t.Error(err) diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 2fb0f1062add..be31355a7e88 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -2864,7 +2864,7 @@ may increase either contention or retry errors, or both.`, } if indexDesc.ID == tableDesc.PrimaryIndex.ID { - keyPrefix := tableDesc.IndexSpan(indexDesc.ID).Key + keyPrefix := sqlbase.MakeIndexKeyPrefix(tableDesc, indexDesc.ID) res, _, err := sqlbase.EncodeIndexKey(tableDesc, indexDesc, colMap, datums, keyPrefix) if err != nil { return nil, err diff --git a/pkg/sql/sequence.go b/pkg/sql/sequence.go index e0bff82103d1..8a5b8f09a3e9 100644 --- a/pkg/sql/sequence.go +++ b/pkg/sql/sequence.go @@ -49,7 +49,7 @@ func (p *planner) IncrementSequence(ctx context.Context, seqName *tree.TableName rowid := builtins.GenerateUniqueInt(p.EvalContext().NodeID) val = int64(rowid) } else { - seqValueKey := keys.MakeSequenceKey(uint32(descriptor.ID)) + seqValueKey := keys.TODOSQLCodec.SequenceKey(uint32(descriptor.ID)) val, err = kv.IncrementValRetryable( ctx, p.txn.DB(), seqValueKey, seqOpts.Increment) if err != nil { @@ -162,7 +162,7 @@ func MakeSequenceKeyVal( newVal = newVal - sequence.SequenceOpts.Increment } - seqValueKey := keys.MakeSequenceKey(uint32(sequence.ID)) + seqValueKey := keys.TODOSQLCodec.SequenceKey(uint32(sequence.ID)) return seqValueKey, newVal, nil } @@ -173,7 +173,7 @@ func (p *planner) GetSequenceValue( if desc.SequenceOpts == nil { return 0, errors.New("descriptor is not a sequence") } - keyValue, err := p.txn.Get(ctx, keys.MakeSequenceKey(uint32(desc.ID))) + keyValue, err := p.txn.Get(ctx, keys.TODOSQLCodec.SequenceKey(uint32(desc.ID))) if err != nil { return 0, err } diff --git a/pkg/sql/span/span_builder.go b/pkg/sql/span/span_builder.go index 9be263624956..132e22dac006 100644 --- a/pkg/sql/span/span_builder.go +++ b/pkg/sql/span/span_builder.go @@ -71,25 +71,22 @@ func MakeBuilder(table *sqlbase.TableDescriptor, index *sqlbase.IndexDescriptor) } // Set up the interstices for encoding interleaved tables later. - s.interstices[0] = sqlbase.MakeIndexKeyPrefix(table, index.ID) + s.interstices[0] = s.KeyPrefix if len(index.Interleave.Ancestors) > 0 { // TODO(rohany): too much of this code is copied from EncodePartialIndexKey. sharedPrefixLen := 0 for i, ancestor := range index.Interleave.Ancestors { // The first ancestor is already encoded in interstices[0]. if i != 0 { - s.interstices[sharedPrefixLen] = - encoding.EncodeUvarintAscending(s.interstices[sharedPrefixLen], uint64(ancestor.TableID)) - s.interstices[sharedPrefixLen] = - encoding.EncodeUvarintAscending(s.interstices[sharedPrefixLen], uint64(ancestor.IndexID)) + s.interstices[sharedPrefixLen] = sqlbase.EncodePartialTableIDIndexID( + s.interstices[sharedPrefixLen], ancestor.TableID, ancestor.IndexID) } sharedPrefixLen += int(ancestor.SharedPrefixLen) - s.interstices[sharedPrefixLen] = encoding.EncodeInterleavedSentinel(s.interstices[sharedPrefixLen]) + s.interstices[sharedPrefixLen] = encoding.EncodeInterleavedSentinel( + s.interstices[sharedPrefixLen]) } - s.interstices[sharedPrefixLen] = - encoding.EncodeUvarintAscending(s.interstices[sharedPrefixLen], uint64(table.ID)) - s.interstices[sharedPrefixLen] = - encoding.EncodeUvarintAscending(s.interstices[sharedPrefixLen], uint64(index.ID)) + s.interstices[sharedPrefixLen] = sqlbase.EncodePartialTableIDIndexID( + s.interstices[sharedPrefixLen], table.ID, index.ID) } return s @@ -129,7 +126,7 @@ func (s *Builder) SpanFromEncDatums( values sqlbase.EncDatumRow, prefixLen int, ) (_ roachpb.Span, containsNull bool, _ error) { return sqlbase.MakeSpanFromEncDatums( - s.KeyPrefix, values[:prefixLen], s.indexColTypes[:prefixLen], s.indexColDirs[:prefixLen], s.table, s.index, &s.alloc) + values[:prefixLen], s.indexColTypes[:prefixLen], s.indexColDirs[:prefixLen], s.table, s.index, &s.alloc, s.KeyPrefix) } // SpanFromDatumRow generates an index span with prefixLen constraint columns from the index. diff --git a/pkg/sql/sqlbase/index_encoding.go b/pkg/sql/sqlbase/index_encoding.go index fddf0568d0fd..bc6a083f73a2 100644 --- a/pkg/sql/sqlbase/index_encoding.go +++ b/pkg/sql/sqlbase/index_encoding.go @@ -35,15 +35,12 @@ import ( // need the corresponding Span, prefer desc.IndexSpan(indexID) or // desc.PrimaryIndexSpan(). func MakeIndexKeyPrefix(desc *TableDescriptor, indexID IndexID) []byte { - var key []byte + keyGen := &keys.TODOSQLCodec if i, err := desc.FindIndexByID(indexID); err == nil && len(i.Interleave.Ancestors) > 0 { - key = encoding.EncodeUvarintAscending(key, uint64(i.Interleave.Ancestors[0].TableID)) - key = encoding.EncodeUvarintAscending(key, uint64(i.Interleave.Ancestors[0].IndexID)) - return key + ancestor := &i.Interleave.Ancestors[0] + return keyGen.IndexPrefix(uint32(ancestor.TableID), uint32(ancestor.IndexID)) } - key = encoding.EncodeUvarintAscending(key, uint64(desc.ID)) - key = encoding.EncodeUvarintAscending(key, uint64(indexID)) - return key + return keyGen.IndexPrefix(uint32(desc.ID), uint32(indexID)) } // EncodeIndexKey creates a key by concatenating keyPrefix with the @@ -140,8 +137,7 @@ func EncodePartialIndexKey( for i, ancestor := range index.Interleave.Ancestors { // The first ancestor is assumed to already be encoded in keyPrefix. if i != 0 { - key = encoding.EncodeUvarintAscending(key, uint64(ancestor.TableID)) - key = encoding.EncodeUvarintAscending(key, uint64(ancestor.IndexID)) + key = EncodePartialTableIDIndexID(key, ancestor.TableID, ancestor.IndexID) } partial := false @@ -168,8 +164,7 @@ func EncodePartialIndexKey( key = encoding.EncodeInterleavedSentinel(key) } - key = encoding.EncodeUvarintAscending(key, uint64(tableDesc.ID)) - key = encoding.EncodeUvarintAscending(key, uint64(index.ID)) + key = EncodePartialTableIDIndexID(key, tableDesc.ID, index.ID) } var n bool @@ -209,15 +204,15 @@ func (d directions) get(i int) (encoding.Direction, error) { // key. An example of one level of interleaving (a parent): // /////NullDesc//// func MakeSpanFromEncDatums( - keyPrefix []byte, values EncDatumRow, types []types.T, dirs []IndexDescriptor_Direction, tableDesc *TableDescriptor, index *IndexDescriptor, alloc *DatumAlloc, + keyPrefix []byte, ) (_ roachpb.Span, containsNull bool, _ error) { - startKey, complete, containsNull, err := makeKeyFromEncDatums(keyPrefix, values, types, dirs, tableDesc, index, alloc) + startKey, complete, containsNull, err := makeKeyFromEncDatums(values, types, dirs, tableDesc, index, alloc, keyPrefix) if err != nil { return roachpb.Span{}, false, err } @@ -395,13 +390,13 @@ func SplitSpanIntoSeparateFamilies( // key. An example of one level of interleaving (a parent): // /////NullDesc//// func makeKeyFromEncDatums( - keyPrefix []byte, values EncDatumRow, types []types.T, dirs []IndexDescriptor_Direction, tableDesc *TableDescriptor, index *IndexDescriptor, alloc *DatumAlloc, + keyPrefix []byte, ) (_ roachpb.Key, complete bool, containsNull bool, _ error) { // Values may be a prefix of the index columns. if len(values) > len(dirs) { @@ -419,8 +414,7 @@ func makeKeyFromEncDatums( for i, ancestor := range index.Interleave.Ancestors { // The first ancestor is assumed to already be encoded in keyPrefix. if i != 0 { - key = encoding.EncodeUvarintAscending(key, uint64(ancestor.TableID)) - key = encoding.EncodeUvarintAscending(key, uint64(ancestor.IndexID)) + key = EncodePartialTableIDIndexID(key, ancestor.TableID, ancestor.IndexID) } partial := false @@ -450,8 +444,7 @@ func makeKeyFromEncDatums( key = encoding.EncodeInterleavedSentinel(key) } - key = encoding.EncodeUvarintAscending(key, uint64(tableDesc.ID)) - key = encoding.EncodeUvarintAscending(key, uint64(index.ID)) + key = EncodePartialTableIDIndexID(key, tableDesc.ID, index.ID) } var ( err error @@ -502,29 +495,17 @@ func appendEncDatumsToKey( return key, containsNull, nil } -// EncodeTableIDIndexID encodes a table id followed by an index id. -func EncodeTableIDIndexID(key []byte, tableID ID, indexID IndexID) []byte { - key = encoding.EncodeUvarintAscending(key, uint64(tableID)) - key = encoding.EncodeUvarintAscending(key, uint64(indexID)) - return key +// EncodePartialTableIDIndexID encodes a table id followed by an index id to an +// existing key. The key must already contain a tenant id. +func EncodePartialTableIDIndexID(key []byte, tableID ID, indexID IndexID) []byte { + return keys.MakeTableIDIndexID(key, uint32(tableID), uint32(indexID)) } -// DecodeTableIDIndexID decodes a table id followed by an index id. -func DecodeTableIDIndexID(key []byte) ([]byte, ID, IndexID, error) { - var tableID uint64 - var indexID uint64 - var err error - - key, tableID, err = encoding.DecodeUvarintAscending(key) - if err != nil { - return nil, 0, 0, err - } - key, indexID, err = encoding.DecodeUvarintAscending(key) - if err != nil { - return nil, 0, 0, err - } - - return key, ID(tableID), IndexID(indexID), nil +// DecodePartialTableIDIndexID decodes a table id followed by an index id. The +// input key must already have its tenant id removed. +func DecodePartialTableIDIndexID(key []byte) ([]byte, ID, IndexID, error) { + key, tableID, indexID, err := keys.DecodeTableIDIndexID(key) + return key, ID(tableID), IndexID(indexID), err } // DecodeIndexKeyPrefix decodes the prefix of an index key and returns the @@ -534,6 +515,11 @@ func DecodeTableIDIndexID(key []byte) ([]byte, ID, IndexID, error) { func DecodeIndexKeyPrefix( desc *TableDescriptor, key []byte, ) (indexID IndexID, remaining []byte, err error) { + key, err = keys.TODOSQLCodec.StripTenantPrefix(key) + if err != nil { + return 0, nil, err + } + // TODO(dan): This whole operation is n^2 because of the interleaves // bookkeeping. We could improve it to n with a prefix tree of components. @@ -541,7 +527,7 @@ func DecodeIndexKeyPrefix( for component := 0; ; component++ { var tableID ID - key, tableID, indexID, err = DecodeTableIDIndexID(key) + key, tableID, indexID, err = DecodePartialTableIDIndexID(key) if err != nil { return 0, nil, err } @@ -604,7 +590,11 @@ func DecodeIndexKey( colDirs []IndexDescriptor_Direction, key []byte, ) (remainingKey []byte, matches bool, foundNull bool, _ error) { - key, _, _, err := DecodeTableIDIndexID(key) + key, err := keys.TODOSQLCodec.StripTenantPrefix(key) + if err != nil { + return nil, false, false, err + } + key, _, _, err = DecodePartialTableIDIndexID(key) if err != nil { return nil, false, false, err } @@ -612,8 +602,8 @@ func DecodeIndexKey( } // DecodeIndexKeyWithoutTableIDIndexIDPrefix is the same as DecodeIndexKey, -// except it expects its index key is missing its first table id / index id -// key prefix. +// except it expects its index key is missing in its tenant id and first table +// id / index id key prefix. func DecodeIndexKeyWithoutTableIDIndexIDPrefix( desc *TableDescriptor, index *IndexDescriptor, @@ -631,7 +621,7 @@ func DecodeIndexKeyWithoutTableIDIndexIDPrefix( // Our input key had its first table id / index id chopped off, so // don't try to decode those for the first ancestor. if i != 0 { - key, decodedTableID, decodedIndexID, err = DecodeTableIDIndexID(key) + key, decodedTableID, decodedIndexID, err = DecodePartialTableIDIndexID(key) if err != nil { return nil, false, false, err } @@ -657,7 +647,7 @@ func DecodeIndexKeyWithoutTableIDIndexIDPrefix( } } - key, decodedTableID, decodedIndexID, err = DecodeTableIDIndexID(key) + key, decodedTableID, decodedIndexID, err = DecodePartialTableIDIndexID(key) if err != nil { return nil, false, false, err } @@ -1451,8 +1441,7 @@ func TableEquivSignatures( // Encode the table's ancestors' TableIDs and IndexIDs. for i, ancestor := range index.Interleave.Ancestors { - fullSignature = encoding.EncodeUvarintAscending(fullSignature, uint64(ancestor.TableID)) - fullSignature = encoding.EncodeUvarintAscending(fullSignature, uint64(ancestor.IndexID)) + fullSignature = EncodePartialTableIDIndexID(fullSignature, ancestor.TableID, ancestor.IndexID) // Create a reference up to this point for the ancestor's // signature. signatures[i] = fullSignature @@ -1461,8 +1450,7 @@ func TableEquivSignatures( } // Encode the table's table and index IDs. - fullSignature = encoding.EncodeUvarintAscending(fullSignature, uint64(desc.ID)) - fullSignature = encoding.EncodeUvarintAscending(fullSignature, uint64(index.ID)) + fullSignature = EncodePartialTableIDIndexID(fullSignature, desc.ID, index.ID) // Create a reference for the given table's signature as the last // element of signatures. signatures[len(signatures)-1] = fullSignature @@ -1536,7 +1524,13 @@ func maxKeyTokens(index *IndexDescriptor, containsNull bool) int { // was /1, we cannot push this forwards since that is the first key we want // to read. func AdjustStartKeyForInterleave(index *IndexDescriptor, start roachpb.Key) (roachpb.Key, error) { - keyTokens, containsNull, err := encoding.DecomposeKeyTokens(start) + // Remove the tenant prefix before decomposing. + strippedStart, err := keys.TODOSQLCodec.StripTenantPrefix(start) + if err != nil { + return roachpb.Key{}, err + } + + keyTokens, containsNull, err := encoding.DecomposeKeyTokens(strippedStart) if err != nil { return roachpb.Key{}, err } @@ -1578,12 +1572,18 @@ func AdjustEndKeyForInterleave( return end.PrefixEnd(), nil } + // Remove the tenant prefix before decomposing. + strippedEnd, err := keys.TODOSQLCodec.StripTenantPrefix(end) + if err != nil { + return roachpb.Key{}, err + } + // To illustrate, suppose we have the interleaved hierarchy // parent // child // grandchild // Suppose our target index is child. - keyTokens, containsNull, err := encoding.DecomposeKeyTokens(end) + keyTokens, containsNull, err := encoding.DecomposeKeyTokens(strippedEnd) if err != nil { return roachpb.Key{}, err } diff --git a/pkg/sql/sqlbase/keys.go b/pkg/sql/sqlbase/keys.go index a7a427228bc6..08d2616e2929 100644 --- a/pkg/sql/sqlbase/keys.go +++ b/pkg/sql/sqlbase/keys.go @@ -24,8 +24,7 @@ import ( // Pass name == "" in order to generate the prefix key to use to scan over all // of the names for the specified parentID. func MakeNameMetadataKey(parentID ID, parentSchemaID ID, name string) roachpb.Key { - k := keys.MakeTablePrefix(uint32(NamespaceTable.ID)) - k = encoding.EncodeUvarintAscending(k, uint64(NamespaceTable.PrimaryIndex.ID)) + k := keys.TODOSQLCodec.IndexPrefix(uint32(NamespaceTable.ID), uint32(NamespaceTable.PrimaryIndex.ID)) k = encoding.EncodeUvarintAscending(k, uint64(parentID)) k = encoding.EncodeUvarintAscending(k, uint64(parentSchemaID)) if name != "" { @@ -38,7 +37,7 @@ func MakeNameMetadataKey(parentID ID, parentSchemaID ID, name string) roachpb.Ke // DecodeNameMetadataKey returns the components that make up the // NameMetadataKey for version >= 20.1. func DecodeNameMetadataKey(k roachpb.Key) (parentID ID, parentSchemaID ID, name string, err error) { - k, _, err = keys.DecodeTablePrefix(k) + k, _, err = keys.TODOSQLCodec.DecodeTablePrefix(k) if err != nil { return 0, 0, "", err } @@ -78,8 +77,8 @@ func DecodeNameMetadataKey(k roachpb.Key) (parentID ID, parentSchemaID ID, name // versions < 20.1. Pass name == "" in order to generate the prefix key to use // to scan over all of the names for the specified parentID. func MakeDeprecatedNameMetadataKey(parentID ID, name string) roachpb.Key { - k := keys.MakeTablePrefix(uint32(DeprecatedNamespaceTable.ID)) - k = encoding.EncodeUvarintAscending(k, uint64(DeprecatedNamespaceTable.PrimaryIndex.ID)) + k := keys.TODOSQLCodec.IndexPrefix( + uint32(DeprecatedNamespaceTable.ID), uint32(DeprecatedNamespaceTable.PrimaryIndex.ID)) k = encoding.EncodeUvarintAscending(k, uint64(parentID)) if name != "" { k = encoding.EncodeBytesAscending(k, []byte(name)) @@ -90,12 +89,12 @@ func MakeDeprecatedNameMetadataKey(parentID ID, name string) roachpb.Key { // MakeAllDescsMetadataKey returns the key for all descriptors. func MakeAllDescsMetadataKey() roachpb.Key { - return keys.DescMetadataPrefix() + return keys.TODOSQLCodec.DescMetadataPrefix() } // MakeDescMetadataKey returns the key for the descriptor. func MakeDescMetadataKey(descID ID) roachpb.Key { - return keys.DescMetadataKey(uint32(descID)) + return keys.TODOSQLCodec.DescMetadataKey(uint32(descID)) } // IndexKeyValDirs returns the corresponding encoding.Directions for all the diff --git a/pkg/sql/sqlbase/metadata.go b/pkg/sql/sqlbase/metadata.go index 3c69fca56ed5..1da40ebfa73e 100644 --- a/pkg/sql/sqlbase/metadata.go +++ b/pkg/sql/sqlbase/metadata.go @@ -70,6 +70,7 @@ func WrapDescriptor(descriptor DescriptorProto) *Descriptor { // installed on the underlying persistent storage before a cockroach store can // start running correctly, thus requiring this special initialization. type MetadataSchema struct { + // TODO(nvanbenschoten): add roachpb.TenantID here. Use in GetInitialValues. descs []metadataDescriptor otherSplitIDs []uint32 otherKV []roachpb.KeyValue @@ -187,7 +188,7 @@ func (ms MetadataSchema) GetInitialValues( Value: value, }) if desc.GetID() > keys.MaxSystemConfigDescID { - splits = append(splits, roachpb.RKey(keys.MakeTablePrefix(uint32(desc.GetID())))) + splits = append(splits, roachpb.RKey(keys.TODOSQLCodec.TablePrefix(uint32(desc.GetID())))) } } @@ -198,7 +199,7 @@ func (ms MetadataSchema) GetInitialValues( } for _, id := range ms.otherSplitIDs { - splits = append(splits, roachpb.RKey(keys.MakeTablePrefix(id))) + splits = append(splits, roachpb.RKey(keys.TODOSQLCodec.TablePrefix(id))) } // Other key/value generation that doesn't fit into databases and diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index e29c3ae6fd95..6b497df45a37 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -3606,7 +3606,7 @@ func (desc *TableDescriptor) IndexSpan(indexID IndexID) roachpb.Span { // TableSpan returns the Span that corresponds to the entire table. func (desc *TableDescriptor) TableSpan() roachpb.Span { - prefix := roachpb.Key(keys.MakeTablePrefix(uint32(desc.ID))) + prefix := keys.TODOSQLCodec.TablePrefix(uint32(desc.ID)) return roachpb.Span{Key: prefix, EndKey: prefix.PrefixEnd()} } diff --git a/pkg/sql/sqlbase/system.go b/pkg/sql/sqlbase/system.go index 357c4e3b6729..ec2e7859f29e 100644 --- a/pkg/sql/sqlbase/system.go +++ b/pkg/sql/sqlbase/system.go @@ -1463,7 +1463,7 @@ func createZoneConfigKV(keyID int, zoneConfig *zonepb.ZoneConfig) roachpb.KeyVal panic(fmt.Sprintf("could not marshal ZoneConfig for ID: %d: %s", keyID, err)) } return roachpb.KeyValue{ - Key: keys.ZoneKey(uint32(keyID)), + Key: keys.TODOSQLCodec.ZoneKey(uint32(keyID)), Value: value, } } diff --git a/pkg/sql/sqlbase/utils_test.go b/pkg/sql/sqlbase/utils_test.go index 0fe071f45311..9f62301e7549 100644 --- a/pkg/sql/sqlbase/utils_test.go +++ b/pkg/sql/sqlbase/utils_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/apd" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -43,7 +44,7 @@ var tableNames = map[string]bool{ // - NULLASC, NULLDESC, NOTNULLASC, NOTNULLDESC // - PrefixEnd func EncodeTestKey(tb testing.TB, kvDB *kv.DB, keyStr string) roachpb.Key { - var key []byte + key := keys.SystemSQLCodec.TenantPrefix() tokens := strings.Split(keyStr, "/") if tokens[0] != "" { panic("missing '/' token at the beginning of long format") @@ -54,7 +55,7 @@ func EncodeTestKey(tb testing.TB, kvDB *kv.DB, keyStr string) roachpb.Key { for _, tok := range tokens { if tok == "PrefixEnd" { - key = roachpb.Key(key).PrefixEnd() + key = key.PrefixEnd() continue } diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go b/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go index ba38cfa3399b..38ff51bccfc6 100644 --- a/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go +++ b/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go @@ -169,7 +169,7 @@ func TestChangePollInterval(t *testing.T) { defer leaktest.AfterTest(t)() // We'll inject a request filter to detect scans due to the polling. - tableStart := roachpb.Key(keys.MakeTablePrefix(keys.StatementDiagnosticsRequestsTableID)) + tableStart := keys.SystemSQLCodec.TablePrefix(keys.StatementDiagnosticsRequestsTableID) tableSpan := roachpb.Span{ Key: tableStart, EndKey: tableStart.PrefixEnd(), diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index 4a3fb57d980f..969fa17a1676 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -21,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -122,9 +122,7 @@ func (td *tableDeleter) deleteAllRowsFast( ctx context.Context, resume roachpb.Span, limit int64, traceKV bool, ) (roachpb.Span, error) { if resume.Key == nil { - tablePrefix := roachpb.Key( - encoding.EncodeUvarintAscending(nil, uint64(td.rd.Helper.TableDesc.ID)), - ) + tablePrefix := keys.TODOSQLCodec.TablePrefix(uint32(td.rd.Helper.TableDesc.ID)) // Delete rows and indexes starting with the table's prefix. resume = roachpb.Span{ Key: tablePrefix, diff --git a/pkg/sql/tests/split_test.go b/pkg/sql/tests/split_test.go index 38bd5650f231..ad5ca008f1f9 100644 --- a/pkg/sql/tests/split_test.go +++ b/pkg/sql/tests/split_test.go @@ -123,7 +123,7 @@ func TestSplitOnTableBoundaries(t *testing.T) { }) // Verify the actual splits. - splits = []roachpb.RKey{keys.MakeTablePrefix(objectID + 3), roachpb.RKeyMax} + splits = []roachpb.RKey{roachpb.RKey(keys.SystemSQLCodec.TablePrefix(objectID + 3)), roachpb.RKeyMax} ranges, err = getRangeKeys(kvDB) if err != nil { t.Fatal(err) diff --git a/pkg/sql/tests/table_split_test.go b/pkg/sql/tests/table_split_test.go index dca4eeeeeb04..130488edf026 100644 --- a/pkg/sql/tests/table_split_test.go +++ b/pkg/sql/tests/table_split_test.go @@ -44,7 +44,7 @@ SELECT tables.id FROM system.namespace tables ` var tableID uint32 runner.QueryRow(t, tableIDQuery, "test", "t").Scan(&tableID) - tableStartKey := keys.MakeTablePrefix(tableID) + tableStartKey := keys.SystemSQLCodec.TablePrefix(tableID) // Wait for new table to split. testutils.SucceedsSoon(t, func() error { diff --git a/pkg/sql/zone_config_test.go b/pkg/sql/zone_config_test.go index b8b4ebf8cb4c..a6a7989c81f7 100644 --- a/pkg/sql/zone_config_test.go +++ b/pkg/sql/zone_config_test.go @@ -18,6 +18,7 @@ import ( "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/server" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -119,7 +120,7 @@ func TestGetZoneConfig(t *testing.T) { for tcNum, tc := range testCases { // Verify SystemConfig.GetZoneConfigForKey. { - key := append(keys.MakeTablePrefix(tc.objectID), tc.keySuffix...) + key := append(roachpb.RKey(keys.SystemSQLCodec.TablePrefix(tc.objectID)), tc.keySuffix...) zoneCfg, err := cfg.GetZoneConfigForKey(key) // Complete ZoneConfig if err != nil { t.Fatalf("#%d: err=%s", tcNum, err) @@ -355,7 +356,7 @@ func TestCascadingZoneConfig(t *testing.T) { for tcNum, tc := range testCases { // Verify SystemConfig.GetZoneConfigForKey. { - key := append(keys.MakeTablePrefix(tc.objectID), tc.keySuffix...) + key := append(roachpb.RKey(keys.SystemSQLCodec.TablePrefix(tc.objectID)), tc.keySuffix...) zoneCfg, err := cfg.GetZoneConfigForKey(key) // Complete ZoneConfig if err != nil { t.Fatalf("#%d: err=%s", tcNum, err) @@ -643,7 +644,8 @@ func BenchmarkGetZoneConfig(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - _, err := cfg.GetZoneConfigForKey(keys.MakeTablePrefix(keys.MinUserDescID)) + key := roachpb.RKey(keys.SystemSQLCodec.TablePrefix(keys.MinUserDescID)) + _, err := cfg.GetZoneConfigForKey(key) if err != nil { b.Fatal(err) } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index b4edd54c1492..d0424b357f21 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -3332,7 +3332,7 @@ func MVCCFindSplitKey( return nil, nil } var minSplitKey roachpb.Key - if _, _, err := keys.DecodeTablePrefix(it.UnsafeKey().Key); err == nil { + if _, _, err := keys.TODOSQLCodec.DecodeTablePrefix(it.UnsafeKey().Key); err == nil { // The first key in this range represents a row in a SQL table. Advance the // minSplitKey past this row to avoid the problems described above. firstRowKey, err := keys.EnsureSafeSplitKey(it.Key().Key) diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index dc5246d05554..c14558845bd4 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -4215,8 +4215,8 @@ func TestValidSplitKeys(t *testing.T) { {roachpb.Key("a"), true}, {roachpb.Key("\xff"), true}, {roachpb.Key("\xff\x01"), true}, - {roachpb.Key(keys.MakeTablePrefix(keys.MaxSystemConfigDescID)), false}, - {roachpb.Key(keys.MakeTablePrefix(keys.MaxSystemConfigDescID + 1)), true}, + {keys.SystemSQLCodec.TablePrefix(keys.MaxSystemConfigDescID), false}, + {keys.SystemSQLCodec.TablePrefix(keys.MaxSystemConfigDescID + 1), true}, } for i, test := range testCases { valid := IsValidSplitKey(test.key) @@ -4294,7 +4294,7 @@ func TestFindValidSplitKeys(t *testing.T) { // Manually creates rows corresponding to the schema: // CREATE TABLE t (id1 STRING, id2 STRING, ... PRIMARY KEY (id1, id2, ...)) addTablePrefix := func(prefix roachpb.Key, id uint32, rowVals ...string) roachpb.Key { - tableKey := append(prefix, keys.MakeTablePrefix(id)...) + tableKey := append(prefix, keys.SystemSQLCodec.TablePrefix(id)...) rowKey := roachpb.Key(encoding.EncodeVarintAscending(tableKey, 1)) for _, rowVal := range rowVals { rowKey = encoding.EncodeStringAscending(rowKey, rowVal) @@ -4333,7 +4333,7 @@ func TestFindValidSplitKeys(t *testing.T) { addColFam(tablePrefix(1, "some", "data"), 1), addColFam(tablePrefix(keys.MaxSystemConfigDescID, "blah"), 1), }, - rangeStart: keys.MakeTablePrefix(1), + rangeStart: keys.SystemSQLCodec.TablePrefix(1), expSplit: nil, expError: false, }, @@ -4454,7 +4454,7 @@ func TestFindValidSplitKeys(t *testing.T) { addColFam(tablePrefix(userID, "c"), 1), addColFam(tablePrefix(userID, "d"), 1), }, - rangeStart: keys.MakeTablePrefix(userID), + rangeStart: keys.SystemSQLCodec.TablePrefix(userID), expSplit: tablePrefix(userID, "c"), expError: false, }, @@ -4470,7 +4470,7 @@ func TestFindValidSplitKeys(t *testing.T) { addColFam(tablePrefix(userID, "b"), 1), addColFam(tablePrefix(userID, "c"), 1), }, - rangeStart: keys.MakeTablePrefix(keys.MinUserDescID), + rangeStart: keys.SystemSQLCodec.TablePrefix(keys.MinUserDescID), expSplit: tablePrefix(userID, "b"), expError: false, }, diff --git a/pkg/storage/row_counter.go b/pkg/storage/row_counter.go index 475ee2375652..e56024aeb7d2 100644 --- a/pkg/storage/row_counter.go +++ b/pkg/storage/row_counter.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/encoding" ) // RowCounter is a helper that counts how many distinct rows appear in the KVs @@ -52,12 +51,7 @@ func (r *RowCounter) Count(key roachpb.Key) error { r.prev = append(r.prev[:0], row...) - rest, tableID, err := keys.DecodeTablePrefix(row) - if err != nil { - return err - } - - _, indexID, err := encoding.DecodeUvarintAscending(rest) + _, tableID, indexID, err := keys.TODOSQLCodec.DecodeIndexPrefix(row) if err != nil { return err } @@ -65,7 +59,7 @@ func (r *RowCounter) Count(key roachpb.Key) error { if r.EntryCounts == nil { r.EntryCounts = make(map[uint64]int64) } - r.EntryCounts[roachpb.BulkOpSummaryID(tableID, indexID)]++ + r.EntryCounts[roachpb.BulkOpSummaryID(uint64(tableID), uint64(indexID))]++ if indexID == 1 { r.DeprecatedRows++ diff --git a/pkg/storage/sst_writer_test.go b/pkg/storage/sst_writer_test.go index cc265f119e30..18e5c90a4afa 100644 --- a/pkg/storage/sst_writer_test.go +++ b/pkg/storage/sst_writer_test.go @@ -24,7 +24,7 @@ import ( ) func makeIntTableKVs(numKeys, valueSize, maxRevisions int) []storage.MVCCKeyValue { - prefix := encoding.EncodeUvarintAscending(keys.MakeTablePrefix(uint32(100)), uint64(1)) + prefix := encoding.EncodeUvarintAscending(keys.SystemSQLCodec.TablePrefix(uint32(100)), uint64(1)) kvs := make([]storage.MVCCKeyValue, numKeys) r, _ := randutil.NewPseudoRand() diff --git a/pkg/testutils/keysutils/pretty_scanner.go b/pkg/testutils/keysutils/pretty_scanner.go index 6673967553f4..b69c013ea5c5 100644 --- a/pkg/testutils/keysutils/pretty_scanner.go +++ b/pkg/testutils/keysutils/pretty_scanner.go @@ -25,6 +25,8 @@ import ( // MakePrettyScannerForNamedTables create a PrettyScanner that, beside what the // PrettyScanner is generally able to decode, can also decode keys of the form // "///1/2/3/..." using supplied maps from names to ids. +// +// TODO(nvanbenschoten): support tenant SQL keys. func MakePrettyScannerForNamedTables( tableNameToID map[string]int, idxNameToID map[string]int, ) keysutil.PrettyScanner { @@ -65,7 +67,7 @@ func parseTableKeysAsAscendingInts( if !ok { panic(fmt.Sprintf("unknown table: %s", tableName)) } - output := roachpb.Key(keys.MakeTablePrefix(uint32(tableID))) + output := keys.TODOSQLCodec.TablePrefix(uint32(tableID)) if remainder == "" { return "", output } diff --git a/pkg/testutils/keysutils/pretty_scanner_test.go b/pkg/testutils/keysutils/pretty_scanner_test.go index d4b5d538fe62..72ea01b91cec 100644 --- a/pkg/testutils/keysutils/pretty_scanner_test.go +++ b/pkg/testutils/keysutils/pretty_scanner_test.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" ) @@ -30,19 +29,19 @@ func TestPrettyScanner(t *testing.T) { { prettyKey: "/Table/t1", expKey: func() roachpb.Key { - return keys.MakeTablePrefix(50) + return keys.SystemSQLCodec.TablePrefix(50) }, }, { prettyKey: "/Table/t1/pk", expKey: func() roachpb.Key { - return sqlbase.EncodeTableIDIndexID(nil /* key */, 50, 1) + return keys.SystemSQLCodec.IndexPrefix(50, 1) }, }, { prettyKey: "/Table/t1/pk/1/2/3", expKey: func() roachpb.Key { - k := sqlbase.EncodeTableIDIndexID(nil /* key */, 50, 1) + k := keys.SystemSQLCodec.IndexPrefix(50, 1) k = encoding.EncodeVarintAscending(k, 1) k = encoding.EncodeVarintAscending(k, 2) k = encoding.EncodeVarintAscending(k, 3) @@ -57,7 +56,7 @@ func TestPrettyScanner(t *testing.T) { { prettyKey: "/Table/t1/idx1/1/2/3", expKey: func() roachpb.Key { - k := sqlbase.EncodeTableIDIndexID(nil /* key */, 50, 5) + k := keys.SystemSQLCodec.IndexPrefix(50, 5) k = encoding.EncodeVarintAscending(k, 1) k = encoding.EncodeVarintAscending(k, 2) k = encoding.EncodeVarintAscending(k, 3) diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index a054ccbe9732..721be04bee00 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -14,7 +14,6 @@ import ( "context" gosql "database/sql" "fmt" - "math" "net" "sync" "testing" @@ -684,7 +683,7 @@ func (tc *TestCluster) FindRangeLeaseHolder( // kv scratch space (it doesn't overlap system spans or SQL tables). The range // is lazily split off on the first call to ScratchRange. func (tc *TestCluster) ScratchRange(t testing.TB) roachpb.Key { - scratchKey := keys.MakeTablePrefix(math.MaxUint32) + scratchKey := keys.TableDataMax if tc.scratchRangeID > 0 { return scratchKey } diff --git a/pkg/testutils/testcluster/testcluster_test.go b/pkg/testutils/testcluster/testcluster_test.go index 7d80a3ae83fd..9c2183edf8e1 100644 --- a/pkg/testutils/testcluster/testcluster_test.go +++ b/pkg/testutils/testcluster/testcluster_test.go @@ -60,7 +60,7 @@ func TestManualReplication(t *testing.T) { kvDB := tc.Servers[0].DB() tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "test") - tableStartKey := keys.MakeTablePrefix(uint32(tableDesc.ID)) + tableStartKey := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.ID)) leftRangeDesc, tableRangeDesc, err := tc.SplitRange(tableStartKey) if err != nil { t.Fatal(err)