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..53016160364b 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 bytes.Compare(k, b) == -1 +} + +// 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..c06c105e7655 --- /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 im not sure", 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/metadata.go b/pkg/roachpb/metadata.go index 26cfc941408d..3e7dde5ff673 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -1006,3 +1006,18 @@ func (h *GCHint) advanceGCTimestamp(gcThreshold hlc.Timestamp) bool { h.GCTimestamp, h.GCTimestampNext = hlc.Timestamp{}, hlc.Timestamp{} return true } + +type RangeDescriptorsByStartKey []RangeDescriptor + +func (r RangeDescriptorsByStartKey) Len() int { + return len(r) +} +func (r RangeDescriptorsByStartKey) Less(i, j int) bool { + return r[i].StartKey.AsRawKey().Less(r[j].StartKey.AsRawKey()) +} + +func (r RangeDescriptorsByStartKey) Swap(i, j int) { + tmp := r[i] + r[i] = r[j] + r[j] = tmp +} diff --git a/pkg/roachpb/metadata_test.go b/pkg/roachpb/metadata_test.go index 6970c44dc23e..77fabc4ecde5 100644 --- a/pkg/roachpb/metadata_test.go +++ b/pkg/roachpb/metadata_test.go @@ -8,6 +8,7 @@ package roachpb import ( "fmt" "reflect" + "sort" "strings" "testing" @@ -662,3 +663,24 @@ func TestGCHint(t *testing.T) { }) } } + +func TestRangeDescriptorsByStartKey(t *testing.T) { + // table-prefix-range-key + tprk := func(t byte) RKey { + return RKey(Key([]byte{t})) + } + ranges := []RangeDescriptor{ + {StartKey: tprk(2), EndKey: tprk(7)}, + {StartKey: tprk(5), EndKey: tprk(5)}, + {StartKey: tprk(7), EndKey: tprk(2)}, + {StartKey: tprk(1), EndKey: tprk(10)}, + {StartKey: tprk(5), EndKey: tprk(5)}, + } + sort.Stable(RangeDescriptorsByStartKey(ranges)) + + for i := 0; i < len(ranges)-1; i++ { + if ranges[i+1].StartKey.AsRawKey().Less(ranges[i].StartKey.AsRawKey()) { + t.Fatalf("expected ranges to be ordered increasing by start key, failed on %d, %d with keys %s, %s", i, i+1, ranges[i].StartKey.AsRawKey(), ranges[i+1].StartKey.AsRawKey()) + } + } +} 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..4a28ddc1f817 --- /dev/null +++ b/pkg/server/apiutil/rangeutil.go @@ -0,0 +1,189 @@ +// 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" +) + +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, codec, 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) { + if spanFromRange(ranges[i]).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 := spanFromRange(rangeDesc).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, codec keys.SQLCodec, 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]bool{} + 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] = true + indexes = append(indexes, IndexNames{ + Database: database.GetName(), + Table: table.GetName(), + Index: index.GetName(), + Span: spanFromIndex(codec, table, index), + }) + } + } + + return indexes, nil +} + +func spanFromRange(rangeDesc roachpb.RangeDescriptor) roachpb.Span { + return rangeDesc.KeySpan().AsRawSpanWithNoLocals() +} + +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..1cd601c9ef58 --- /dev/null +++ b/pkg/server/apiutil/rangeutil_test.go @@ -0,0 +1,228 @@ +// 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 ( + "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) + + if len(result) != len(expected) { + t.Fatalf("expected %d ranges, got %d", len(expected), len(result)) + } + + for rangeID, expectedIndexes := range expected { + actualIndexes, ok := result[rangeID] + if !ok { + t.Fatalf("expected rangeID %d not found", rangeID) + } + if len(actualIndexes) != len(expectedIndexes) { + t.Fatalf("expected %d indexes for rangeID %d, got %d", len(expectedIndexes), rangeID, len(actualIndexes)) + } + for i, expectedIndex := range expectedIndexes { + if actualIndexes[i].Index != expectedIndex.Index { + t.Fatalf("expected index %v, got %v", expectedIndex, actualIndexes[i]) + } + } + } +} + +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)), + }, + } + + expectedSpans := []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) + + if len(result) != len(expectedSpans) { + t.Fatalf("expected %d spans, got %d", len(expectedSpans), len(result)) + } + + for i, expectedSpan := range expectedSpans { + if !result[i].Equal(expectedSpan) { + t.Fatalf("expected span %v, got %v", expectedSpan, 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) {