Skip to content

Commit

Permalink
Merge #53211
Browse files Browse the repository at this point in the history
53211: sql/multitenancy: implement SHOW/CANCEL QUERIES/SESSIONS r=tbg a=asubiotto

This PR narrows down the StatusServer service into an interface used by SQL minus `Nodes` which remains in the `TenantDeprecatedWrapper` as a separate interface to discourage use of any other StatusServer methods. The narrowed-down interface is then implemented using a local-only `tenantStatusServer` which gives us the ability to SHOW/CANCEL QUERIES/SESSIONS.

Release note: None

Closes #47895

Co-authored-by: Alfonso Subiotto Marques <[email protected]>
  • Loading branch information
craig[bot] and asubiotto committed Sep 9, 2020
2 parents eba9eed + 39e7448 commit da982a5
Show file tree
Hide file tree
Showing 20 changed files with 610 additions and 449 deletions.
36 changes: 23 additions & 13 deletions pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ var errAdminAPIError = status.Errorf(codes.Internal, "An internal server error "
// A adminServer provides a RESTful HTTP API to administration of
// the cockroach cluster.
type adminServer struct {
*adminPrivilegeChecker
server *Server
memMonitor *mon.BytesMonitor
}
Expand All @@ -103,8 +104,11 @@ var noteworthyAdminMemoryUsageBytes = envutil.EnvOrDefaultInt64("COCKROACH_NOTEW

// newAdminServer allocates and returns a new REST server for
// administrative APIs.
func newAdminServer(s *Server) *adminServer {
server := &adminServer{server: s}
func newAdminServer(s *Server, ie *sql.InternalExecutor) *adminServer {
server := &adminServer{
adminPrivilegeChecker: &adminPrivilegeChecker{ie: ie},
server: s,
}
// TODO(knz): We do not limit memory usage by admin operations
// yet. Is this wise?
server.memMonitor = mon.NewUnlimitedMonitor(
Expand Down Expand Up @@ -2398,8 +2402,14 @@ func (s *adminServer) dialNode(
return serverpb.NewAdminClient(conn), nil
}

func (s *adminServer) requireAdminUser(ctx context.Context) (userName string, err error) {
userName, isAdmin, err := s.getUserAndRole(ctx)
// adminPrivilegeChecker is a helper struct to check whether given usernames
// have admin privileges.
type adminPrivilegeChecker struct {
ie *sql.InternalExecutor
}

func (c *adminPrivilegeChecker) requireAdminUser(ctx context.Context) (userName string, err error) {
userName, isAdmin, err := c.getUserAndRole(ctx)
if err != nil {
return "", err
}
Expand All @@ -2409,15 +2419,15 @@ func (s *adminServer) requireAdminUser(ctx context.Context) (userName string, er
return userName, nil
}

func (s *adminServer) requireViewActivityPermission(
func (c *adminPrivilegeChecker) requireViewActivityPermission(
ctx context.Context,
) (userName string, err error) {
userName, isAdmin, err := s.getUserAndRole(ctx)
userName, isAdmin, err := c.getUserAndRole(ctx)
if err != nil {
return "", err
}
if !isAdmin {
hasViewActivity, err := s.hasRoleOption(ctx, userName, roleoption.VIEWACTIVITY)
hasViewActivity, err := c.hasRoleOption(ctx, userName, roleoption.VIEWACTIVITY)
if err != nil {
return "", err
}
Expand All @@ -2430,23 +2440,23 @@ func (s *adminServer) requireViewActivityPermission(
return userName, nil
}

func (s *adminServer) getUserAndRole(
func (c *adminPrivilegeChecker) getUserAndRole(
ctx context.Context,
) (userName string, isAdmin bool, err error) {
userName, err = userFromContext(ctx)
if err != nil {
return "", false, err
}
isAdmin, err = s.hasAdminRole(ctx, userName)
isAdmin, err = c.hasAdminRole(ctx, userName)
return userName, isAdmin, err
}

func (s *adminServer) hasAdminRole(ctx context.Context, user string) (bool, error) {
func (c *adminPrivilegeChecker) hasAdminRole(ctx context.Context, user string) (bool, error) {
if user == security.RootUser {
// Shortcut.
return true, nil
}
rows, _, err := s.server.sqlServer.internalExecutor.QueryWithCols(
rows, _, err := c.ie.QueryWithCols(
ctx, "check-is-admin", nil, /* txn */
sessiondata.InternalExecutorOverride{User: user},
"SELECT crdb_internal.is_admin()")
Expand All @@ -2466,14 +2476,14 @@ func (s *adminServer) hasAdminRole(ctx context.Context, user string) (bool, erro
return bool(dbDatum), nil
}

func (s *adminServer) hasRoleOption(
func (c *adminPrivilegeChecker) hasRoleOption(
ctx context.Context, user string, roleOption roleoption.Option,
) (bool, error) {
if user == security.RootUser {
// Shortcut.
return true, nil
}
rows, _, err := s.server.sqlServer.internalExecutor.QueryWithCols(
rows, _, err := c.ie.QueryWithCols(
ctx, "check-role-option", nil, /* txn */
sessiondata.InternalExecutorOverride{User: user},
"SELECT crdb_internal.has_role_option($1)", roleOption.String())
Expand Down
5 changes: 3 additions & 2 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -542,7 +542,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {

lateBoundServer := &Server{}
// TODO(tbg): give adminServer only what it needs (and avoid circular deps).
sAdmin := newAdminServer(lateBoundServer)
sAdmin := newAdminServer(lateBoundServer, internalExecutor)
sessionRegistry := sql.NewSessionRegistry()

sStatus := newStatusServer(
Expand Down Expand Up @@ -580,7 +580,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {

sqlServer, err := newSQLServer(ctx, sqlServerArgs{
sqlServerOptionalKVArgs: sqlServerOptionalKVArgs{
statusServer: serverpb.MakeOptionalStatusServer(sStatus),
nodesStatusServer: serverpb.MakeOptionalNodesStatusServer(sStatus),
nodeLiveness: optionalnodeliveness.MakeContainer(nodeLiveness),
gossip: gossip.MakeOptionalGossip(g),
grpcServer: grpcServer.Server,
Expand All @@ -607,6 +607,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
circularJobRegistry: jobRegistry,
jobAdoptionStopFile: jobAdoptionStopFile,
protectedtsProvider: protectedtsProvider,
sqlStatusServer: sStatus,
})
if err != nil {
return nil, err
Expand Down
12 changes: 8 additions & 4 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,8 +101,8 @@ type sqlServer struct {
// respective object is available. When it is not, return
// UnsupportedWithMultiTenancy.
type sqlServerOptionalKVArgs struct {
// statusServer gives access to the Status service.
statusServer serverpb.OptionalStatusServer
// nodesStatusServer gives access to the NodesStatus service.
nodesStatusServer serverpb.OptionalNodesStatusServer
// Narrowed down version of *NodeLiveness. Used by jobs and DistSQLPlanner
nodeLiveness optionalnodeliveness.Container
// Gossip is relied upon by distSQLCfg (execinfra.ServerConfig), the executor
Expand Down Expand Up @@ -191,6 +191,9 @@ type sqlServerArgs struct {

// The executorConfig uses the provider.
protectedtsProvider protectedts.Provider

// Used to list sessions and cancel sessions/queries.
sqlStatusServer serverpb.SQLStatusServer
}

func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
Expand Down Expand Up @@ -436,7 +439,8 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
LeaseManager: leaseMgr,
Clock: cfg.clock,
DistSQLSrv: distSQLServer,
StatusServer: cfg.statusServer,
NodesStatusServer: cfg.nodesStatusServer,
SQLStatusServer: cfg.sqlStatusServer,
SessionRegistry: cfg.sessionRegistry,
SQLLivenessReader: cfg.sqlLivenessProvider,
JobRegistry: jobRegistry,
Expand Down Expand Up @@ -615,7 +619,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
codec,
cfg.registry,
distSQLServer.ServerConfig.SessionBoundInternalExecutorFactory,
cfg.statusServer,
cfg.sqlStatusServer,
cfg.isMeta1Leaseholder,
sqlExecutorTestingKnobs,
)
Expand Down
52 changes: 37 additions & 15 deletions pkg/server/serverpb/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,32 +10,54 @@

package serverpb

import "github.com/cockroachdb/cockroach/pkg/util/errorutil"
import (
context "context"

// OptionalStatusServer is a StatusServer that is only optionally present inside
// the SQL subsystem. In practice, it is present on the system tenant, and not
// present on "regular" tenants.
type OptionalStatusServer struct {
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
)

// SQLStatusServer is a smaller version of the serverpb.StatusInterface which
// includes only the methods used by the SQL subsystem.
type SQLStatusServer interface {
ListSessions(context.Context, *ListSessionsRequest) (*ListSessionsResponse, error)
ListLocalSessions(context.Context, *ListSessionsRequest) (*ListSessionsResponse, error)
CancelQuery(context.Context, *CancelQueryRequest) (*CancelQueryResponse, error)
CancelSession(context.Context, *CancelSessionRequest) (*CancelSessionResponse, error)
}

// OptionalNodesStatusServer is a StatusServer that is only optionally present
// inside the SQL subsystem. In practice, it is present on the system tenant,
// and not present on "regular" tenants.
type OptionalNodesStatusServer struct {
w errorutil.TenantSQLDeprecatedWrapper // stores serverpb.StatusServer
}

// MakeOptionalStatusServer initializes and returns an OptionalStatusServer. The
// provided server will be returned via OptionalErr() if and only if it is not
// nil.
func MakeOptionalStatusServer(s StatusServer) OptionalStatusServer {
return OptionalStatusServer{
// Return the status server from OptionalErr() only if one was provided.
// MakeOptionalNodesStatusServer initializes and returns an
// OptionalNodesStatusServer. The provided server will be returned via
// OptionalNodesStatusServer() if and only if it is not nil.
func MakeOptionalNodesStatusServer(s NodesStatusServer) OptionalNodesStatusServer {
return OptionalNodesStatusServer{
// Return the status server from OptionalSQLStatusServer() only if one was provided.
// We don't have any calls to .Deprecated().
w: errorutil.MakeTenantSQLDeprecatedWrapper(s, s != nil /* exposed */),
}
}

// OptionalErr returns the wrapped StatusServer, if it is available. If it is
// not, an error referring to the optionally supplied issues is returned.
func (s *OptionalStatusServer) OptionalErr(issueNos ...int) (StatusServer, error) {
// NodesStatusServer is the subset of the serverpb.StatusInterface that is used
// by the SQL subsystem but is unavailable to tenants.
type NodesStatusServer interface {
Nodes(context.Context, *NodesRequest) (*NodesResponse, error)
}

// OptionalNodesStatusServer returns the wrapped NodesStatusServer, if it is
// available. If it is not, an error referring to the optionally supplied issues
// is returned.
func (s *OptionalNodesStatusServer) OptionalNodesStatusServer(
issueNos ...int,
) (NodesStatusServer, error) {
v, err := s.w.OptionalErr(issueNos...)
if err != nil {
return nil, err
}
return v.(StatusServer), nil
return v.(NodesStatusServer), nil
}
6 changes: 3 additions & 3 deletions pkg/server/statement_diagnostics_requests.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ func (s *statusServer) CreateStatementDiagnosticsReport(
ctx = propagateGatewayMetadata(ctx)
ctx = s.AnnotateCtx(ctx)

if _, err := s.admin.requireViewActivityPermission(ctx); err != nil {
if _, err := s.privilegeChecker.requireViewActivityPermission(ctx); err != nil {
return nil, err
}

Expand All @@ -93,7 +93,7 @@ func (s *statusServer) StatementDiagnosticsRequests(
ctx = propagateGatewayMetadata(ctx)
ctx = s.AnnotateCtx(ctx)

if _, err := s.admin.requireViewActivityPermission(ctx); err != nil {
if _, err := s.privilegeChecker.requireViewActivityPermission(ctx); err != nil {
return nil, err
}

Expand Down Expand Up @@ -162,7 +162,7 @@ func (s *statusServer) StatementDiagnostics(
ctx = propagateGatewayMetadata(ctx)
ctx = s.AnnotateCtx(ctx)

if _, err := s.admin.requireViewActivityPermission(ctx); err != nil {
if _, err := s.privilegeChecker.requireViewActivityPermission(ctx); err != nil {
return nil, err
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/server/statements.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ func (s *statusServer) Statements(
ctx = propagateGatewayMetadata(ctx)
ctx = s.AnnotateCtx(ctx)

if _, err := s.admin.requireViewActivityPermission(ctx); err != nil {
if _, err := s.privilegeChecker.requireViewActivityPermission(ctx); err != nil {
return nil, err
}

Expand Down
Loading

0 comments on commit da982a5

Please sign in to comment.