Skip to content

Commit

Permalink
Merge #90391
Browse files Browse the repository at this point in the history
90391: multitenant: move SPLIT AT tenant check from SQL to KV layer r=knz,dt a=ecwall

fixes #90435

Move the SPLIT AT tenant check from the SQL to KV layer to prepare for tenant
permissions work. This check will eventually be replaced by a tenant 
permissions lookup to determine if the tenant can perform the requested KV 
operation (AdminSplit in this case). `Codec.ForSystemTenant()` checks still 
exist in the SQL and will be handled in separate PRs.

Release note: None


Co-authored-by: Evan Wall <[email protected]>
  • Loading branch information
craig[bot] and ecwall committed Nov 3, 2022
2 parents dd96a71 + 19382ae commit 52d7d65
Show file tree
Hide file tree
Showing 43 changed files with 499 additions and 239 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -1545,6 +1545,7 @@ GO_TARGETS = [
"//pkg/sql/mutations:mutations",
"//pkg/sql/oidext:oidext",
"//pkg/sql/oidext:oidext_test",
"//pkg/sql/oppurpose:oppurpose",
"//pkg/sql/opt/bench:bench",
"//pkg/sql/opt/bench:bench_test",
"//pkg/sql/opt/cat:cat",
Expand Down Expand Up @@ -2710,6 +2711,7 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/memsize:get_x_data",
"//pkg/sql/mutations:get_x_data",
"//pkg/sql/oidext:get_x_data",
"//pkg/sql/oppurpose:get_x_data",
"//pkg/sql/opt:get_x_data",
"//pkg/sql/opt/bench:get_x_data",
"//pkg/sql/opt/cat:get_x_data",
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ go_library(
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/exprutil",
"//pkg/sql/oppurpose",
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
Expand Down Expand Up @@ -245,6 +246,7 @@ go_test(
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/importer",
"//pkg/sql/oppurpose",
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/randgen",
Expand Down
15 changes: 13 additions & 2 deletions pkg/ccl/backupccl/restore_data_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/oppurpose"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -359,10 +360,20 @@ func runTestIngest(t *testing.T, init func(*cluster.Settings)) {
t.Fatalf("failed to rewrite key: %s", reqMidKey2)
}

if err := kvDB.AdminSplit(ctx, reqMidKey1, hlc.MaxTimestamp /* expirationTime */); err != nil {
if err := kvDB.AdminSplit(
ctx,
reqMidKey1,
hlc.MaxTimestamp, /* expirationTime */
oppurpose.SplitBackup,
); err != nil {
t.Fatal(err)
}
if err := kvDB.AdminSplit(ctx, reqMidKey2, hlc.MaxTimestamp /* expirationTime */); err != nil {
if err := kvDB.AdminSplit(
ctx,
reqMidKey2,
hlc.MaxTimestamp, /* expirationTime */
oppurpose.SplitBackup,
); err != nil {
t.Fatal(err)
}

Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/backupccl/split_and_scatter_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/oppurpose"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -96,7 +97,7 @@ func (s dbSplitAndScatterer) split(
newSplitKey = splitAt
}
log.VEventf(ctx, 1, "presplitting new key %+v", newSplitKey)
if err := s.db.AdminSplit(ctx, newSplitKey, expirationTime); err != nil {
if err := s.db.AdminSplit(ctx, newSplitKey, expirationTime, oppurpose.SplitBackup); err != nil {
return errors.Wrapf(err, "splitting key %s", newSplitKey)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -390,7 +390,7 @@ test {}
statement ok
CREATE TABLE foo (a INT PRIMARY KEY, INDEX idx(a)); INSERT INTO foo VALUES(1)

statement error unsupported in multi-tenancy mode
statement error pq: rpc error: code = Unauthenticated desc = request \[1 AdmSplit\] not permitted
ALTER TABLE foo SPLIT AT VALUES(2)

# Make sure that the cluster id isn't unset.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ SELECT * FROM crdb_internal.kv_node_status

# Cannot perform operations that issue Admin requests.

statement error operation is unsupported in multi-tenancy mode
statement error pq: rpc error: code = Unauthenticated desc = request \[1 AdmSplit\] not permitted
ALTER TABLE kv SPLIT AT VALUES ('foo')

statement error operation is unsupported in multi-tenancy mode
Expand Down
6 changes: 5 additions & 1 deletion pkg/kv/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -699,7 +699,10 @@ func (b *Batch) adminMerge(key interface{}) {
// adminSplit is only exported on DB. It is here for symmetry with the
// other operations.
func (b *Batch) adminSplit(
splitKeyIn interface{}, expirationTime hlc.Timestamp, predicateKeys []roachpb.Key,
splitKeyIn interface{},
expirationTime hlc.Timestamp,
class roachpb.AdminSplitRequest_Class,
predicateKeys []roachpb.Key,
) {
splitKey, err := marshalKey(splitKeyIn)
if err != nil {
Expand All @@ -713,6 +716,7 @@ func (b *Batch) adminSplit(
SplitKey: splitKey,
ExpirationTime: expirationTime,
PredicateKeys: predicateKeys,
Class: class,
}
b.appendReqs(req)
b.initResult(1, 0, notRaw, nil)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/bulk/buffering_adder.go
Original file line number Diff line number Diff line change
Expand Up @@ -403,7 +403,7 @@ func (b *BufferingAdder) createInitialSplits(ctx context.Context) error {
}
predicateKey := b.curBuf.Key(predicateAt)
log.VEventf(ctx, 1, "pre-splitting span %d of %d at %s", i, b.initialSplits, splitKey)
if err := b.sink.db.AdminSplit(ctx, splitKey, expire, predicateKey); err != nil {
if err := b.sink.db.AdminSplit(ctx, splitKey, expire, roachpb.AdminSplitRequest_INGESTION, predicateKey); err != nil {
// TODO(dt): a typed error would be nice here.
if strings.Contains(err.Error(), "predicate") {
log.VEventf(ctx, 1, "%s adder split at %s rejected, had previously split and no longer included %s",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/bulk/sst_batcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -521,7 +521,7 @@ func (b *SSTBatcher) doFlush(ctx context.Context, reason int) error {
log.Warningf(ctx, "%s failed to generate split-above key: %v", b.name, err)
} else {
beforeSplit := timeutil.Now()
err := b.db.AdminSplit(ctx, splitAbove, expire)
err := b.db.AdminSplit(ctx, splitAbove, expire, roachpb.AdminSplitRequest_INGESTION)
b.currentStats.SplitWait += timeutil.Since(beforeSplit)
if err != nil {
log.Warningf(ctx, "%s failed to split-above: %v", b.name, err)
Expand All @@ -536,7 +536,7 @@ func (b *SSTBatcher) doFlush(ctx context.Context, reason int) error {
log.Warningf(ctx, "%s failed to generate split key: %v", b.name, err)
} else {
beforeSplit := timeutil.Now()
err := b.db.AdminSplit(ctx, splitAt, expire)
err := b.db.AdminSplit(ctx, splitAt, expire, roachpb.AdminSplitRequest_INGESTION)
b.currentStats.SplitWait += timeutil.Since(beforeSplit)
if err != nil {
log.Warningf(ctx, "%s failed to split: %v", b.name, err)
Expand Down
14 changes: 12 additions & 2 deletions pkg/kv/bulk/sst_batcher_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -277,7 +277,12 @@ func runTestImport(t *testing.T, batchSizeValue int64) {
}

t.Logf("splitting at %s", key(split1))
require.NoError(t, kvDB.AdminSplit(ctx, key(split1), hlc.MaxTimestamp /* expirationTime */))
require.NoError(t, kvDB.AdminSplit(
ctx,
key(split1),
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
))

// We want to make sure our range-aware batching knows about one of our
// splits to exercise that code path, but we also want to make sure we
Expand All @@ -293,7 +298,12 @@ func runTestImport(t *testing.T, batchSizeValue int64) {
}

t.Logf("splitting at %s", key(split2))
require.NoError(t, kvDB.AdminSplit(ctx, key(split2), hlc.MaxTimestamp /* expirationTime */))
require.NoError(t, kvDB.AdminSplit(
ctx,
key(split2),
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
))

ts := hlc.Timestamp{WallTime: 100}
mem := mon.NewUnlimitedMonitor(ctx, "lots", mon.MemoryResource, nil, nil, 0, nil)
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -590,10 +590,11 @@ func (db *DB) AdminSplit(
ctx context.Context,
splitKey interface{},
expirationTime hlc.Timestamp,
class roachpb.AdminSplitRequest_Class,
predicateKeys ...roachpb.Key,
) error {
b := &Batch{}
b.adminSplit(splitKey, expirationTime, predicateKeys)
b.adminSplit(splitKey, expirationTime, class, predicateKeys)
return getOneErr(db.Run(ctx, b), b)
}

Expand Down
66 changes: 53 additions & 13 deletions pkg/kv/kvclient/kvcoord/dist_sender_server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,8 +119,13 @@ func TestRangeLookupWithOpenTransaction(t *testing.T) {
// client.
func setupMultipleRanges(ctx context.Context, db *kv.DB, splitAt ...string) error {
// Split the keyspace at the given keys.
for _, key := range splitAt {
if err := db.AdminSplit(ctx, key /* splitKey */, hlc.MaxTimestamp /* expirationTime */); err != nil {
for _, splitKey := range splitAt {
if err := db.AdminSplit(
ctx,
splitKey,
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
); err != nil {
return err
}
}
Expand Down Expand Up @@ -1173,7 +1178,12 @@ func TestMultiRangeScanDeleteRange(t *testing.T) {
defer s.Stopper().Stop(ctx)
tds := db.NonTransactionalSender()

if err := db.AdminSplit(ctx, "m", hlc.MaxTimestamp /* expirationTime */); err != nil {
if err := db.AdminSplit(
ctx,
"m", /* splitKey */
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
); err != nil {
t.Fatal(err)
}
writes := []roachpb.Key{roachpb.Key("a"), roachpb.Key("z")}
Expand Down Expand Up @@ -1325,8 +1335,13 @@ func TestMultiRangeScanWithPagination(t *testing.T) {
defer s.Stopper().Stop(ctx)
tds := db.NonTransactionalSender()

for _, sk := range tc.splitKeys {
if err := db.AdminSplit(ctx, sk, hlc.MaxTimestamp /* expirationTime */); err != nil {
for _, splitKey := range tc.splitKeys {
if err := db.AdminSplit(
ctx,
splitKey,
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
); err != nil {
t.Fatal(err)
}
}
Expand Down Expand Up @@ -1472,8 +1487,13 @@ func TestParallelSender(t *testing.T) {

// Split into multiple ranges.
splitKeys := []string{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}
for _, key := range splitKeys {
if err := db.AdminSplit(context.Background(), key, hlc.MaxTimestamp /* expirationTime */); err != nil {
for _, splitKey := range splitKeys {
if err := db.AdminSplit(
context.Background(),
splitKey,
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
); err != nil {
t.Fatal(err)
}
}
Expand Down Expand Up @@ -1516,9 +1536,14 @@ func initReverseScanTestEnv(s serverutils.TestServerInterface, t *testing.T) *kv

// Set up multiple ranges:
// ["", "b"),["b", "e") ,["e", "g") and ["g", "\xff\xff").
for _, key := range []string{"b", "e", "g"} {
for _, splitKey := range []string{"b", "e", "g"} {
// Split the keyspace at the given key.
if err := db.AdminSplit(context.Background(), key, hlc.MaxTimestamp /* expirationTime */); err != nil {
if err := db.AdminSplit(
context.Background(),
splitKey,
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
); err != nil {
t.Fatal(err)
}
}
Expand Down Expand Up @@ -1618,8 +1643,13 @@ func TestBatchPutWithConcurrentSplit(t *testing.T) {

// Split first using the default client and scan to make sure that
// the range descriptor cache reflects the split.
for _, key := range []string{"b", "f"} {
if err := db.AdminSplit(context.Background(), key, hlc.MaxTimestamp /* expirationTime */); err != nil {
for _, splitKey := range []string{"b", "f"} {
if err := db.AdminSplit(
context.Background(),
splitKey,
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
); err != nil {
t.Fatal(err)
}
}
Expand Down Expand Up @@ -1676,7 +1706,12 @@ func TestReverseScanWithSplitAndMerge(t *testing.T) {

// Case 1: An encounter with a range split.
// Split the range ["b", "e") at "c".
if err := db.AdminSplit(context.Background(), "c", hlc.MaxTimestamp /* expirationTime */); err != nil {
if err := db.AdminSplit(
context.Background(),
"c", /* splitKey */
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
); err != nil {
t.Fatal(err)
}

Expand Down Expand Up @@ -3538,7 +3573,12 @@ func BenchmarkReturnOnRangeBoundary(b *testing.B) {

for r := 0; r < Ranges; r++ {
rangeKey := string(rune('a' + r))
require.NoError(b, db.AdminSplit(ctx, rangeKey, hlc.MaxTimestamp))
require.NoError(b, db.AdminSplit(
ctx,
rangeKey,
hlc.MaxTimestamp, /* expirationTime */
roachpb.AdminSplitRequest_INGESTION,
))

for k := 0; k < KeysPerRange; k++ {
key := fmt.Sprintf("%s%d", rangeKey, k)
Expand Down
Loading

0 comments on commit 52d7d65

Please sign in to comment.