From 9908abc666a446a9c42917c749256e755dcf4aa3 Mon Sep 17 00:00:00 2001 From: Brian Dillmann Date: Wed, 30 Oct 2024 09:58:04 -0400 Subject: [PATCH 1/2] 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 cavets 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: none Fixes: #130997 Release note: None --- pkg/roachpb/metadata.go | 15 +++++++++++++++ pkg/roachpb/metadata_test.go | 22 ++++++++++++++++++++++ 2 files changed, 37 insertions(+) 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()) + } + } +} From a609fd6aeb5f9dca8a0f12fcec841b68c158563d Mon Sep 17 00:00:00 2001 From: Brian Dillmann Date: Fri, 1 Nov 2024 16:28:04 -0400 Subject: [PATCH 2/2] ui, server: modify hot ranges api and table to use new contents approx. This change is the last in a set of commits to change the hot ranges page from only showing one table, index per range to many. It builds on top of the changes in the catalog reader (10b9ee03fba37f314771d8624b2619f10b5bbb59) and the range utilities (109219d5fca6d347663127bf21b8dd162ef51d4a) to surface a set of tables, indexes for each range. The primary changes in this commit specifically are the modification of the status server to use the new `rangeutil` utilities, and changing the wire, presentation format of the information. Epic: CRDB-43151 Fixes: #130997 Release note (bug fix): changes the table, index contents of the hot ranges page in DB console. --- docs/generated/http/full.md | 9 +- pkg/roachpb/BUILD.bazel | 1 + pkg/roachpb/data.go | 31 +++-- pkg/roachpb/key_test.go | 25 +++- pkg/roachpb/span_test.go | 17 ++- pkg/server/BUILD.bazel | 1 - pkg/server/apiutil/BUILD.bazel | 6 +- pkg/server/apiutil/index_names.go | 91 ++++++++++++++ pkg/server/apiutil/index_names_test.go | 119 ++++++++++++++++++ pkg/server/apiutil/rangeutil.go | 82 ++++++------ pkg/server/apiutil/rangeutil_test.go | 3 +- pkg/server/serverpb/status.proto | 14 ++- pkg/server/status.go | 112 ++++------------- pkg/sql/catalog/descs/collection.go | 10 +- .../catalog/internal/catkv/catalog_reader.go | 9 +- .../src/views/hotRanges/hotRangesTable.tsx | 25 +--- 16 files changed, 374 insertions(+), 181 deletions(-) create mode 100644 pkg/server/apiutil/index_names.go create mode 100644 pkg/server/apiutil/index_names_test.go diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index fabce1b5539f..31d96353028d 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -3653,9 +3653,9 @@ HotRange message describes a single hot range, ie its QPS, node ID it belongs to | range_id | [int32](#cockroach.server.serverpb.HotRangesResponseV2-int32) | | range_id indicates Range ID that's identified as hot range. | [reserved](#support-status) | | node_id | [int32](#cockroach.server.serverpb.HotRangesResponseV2-int32) | | node_id indicates the node that contains the current hot range. | [reserved](#support-status) | | qps | [double](#cockroach.server.serverpb.HotRangesResponseV2-double) | | qps (queries per second) shows the amount of queries that interact with current range. | [reserved](#support-status) | -| table_name | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | | table_name indicates the SQL table that the range belongs to. | [reserved](#support-status) | -| database_name | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | | database_name indicates on database that has current hot range. | [reserved](#support-status) | -| index_name | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | | index_name indicates the index name for current range. | [reserved](#support-status) | +| table_name | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | | table_name has been deprecated in favor of tables = 16; | [reserved](#support-status) | +| database_name | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | | database_name has been deprecated in favor of databases = 17; | [reserved](#support-status) | +| index_name | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | | index_name has been deprecated in favor of indexes = 17; | [reserved](#support-status) | | replica_node_ids | [int32](#cockroach.server.serverpb.HotRangesResponseV2-int32) | repeated | replica_node_ids specifies the list of node ids that contain replicas with current hot range. | [reserved](#support-status) | | leaseholder_node_id | [int32](#cockroach.server.serverpb.HotRangesResponseV2-int32) | | leaseholder_node_id indicates the Node ID that is the current leaseholder for the given range. | [reserved](#support-status) | | schema_name | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | | schema_name provides the name of schema (if exists) for table in current range. | [reserved](#support-status) | @@ -3665,6 +3665,9 @@ HotRange message describes a single hot range, ie its QPS, node ID it belongs to | write_bytes_per_second | [double](#cockroach.server.serverpb.HotRangesResponseV2-double) | | write_bytes_per_second is the recent number of bytes written per second on this range. | [reserved](#support-status) | | read_bytes_per_second | [double](#cockroach.server.serverpb.HotRangesResponseV2-double) | | read_bytes_per_second is the recent number of bytes read per second on this range. | [reserved](#support-status) | | cpu_time_per_second | [double](#cockroach.server.serverpb.HotRangesResponseV2-double) | | CPU time (ns) per second is the recent cpu usage per second on this range. | [reserved](#support-status) | +| databases | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | repeated | Databases for the range. | [reserved](#support-status) | +| tables | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | repeated | Tables for the range | [reserved](#support-status) | +| indexes | [string](#cockroach.server.serverpb.HotRangesResponseV2-string) | repeated | Indexes for the range | [reserved](#support-status) | diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index 92d96122981a..c2a6d7637475 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -84,6 +84,7 @@ go_test( "//pkg/raft/raftpb", "//pkg/raft/tracker", "//pkg/storage/enginepb", + "//pkg/testutils", "//pkg/testutils/zerofields", "//pkg/util", "//pkg/util/bitarray", diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 7ec6470a9745..2ba82c4b135a 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -224,15 +224,18 @@ func (k Key) Less(b Key) bool { } // Clamp fixes the key to something within the range a < k < b. -func (k Key) Clamp(a, b Key) Key { +func (k Key) Clamp(min, max Key) (Key, error) { + if max.Less(min) { + return nil, errors.Newf("cannot clamp when min '%s' is larger than max '%s'", min, max) + } result := k - if k.Less(a) { - result = a + if k.Less(min) { + result = min } - if b.Less(k) { - result = b + if max.Less(k) { + result = max } - return result + return result, nil } // SafeFormat implements the redact.SafeFormatter interface. @@ -2383,11 +2386,19 @@ func (s Span) ZeroLength() bool { } // 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), +func (s Span) Clamp(bounds Span) (Span, error) { + start, err := s.Key.Clamp(bounds.Key, bounds.EndKey) + if err != nil { + return Span{}, err } + end, err := s.EndKey.Clamp(bounds.Key, bounds.EndKey) + if err != nil { + return Span{}, err + } + return Span{ + Key: start, + EndKey: end, + }, nil } // Overlaps returns true WLOG for span A and B iff: diff --git a/pkg/roachpb/key_test.go b/pkg/roachpb/key_test.go index 3cdb9406084f..55a2883ed78f 100644 --- a/pkg/roachpb/key_test.go +++ b/pkg/roachpb/key_test.go @@ -11,7 +11,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/stretchr/testify/require" ) func TestKeyClampTenants(t *testing.T) { @@ -33,7 +35,8 @@ func TestKeyClampTenants(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - result := tt.k.Clamp(tt.a, tt.b) + result, err := tt.k.Clamp(tt.a, tt.b) + require.NoError(t, err) if !result.Equal(tt.expected) { t.Errorf("Clamp(%v, %v, %v) = %v; want %v", tt.k, tt.a, tt.b, result, tt.expected) } @@ -58,7 +61,8 @@ func TestKeyClampTables(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - result := tt.k.Clamp(tt.a, tt.b) + result, err := tt.k.Clamp(tt.a, tt.b) + require.NoError(t, err) if !result.Equal(tt.expected) { t.Errorf("Clamp(%v, %v, %v) = %v; want %v", tt.k, tt.a, tt.b, result, tt.expected) } @@ -99,10 +103,25 @@ func TestKeyClampTenantTablespace(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - result := tt.k.Clamp(tt.a, tt.b) + result, err := tt.k.Clamp(tt.a, tt.b) + require.NoError(t, err) if !result.Equal(tt.expected) { t.Errorf("Clamp(%v, %v, %v) = %v; want %v", tt.k, tt.a, tt.b, result, tt.expected) } }) } } + +func TestKeyClampError(t *testing.T) { + // verify that max < min throws error + a, b := roachpb.Key([]byte{'a'}), roachpb.Key([]byte{'b'}) + expected := `cannot clamp when min '"b"' is larger than max '"a"'` + _, err := a.Clamp(b, a) + if !testutils.IsError(err, expected) { + t.Fatalf("expected error to be '%s', got '%s'", expected, err) + } + + // verify that max = min throws no error + _, err = a.Clamp(a, a) + require.NoError(t, err) +} diff --git a/pkg/roachpb/span_test.go b/pkg/roachpb/span_test.go index a6214419446a..baaecaf6ac02 100644 --- a/pkg/roachpb/span_test.go +++ b/pkg/roachpb/span_test.go @@ -10,6 +10,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" ) func TestSpanZeroLength(t *testing.T) { @@ -38,6 +39,7 @@ func TestSpanClamp(t *testing.T) { span roachpb.Span bounds roachpb.Span want roachpb.Span + error string }{ { name: "within bounds", @@ -63,12 +65,23 @@ func TestSpanClamp(t *testing.T) { bounds: roachpb.Span{tp(5), tp(15)}, want: roachpb.Span{tp(5), tp(15)}, }, + { + name: "clamp start error", + span: roachpb.Span{}, + bounds: roachpb.Span{tp(2), tp(1)}, + want: roachpb.Span{}, + error: "cannot clamp when min '/Table/2' is larger than max '/Table/1'", + }, } 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) + span, err := tt.span.Clamp(tt.bounds) + if !testutils.IsError(err, tt.error) { + t.Fatalf("expected error to be '%s', got '%s'", tt.error, err) + } + if !span.Equal(tt.want) { + t.Errorf("Clamp() = %v, want %v", span, tt.want) } }) } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index d98c78705fb7..2057f3affa93 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -209,7 +209,6 @@ go_library( "//pkg/sql", "//pkg/sql/appstatspb", "//pkg/sql/auditlogging", - "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catsessiondata", diff --git a/pkg/server/apiutil/BUILD.bazel b/pkg/server/apiutil/BUILD.bazel index 74fa6d52869c..57a42fc48b17 100644 --- a/pkg/server/apiutil/BUILD.bazel +++ b/pkg/server/apiutil/BUILD.bazel @@ -4,6 +4,7 @@ go_library( name = "apiutil", srcs = [ "apiutil.go", + "index_names.go", "rangeutil.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/server/apiutil", @@ -21,7 +22,10 @@ go_library( go_test( name = "apiutil_test", - srcs = ["rangeutil_test.go"], + srcs = [ + "index_names_test.go", + "rangeutil_test.go", + ], deps = [ ":apiutil", "//pkg/keys", diff --git a/pkg/server/apiutil/index_names.go b/pkg/server/apiutil/index_names.go new file mode 100644 index 000000000000..b68189171949 --- /dev/null +++ b/pkg/server/apiutil/index_names.go @@ -0,0 +1,91 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package apiutil + +import ( + "fmt" + "strings" + + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// IndexNames and IndexNamesList are utilities for representing an +// index span by its corresponding identifiers. +// They include the underlying span for comparison against other spans +// and omit the server and schema parts of their four part name. +type IndexNames struct { + Database string + Table string + Index string + Span roachpb.Span +} + +func (idx IndexNames) String() string { + return fmt.Sprintf("%s.%s.%s", idx.Database, idx.Table, idx.Index) +} + +type IndexNamesList []IndexNames + +// quoteIfContainsDot adds quotes to an identifier if it contains +// the four part delimiter '.'. +func quoteIfContainsDot(identifier string) string { + if strings.Contains(identifier, ".") { + return `"` + identifier + `"` + } + return identifier +} + +// ToOutput is a simple function which returns a set of +// de-duplicated, fully referenced database, table, and index names +// depending on the number of databases and tables which appear. +func (idxl IndexNamesList) ToOutput() ([]string, []string, []string) { + fpi := quoteIfContainsDot + seenDatabases := map[string]bool{} + databases := []string{} + for _, idx := range idxl { + database := fpi(idx.Database) + if !seenDatabases[database] { + seenDatabases[database] = true + databases = append(databases, database) + } + } + + multipleDatabases := len(databases) > 1 + seenTables := map[string]bool{} + tables := []string{} + for _, idx := range idxl { + table := fpi(idx.Table) + if multipleDatabases { + table = fpi(idx.Database) + "." + table + } + if !seenTables[table] { + seenTables[table] = true + tables = append(tables, table) + } + } + + multipleTables := len(tables) > 1 + indexes := []string{} + for _, idx := range idxl { + index := fpi(idx.Index) + if multipleTables { + index = fpi(idx.Table) + "." + index + if multipleDatabases { + index = fpi(idx.Database) + "." + index + } + } + indexes = append(indexes, index) + } + + return databases, tables, indexes +} + +// 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 +} diff --git a/pkg/server/apiutil/index_names_test.go b/pkg/server/apiutil/index_names_test.go new file mode 100644 index 000000000000..b24ecfff5cd5 --- /dev/null +++ b/pkg/server/apiutil/index_names_test.go @@ -0,0 +1,119 @@ +// 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 ( + "reflect" + "testing" + + "github.com/cockroachdb/cockroach/pkg/server/apiutil" +) + +func TestToOutput(t *testing.T) { + tests := []struct { + name string + input apiutil.IndexNamesList + expected struct { + databases []string + tables []string + indexes []string + } + }{ + { + name: "Single database, single table", + input: apiutil.IndexNamesList{ + {Database: "db1", Table: "table1", Index: "index1"}, + }, + expected: struct { + databases []string + tables []string + indexes []string + }{ + databases: []string{"db1"}, + tables: []string{"table1"}, + indexes: []string{"index1"}, + }, + }, + { + name: "Single database, multiple tables", + input: apiutil.IndexNamesList{ + {Database: "db1", Table: "table1", Index: "index1"}, + {Database: "db1", Table: "table2", Index: "index2"}, + }, + expected: struct { + databases []string + tables []string + indexes []string + }{ + databases: []string{"db1"}, + tables: []string{"table1", "table2"}, + indexes: []string{"table1.index1", "table2.index2"}, + }, + }, + { + name: "Multiple databases, multiple tables", + input: apiutil.IndexNamesList{ + {Database: "db1", Table: "table1", Index: "index1"}, + {Database: "db2", Table: "table2", Index: "index2"}, + }, + expected: struct { + databases []string + tables []string + indexes []string + }{ + databases: []string{"db1", "db2"}, + tables: []string{"db1.table1", "db2.table2"}, + indexes: []string{"db1.table1.index1", "db2.table2.index2"}, + }, + }, + { + name: "Duplicate entries", + input: apiutil.IndexNamesList{ + {Database: "db1", Table: "table1", Index: "index1"}, + {Database: "db1", Table: "table1", Index: "index1"}, + }, + expected: struct { + databases []string + tables []string + indexes []string + }{ + databases: []string{"db1"}, + tables: []string{"table1"}, + indexes: []string{"index1", "index1"}, + }, + }, + { + name: "Identifiers with dot in the name", + input: apiutil.IndexNamesList{ + {Database: "db.1", Table: "table.1", Index: "index1"}, + }, + expected: struct { + databases []string + tables []string + indexes []string + }{ + databases: []string{"\"db.1\""}, + tables: []string{"\"table.1\""}, + indexes: []string{"index1"}, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + databases, tables, indexes := tt.input.ToOutput() + if !reflect.DeepEqual(databases, tt.expected.databases) { + t.Errorf("expected databases %v, got %v", tt.expected.databases, databases) + } + if !reflect.DeepEqual(tables, tt.expected.tables) { + t.Errorf("expected tables %v, got %v", tt.expected.tables, tables) + } + if !reflect.DeepEqual(indexes, tt.expected.indexes) { + t.Errorf("expected indexes %v, got %v", tt.expected.indexes, indexes) + } + }) + } +} diff --git a/pkg/server/apiutil/rangeutil.go b/pkg/server/apiutil/rangeutil.go index b80fb9ed3f7d..e0269a39531e 100644 --- a/pkg/server/apiutil/rangeutil.go +++ b/pkg/server/apiutil/rangeutil.go @@ -1,4 +1,4 @@ -// Copyright 2023 The Cockroach Authors. +// Copyright 2024 The Cockroach Authors. // // Use of this software is governed by the CockroachDB Software License // included in the /LICENSE file. @@ -8,6 +8,7 @@ package apiutil import ( "context" "math" + "sort" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -33,37 +34,33 @@ import ( // passed in and a mapping from those ranges to indexes can be // returned. -// IndexNames is a set of identifiers for an index span. -// It includes the database and table from the span's four part name -// as well as the index name itself. -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 +// rangeLess encapsulate a slice of ranges and returns a comparator function +// so that it can be sorted by go's builtin sort package. +func rangeLess(ranges []roachpb.RangeDescriptor) func(i, j int) bool { + return func(i, j int) bool { + return ranges[i].StartKey.Less(ranges[j].StartKey) + } } // 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. +// RangeID -> IndexNamesList mapping. It does this by executing the following steps: +// 1. Sort the incoming ranges by start key +// 2. Convert the set of ranges to a set of spans. +// 3. Get the table descriptors that fall within the given spans. +// 4. Get the database, table and index name for all indexes found in the descriptors. +// 5. 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) +) (map[roachpb.RangeID]IndexNamesList, error) { + sort.Slice(ranges, rangeLess(ranges)) + spans, err := RangesToTableSpans(codec, ranges) + if err != nil { + return nil, err + } tables, err := SpansToOrderedTableDescriptors(ctx, txn, spans) if err != nil { @@ -82,13 +79,13 @@ func GetRangeIndexMapping( // 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{} + ranges []roachpb.RangeDescriptor, indexes IndexNamesList, +) map[roachpb.RangeID]IndexNamesList { + results := map[roachpb.RangeID]IndexNamesList{} + contents := IndexNamesList{} flushToResults := func(rangeID roachpb.RangeID) { results[rangeID] = contents - contents = []IndexNames{} + contents = IndexNamesList{} } // move through the ranges + descriptors @@ -116,10 +113,12 @@ func MapRangesToIndexes( 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 { +// RangeToTableSpans 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, error) { spans := []roachpb.Span{} // cannot use keys.TableDataMin/Max @@ -129,13 +128,16 @@ func RangesToTableSpans(codec keys.SQLCodec, ranges []roachpb.RangeDescriptor) [ EndKey: codec.TablePrefix(math.MaxUint32), } for _, rangeDesc := range ranges { - span := rangeDesc.KeySpan().AsRawSpanWithNoLocals().Clamp(bounds) + span, err := rangeDesc.KeySpan().AsRawSpanWithNoLocals().Clamp(bounds) + if err != nil { + return nil, err + } if !span.ZeroLength() { spans = append(spans, span) } } - return spans + return spans, nil } // SpansToOrderedTableDescriptors uses the transaction's collection to turn a set of @@ -159,17 +161,17 @@ func SpansToOrderedTableDescriptors( 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. +// TableDescriptorsToIndexNames 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) { +) (IndexNamesList, error) { seen := map[string]struct{}{} - indexes := []IndexNames{} + indexes := IndexNamesList{} for _, table := range tables { database, ok := databases[table.GetParentID()] diff --git a/pkg/server/apiutil/rangeutil_test.go b/pkg/server/apiutil/rangeutil_test.go index 79f704ac3fcd..4dabbea4a757 100644 --- a/pkg/server/apiutil/rangeutil_test.go +++ b/pkg/server/apiutil/rangeutil_test.go @@ -117,8 +117,9 @@ func TestRangesToTableSpans(t *testing.T) { }, } - result := apiutil.RangesToTableSpans(codec, ranges) + result, err := apiutil.RangesToTableSpans(codec, ranges) + require.NoError(t, err) require.Equal(t, len(result), len(expected)) for i, span := range expected { diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index a85c09072be1..cbeec3d0a58d 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -1484,11 +1484,11 @@ message HotRangesResponseV2 { double qps = 3 [ (gogoproto.customname) = "QPS" ]; - // table_name indicates the SQL table that the range belongs to. - string table_name = 4; - // database_name indicates on database that has current hot range. + // table_name has been deprecated in favor of tables = 16; + string table_name = 4 [deprecated = true]; + // database_name has been deprecated in favor of databases = 17; string database_name = 5; - // index_name indicates the index name for current range. + // index_name has been deprecated in favor of indexes = 17; string index_name = 6; // replica_node_ids specifies the list of node ids that contain replicas with current hot range. repeated int32 replica_node_ids = 7 [ @@ -1524,6 +1524,12 @@ message HotRangesResponseV2 { // CPU time (ns) per second is the recent cpu usage per second on this // range. double cpu_time_per_second = 15 [(gogoproto.customname) = "CPUTimePerSecond"]; + // Databases for the range. + repeated string databases = 16; + // Tables for the range + repeated string tables = 17; + // Indexes for the range + repeated string indexes = 18; } // Ranges contain list of hot ranges info that has highest number of QPS. repeated HotRange ranges = 1; diff --git a/pkg/server/status.go b/pkg/server/status.go index 194274923ff7..40b7fdd50fee 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/apiconstants" + "github.com/cockroachdb/cockroach/pkg/server/apiutil" "github.com/cockroachdb/cockroach/pkg/server/authserver" "github.com/cockroachdb/cockroach/pkg/server/debug" "github.com/cockroachdb/cockroach/pkg/server/diagnostics/diagnosticspb" @@ -57,8 +58,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" - "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/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/sql/contention" @@ -2825,13 +2824,6 @@ func (s *systemStatusServer) HotRanges( return response, nil } -type tableMeta struct { - dbName string - tableName string - schemaName string - indexName string -} - func (t *statusServer) HotRangesV2( ctx context.Context, req *serverpb.HotRangesRequest, ) (*serverpb.HotRangesResponseV2, error) { @@ -2874,8 +2866,6 @@ func (s *systemStatusServer) HotRangesV2( } } - var tableMetaCache syncutil.Map[roachpb.RangeID, tableMeta] - response := &serverpb.HotRangesResponseV2{ ErrorsByNodeID: make(map[roachpb.NodeID]string), } @@ -2889,70 +2879,30 @@ func (s *systemStatusServer) HotRangesV2( if local { resp := s.localHotRanges(ctx, tenantID) var ranges []*serverpb.HotRangesResponseV2_HotRange + var rangeIndexMappings map[roachpb.RangeID]apiutil.IndexNamesList for _, store := range resp.Stores { + rangeDescriptors := []roachpb.RangeDescriptor{} + for _, r := range store.HotRanges { + rangeDescriptors = append(rangeDescriptors, r.Desc) + } + if err = s.sqlServer.distSQLServer.DB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error { + databases, err := txn.Descriptors().GetAllDatabaseDescriptorsMap(ctx, txn.KV()) + if err != nil { + return err + } + rangeIndexMappings, err = apiutil.GetRangeIndexMapping(ctx, txn, s.sqlServer.execCfg.Codec, databases, rangeDescriptors) + return err + }); err != nil { + return nil, err + } for _, r := range store.HotRanges { - var ( - dbName, tableName, indexName, schemaName string - replicaNodeIDs []roachpb.NodeID - ) - rangeID := r.Desc.RangeID + var replicaNodeIDs []roachpb.NodeID + for _, repl := range r.Desc.Replicas().Descriptors() { replicaNodeIDs = append(replicaNodeIDs, repl.NodeID) } - if maybeIndexPrefix, tableID, ok := decodeTableID(s.sqlServer.execCfg.Codec, r.Desc.StartKey.AsRawKey()); !ok { - dbName = "system" - tableName = r.Desc.StartKey.String() - } else if meta, ok := tableMetaCache.Load(rangeID); ok { - dbName = meta.dbName - tableName = meta.tableName - schemaName = meta.schemaName - indexName = meta.indexName - } else { - if err = s.sqlServer.distSQLServer.DB.DescsTxn( - ctx, func(ctx context.Context, txn descs.Txn) error { - col := txn.Descriptors() - desc, err := col.ByIDWithoutLeased(txn.KV()).WithoutNonPublic().Get().Table(ctx, descpb.ID(tableID)) - if err != nil { - return errors.Wrapf(err, "cannot get table descriptor with tableID: %d, %s", tableID, r.Desc) - } - tableName = desc.GetName() - - if !maybeIndexPrefix.Equal(roachpb.KeyMin) { - if _, _, idxID, err := s.sqlServer.execCfg.Codec.DecodeIndexPrefix(r.Desc.StartKey.AsRawKey()); err != nil { - log.Warningf(ctx, "cannot decode index prefix for range descriptor: %s: %v", r.Desc, err) - } else { - if index := catalog.FindIndexByID(desc, descpb.IndexID(idxID)); index == nil { - log.Warningf(ctx, "cannot get index name for range descriptor: %s: index with ID %d not found", r.Desc, idxID) - } else { - indexName = index.GetName() - } - } - } - - if dbDesc, err := col.ByIDWithoutLeased(txn.KV()).WithoutNonPublic().Get().Database(ctx, desc.GetParentID()); err != nil { - log.Warningf(ctx, "cannot get database by descriptor ID: %s: %v", r.Desc, err) - } else { - dbName = dbDesc.GetName() - } - - if schemaDesc, err := col.ByIDWithoutLeased(txn.KV()).WithoutNonPublic().Get().Schema(ctx, desc.GetParentSchemaID()); err != nil { - log.Warningf(ctx, "cannot get schema name for range descriptor: %s: %v", r.Desc, err) - } else { - schemaName = schemaDesc.GetName() - } - return nil - }); err != nil { - log.Warningf(ctx, "failed to get table info for %s: %v", r.Desc, err) - continue - } - tableMetaCache.Store(rangeID, &tableMeta{ - dbName: dbName, - tableName: tableName, - schemaName: schemaName, - indexName: indexName, - }) - } + databases, tables, indexes := rangeIndexMappings[r.Desc.RangeID].ToOutput() ranges = append(ranges, &serverpb.HotRangesResponseV2_HotRange{ RangeID: r.Desc.RangeID, @@ -2963,13 +2913,12 @@ func (s *systemStatusServer) HotRangesV2( WriteBytesPerSecond: r.WriteBytesPerSecond, ReadBytesPerSecond: r.ReadBytesPerSecond, CPUTimePerSecond: r.CPUTimePerSecond, - TableName: tableName, - SchemaName: schemaName, - DatabaseName: dbName, - IndexName: indexName, ReplicaNodeIds: replicaNodeIDs, LeaseholderNodeID: r.LeaseholderNodeID, StoreID: store.StoreID, + Databases: databases, + Tables: tables, + Indexes: indexes, }) } } @@ -3014,23 +2963,6 @@ func (s *systemStatusServer) HotRangesV2( return response, nil } -func decodeTableID(codec keys.SQLCodec, key roachpb.Key) (roachpb.Key, uint32, bool) { - remaining, tableID, err := codec.DecodeTablePrefix(key) - if err != nil { - return nil, 0, false - } - // Validate that tableID doesn't belong to system or pseudo table. - if key.Equal(roachpb.KeyMin) || - tableID <= keys.SystemDatabaseID || - keys.IsPseudoTableID(tableID) || - bytes.HasPrefix(key, keys.Meta1Prefix) || - bytes.HasPrefix(key, keys.Meta2Prefix) || - bytes.HasPrefix(key, keys.SystemPrefix) { - return nil, 0, false - } - return remaining, tableID, true -} - func (s *systemStatusServer) localHotRanges( ctx context.Context, tenantID roachpb.TenantID, ) serverpb.HotRangesResponse_NodeResponse { diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 61fabafc93ee..48fb9a6ceae9 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -1132,10 +1132,12 @@ func (tc *Collection) GetAllDatabaseDescriptors( return ret, nil } -// GetAllDatabaseDescriptorsMap returns the results of GetAllDatabaseDescriptors +// GetAllDatabaseDescriptorsMap returns the results of +// GetAllDatabaseDescriptors but as a map with the database ID as the +// key. func (tc *Collection) GetAllDatabaseDescriptorsMap( ctx context.Context, txn *kv.Txn, -) (ret map[descpb.ID]catalog.DatabaseDescriptor, _ error) { +) (map[descpb.ID]catalog.DatabaseDescriptor, error) { descriptors, err := tc.GetAllDatabaseDescriptors(ctx, txn) result := map[descpb.ID]catalog.DatabaseDescriptor{} if err != nil { @@ -1149,7 +1151,9 @@ func (tc *Collection) GetAllDatabaseDescriptorsMap( return result, nil } -// but as a map with the database ID as the key. +// GetSchemasForDatabase returns the schemas for a given database +// visible by the transaction. +// Deprecated: prefer GetAllSchemasInDatabase. func (tc *Collection) GetSchemasForDatabase( ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, ) (map[descpb.ID]string, error) { diff --git a/pkg/sql/catalog/internal/catkv/catalog_reader.go b/pkg/sql/catalog/internal/catkv/catalog_reader.go index ed316ce5b823..99397d6fbcf5 100644 --- a/pkg/sql/catalog/internal/catkv/catalog_reader.go +++ b/pkg/sql/catalog/internal/catkv/catalog_reader.go @@ -209,13 +209,16 @@ func (cr catalogReader) ScanDescriptorsInSpans( ) (nstree.Catalog, error) { var mc nstree.MutableCatalog - descSpans := make([]roachpb.Span, len(spans)) - for i, span := range spans { + descSpans := []roachpb.Span{} + for _, span := range spans { descSpan, err := getDescriptorSpanFromSpan(cr.Codec(), span) if err != nil { return mc.Catalog, err } - descSpans[i] = descSpan + if descSpan.ZeroLength() { + continue + } + descSpans = append(descSpans, descSpan) } cq := catalogQuery{codec: cr.codec} diff --git a/pkg/ui/workspaces/db-console/src/views/hotRanges/hotRangesTable.tsx b/pkg/ui/workspaces/db-console/src/views/hotRanges/hotRangesTable.tsx index c7f94b57e40b..8b3c002cf304 100644 --- a/pkg/ui/workspaces/db-console/src/views/hotRanges/hotRangesTable.tsx +++ b/pkg/ui/workspaces/db-console/src/views/hotRanges/hotRangesTable.tsx @@ -231,8 +231,8 @@ const HotRangesTable = ({ Database ), - cell: val => <>{val.database_name}, - sort: val => val.database_name, + cell: val => <>{val.databases.join(", ")}, + sort: val => val.databases.join(", "), }, { name: "table", @@ -244,22 +244,7 @@ const HotRangesTable = ({ Table ), - cell: val => - // A hot range may not necessarily back a SQL table. If we see a - // "table name" that starts with a slash, it is not a table name but - // instead the start key of the range, and we should not link it. - val.table_name.startsWith("/") ? ( - val.table_name - ) : ( - - {val.table_name} - - ), + cell: val => val.tables.join(", "), sort: val => val.table_name, }, { @@ -272,8 +257,8 @@ const HotRangesTable = ({ Index ), - cell: val => <>{val.index_name}, - sort: val => val.index_name, + cell: val => <>{val.indexes.join(", ")}, + sort: val => val.indexes.join(", "), }, { name: "locality",