Skip to content

Commit

Permalink
Merge #133190
Browse files Browse the repository at this point in the history
133190: sql: using catalog reader, allow descriptor scanning by span r=angles-n-daemons a=angles-n-daemons

Summary: new `catalogReader.GetDescriptorsInSpans(..., spans []roachpb.Span)` function which acts as named.

-----------
sql: using catalog reader, allow descriptor scanning by span

Historically, to estimate which tables and indexes existed within the
span of a range, we used the Collection's `GetAllDescriptors` function
and returned the values which fell in between the start and end keys of
the range's span. This approach had the benefit of being precise, but
the drawback of being computationally expensive - since the system can
have hundreds of nodes and theoretically millions of tables, using
`GetAllDescriptors` begins to become a bottleneck.

This approach was modified so that instead of pulling all descriptors
for the system when computing range contents, the system instead used
the start key of the range to identify the exact table + index at that
point within the keyspace
([change](https://github.com/cockroachdb/cockroach/pull/77277/files)).
This works if each table, index has their own range, but quickly breaks
down if tables share a range. Consider the following layout of data:

```
T1=Table 1
T2=Table 2
R1=Range1

└─────T1─────┴─────T2────┴─────T3─────┘
└────────┴─────────R1───────┴─────────┘
```

Since the start key of the range falls with Table 1, the system
associates the range with only Table 1, despite it containing Tables 2
and 3.

Using this information, it becomes necessary to identify a set of
descriptors within a certain span. This PR introduces the
`ScanDescriptorsInSpans` function which does just that, allows the user
to specify a set of spans whose descriptors are important and then
return a catalog including those descriptors.

It does this by translating the span keys into descriptor span keys and
scanning them from the descriptors table. For example given a span
`[/Table/Users/PKEY/1, /Table/Users/SECONDARY/chicago]` where the ID for
the Users table is `5`, it will generate a descriptor span
`[/Table/Descriptors/PKEY/5, /Table/Descriptors/PKEY/6]`.

This descriptor too comes with its drawbacks in that within the
descriptor space, keys are scoped by table, and not necessarily indexes.
That means in a following PR, the status server will be responsible for
taking these descriptors, which include all indexes in the tables
pulled, and filtering it down to only the indexes which appear in the
specified range.

The bulk of the changeset is in updating the datadriven tests to test
this behavior, the primary area of focus for review should be the
`pkg/sql/catalog/internal/catkv/catalog_reader.go` file (~75 LOC).

Epic: CRDB-43151
Fixes: #130997

Release note: None

Co-authored-by: Brian Dillmann <[email protected]>
  • Loading branch information
craig[bot] and angles-n-daemons committed Nov 8, 2024
2 parents b4f7fd2 + 7bd74a1 commit 16da9ff
Show file tree
Hide file tree
Showing 7 changed files with 570 additions and 0 deletions.
7 changes: 7 additions & 0 deletions pkg/sql/catalog/descs/collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -743,6 +743,13 @@ func (tc *Collection) GetAll(ctx context.Context, txn *kv.Txn) (nstree.Catalog,
return ret.Catalog, nil
}

// GetDescriptorsInSpans returns all descriptors within a given span.
func (tc *Collection) GetDescriptorsInSpans(
ctx context.Context, txn *kv.Txn, spans []roachpb.Span,
) (nstree.Catalog, error) {
return tc.cr.ScanDescriptorsInSpans(ctx, txn, spans)
}

// GetAllComments gets all comments for all descriptors in the given database.
// This method never returns the underlying catalog, since it will be incomplete and only
// contain comments.
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/catalog/internal/catkv/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ go_test(
":catkv",
"//pkg/base",
"//pkg/kv",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/server",
Expand All @@ -67,6 +68,7 @@ go_test(
"//pkg/testutils/datapathutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/util/encoding",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
Expand Down
73 changes: 73 additions & 0 deletions pkg/sql/catalog/internal/catkv/catalog_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,9 @@ type CatalogReader interface {
// ScanAll scans the entirety of the descriptor and namespace tables.
ScanAll(ctx context.Context, txn *kv.Txn) (nstree.Catalog, error)

// ScanDescriptorsInSpans scans the descriptors specified in a given span.
ScanDescriptorsInSpans(ctx context.Context, txn *kv.Txn, span []roachpb.Span) (nstree.Catalog, error)

// ScanAllComments scans the entirety of the comments table as well as the namespace entries for the given database.
// If the dbContext is nil, we scan the database-level namespace entries.
ScanAllComments(ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor) (nstree.Catalog, error)
Expand Down Expand Up @@ -158,6 +161,76 @@ func (cr catalogReader) ScanAll(ctx context.Context, txn *kv.Txn) (nstree.Catalo
return mc.Catalog, nil
}

// getDescriptorIDFromExclusiveKey translates an exclusive upper bound roach key
// into an upper bound descriptor ID. It does this by turning the key into a
// descriptor ID, and then moving it upwards if and only if it is not the prefix
// of the current index / table span.
func getDescriptorIDFromExclusiveKey(codec keys.SQLCodec, key roachpb.Key) (uint32, error) {
keyWithoutTable, endID, err := codec.DecodeTablePrefix(key)
if err != nil {
return 0, err
}
if len(keyWithoutTable) == 0 {
return endID, nil
}

keyWithoutIndex, _, indexId, err := codec.DecodeIndexPrefix(key)
if err != nil {
return 0, err
}
// if there's remaining bytes or the index isn't the primary, increment
// the end so that the descriptor under the key is included.
if len(keyWithoutIndex) != 0 || indexId > 1 {
endID++
}
return endID, nil
}

// getDescriptorSpanFromSpan returns a start and end descriptor ID from a given span
func getDescriptorSpanFromSpan(codec keys.SQLCodec, span roachpb.Span) (roachpb.Span, error) {
_, startID, err := codec.DecodeTablePrefix(span.Key)
if err != nil {
return roachpb.Span{}, err
}
endID, err := getDescriptorIDFromExclusiveKey(codec, span.EndKey)
if err != nil {
return roachpb.Span{}, err
}

return roachpb.Span{
Key: catalogkeys.MakeDescMetadataKey(codec, descpb.ID(startID)),
EndKey: catalogkeys.MakeDescMetadataKey(codec, descpb.ID(endID)),
}, nil
}

// ScanDescriptorsInSpans is part of the CatalogReader interface.
func (cr catalogReader) ScanDescriptorsInSpans(
ctx context.Context, txn *kv.Txn, spans []roachpb.Span,
) (nstree.Catalog, error) {
var mc nstree.MutableCatalog

descSpans := make([]roachpb.Span, len(spans))
for i, span := range spans {
descSpan, err := getDescriptorSpanFromSpan(cr.Codec(), span)
if err != nil {
return mc.Catalog, err
}
descSpans[i] = descSpan
}

cq := catalogQuery{codec: cr.codec}
err := cq.query(ctx, txn, &mc, func(codec keys.SQLCodec, b *kv.Batch) {
for _, descSpan := range descSpans {
scanRange(ctx, b, descSpan.Key, descSpan.EndKey)
}
})
if err != nil {
return mc.Catalog, err
}

return mc.Catalog, nil
}

// ScanAllComments is part of the CatalogReader interface.
func (cr catalogReader) ScanAllComments(
ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor,
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/catalog/internal/catkv/catalog_reader_cached.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"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/nstree"
Expand Down Expand Up @@ -220,6 +221,15 @@ func (c *cachedCatalogReader) ScanAll(ctx context.Context, txn *kv.Txn) (nstree.
return read, nil
}

// ScanDescriptorsInSpans is part of the CatalogReader interface.
func (c *cachedCatalogReader) ScanDescriptorsInSpans(
ctx context.Context, txn *kv.Txn, spans []roachpb.Span,
) (nstree.Catalog, error) {
// TODO (brian.dillmann@): explore caching these calls.
// https://github.com/cockroachdb/cockroach/issues/134666
return c.cr.ScanDescriptorsInSpans(ctx, txn, spans)
}

// ScanNamespaceForDatabases is part of the CatalogReader interface.
func (c *cachedCatalogReader) ScanNamespaceForDatabases(
ctx context.Context, txn *kv.Txn,
Expand Down
72 changes: 72 additions & 0 deletions pkg/sql/catalog/internal/catkv/catalog_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
Expand All @@ -26,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
Expand Down Expand Up @@ -203,6 +205,24 @@ func TestDataDriven(t *testing.T) {
return cr.GetByNames(ctx, txn, nis)
}
return h.doCatalogQuery(ctx, q)
case "scan_descriptors_in_span":
{
start := h.parseKeyFromArgKey("start")
end := h.parseKeyFromArgKey("end")
q := func(ctx context.Context, txn *kv.Txn, cr catkv.CatalogReader) (nstree.Catalog, error) {
return cr.ScanDescriptorsInSpans(ctx, txn, []roachpb.Span{{Key: start, EndKey: end}})
}
return h.doCatalogQuery(ctx, q)
}
case "scan_descriptors_in_multiple_spans":
{
first := h.parseSpanFromArgKey("first")
second := h.parseSpanFromArgKey("second")
q := func(ctx context.Context, txn *kv.Txn, cr catkv.CatalogReader) (nstree.Catalog, error) {
return cr.ScanDescriptorsInSpans(ctx, txn, []roachpb.Span{first, second})
}
return h.doCatalogQuery(ctx, q)
}
}
return fmt.Sprintf("%s: unknown command: %s", d.Pos, d.Cmd)
})
Expand All @@ -217,6 +237,58 @@ type testHelper struct {
ucr, ccr catkv.CatalogReader
}

func (h testHelper) parseSpanFromArgKey(argkey string) roachpb.Span {
arg, exists := h.d.Arg(argkey)
if !exists {
h.t.Fatalf("scan_descriptors_in_span requires '%s' arg", argkey)
}
start, end := arg.TwoVals(h.t)
return roachpb.Span{
Key: h.parseKeyFromArgStr(start),
EndKey: h.parseKeyFromArgStr(end),
}
}

func (h testHelper) parseKeyFromArgKey(argkey string) roachpb.Key {
arg, exists := h.d.Arg(argkey)
if !exists {
h.t.Fatalf("scan_descriptors_in_span requires '%s' arg", argkey)
}
return h.parseKeyFromArgStr(arg.SingleVal(h.t))
}

func (h testHelper) parseKeyFromArgStr(argstr string) roachpb.Key {
parts := strings.Split(argstr, "/")
if len(parts) == 0 {
h.t.Fatal("cannot parse key without at least one key part")
} else if len(parts) > 4 {
h.t.Fatal("key argument has too many parts")
}

tableId, err := strconv.Atoi(parts[0])
require.NoError(h.t, err)
if len(parts) == 1 {
return h.execCfg.Codec.TablePrefix(uint32(tableId))
}

indexId, err := strconv.Atoi(parts[1])
require.NoError(h.t, err)

key := h.execCfg.Codec.IndexPrefix(uint32(tableId), uint32(indexId))
if len(parts) == 3 && parts[2] != "" {
// only supports integer and string key values
if encoding.PeekType([]byte(parts[2])) == encoding.Int {
pkey, err := strconv.Atoi(parts[1])
require.NoError(h.t, err)
return encoding.EncodeVarintAscending(key, int64(pkey))
} else {
return encoding.EncodeStringAscending(key, parts[2])
}
}

return key
}

func (h testHelper) argDesc(
ctx context.Context, idArgName string, expectedType catalog.DescriptorType,
) catalog.Descriptor {
Expand Down
Loading

0 comments on commit 16da9ff

Please sign in to comment.