Skip to content

Commit

Permalink
Merge pull request #134988 from angles-n-daemons/backport24.3-133190-…
Browse files Browse the repository at this point in the history
…133840-134106

release-24.3: ui, apiutil, server: surface multiple indexes within a range in the hot ranges page
  • Loading branch information
angles-n-daemons authored Dec 6, 2024
2 parents 812c92c + 67c45eb commit 480af61
Show file tree
Hide file tree
Showing 24 changed files with 1,610 additions and 121 deletions.
9 changes: 6 additions & 3 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -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) |
Expand All @@ -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) |



Expand Down
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -325,6 +325,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",
Expand Down Expand Up @@ -1660,6 +1661,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",
Expand Down
3 changes: 3 additions & 0 deletions pkg/roachpb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -60,13 +60,15 @@ 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",
"metadata_test.go",
"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",
Expand All @@ -82,6 +84,7 @@ go_test(
"//pkg/raft/raftpb",
"//pkg/raft/tracker",
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/testutils/zerofields",
"//pkg/util",
"//pkg/util/bitarray",
Expand Down
41 changes: 41 additions & 0 deletions pkg/roachpb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,26 @@ 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(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(min) {
result = min
}
if max.Less(k) {
result = max
}
return result, nil
}

// SafeFormat implements the redact.SafeFormatter interface.
func (k Key) SafeFormat(w redact.SafePrinter, _ rune) {
SafeFormatKey(w, nil /* valDirs */, k)
Expand Down Expand Up @@ -2360,6 +2380,27 @@ 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, 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:
// 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
Expand Down
127 changes: 127 additions & 0 deletions pkg/roachpb/key_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
// 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/testutils"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/stretchr/testify/require"
)

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, 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 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, 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 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, 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)
}
15 changes: 15 additions & 0 deletions pkg/roachpb/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
22 changes: 22 additions & 0 deletions pkg/roachpb/metadata_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ package roachpb
import (
"fmt"
"reflect"
"sort"
"strings"
"testing"

Expand Down Expand Up @@ -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())
}
}
}
88 changes: 88 additions & 0 deletions pkg/roachpb/span_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// 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"
"github.com/cockroachdb/cockroach/pkg/testutils"
)

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
error string
}{
{
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)},
},
{
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) {
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)
}
})
}
}
1 change: 0 additions & 1 deletion pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -210,7 +210,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",
Expand Down
Loading

0 comments on commit 480af61

Please sign in to comment.