Skip to content

Commit

Permalink
rpc: implement tenant access control policies at KV RPC boundary
Browse files Browse the repository at this point in the history
Fixes #47898.

Rebased on #51503 and #52034. Ignore all but the last 3 commits.

This commit adds a collection of access control policies for the newly
exposed tenant RPC server. These authorization policies ensure that an
authenticated tenant is only able to access keys within its keyspace and
that no tenant is able to access data from another tenant's keyspace
through the tenant RPC server. This is a major step in providing
crypto-backed logical isolation between tenants in a multi-tenant
cluster.

The existing auth mechanism is retained on the standard RPC server,
which means that the system tenant is still able to access any key in
the system.
  • Loading branch information
nvanbenschoten committed Aug 4, 2020
1 parent acfa3d4 commit f21b856
Show file tree
Hide file tree
Showing 6 changed files with 702 additions and 59 deletions.
10 changes: 8 additions & 2 deletions pkg/ccl/kvccl/kvtenantccl/proxy.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight"
"github.com/cockroachdb/errors"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)

func init() {
Expand Down Expand Up @@ -294,13 +296,17 @@ func (p *Proxy) RangeLookup(
PrefetchReverse: useReverseScan,
})
if err != nil {
// Soft RPC error. Drop client and retry.
log.Warningf(ctx, "error issuing RangeLookup RPC: %v", err)
if status.Code(err) == codes.Unauthenticated {
// Authentication error. Propagate.
return nil, nil, err
}
// Soft RPC error. Drop client and retry.
p.tryForgetClient(ctx, client)
continue
}
if resp.Error != nil {
// Hard logical error.
// Hard logical error. Propagate.
return nil, nil, resp.Error.GoError()
}
return resp.Descriptors, resp.PrefetchedDescriptors, nil
Expand Down
17 changes: 17 additions & 0 deletions pkg/roachpb/tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package roachpb

import (
"context"
"math"
"strconv"
)
Expand Down Expand Up @@ -76,3 +77,19 @@ func checkValid(id uint64) {
func IsSystemTenantID(id uint64) bool {
return id == SystemTenantID.ToUint64()
}

type tenantKey struct{}

// NewContextForTenant creates a new context with tenant information attached.
func NewContextForTenant(ctx context.Context, tenID TenantID) context.Context {
return context.WithValue(ctx, tenantKey{}, tenID)
}

// TenantFromContext returns the tenant information in ctx if it exists.
func TenantFromContext(ctx context.Context) (tenID TenantID, ok bool) {
tenID, ok = ctx.Value(tenantKey{}).(TenantID)
return
}

// Silence unused warning.
var _ = TenantFromContext
101 changes: 101 additions & 0 deletions pkg/rpc/auth.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package rpc

import (
"context"

"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/peer"
"google.golang.org/grpc/status"
)

var errTLSInfoMissing = authError("TLSInfo is not available in request context")

func authError(msg string) error {
return status.Error(codes.Unauthenticated, msg)
}

func authErrorf(format string, a ...interface{}) error {
return status.Errorf(codes.Unauthenticated, format, a...)
}

// auth is a policy that performs authentication and authorization for unary and
// streaming RPC invocations. An auth enforces its policy through a pair of gRPC
// interceptors that it exports.
type auth interface {
// AuthUnary returns an interceptor to validate unary RPCs.
AuthUnary() grpc.UnaryServerInterceptor

// AuthUnary returns an interceptor to validate streaming RPCs.
AuthStream() grpc.StreamServerInterceptor
}

// kvAuth is the standard auth policy used for RPCs sent to an RPC server. It
// validates that client TLS certificate provided by the incoming connection
// contains a sufficiently privileged user.
type kvAuth struct{}

// kvAuth implements the auth interface.
func (a kvAuth) AuthUnary() grpc.UnaryServerInterceptor { return a.unaryInterceptor }
func (a kvAuth) AuthStream() grpc.StreamServerInterceptor { return a.streamInterceptor }

func (a kvAuth) unaryInterceptor(
ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler,
) (interface{}, error) {
if err := a.requireSuperUser(ctx); err != nil {
return nil, err
}
return handler(ctx, req)
}

func (a kvAuth) streamInterceptor(
srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler,
) error {
if err := a.requireSuperUser(ss.Context()); err != nil {
return err
}
return handler(srv, ss)
}

func (a kvAuth) requireSuperUser(ctx context.Context) error {
// TODO(marc): grpc's authentication model (which gives credential access in
// the request handler) doesn't really fit with the current design of the
// security package (which assumes that TLS state is only given at connection
// time) - that should be fixed.
if grpcutil.IsLocalRequestContext(ctx) {
// This is an in-process request. Bypass authentication check.
} else if peer, ok := peer.FromContext(ctx); ok {
if tlsInfo, ok := peer.AuthInfo.(credentials.TLSInfo); ok {
certUsers, err := security.GetCertificateUsers(&tlsInfo.State)
if err != nil {
return err
}
// TODO(benesch): the vast majority of RPCs should be limited to just
// NodeUser. This is not a security concern, as RootUser has access to
// read and write all data, merely good hygiene. For example, there is
// no reason to permit the root user to send raw Raft RPCs.
if !security.ContainsUser(security.NodeUser, certUsers) &&
!security.ContainsUser(security.RootUser, certUsers) {
return authErrorf("user %s is not allowed to perform this RPC", certUsers)
}
} else {
// TODO DURING REVIEW: is this a typo or intentional?
}
} else {
return errTLSInfoMissing
}
return nil
}
Loading

0 comments on commit f21b856

Please sign in to comment.