Skip to content

Commit

Permalink
builtins: new function crdb_internal.ranges_in_span
Browse files Browse the repository at this point in the history
Part of: #96011

This PR introduces a new SRF: `crdb_internal.ranges_in_span(start_key,
end_key)` - returns the set of ranges encompassing this span in the form
of `(range_id, start_key, end_key)`

Release note (sql change): introduce new builtin function
`crdb_internal.ranges_in_span(start_key, end_key), returns the set of
ranges encompassing the given start and end key.
  • Loading branch information
Thomas Hardy committed Feb 24, 2023
1 parent 4806560 commit b6756c3
Show file tree
Hide file tree
Showing 12 changed files with 185 additions and 0 deletions.
4 changes: 4 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -1260,6 +1260,10 @@ the locality flag on node startup. Returns an error if no region is set.</p>
<tbody>
<tr><td><a name="aclexplode"></a><code>aclexplode(aclitems: <a href="string.html">string</a>[]) &rarr; tuple{oid AS grantor, oid AS grantee, string AS privilege_type, bool AS is_grantable}</code></td><td><span class="funcdesc"><p>Produces a virtual table containing aclitem stuff (returns no rows as this feature is unsupported in CockroachDB)</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.ranges_in_span"></a><code>crdb_internal.ranges_in_span(span_keys: <a href="bytes.html">bytes</a>[]) &rarr; tuple{int AS range_id, bytes AS start_key, bytes AS end_key}</code></td><td><span class="funcdesc"><p>Returns ranges (id, start key, end key) within the provided span.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.ranges_in_span"></a><code>crdb_internal.ranges_in_span(start_key: <a href="bytes.html">bytes</a>, end_key: <a href="bytes.html">bytes</a>) &rarr; tuple{int AS range_id, bytes AS start_key, bytes AS end_key}</code></td><td><span class="funcdesc"><p>Returns ranges (id, start key, end key) within the provided span.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.scan"></a><code>crdb_internal.scan(span: <a href="bytes.html">bytes</a>[]) &rarr; tuple{bytes AS key, bytes AS value, string AS ts}</code></td><td><span class="funcdesc"><p>Returns the raw keys and values from the specified span</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.scan"></a><code>crdb_internal.scan(start_key: <a href="bytes.html">bytes</a>, end_key: <a href="bytes.html">bytes</a>) &rarr; tuple{bytes AS key, bytes AS value, string AS ts}</code></td><td><span class="funcdesc"><p>Returns the raw keys and values with their timestamp from the specified span</p>
Expand Down
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -1466,6 +1466,7 @@ GO_TARGETS = [
"//pkg/spanconfig/spanconfigtestutils:spanconfigtestutils_test",
"//pkg/spanconfig:spanconfig",
"//pkg/spanconfig:spanconfig_test",
"//pkg/spanstats:spanstats",
"//pkg/sql/appstatspb:appstatspb",
"//pkg/sql/appstatspb:appstatspb_test",
"//pkg/sql/backfill:backfill",
Expand Down Expand Up @@ -2797,6 +2798,7 @@ GET_X_DATA_TARGETS = [
"//pkg/spanconfig/spanconfigstore:get_x_data",
"//pkg/spanconfig/spanconfigtestutils:get_x_data",
"//pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster:get_x_data",
"//pkg/spanstats:get_x_data",
"//pkg/sql:get_x_data",
"//pkg/sql/appstatspb:get_x_data",
"//pkg/sql/backfill:get_x_data",
Expand Down
7 changes: 7 additions & 0 deletions pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions pkg/sql/faketreeeval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//pkg/util/duration",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/mon",
"//pkg/util/rangedesc",
"@com_github_cockroachdb_errors//:errors",
"@com_github_lib_pq//oid",
],
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/faketreeeval/evalctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)
Expand Down Expand Up @@ -476,6 +477,13 @@ func (ep *DummyEvalPlanner) GetRangeDescByID(
return
}

// GetRangeIteratorWithinSpan is part of the eval.Planner interface.
func (ep *DummyEvalPlanner) GetRangeIteratorWithinSpan(
ctx context.Context, span roachpb.Span,
) (it rangedesc.Iterator, err error) {
return
}

// DummyPrivilegedAccessor implements the tree.PrivilegedAccessor interface by returning errors.
type DummyPrivilegedAccessor struct{}

Expand Down
31 changes: 31 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/ranges_in_span
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
# LogicTest: 3node-tenant

# SELECT * FROM crdb_internal.ranges_in_span: all ranges within given span.
# Assert the schema, using 'system.descriptor' table start/end keys as separate arguments.
query ITT colnames
SELECT * FROM crdb_internal.ranges_in_span('\x8b', '\x8c') LIMIT 0
----
range_id start_key end_key

# SELECT * FROM crdb_internal.ranges_in_span: all ranges within given span.
# Assert correct values, using 'system.descriptor' table start/end keys as separate arguments.
query ITT colnames
SELECT range_id, crdb_internal.pretty_key(start_key, -1) as start_key, crdb_internal.pretty_key(end_key, -1) as end_key FROM crdb_internal.ranges_in_span('\x8b', '\x8c')
----
range_id start_key end_key
7 /Table/3 /Table/4

# SELECT * FROM crdb_internal.ranges_in_span: all ranges within given span.
# Assert the schema, using 'system.descriptor' table start/end keys as single byte array argument.
query ITT colnames
SELECT * FROM crdb_internal.ranges_in_span('{"\x8b", "\x8c"}') LIMIT 0
----
range_id start_key end_key

# SELECT * FROM crdb_internal.ranges_in_span: all ranges within given span.
# Assert correct values, using 'system.descriptor' table start/end keys as single byte array argument.
query ITT colnames
SELECT range_id, crdb_internal.pretty_key(start_key, -1) as start_key, crdb_internal.pretty_key(end_key, -1) as end_key FROM crdb_internal.ranges_in_span('{"\x8b", "\x8c"}')
----
range_id start_key end_key
7 /Table/3 /Table/4
13 changes: 13 additions & 0 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keyvisualizer"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/repstream"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
Expand Down Expand Up @@ -51,6 +52,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
"github.com/lib/pq/oid"
Expand Down Expand Up @@ -834,3 +836,14 @@ func (p *planner) GetReplicationStreamManager(
func (p *planner) GetStreamIngestManager(ctx context.Context) (eval.StreamIngestManager, error) {
return repstream.GetStreamIngestManager(ctx, p.EvalContext(), p.InternalSQLTxn())
}

// GetRangeIteratorWithinSpan is part of the eval.Planner interface.
func (p *planner) GetRangeIteratorWithinSpan(
ctx context.Context, span roachpb.Span,
) (rangedesc.Iterator, error) {
rangeDescIterator, err := p.execCfg.RangeDescIteratorFactory.NewIterator(ctx, span)
if err != nil {
return nil, err
}
return rangeDescIterator, nil
}
1 change: 1 addition & 0 deletions pkg/sql/sem/builtins/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@ go_library(
"//pkg/util/randident",
"//pkg/util/randident/randidentcfg",
"//pkg/util/randutil",
"//pkg/util/rangedesc",
"//pkg/util/ring",
"//pkg/util/syncutil",
"//pkg/util/timeofday",
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/sem/builtins/fixed_oids.go
Original file line number Diff line number Diff line change
Expand Up @@ -2045,6 +2045,8 @@ var builtinOidsArray = []string{
2069: `crdb_internal.create_tenant(parameters: jsonb) -> int`,
2070: `crdb_internal.num_inverted_index_entries(val: tsvector, version: int) -> int`,
2072: `crdb_internal.upsert_dropped_relation_gc_ttl(desc_id: int, gc_ttl: interval) -> bool`,
2073: `crdb_internal.ranges_in_span(start_key: bytes, end_key: bytes) -> tuple{int AS range_id, bytes AS start_key, bytes AS end_key}`,
2074: `crdb_internal.ranges_in_span(span_keys: bytes[]) -> tuple{int AS range_id, bytes AS start_key, bytes AS end_key}`,
}

var builtinOidsBySignature map[string]oid.Oid
Expand Down
111 changes: 111 additions & 0 deletions pkg/sql/sem/builtins/generator_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/randident"
"github.com/cockroachdb/cockroach/pkg/util/randident/randidentcfg"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
Expand Down Expand Up @@ -465,6 +466,27 @@ var generators = map[string]builtinDefinition{
volatility.Volatile,
),
),
"crdb_internal.ranges_in_span": makeBuiltin(genProps(),
makeGeneratorOverload(
tree.ParamTypes{
{Name: "start_key", Typ: types.Bytes},
{Name: "end_key", Typ: types.Bytes},
},
rangesInSpanGeneratorType,
makeRangesInSpanGenerator,
"Returns ranges (id, start key, end key) within the provided span.",
volatility.Stable,
),
makeGeneratorOverload(
tree.ParamTypes{
{Name: "span_keys", Typ: types.BytesArray},
},
rangesInSpanGeneratorType,
makeRangesInSpanGenerator,
"Returns ranges (id, start key, end key) within the provided span.",
volatility.Stable,
),
),
"crdb_internal.show_create_all_schemas": makeBuiltin(
tree.FunctionProperties{},
makeGeneratorOverload(
Expand Down Expand Up @@ -2895,3 +2917,92 @@ func makeIdentGenerator(
count: count,
}, nil
}

// rangeSpanIterator is a ValueGenerator that iterates over all
// ranges of a target span.
type rangeSpanIterator struct {
// The span to iterate
span roachpb.Span
p eval.Planner
currRangeDesc roachpb.RangeDescriptor
rangeIter rangedesc.Iterator

// A buffer to avoid allocating an array on every call to Values().
buf [3]tree.Datum
}

func newRangeSpanIterator(eval *eval.Context, span roachpb.Span) *rangeSpanIterator {
return &rangeSpanIterator{span: span, p: eval.Planner}
}

// Start implements the tree.ValueGenerator interface.
func (rs *rangeSpanIterator) Start(ctx context.Context, _ *kv.Txn) error {
var err error
rs.rangeIter, err = rs.p.GetRangeIteratorWithinSpan(ctx, rs.span)
if err != nil {
return err
}
return nil
}

// Next implements the tree.ValueGenerator interface.
func (rs *rangeSpanIterator) Next(_ context.Context) (bool, error) {
exists := rs.rangeIter.Valid()
if exists {
rs.currRangeDesc = rs.rangeIter.CurRangeDescriptor()
rs.rangeIter.Next()
}
return exists, nil
}

// Values implements the tree.ValueGenerator interface.
func (rs *rangeSpanIterator) Values() (tree.Datums, error) {
rs.buf[0] = tree.NewDInt(tree.DInt(rs.currRangeDesc.RangeID))
rs.buf[1] = tree.NewDBytes(tree.DBytes(rs.currRangeDesc.StartKey))
rs.buf[2] = tree.NewDBytes(tree.DBytes(rs.currRangeDesc.EndKey))
return rs.buf[:], nil
}

// Close implements the tree.ValueGenerator interface.
func (rs *rangeSpanIterator) Close(_ context.Context) {}

// ResolvedType implements the tree.ValueGenerator interface.
func (rs *rangeSpanIterator) ResolvedType() *types.T {
return rangesInSpanGeneratorType
}

var rangesInSpanGeneratorType = types.MakeLabeledTuple(
[]*types.T{types.Int, types.Bytes, types.Bytes},
[]string{"range_id", "start_key", "end_key"},
)

func makeRangesInSpanGenerator(
ctx context.Context, evalCtx *eval.Context, args tree.Datums,
) (eval.ValueGenerator, error) {
// The user must be an admin to use this builtin.
isAdmin, err := evalCtx.SessionAccessor.HasAdminRole(ctx)
if err != nil {
return nil, err
}
if !isAdmin {
return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "user needs the admin role to view range data")
}
var startKey []byte
var endKey []byte
if len(args) == 1 {
arr := tree.MustBeDArray(args[0])
if arr.Len() != 2 {
return nil, errors.New("expected an array of two elements")
}
startKey = []byte(tree.MustBeDBytes(arr.Array[0]))
endKey = []byte(tree.MustBeDBytes(arr.Array[1]))
}
if len(args) == 2 {
startKey = []byte(tree.MustBeDBytes(args[0]))
endKey = []byte(tree.MustBeDBytes(args[1]))
}
return newRangeSpanIterator(evalCtx, roachpb.Span{
Key: startKey,
EndKey: endKey,
}), nil
}
1 change: 1 addition & 0 deletions pkg/sql/sem/eval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ go_library(
"//pkg/util/hlc",
"//pkg/util/json",
"//pkg/util/mon",
"//pkg/util/rangedesc",
"//pkg/util/ring",
"//pkg/util/timeofday",
"//pkg/util/timeutil",
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/sem/eval/deps.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
"github.com/lib/pq/oid"
)

Expand Down Expand Up @@ -371,6 +372,9 @@ type Planner interface {

// GetRangeDescByID gets the RangeDescriptor by the specified RangeID.
GetRangeDescByID(context.Context, roachpb.RangeID) (roachpb.RangeDescriptor, error)

// GetRangeIteratorWithinSpan gets a range iterator for the provided span.
GetRangeIteratorWithinSpan(context.Context, roachpb.Span) (rangedesc.Iterator, error)
}

// InternalRows is an iterator interface that's exposed by the internal
Expand Down

0 comments on commit b6756c3

Please sign in to comment.