From d0e2005a30142de7d2a53f63d46897117ed13934 Mon Sep 17 00:00:00 2001 From: Brian Dillmann Date: Wed, 30 Oct 2024 09:58:04 -0400 Subject: [PATCH] apiutil, roachpb: create utilities to map descriptors to ranges Previously each range correlated to a single table, or even a single index in a database, so all that was required to identify which tables, indexes were in the range were to look at the start key of the range and map it accordingly. With range coalescing however, it's possible for one, or many, tables, indexes and the like to reside within the same range. To properly identify the contents of a range, this PR adds the following utilities: 1. A utility function which turns a range into a span, and clamps it to its tenant's table space. 2. A utility function which takes the above spans and uses the catalog and new descriptor by span utility to turn those spans into a set of table descriptors ordered by id. 3. A utility function which transforms those table descriptors into a set of (database, table, index) names which deduplicate and identify each index uniquely. 4. A utility function, which merges the ranges and indexes into a map keyed by RangeID whose values are the above index names. 5. A primary entrypoint for consumers from which a set of ranges can be passed in and a mapping from those ranges to indexes can be returned. A variety of caveats come with this approach. It attempts to scan the desciptors all at once, but it still will scan a sizable portion of the descriptors table if the request is large enough. This makes no attempt to describe system information which does not have a descriptor. It will describe system tables which appear in the descriptors table, but it will not try to explain "tables" which do not have descriptors (example tsdb), or any other information stored in the keyspace without a descriptor (PseudoTableIDs, GossipKeys for example). Throughout this work, many existing utilities were duplicated, and then un-duplicated (`keys.TableDataMin`, `roachpb.Span.Overlap`, etc). If you see anything that seems to already exist, feel free to point it out accordingly. Epic: CRDB-43151 Fixes: #130997 Release note: None --- pkg/BUILD.bazel | 2 + pkg/roachpb/BUILD.bazel | 2 + pkg/roachpb/data.go | 30 ++++ pkg/roachpb/key_test.go | 108 +++++++++++++ pkg/roachpb/span_test.go | 75 +++++++++ pkg/server/apiutil/BUILD.bazel | 33 +++- pkg/server/apiutil/rangeutil.go | 202 ++++++++++++++++++++++++ pkg/server/apiutil/rangeutil_test.go | 225 +++++++++++++++++++++++++++ pkg/sql/catalog/descs/collection.go | 21 ++- 9 files changed, 692 insertions(+), 6 deletions(-) create mode 100644 pkg/roachpb/key_test.go create mode 100644 pkg/roachpb/span_test.go create mode 100644 pkg/server/apiutil/rangeutil.go create mode 100644 pkg/server/apiutil/rangeutil_test.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index bdf953d5406c..94e50194f224 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -323,6 +323,7 @@ ALL_TESTS = [ "//pkg/security/username:username_disallowed_imports_test", "//pkg/security/username:username_test", "//pkg/security:security_test", + "//pkg/server/apiutil:apiutil_test", "//pkg/server/application_api:application_api_test", "//pkg/server/authserver:authserver_test", "//pkg/server/debug/goroutineui:goroutineui_test", @@ -1657,6 +1658,7 @@ GO_TARGETS = [ "//pkg/security:security_test", "//pkg/server/apiconstants:apiconstants", "//pkg/server/apiutil:apiutil", + "//pkg/server/apiutil:apiutil_test", "//pkg/server/application_api:application_api", "//pkg/server/application_api:application_api_test", "//pkg/server/authserver:authserver", diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index e8f4eed43884..92d96122981a 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -60,6 +60,7 @@ go_test( srcs = [ "data_test.go", "index_usage_stats_test.go", + "key_test.go", "main_test.go", "merge_spans_test.go", "metadata_replicas_test.go", @@ -67,6 +68,7 @@ go_test( "span_config_conformance_report_test.go", "span_config_test.go", "span_group_test.go", + "span_test.go", "string_test.go", "tenant_test.go", "version_test.go", diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 5ad9ad1b19fe..7ec6470a9745 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -218,6 +218,23 @@ func (k Key) Compare(b Key) int { return bytes.Compare(k, b) } +// Less says whether key k is less than key b. +func (k Key) Less(b Key) bool { + return k.Compare(b) < 0 +} + +// Clamp fixes the key to something within the range a < k < b. +func (k Key) Clamp(a, b Key) Key { + result := k + if k.Less(a) { + result = a + } + if b.Less(k) { + result = b + } + return result +} + // SafeFormat implements the redact.SafeFormatter interface. func (k Key) SafeFormat(w redact.SafePrinter, _ rune) { SafeFormatKey(w, nil /* valDirs */, k) @@ -2360,6 +2377,19 @@ func (s Span) Equal(o Span) bool { return s.Key.Equal(o.Key) && s.EndKey.Equal(o.EndKey) } +// ZeroLength returns true if the distance between the start and end key is 0. +func (s Span) ZeroLength() bool { + return s.Key.Equal(s.EndKey) +} + +// Clamp clamps span s's keys within the span defined in bounds. +func (s Span) Clamp(bounds Span) Span { + return Span{ + s.Key.Clamp(bounds.Key, bounds.EndKey), + s.EndKey.Clamp(bounds.Key, bounds.EndKey), + } +} + // Overlaps returns true WLOG for span A and B iff: // 1. Both spans contain one key (just the start key) and they are equal; or // 2. The span with only one key is contained inside the other span; or diff --git a/pkg/roachpb/key_test.go b/pkg/roachpb/key_test.go new file mode 100644 index 000000000000..3cdb9406084f --- /dev/null +++ b/pkg/roachpb/key_test.go @@ -0,0 +1,108 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package roachpb_test + +import ( + "math" + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/encoding" +) + +func TestKeyClampTenants(t *testing.T) { + // tp = TablePrefix + tp := keys.MakeSQLCodec(roachpb.MustMakeTenantID(3)).TablePrefix + lowTp := keys.MakeSQLCodec(roachpb.MustMakeTenantID(1)).TablePrefix + highTp := keys.MakeSQLCodec(roachpb.MustMakeTenantID(5)).TablePrefix + sysTp := keys.SystemSQLCodec.TablePrefix + tests := []struct { + name string + k, a, b roachpb.Key + expected roachpb.Key + }{ + {"key within main tenant is unchanged", tp(5), tp(1), tp(10), tp(5)}, + {"low tenant codec gets clamped to lower bound", lowTp(5), tp(1), tp(10), tp(1)}, + {"high tenant codec gets clamped to upper bound", highTp(5), tp(1), tp(10), tp(10)}, + {"system codec occurs below the tenant table boundaries", sysTp(5), tp(1), tp(10), tp(1)}, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + result := tt.k.Clamp(tt.a, tt.b) + if !result.Equal(tt.expected) { + t.Errorf("Clamp(%v, %v, %v) = %v; want %v", tt.k, tt.a, tt.b, result, tt.expected) + } + }) + } +} + +func TestKeyClampTables(t *testing.T) { + // tp = TablePrefix + tp := keys.MakeSQLCodec(roachpb.MustMakeTenantID(3)).TablePrefix + tests := []struct { + name string + k, a, b roachpb.Key + expected roachpb.Key + }{ + {"table within prefix is unchanged", tp(5), tp(1), tp(10), tp(5)}, + {"low table gets clamped to lower bound", tp(0), tp(1), tp(10), tp(1)}, + {"high table gets clamped to upper bound", tp(11), tp(1), tp(10), tp(10)}, + {"low table on lower bound is unchanged", tp(1), tp(1), tp(10), tp(1)}, + {"high table on upper bound is unchanged", tp(10), tp(1), tp(10), tp(10)}, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + result := tt.k.Clamp(tt.a, tt.b) + if !result.Equal(tt.expected) { + t.Errorf("Clamp(%v, %v, %v) = %v; want %v", tt.k, tt.a, tt.b, result, tt.expected) + } + }) + } +} + +func TestKeyClampTenantTablespace(t *testing.T) { + timeseriesKeyPrefix := encoding.EncodeVarintAscending( + encoding.EncodeBytesAscending( + append(roachpb.Key(nil), keys.TimeseriesPrefix...), + []byte("my.fake.metric"), + ), + int64(10), + ) + tsKey := func(source string, timestamp int64) roachpb.Key { + return append(encoding.EncodeVarintAscending(timeseriesKeyPrefix, timestamp), source...) + } + + tp := keys.MakeSQLCodec(roachpb.MustMakeTenantID(3)).TablePrefix + lower := tp(0) + upper := tp(math.MaxUint32) + tests := []struct { + name string + k, a, b roachpb.Key + expected roachpb.Key + }{ + {"KeyMin gets clamped to lower", roachpb.KeyMin, lower, upper, lower}, + {"KeyMax gets clamped to upper", roachpb.KeyMax, lower, upper, upper}, + {"Meta1Prefix gets clamped to lower", keys.Meta1Prefix, lower, upper, lower}, + {"Meta2Prefix gets clamped to lower", keys.Meta2Prefix, lower, upper, lower}, + {"TableDataMin gets clamped to lower", keys.TableDataMin, lower, upper, lower}, + // below is an unexpected test case for a tenant codec + {"TableDataMax also gets clamped to lower", keys.TableDataMax, lower, upper, lower}, + {"SystemPrefix gets clamped to lower", keys.SystemPrefix, lower, upper, lower}, + {"TimeseriesKey gets clamped to lower", tsKey("5", 123), lower, upper, lower}, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + result := tt.k.Clamp(tt.a, tt.b) + if !result.Equal(tt.expected) { + t.Errorf("Clamp(%v, %v, %v) = %v; want %v", tt.k, tt.a, tt.b, result, tt.expected) + } + }) + } +} diff --git a/pkg/roachpb/span_test.go b/pkg/roachpb/span_test.go new file mode 100644 index 000000000000..a6214419446a --- /dev/null +++ b/pkg/roachpb/span_test.go @@ -0,0 +1,75 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package roachpb_test + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +func TestSpanZeroLength(t *testing.T) { + // create two separate references here. + shouldBeEmpty := roachpb.Span{ + Key: keys.SystemSQLCodec.TablePrefix(1), + EndKey: keys.SystemSQLCodec.TablePrefix(1), + } + if !shouldBeEmpty.ZeroLength() { + t.Fatalf("expected span %s to be empty.", shouldBeEmpty) + } + + shouldNotBeEmpty := roachpb.Span{ + Key: keys.SystemSQLCodec.TablePrefix(1), + EndKey: keys.SystemSQLCodec.TablePrefix(1).Next(), + } + if shouldNotBeEmpty.ZeroLength() { + t.Fatalf("expected span %s to not be empty.", shouldNotBeEmpty) + } +} + +func TestSpanClamp(t *testing.T) { + tp := keys.SystemSQLCodec.TablePrefix + tests := []struct { + name string + span roachpb.Span + bounds roachpb.Span + want roachpb.Span + }{ + { + name: "within bounds", + span: roachpb.Span{tp(5), tp(10)}, + bounds: roachpb.Span{tp(0), tp(15)}, + want: roachpb.Span{tp(5), tp(10)}, + }, + { + name: "clamp lower bound", + span: roachpb.Span{tp(0), tp(10)}, + bounds: roachpb.Span{tp(5), tp(15)}, + want: roachpb.Span{tp(5), tp(10)}, + }, + { + name: "clamp upper bound", + span: roachpb.Span{tp(5), tp(20)}, + bounds: roachpb.Span{tp(0), tp(15)}, + want: roachpb.Span{tp(5), tp(15)}, + }, + { + name: "clamp both bounds", + span: roachpb.Span{tp(0), tp(20)}, + bounds: roachpb.Span{tp(5), tp(15)}, + want: roachpb.Span{tp(5), tp(15)}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if got := tt.span.Clamp(tt.bounds); !got.Equal(tt.want) { + t.Errorf("Clamp() = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/pkg/server/apiutil/BUILD.bazel b/pkg/server/apiutil/BUILD.bazel index 0d6bba28b829..74fa6d52869c 100644 --- a/pkg/server/apiutil/BUILD.bazel +++ b/pkg/server/apiutil/BUILD.bazel @@ -1,9 +1,36 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "apiutil", - srcs = ["apiutil.go"], + srcs = [ + "apiutil.go", + "rangeutil.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/server/apiutil", visibility = ["//visibility:public"], - deps = ["//pkg/server/srverrors"], + deps = [ + "//pkg/keys", + "//pkg/roachpb", + "//pkg/server/srverrors", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "apiutil_test", + srcs = ["rangeutil_test.go"], + deps = [ + ":apiutil", + "//pkg/keys", + "//pkg/roachpb", + "//pkg/sql/catalog", + "//pkg/sql/catalog/dbdesc", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/sem/catid", + "@com_github_stretchr_testify//require", + ], ) diff --git a/pkg/server/apiutil/rangeutil.go b/pkg/server/apiutil/rangeutil.go new file mode 100644 index 000000000000..b1bc0e1846bf --- /dev/null +++ b/pkg/server/apiutil/rangeutil.go @@ -0,0 +1,202 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package apiutil + +import ( + "context" + "math" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/errors" +) + +// This file contains a set of helper functions which are useful for turning +// ranges into the SQL related contents which reside within. It includes: +// 1. A utility function which turns a range into a span, and clamps it +// to its tenant's table space. +// 2. A utility function which takes the above spans and uses the catalog +// and new descriptor by span utility to turn those spans into a set of +// table descriptors ordered by id. +// 3. A utility function which transforms those table descriptors into a +// set of (database, table, index) names which deduplicate and identify +// each index uniquely. +// 4. A utility function, which merges the ranges and indexes into a map +// keyed by RangeID whose values are the above index names. +// 5. A primary entrypoint for consumers from which a set of ranges can be +// passed in and a mapping from those ranges to indexes can be +// returned. + +type IndexNames struct { + Database string + Table string + Index string + Span roachpb.Span +} + +// Equal only compares the names, not the spans +func (idx IndexNames) Equal(other IndexNames) bool { + return idx.Database == other.Database && + idx.Table == other.Table && + idx.Index == other.Index +} + +// GetRangeIndexMappings translates a set of ordered ranges into a +// RangeID -> []IndexNames mapping. It does this by executing the fololowing steps: +// 1. Convert the set of ranges to a set of spans. +// 2. Get the table descriptors that fall within the given spans. +// 3. Get the database, table and index name for all indexes found in the descriptors. +// 4. Return a mapping of the indexes which appear in each range. +func GetRangeIndexMapping( + ctx context.Context, + txn descs.Txn, + codec keys.SQLCodec, + databases map[descpb.ID]catalog.DatabaseDescriptor, + ranges []roachpb.RangeDescriptor, +) (map[roachpb.RangeID][]IndexNames, error) { + spans := RangesToTableSpans(codec, ranges) + + tables, err := SpansToOrderedTableDescriptors(ctx, txn, spans) + if err != nil { + return nil, err + } + + indexes, err := TableDescriptorsToIndexNames(codec, databases, tables) + if err != nil { + return nil, err + } + + return MapRangesToIndexes(ranges, indexes), nil +} + +// MapRangesToIndexes is a utility function which iterates over two lists, +// one consisting of ordered ranges, and the other consisting of ordered index names +// and outputs a mapping from range to index. +func MapRangesToIndexes( + ranges []roachpb.RangeDescriptor, indexes []IndexNames, +) map[roachpb.RangeID][]IndexNames { + results := map[roachpb.RangeID][]IndexNames{} + contents := []IndexNames{} + flushToResults := func(rangeID roachpb.RangeID) { + results[rangeID] = contents + contents = []IndexNames{} + } + + // move through the ranges + descriptors + // using two indexes, i, j. + // while i and j are valid + i := 0 + j := 0 + for i < len(ranges) && j < len(indexes) { + rangeSpan := ranges[i].KeySpan().AsRawSpanWithNoLocals() + if rangeSpan.Overlaps(indexes[j].Span) { + contents = append(contents, indexes[j]) + } + + if ranges[i].EndKey.AsRawKey().Less(indexes[j].Span.EndKey) { + flushToResults(ranges[i].RangeID) + i++ + } else { + j++ + } + } + + if i < len(ranges) { + flushToResults(ranges[i].RangeID) + } + return results +} + +// RangeToTableSpans is a simple utility function which converts a set of ranges +// to a set of spans bound to the codec's SQL table space, and removed if the bound +// span is zero length. +func RangesToTableSpans(codec keys.SQLCodec, ranges []roachpb.RangeDescriptor) []roachpb.Span { + spans := []roachpb.Span{} + + // cannot use keys.TableDataMin/Max + // Check the following: keys.TableDataMax.Less(keys.MakeSQLCodec(3).TablePrefix(1)) == true + bounds := roachpb.Span{ + Key: codec.TablePrefix(0), + EndKey: codec.TablePrefix(math.MaxUint32), + } + for _, rangeDesc := range ranges { + span := rangeDesc.KeySpan().AsRawSpanWithNoLocals().Clamp(bounds) + if !span.ZeroLength() { + spans = append(spans, span) + } + } + + return spans +} + +// SpansToOrderedTableDescriptors uses the transaction's collection to turn a set of +// spans to a set of descriptors which describe the table space in which those spans lie. +func SpansToOrderedTableDescriptors( + ctx context.Context, txn descs.Txn, spans []roachpb.Span, +) ([]catalog.TableDescriptor, error) { + descriptors := []catalog.TableDescriptor{} + collection := txn.Descriptors() + nscatalog, err := collection.GetDescriptorsInSpans(ctx, txn.KV(), spans) + if err != nil { + return nil, err + } + + allDescriptors := nscatalog.OrderedDescriptors() + for _, iDescriptor := range allDescriptors { + if table, ok := iDescriptor.(catalog.TableDescriptor); ok { + descriptors = append(descriptors, table) + } + } + return descriptors, nil +} + +// TableDescriptorsToIndexNames is a simple function which maps a set of descriptors to the +// database, table, index combinations within. It assumes that every table +// has at least one index, the descriptors input are ordered, and that +// there can be duplicates of the descriptors. +func TableDescriptorsToIndexNames( + codec keys.SQLCodec, + databases map[descpb.ID]catalog.DatabaseDescriptor, + tables []catalog.TableDescriptor, +) ([]IndexNames, error) { + seen := map[string]struct{}{} + indexes := []IndexNames{} + + for _, table := range tables { + database, ok := databases[table.GetParentID()] + if !ok { + return nil, errors.Errorf("could not find database for table %s", table.GetName()) + } + for _, index := range table.AllIndexes() { + key := database.GetName() + table.GetName() + index.GetName() + if _, ok := seen[key]; ok { + continue + } + seen[key] = struct{}{} + indexes = append(indexes, IndexNames{ + Database: database.GetName(), + Table: table.GetName(), + Index: index.GetName(), + Span: spanFromIndex(codec, table, index), + }) + } + } + + return indexes, nil +} + +func spanFromIndex( + codec keys.SQLCodec, table catalog.TableDescriptor, index catalog.Index, +) roachpb.Span { + prefix := codec.IndexPrefix(uint32(table.GetID()), uint32(index.GetID())) + return roachpb.Span{ + Key: prefix, + EndKey: prefix.PrefixEnd(), + } +} diff --git a/pkg/server/apiutil/rangeutil_test.go b/pkg/server/apiutil/rangeutil_test.go new file mode 100644 index 000000000000..79f704ac3fcd --- /dev/null +++ b/pkg/server/apiutil/rangeutil_test.go @@ -0,0 +1,225 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package apiutil_test + +import ( + "encoding/json" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/apiutil" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/stretchr/testify/require" +) + +func TestMapRangesToIndexes(t *testing.T) { + k := func(c byte) roachpb.Key { + return roachpb.Key([]byte{c}) + } + rk := func(c byte) roachpb.RKey { + return roachpb.RKey(k(c)) + } + ranges := []roachpb.RangeDescriptor{ + {RangeID: 1, StartKey: rk('d'), EndKey: rk('m')}, + {RangeID: 2, StartKey: rk('m'), EndKey: rk('x')}, + } + + indexes := []apiutil.IndexNames{ + {Index: "Totally before first", Span: roachpb.Span{Key: k('a'), EndKey: k('c')}}, + {Index: "Start and before first", Span: roachpb.Span{Key: k('c'), EndKey: k('e')}}, + {Index: "Middle of first range", Span: roachpb.Span{Key: k('e'), EndKey: k('f')}}, + {Index: "Overlaps with both", Span: roachpb.Span{Key: k('f'), EndKey: k('o')}}, + {Index: "Middle of second range", Span: roachpb.Span{Key: k('o'), EndKey: k('q')}}, + {Index: "End and after second", Span: roachpb.Span{Key: k('q'), EndKey: k('y')}}, + {Index: "Totally after end", Span: roachpb.Span{Key: k('y'), EndKey: k('z')}}, + } + + expected := map[roachpb.RangeID][]apiutil.IndexNames{ + 1: { + {Index: "Start and before first"}, + {Index: "Middle of first range"}, + {Index: "Overlaps with both"}, + }, + 2: { + {Index: "Overlaps with both"}, + {Index: "Middle of second range"}, + {Index: "End and after second"}, + }, + } + + result := apiutil.MapRangesToIndexes(ranges, indexes) + + require.Equal(t, len(result), len(expected)) + b, _ := json.MarshalIndent(result, "", "\t") + fmt.Println(string(b)) + b, _ = json.MarshalIndent(indexes, "", "\t") + fmt.Println(string(b)) + + for rangeID, expectedIndexes := range expected { + actualIndexes, ok := result[rangeID] + require.True(t, ok) + require.Equal(t, len(actualIndexes), len(expectedIndexes)) + for i, expectedIndex := range expectedIndexes { + fmt.Println(rangeID, i, expectedIndex, actualIndexes[i]) + require.Equal(t, expectedIndex.Index, actualIndexes[i].Index) + } + } +} + +func TestRangesToTableSpans(t *testing.T) { + codec := keys.MakeSQLCodec(roachpb.MustMakeTenantID(1)) + ranges := []roachpb.RangeDescriptor{ + // should be zero len + { + StartKey: roachpb.RKey(codec.TablePrefix(0).Prevish(1)), + EndKey: roachpb.RKey(codec.TablePrefix(0)), + }, + // should also be zero len + { + StartKey: roachpb.RKey(codec.TablePrefix(1)), + EndKey: roachpb.RKey(codec.TablePrefix(1)), + }, + { + StartKey: roachpb.RKey(codec.TablePrefix(1)), + EndKey: roachpb.RKey(codec.TablePrefix(3)), + }, + { + StartKey: roachpb.RKey(codec.TablePrefix(3)), + EndKey: roachpb.RKey(codec.TablePrefix(5)), + }, + { + StartKey: roachpb.RKey(codec.TablePrefix(5)), + EndKey: roachpb.RKey(codec.TablePrefix(6)), + }, + } + + expected := []roachpb.Span{ + { + Key: codec.TablePrefix(1), + EndKey: codec.TablePrefix(3), + }, + { + Key: codec.TablePrefix(3), + EndKey: codec.TablePrefix(5), + }, + { + Key: codec.TablePrefix(5), + EndKey: codec.TablePrefix(6), + }, + } + + result := apiutil.RangesToTableSpans(codec, ranges) + + require.Equal(t, len(result), len(expected)) + + for i, span := range expected { + if !result[i].Equal(span) { + t.Fatalf("expected span %v, got %v", span, result[i]) + } + } +} + +func makeDBDesc(id uint32, name string) catalog.DatabaseDescriptor { + + db := &dbdesc.Mutable{} + db.SetName(name) + db.ID = catid.DescID(id) + descriptor := db.ImmutableCopy() + return descriptor.(catalog.DatabaseDescriptor) +} + +func makeTableDesc(databaseID uint32, name string, indexes []string) catalog.TableDescriptor { + descIndexes := []descpb.IndexDescriptor{} + table := tabledesc.NewBuilder(&descpb.TableDescriptor{ + Name: name, + ParentID: catid.DescID(databaseID), + Indexes: descIndexes, + }).BuildCreatedMutableTable() + for i, name := range indexes { + if i == 0 { + _ = table.AddPrimaryIndex(descpb.IndexDescriptor{Name: name}) + } else { + + _ = table.AddSecondaryIndex(descpb.IndexDescriptor{Name: name}) + } + } + return table.NewBuilder().BuildImmutable().(catalog.TableDescriptor) +} + +func TestTableDescriptorsToIndexNames(t *testing.T) { + // test straightforward path with three tables, two databases + codec := keys.MakeSQLCodec(roachpb.MustMakeTenantID(1)) + databases := map[descpb.ID]catalog.DatabaseDescriptor{ + 1: makeDBDesc(1, "db1"), + 2: makeDBDesc(2, "db2"), + } + tables := []catalog.TableDescriptor{ + makeTableDesc(1, "table1", []string{"pkey"}), + makeTableDesc(2, "table2", []string{"pkey", "table2_secondary_column"}), + makeTableDesc(2, "table3", []string{"pkey"}), + } + + expected := []apiutil.IndexNames{ + {Database: "db1", Table: "table1", Index: "pkey"}, + {Database: "db2", Table: "table2", Index: "pkey"}, + {Database: "db2", Table: "table2", Index: "table2_secondary_column"}, + {Database: "db2", Table: "table3", Index: "pkey"}, + } + indexes, err := apiutil.TableDescriptorsToIndexNames(codec, databases, tables) + + require.NoError(t, err) + require.Equal(t, len(expected), len(indexes)) + for i, index := range indexes { + if !index.Equal(expected[i]) { + t.Fatalf("resulting index did not match expected output: %s %s", index, expected[i]) + } + } +} + +func TestTableDescriptorsToIndexNamesDeduplicates(t *testing.T) { + // verify that duplicate descriptors are de-duplicated + codec := keys.MakeSQLCodec(roachpb.MustMakeTenantID(1)) + databases := map[descpb.ID]catalog.DatabaseDescriptor{ + 1: makeDBDesc(1, "db1"), + } + tables := []catalog.TableDescriptor{ + makeTableDesc(1, "table1", []string{"pkey", "table1_secondary_column"}), + makeTableDesc(1, "table1", []string{"pkey", "table1_secondary_column"}), + } + + expected := []apiutil.IndexNames{ + {Database: "db1", Table: "table1", Index: "pkey"}, + {Database: "db1", Table: "table1", Index: "table1_secondary_column"}, + } + indexes, err := apiutil.TableDescriptorsToIndexNames(codec, databases, tables) + + require.NoError(t, err) + require.Equal(t, len(expected), len(indexes)) + for i, index := range indexes { + if !index.Equal(expected[i]) { + t.Fatalf("resulting index did not match expected output: %s %s", index, expected[i]) + } + } +} + +func TestGetIndexNamesFromDescriptorsMissingDatabase(t *testing.T) { + codec := keys.MakeSQLCodec(roachpb.MustMakeTenantID(1)) + databases := map[descpb.ID]catalog.DatabaseDescriptor{ + 1: makeDBDesc(1, "db1"), + } + tables := []catalog.TableDescriptor{ + makeTableDesc(2, "table2", []string{"pkey", "table2_secondary_column"}), + } + + _, err := apiutil.TableDescriptorsToIndexNames(codec, databases, tables) + require.Errorf(t, err, "could not find database for table %s", tables[0].GetName()) +} diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 898db4063400..61fabafc93ee 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -1132,9 +1132,24 @@ func (tc *Collection) GetAllDatabaseDescriptors( return ret, nil } -// GetSchemasForDatabase returns the schemas for a given database -// visible by the transaction. -// Deprecated: prefer GetAllSchemasInDatabase. +// GetAllDatabaseDescriptorsMap returns the results of GetAllDatabaseDescriptors +func (tc *Collection) GetAllDatabaseDescriptorsMap( + ctx context.Context, txn *kv.Txn, +) (ret map[descpb.ID]catalog.DatabaseDescriptor, _ error) { + descriptors, err := tc.GetAllDatabaseDescriptors(ctx, txn) + result := map[descpb.ID]catalog.DatabaseDescriptor{} + if err != nil { + return nil, err + } + + for _, descriptor := range descriptors { + result[descriptor.GetID()] = descriptor + } + + return result, nil +} + +// but as a map with the database ID as the key. func (tc *Collection) GetSchemasForDatabase( ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, ) (map[descpb.ID]string, error) {