Skip to content

Commit

Permalink
apiutil, roachpb: create utilities to map descriptors to ranges
Browse files Browse the repository at this point in the history
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
  • Loading branch information
angles-n-daemons committed Oct 30, 2024
1 parent 10b9ee0 commit 109219d
Show file tree
Hide file tree
Showing 11 changed files with 719 additions and 6 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
2 changes: 2 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 Down
30 changes: 30 additions & 0 deletions pkg/roachpb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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
Expand Down
108 changes: 108 additions & 0 deletions pkg/roachpb/key_test.go
Original file line number Diff line number Diff line change
@@ -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)
}
})
}
}
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())
}
}
}
75 changes: 75 additions & 0 deletions pkg/roachpb/span_test.go
Original file line number Diff line number Diff line change
@@ -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)
}
})
}
}
33 changes: 30 additions & 3 deletions pkg/server/apiutil/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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",
],
)
Loading

0 comments on commit 109219d

Please sign in to comment.