Skip to content

Commit

Permalink
Merge #52595 #52616
Browse files Browse the repository at this point in the history
52595: sql: don't query BootstrapVersionKey on tenant SQL startup r=nvanbenschoten a=nvanbenschoten

See #52094 (review).

We don't currently track the bootstrap version of each secondary tenant.
For this to be meaningful, we'd need to record the binary version of the
SQL gateway that processed the crdb_internal.create_tenant function which
created the tenant, as this is what dictates the MetadataSchema that was
in effect when the secondary tenant was constructed. This binary version
very well may differ from the cluster-wide bootstrap version at which the
system tenant was bootstrapped.

Since we don't record this version anywhere, we do the next-best thing
and pass a lower-bound on the bootstrap version. We know that no tenants
could have been created before the start of the v20.2 dev cycle, so we
pass VersionStart20_2. bootstrapVersion is only used to avoid performing
superfluous but necessarily idempotent SQL migrations, so at worst, we're
doing more work than strictly necessary during the first time that the
migrations are run.

Now that we don't query BootstrapVersionKey, we don't need to have it
in the allowlists in the tenantAuth policy for Batch and RangeLookup
RPCs.

52616: CODEOWNERS: add notification patterns for SQL syntax and APIs r=rohany a=knz

Release note: None

Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
  • Loading branch information
3 people committed Aug 11, 2020
3 parents 53df364 + 5ffaf44 + 7e06d9a commit e9abbac
Show file tree
Hide file tree
Showing 3 changed files with 40 additions and 39 deletions.
9 changes: 9 additions & 0 deletions .github/CODEOWNERS
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,15 @@
/pkg/sql/opt/ @cockroachdb/sql-opt-prs
/pkg/sql/stats/ @cockroachdb/sql-opt-prs

/pkg/sql/sem/tree/ @cockroachdb/sql-syntax-prs
/pkg/sql/parser/ @cockroachdb/sql-syntax-prs
/pkg/sql/lex/ @cockroachdb/sql-syntax-prs
/pkg/sql/show_create*.go @cockroachdb/sql-syntax-prs

/pkg/sql/crdb_internal.go @cockroachdb/sql-api-prs
/pkg/sql/pg_catalog.go @cockroachdb/sql-api-prs
/pkg/sql/sem/builtins/ @cockroachdb/sql-api-prs

/pkg/cli/ @cockroachdb/cli-prs

/pkg/ccl/backupccl/ @cockroachdb/bulk-prs
Expand Down
41 changes: 6 additions & 35 deletions pkg/rpc/auth_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,26 +121,10 @@ func (a tenantAuth) authBatch(tenID roachpb.TenantID, args *roachpb.BatchRequest
return authError(err.Error())
}
tenSpan := tenantPrefix(tenID)
if tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) {
return nil
}
for _, allow := range batchSpanAllowlist {
if rSpan.Equal(allow) {
return nil
}
if !tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) {
return authErrorf("requested key span %s not fully contained in tenant keyspace %s", rSpan, tenSpan)
}
return authErrorf("requested key span %s not fully contained in tenant keyspace %s", rSpan, tenSpan)
}

// batchSpanAllowlist contains spans outside of a tenant's keyspace that Batch
// RPC invocations are allowed to touch.
var batchSpanAllowlist = []roachpb.RSpan{
// TODO(nvanbenschoten): Explore whether we can get rid of this by no longer
// reading this key in sqlServer.start.
{
Key: roachpb.RKey(keys.BootstrapVersionKey),
EndKey: roachpb.RKey(keys.BootstrapVersionKey.Next()),
},
return nil
}

// authRangeLookup authorizes the provided tenant to invoke the RangeLookup RPC
Expand All @@ -149,23 +133,10 @@ func (a tenantAuth) authRangeLookup(
tenID roachpb.TenantID, args *roachpb.RangeLookupRequest,
) error {
tenSpan := tenantPrefix(tenID)
if tenSpan.ContainsKey(args.Key) {
return nil
}
for _, allow := range rangeLookupKeyAllowlist {
if args.Key.Equal(allow) {
return nil
}
if !tenSpan.ContainsKey(args.Key) {
return authErrorf("requested key %s not fully contained in tenant keyspace %s", args.Key, tenSpan)
}
return authErrorf("requested key %s not fully contained in tenant keyspace %s", args.Key, tenSpan)
}

// rangeLookupKeyAllowlist contains keys outside of a tenant's keyspace that
// RangeLookup RPC invocations are allowed to touch.
var rangeLookupKeyAllowlist = []roachpb.Key{
// TODO(nvanbenschoten): Explore whether we can get rid of this by no longer
// reading this key in sqlServer.start.
keys.BootstrapVersionKey,
return nil
}

// authRangeFeed authorizes the provided tenant to invoke the RangeFeed RPC with
Expand Down
29 changes: 25 additions & 4 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/blobs"
"github.com/cockroachdb/cockroach/pkg/blobs/blobspb"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/jobs"
Expand Down Expand Up @@ -673,11 +674,31 @@ func (s *sqlServer) start(
}

var bootstrapVersion roachpb.Version
if err := s.execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
return txn.GetProto(ctx, keys.BootstrapVersionKey, &bootstrapVersion)
}); err != nil {
return err
if s.execCfg.Codec.ForSystemTenant() {
if err := s.execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
return txn.GetProto(ctx, keys.BootstrapVersionKey, &bootstrapVersion)
}); err != nil {
return err
}
} else {
// We don't currently track the bootstrap version of each secondary tenant.
// For this to be meaningful, we'd need to record the binary version of the
// SQL gateway that processed the crdb_internal.create_tenant function which
// created the tenant, as this is what dictates the MetadataSchema that was
// in effect when the secondary tenant was constructed. This binary version
// very well may differ from the cluster-wide bootstrap version at which the
// system tenant was bootstrapped.
//
// Since we don't record this version anywhere, we do the next-best thing
// and pass a lower-bound on the bootstrap version. We know that no tenants
// could have been created before the start of the v20.2 dev cycle, so we
// pass VersionStart20_2. bootstrapVersion is only used to avoid performing
// superfluous but necessarily idempotent SQL migrations, so at worst, we're
// doing more work than strictly necessary during the first time that the
// migrations are run.
bootstrapVersion = clusterversion.VersionByKey(clusterversion.VersionStart20_2)
}

// Run startup migrations (note: these depend on jobs subsystem running).
if err := migMgr.EnsureMigrations(ctx, bootstrapVersion); err != nil {
return errors.Wrap(err, "ensuring SQL migrations")
Expand Down

0 comments on commit e9abbac

Please sign in to comment.