auth, etcdserver: let maintenance services require root role

This commit lets maintenance services require root privilege. It also
moves AuthInfoFromCtx() from etcdserver to auth pkg for cleaning purpose.
This commit is contained in:
Hitoshi Mitake 2016-11-24 16:34:29 +09:00 committed by Hitoshi Mitake
parent c89eae790d
commit 9886e9448e
7 changed files with 128 additions and 59 deletions

View File

@ -20,6 +20,7 @@ import (
"errors" "errors"
"fmt" "fmt"
"sort" "sort"
"strconv"
"strings" "strings"
"sync" "sync"
@ -29,6 +30,7 @@ import (
"github.com/coreos/pkg/capnslog" "github.com/coreos/pkg/capnslog"
"golang.org/x/crypto/bcrypt" "golang.org/x/crypto/bcrypt"
"golang.org/x/net/context" "golang.org/x/net/context"
"google.golang.org/grpc/metadata"
) )
var ( var (
@ -57,6 +59,7 @@ var (
ErrPermissionNotGranted = errors.New("auth: permission is not granted to the role") ErrPermissionNotGranted = errors.New("auth: permission is not granted to the role")
ErrAuthNotEnabled = errors.New("auth: authentication is not enabled") ErrAuthNotEnabled = errors.New("auth: authentication is not enabled")
ErrAuthOldRevision = errors.New("auth: revision in header is old") ErrAuthOldRevision = errors.New("auth: revision in header is old")
ErrInvalidAuthToken = errors.New("auth: invalid auth token")
// BcryptCost is the algorithm cost / strength for hashing auth passwords // BcryptCost is the algorithm cost / strength for hashing auth passwords
BcryptCost = bcrypt.DefaultCost BcryptCost = bcrypt.DefaultCost
@ -153,6 +156,9 @@ type AuthStore interface {
// Close does cleanup of AuthStore // Close does cleanup of AuthStore
Close() error Close() error
// AuthInfoFromCtx gets AuthInfo from gRPC's context
AuthInfoFromCtx(ctx context.Context) (*AuthInfo, error)
} }
type authStore struct { type authStore struct {
@ -167,6 +173,8 @@ type authStore struct {
simpleTokenKeeper *simpleTokenTTLKeeper simpleTokenKeeper *simpleTokenTTLKeeper
revision uint64 revision uint64
indexWaiter func(uint64) <-chan struct{}
} }
func (as *authStore) AuthEnable() error { func (as *authStore) AuthEnable() error {
@ -871,7 +879,7 @@ func (as *authStore) isAuthEnabled() bool {
return as.enabled return as.enabled
} }
func NewAuthStore(be backend.Backend) *authStore { func NewAuthStore(be backend.Backend, indexWaiter func(uint64) <-chan struct{}) *authStore {
tx := be.BatchTx() tx := be.BatchTx()
tx.Lock() tx.Lock()
@ -883,6 +891,7 @@ func NewAuthStore(be backend.Backend) *authStore {
be: be, be: be,
simpleTokens: make(map[string]string), simpleTokens: make(map[string]string),
revision: 0, revision: 0,
indexWaiter: indexWaiter,
} }
as.commitRevision(tx) as.commitRevision(tx)
@ -921,3 +930,46 @@ func getRevision(tx backend.BatchTx) uint64 {
func (as *authStore) Revision() uint64 { func (as *authStore) Revision() uint64 {
return as.revision return as.revision
} }
func (as *authStore) isValidSimpleToken(token string, ctx context.Context) bool {
splitted := strings.Split(token, ".")
if len(splitted) != 2 {
return false
}
index, err := strconv.Atoi(splitted[1])
if err != nil {
return false
}
select {
case <-as.indexWaiter(uint64(index)):
return true
case <-ctx.Done():
}
return false
}
func (as *authStore) AuthInfoFromCtx(ctx context.Context) (*AuthInfo, error) {
md, ok := metadata.FromContext(ctx)
if !ok {
return nil, nil
}
ts, tok := md["token"]
if !tok {
return nil, nil
}
token := ts[0]
if !as.isValidSimpleToken(token, ctx) {
return nil, ErrInvalidAuthToken
}
authInfo, uok := as.AuthInfoFromToken(token)
if !uok {
plog.Warningf("invalid auth token: %s", token)
return nil, ErrInvalidAuthToken
}
return authInfo, nil
}

View File

@ -26,6 +26,14 @@ import (
func init() { BcryptCost = bcrypt.MinCost } func init() { BcryptCost = bcrypt.MinCost }
func dummyIndexWaiter(index uint64) <-chan struct{} {
ch := make(chan struct{})
go func() {
ch <- struct{}{}
}()
return ch
}
func TestUserAdd(t *testing.T) { func TestUserAdd(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend() b, tPath := backend.NewDefaultTmpBackend()
defer func() { defer func() {
@ -33,7 +41,7 @@ func TestUserAdd(t *testing.T) {
os.Remove(tPath) os.Remove(tPath)
}() }()
as := NewAuthStore(b) as := NewAuthStore(b, dummyIndexWaiter)
ua := &pb.AuthUserAddRequest{Name: "foo"} ua := &pb.AuthUserAddRequest{Name: "foo"}
_, err := as.UserAdd(ua) // add a non-existing user _, err := as.UserAdd(ua) // add a non-existing user
if err != nil { if err != nil {
@ -80,7 +88,7 @@ func TestCheckPassword(t *testing.T) {
os.Remove(tPath) os.Remove(tPath)
}() }()
as := NewAuthStore(b) as := NewAuthStore(b, dummyIndexWaiter)
defer as.Close() defer as.Close()
err := enableAuthAndCreateRoot(as) err := enableAuthAndCreateRoot(as)
if err != nil { if err != nil {
@ -125,7 +133,7 @@ func TestUserDelete(t *testing.T) {
os.Remove(tPath) os.Remove(tPath)
}() }()
as := NewAuthStore(b) as := NewAuthStore(b, dummyIndexWaiter)
defer as.Close() defer as.Close()
err := enableAuthAndCreateRoot(as) err := enableAuthAndCreateRoot(as)
if err != nil { if err != nil {
@ -162,7 +170,7 @@ func TestUserChangePassword(t *testing.T) {
os.Remove(tPath) os.Remove(tPath)
}() }()
as := NewAuthStore(b) as := NewAuthStore(b, dummyIndexWaiter)
defer as.Close() defer as.Close()
err := enableAuthAndCreateRoot(as) err := enableAuthAndCreateRoot(as)
if err != nil { if err != nil {
@ -208,7 +216,7 @@ func TestRoleAdd(t *testing.T) {
os.Remove(tPath) os.Remove(tPath)
}() }()
as := NewAuthStore(b) as := NewAuthStore(b, dummyIndexWaiter)
defer as.Close() defer as.Close()
err := enableAuthAndCreateRoot(as) err := enableAuthAndCreateRoot(as)
if err != nil { if err != nil {
@ -229,7 +237,7 @@ func TestUserGrant(t *testing.T) {
os.Remove(tPath) os.Remove(tPath)
}() }()
as := NewAuthStore(b) as := NewAuthStore(b, dummyIndexWaiter)
defer as.Close() defer as.Close()
err := enableAuthAndCreateRoot(as) err := enableAuthAndCreateRoot(as)
if err != nil { if err != nil {

View File

@ -18,6 +18,7 @@ import (
"crypto/sha256" "crypto/sha256"
"io" "io"
"github.com/coreos/etcd/auth"
"github.com/coreos/etcd/etcdserver" "github.com/coreos/etcd/etcdserver"
pb "github.com/coreos/etcd/etcdserver/etcdserverpb" pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
"github.com/coreos/etcd/mvcc" "github.com/coreos/etcd/mvcc"
@ -45,6 +46,10 @@ type RaftStatusGetter interface {
Leader() types.ID Leader() types.ID
} }
type AuthGetter interface {
AuthStore() auth.AuthStore
}
type maintenanceServer struct { type maintenanceServer struct {
rg RaftStatusGetter rg RaftStatusGetter
kg KVGetter kg KVGetter
@ -54,7 +59,8 @@ type maintenanceServer struct {
} }
func NewMaintenanceServer(s *etcdserver.EtcdServer) pb.MaintenanceServer { func NewMaintenanceServer(s *etcdserver.EtcdServer) pb.MaintenanceServer {
return &maintenanceServer{rg: s, kg: s, bg: s, a: s, hdr: newHeader(s)} srv := &maintenanceServer{rg: s, kg: s, bg: s, a: s, hdr: newHeader(s)}
return &authMaintenanceServer{srv, s}
} }
func (ms *maintenanceServer) Defragment(ctx context.Context, sr *pb.DefragmentRequest) (*pb.DefragmentResponse, error) { func (ms *maintenanceServer) Defragment(ctx context.Context, sr *pb.DefragmentRequest) (*pb.DefragmentResponse, error) {
@ -139,3 +145,49 @@ func (ms *maintenanceServer) Status(ctx context.Context, ar *pb.StatusRequest) (
ms.hdr.fill(resp.Header) ms.hdr.fill(resp.Header)
return resp, nil return resp, nil
} }
type authMaintenanceServer struct {
*maintenanceServer
ag AuthGetter
}
func (ams *authMaintenanceServer) isAuthenticated(ctx context.Context) error {
authInfo, err := ams.ag.AuthStore().AuthInfoFromCtx(ctx)
if err != nil {
return err
}
return ams.ag.AuthStore().IsAdminPermitted(authInfo)
}
func (ams *authMaintenanceServer) Defragment(ctx context.Context, sr *pb.DefragmentRequest) (*pb.DefragmentResponse, error) {
if err := ams.isAuthenticated(ctx); err != nil {
return nil, err
}
return ams.maintenanceServer.Defragment(ctx, sr)
}
func (ams *authMaintenanceServer) Snapshot(sr *pb.SnapshotRequest, srv pb.Maintenance_SnapshotServer) error {
if err := ams.isAuthenticated(srv.Context()); err != nil {
return err
}
return ams.maintenanceServer.Snapshot(sr, srv)
}
func (ams *authMaintenanceServer) Hash(ctx context.Context, r *pb.HashRequest) (*pb.HashResponse, error) {
if err := ams.isAuthenticated(ctx); err != nil {
return nil, err
}
return ams.maintenanceServer.Hash(ctx, r)
}
func (ams *authMaintenanceServer) Status(ctx context.Context, ar *pb.StatusRequest) (*pb.StatusResponse, error) {
if err := ams.isAuthenticated(ctx); err != nil {
return nil, err
}
return ams.maintenanceServer.Status(ctx, ar)
}

View File

@ -93,7 +93,7 @@ func togRPCError(err error) error {
return rpctypes.ErrGRPCPermissionNotGranted return rpctypes.ErrGRPCPermissionNotGranted
case auth.ErrAuthNotEnabled: case auth.ErrAuthNotEnabled:
return rpctypes.ErrGRPCAuthNotEnabled return rpctypes.ErrGRPCAuthNotEnabled
case etcdserver.ErrInvalidAuthToken: case auth.ErrInvalidAuthToken:
return rpctypes.ErrGRPCInvalidAuthToken return rpctypes.ErrGRPCInvalidAuthToken
default: default:
return grpc.Errorf(codes.Unknown, err.Error()) return grpc.Errorf(codes.Unknown, err.Error())

View File

@ -31,7 +31,6 @@ var (
ErrNoLeader = errors.New("etcdserver: no leader") ErrNoLeader = errors.New("etcdserver: no leader")
ErrRequestTooLarge = errors.New("etcdserver: request is too large") ErrRequestTooLarge = errors.New("etcdserver: request is too large")
ErrNoSpace = errors.New("etcdserver: no space") ErrNoSpace = errors.New("etcdserver: no space")
ErrInvalidAuthToken = errors.New("etcdserver: invalid auth token")
ErrTooManyRequests = errors.New("etcdserver: too many requests") ErrTooManyRequests = errors.New("etcdserver: too many requests")
ErrUnhealthy = errors.New("etcdserver: unhealthy cluster") ErrUnhealthy = errors.New("etcdserver: unhealthy cluster")
) )

View File

@ -459,7 +459,10 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
} }
srv.consistIndex.setConsistentIndex(srv.kv.ConsistentIndex()) srv.consistIndex.setConsistentIndex(srv.kv.ConsistentIndex())
srv.authStore = auth.NewAuthStore(srv.be) srv.authStore = auth.NewAuthStore(srv.be,
func(index uint64) <-chan struct{} {
return srv.applyWait.Wait(index)
})
if h := cfg.AutoCompactionRetention; h != 0 { if h := cfg.AutoCompactionRetention; h != 0 {
srv.compactor = compactor.NewPeriodic(h, srv.kv, srv) srv.compactor = compactor.NewPeriodic(h, srv.kv, srv)
srv.compactor.Run() srv.compactor.Run()
@ -1019,7 +1022,7 @@ func (s *EtcdServer) checkMembershipOperationPermission(ctx context.Context) err
// in the state machine layer // in the state machine layer
// However, both of membership change and role management requires the root privilege. // However, both of membership change and role management requires the root privilege.
// So careful operation by admins can prevent the problem. // So careful operation by admins can prevent the problem.
authInfo, err := s.authInfoFromCtx(ctx) authInfo, err := s.AuthStore().AuthInfoFromCtx(ctx)
if err != nil { if err != nil {
return err return err
} }

View File

@ -17,8 +17,6 @@ package etcdserver
import ( import (
"bytes" "bytes"
"encoding/binary" "encoding/binary"
"strconv"
"strings"
"time" "time"
"github.com/coreos/etcd/auth" "github.com/coreos/etcd/auth"
@ -31,7 +29,6 @@ import (
"github.com/coreos/go-semver/semver" "github.com/coreos/go-semver/semver"
"golang.org/x/net/context" "golang.org/x/net/context"
"google.golang.org/grpc/metadata"
) )
const ( const (
@ -617,52 +614,10 @@ func (s *EtcdServer) RoleDelete(ctx context.Context, r *pb.AuthRoleDeleteRequest
return result.resp.(*pb.AuthRoleDeleteResponse), nil return result.resp.(*pb.AuthRoleDeleteResponse), nil
} }
func (s *EtcdServer) isValidSimpleToken(token string) bool {
splitted := strings.Split(token, ".")
if len(splitted) != 2 {
return false
}
index, err := strconv.Atoi(splitted[1])
if err != nil {
return false
}
select {
case <-s.applyWait.Wait(uint64(index)):
return true
case <-s.stop:
return true
}
}
func (s *EtcdServer) authInfoFromCtx(ctx context.Context) (*auth.AuthInfo, error) {
md, ok := metadata.FromContext(ctx)
if !ok {
return nil, nil
}
ts, tok := md["token"]
if !tok {
return nil, nil
}
token := ts[0]
if !s.isValidSimpleToken(token) {
return nil, ErrInvalidAuthToken
}
authInfo, uok := s.AuthStore().AuthInfoFromToken(token)
if !uok {
plog.Warningf("invalid auth token: %s", token)
return nil, ErrInvalidAuthToken
}
return authInfo, nil
}
// doSerialize handles the auth logic, with permissions checked by "chk", for a serialized request "get". Returns a non-nil error on authentication failure. // doSerialize handles the auth logic, with permissions checked by "chk", for a serialized request "get". Returns a non-nil error on authentication failure.
func (s *EtcdServer) doSerialize(ctx context.Context, chk func(*auth.AuthInfo) error, get func()) error { func (s *EtcdServer) doSerialize(ctx context.Context, chk func(*auth.AuthInfo) error, get func()) error {
for { for {
ai, err := s.authInfoFromCtx(ctx) ai, err := s.AuthStore().AuthInfoFromCtx(ctx)
if err != nil { if err != nil {
return err return err
} }
@ -697,7 +652,7 @@ func (s *EtcdServer) processInternalRaftRequestOnce(ctx context.Context, r pb.In
ID: s.reqIDGen.Next(), ID: s.reqIDGen.Next(),
} }
authInfo, err := s.authInfoFromCtx(ctx) authInfo, err := s.AuthStore().AuthInfoFromCtx(ctx)
if err != nil { if err != nil {
return nil, err return nil, err
} }