auth, etcdserver: introduce revision of authStore for avoiding TOCTOU problem

This commit introduces revision of authStore. The revision number
represents a version of authStore that is incremented by updating auth
related information.

The revision is required for avoiding TOCTOU problems. Currently there
are two types of the TOCTOU problems in v3 auth.

The first one is in ordinal linearizable requests with a sequence like
below ():
1. Request from client CA is processed in follower FA. FA looks up the
   username (let it U) for the request from a token of the request. At
   this time, the request is authorized correctly.
2. Another request from client CB is processed in follower FB. CB
   is for changing U's password.
3. FB forwards the request from CB to the leader before FA. Now U's
   password is updated and the request from CA should be rejected.
4. However, the request from CA is processed by the leader because
   authentication is already done in FA.

For avoiding the above sequence, this commit lets
etcdserverpb.RequestHeader have a member revision. The member is
initialized during authentication by followers and checked in a
leader. If the revision in RequestHeader is lower than the leader's
authStore revision, it means a sequence like above happened. In such a
case, the state machine returns auth.ErrAuthRevisionObsolete. The
error code lets nodes retry their requests.

The second one, a case of serializable range and txn, is more
subtle. Because these requests are processed in follower directly. The
TOCTOU problem can be caused by a sequence like below:
1. Serializable request from client CA is processed in follower FA. At
   first, FA looks up the username (let it U) and its permission
   before actual access to KV.
2. Another request from client CB is processed in follower FB and
   forwarded to the leader. The cluster including FA now commits a log
   entry of the request from CB. Assume the request changed the
   permission or password of U.
3. Now the serializable request from CA is accessing to KV. Even if
   the access is allowed at the point of 1, now it can be invalid
   because of the change introduced in 2.

For avoiding the above sequence, this commit lets the functions of
serializable requests (EtcdServer.Range() and EtcdServer.Txn())
compare the revision in the request header with the latest revision of
authStore after the actual access. If the saved revision is lower than
the latest one, it means the permission can be changed. Although it
would introduce false positives (e.g. changing other user's password),
it prevents the TOCTOU problem. This idea is an implementation of
Anthony's comment:
https://github.com/coreos/etcd/pull/5739#issuecomment-228128254
This commit is contained in:
Hitoshi Mitake 2016-06-23 18:31:12 +09:00
parent 6e290abee2
commit ef6b74411c
6 changed files with 341 additions and 145 deletions

View File

@ -20,6 +20,7 @@ package auth
import ( import (
"crypto/rand" "crypto/rand"
"math/big" "math/big"
"strings"
) )
const ( const (
@ -53,3 +54,14 @@ func (as *authStore) assignSimpleTokenToUser(username, token string) {
as.simpleTokens[token] = username as.simpleTokens[token] = username
as.simpleTokensMu.Unlock() as.simpleTokensMu.Unlock()
} }
func (as *authStore) invalidateUser(username string) {
as.simpleTokensMu.Lock()
defer as.simpleTokensMu.Unlock()
for token, name := range as.simpleTokens {
if strings.Compare(name, username) == 0 {
delete(as.simpleTokens, token)
}
}
}

View File

@ -16,6 +16,7 @@ package auth
import ( import (
"bytes" "bytes"
"encoding/binary"
"errors" "errors"
"fmt" "fmt"
"sort" "sort"
@ -35,6 +36,8 @@ var (
authEnabled = []byte{1} authEnabled = []byte{1}
authDisabled = []byte{0} authDisabled = []byte{0}
revisionKey = []byte("authRevision")
authBucketName = []byte("auth") authBucketName = []byte("auth")
authUsersBucketName = []byte("authUsers") authUsersBucketName = []byte("authUsers")
authRolesBucketName = []byte("authRoles") authRolesBucketName = []byte("authRoles")
@ -52,6 +55,7 @@ var (
ErrRoleNotGranted = errors.New("auth: role is not granted to the user") ErrRoleNotGranted = errors.New("auth: role is not granted to the user")
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")
// 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
@ -60,8 +64,15 @@ var (
const ( const (
rootUser = "root" rootUser = "root"
rootRole = "root" rootRole = "root"
revBytesLen = 8
) )
type AuthInfo struct {
Username string
Revision uint64
}
type AuthStore interface { type AuthStore interface {
// AuthEnable turns on the authentication feature // AuthEnable turns on the authentication feature
AuthEnable() error AuthEnable() error
@ -114,23 +125,27 @@ type AuthStore interface {
// RoleList gets a list of all roles // RoleList gets a list of all roles
RoleList(r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error) RoleList(r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error)
// UsernameFromToken gets a username from the given Token // AuthInfoFromToken gets a username from the given Token and current revision number
UsernameFromToken(token string) (string, bool) // (The revision number is used for preventing the TOCTOU problem)
AuthInfoFromToken(token string) (*AuthInfo, bool)
// IsPutPermitted checks put permission of the user // IsPutPermitted checks put permission of the user
IsPutPermitted(username string, key []byte) bool IsPutPermitted(authInfo *AuthInfo, key []byte) error
// IsRangePermitted checks range permission of the user // IsRangePermitted checks range permission of the user
IsRangePermitted(username string, key, rangeEnd []byte) bool IsRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error
// IsDeleteRangePermitted checks delete-range permission of the user // IsDeleteRangePermitted checks delete-range permission of the user
IsDeleteRangePermitted(username string, key, rangeEnd []byte) bool IsDeleteRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error
// IsAdminPermitted checks admin permission of the user // IsAdminPermitted checks admin permission of the user
IsAdminPermitted(username string) bool IsAdminPermitted(authInfo *AuthInfo) error
// GenSimpleToken produces a simple random string // GenSimpleToken produces a simple random string
GenSimpleToken() (string, error) GenSimpleToken() (string, error)
// Revision gets current revision of authStore
Revision() uint64
} }
type authStore struct { type authStore struct {
@ -142,6 +157,8 @@ type authStore struct {
simpleTokensMu sync.RWMutex simpleTokensMu sync.RWMutex
simpleTokens map[string]string // token -> username simpleTokens map[string]string // token -> username
revision uint64
} }
func (as *authStore) AuthEnable() error { func (as *authStore) AuthEnable() error {
@ -170,6 +187,8 @@ func (as *authStore) AuthEnable() error {
as.rangePermCache = make(map[string]*unifiedRangePermissions) as.rangePermCache = make(map[string]*unifiedRangePermissions)
as.revision = getRevision(tx)
plog.Noticef("Authentication enabled") plog.Noticef("Authentication enabled")
return nil return nil
@ -180,6 +199,7 @@ func (as *authStore) AuthDisable() {
tx := b.BatchTx() tx := b.BatchTx()
tx.Lock() tx.Lock()
tx.UnsafePut(authBucketName, enableFlagKey, authDisabled) tx.UnsafePut(authBucketName, enableFlagKey, authDisabled)
as.commitRevision(tx)
tx.Unlock() tx.Unlock()
b.ForceCommit() b.ForceCommit()
@ -235,6 +255,9 @@ func (as *authStore) Recover(be backend.Backend) {
enabled = true enabled = true
} }
} }
as.revision = getRevision(tx)
tx.Unlock() tx.Unlock()
as.enabledMu.Lock() as.enabledMu.Lock()
@ -265,6 +288,8 @@ func (as *authStore) UserAdd(r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse,
putUser(tx, newUser) putUser(tx, newUser)
as.commitRevision(tx)
plog.Noticef("added a new user: %s", r.Name) plog.Noticef("added a new user: %s", r.Name)
return &pb.AuthUserAddResponse{}, nil return &pb.AuthUserAddResponse{}, nil
@ -282,6 +307,11 @@ func (as *authStore) UserDelete(r *pb.AuthUserDeleteRequest) (*pb.AuthUserDelete
delUser(tx, r.Name) delUser(tx, r.Name)
as.commitRevision(tx)
as.invalidateCachedPerm(r.Name)
as.invalidateUser(r.Name)
plog.Noticef("deleted a user: %s", r.Name) plog.Noticef("deleted a user: %s", r.Name)
return &pb.AuthUserDeleteResponse{}, nil return &pb.AuthUserDeleteResponse{}, nil
@ -313,6 +343,11 @@ func (as *authStore) UserChangePassword(r *pb.AuthUserChangePasswordRequest) (*p
putUser(tx, updatedUser) putUser(tx, updatedUser)
as.commitRevision(tx)
as.invalidateCachedPerm(r.Name)
as.invalidateUser(r.Name)
plog.Noticef("changed a password of a user: %s", r.Name) plog.Noticef("changed a password of a user: %s", r.Name)
return &pb.AuthUserChangePasswordResponse{}, nil return &pb.AuthUserChangePasswordResponse{}, nil
@ -348,6 +383,8 @@ func (as *authStore) UserGrantRole(r *pb.AuthUserGrantRoleRequest) (*pb.AuthUser
as.invalidateCachedPerm(r.User) as.invalidateCachedPerm(r.User)
as.commitRevision(tx)
plog.Noticef("granted role %s to user %s", r.Role, r.User) plog.Noticef("granted role %s to user %s", r.Role, r.User)
return &pb.AuthUserGrantRoleResponse{}, nil return &pb.AuthUserGrantRoleResponse{}, nil
} }
@ -416,6 +453,8 @@ func (as *authStore) UserRevokeRole(r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUs
as.invalidateCachedPerm(r.Name) as.invalidateCachedPerm(r.Name)
as.commitRevision(tx)
plog.Noticef("revoked role %s from user %s", r.Role, r.Name) plog.Noticef("revoked role %s from user %s", r.Role, r.Name)
return &pb.AuthUserRevokeRoleResponse{}, nil return &pb.AuthUserRevokeRoleResponse{}, nil
} }
@ -485,6 +524,8 @@ func (as *authStore) RoleRevokePermission(r *pb.AuthRoleRevokePermissionRequest)
// It should be optimized. // It should be optimized.
as.clearCachedPerm() as.clearCachedPerm()
as.commitRevision(tx)
plog.Noticef("revoked key %s from role %s", r.Key, r.Role) plog.Noticef("revoked key %s from role %s", r.Key, r.Role)
return &pb.AuthRoleRevokePermissionResponse{}, nil return &pb.AuthRoleRevokePermissionResponse{}, nil
} }
@ -513,6 +554,8 @@ func (as *authStore) RoleDelete(r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDelete
delRole(tx, r.Role) delRole(tx, r.Role)
as.commitRevision(tx)
plog.Noticef("deleted role %s", r.Role) plog.Noticef("deleted role %s", r.Role)
return &pb.AuthRoleDeleteResponse{}, nil return &pb.AuthRoleDeleteResponse{}, nil
} }
@ -533,16 +576,18 @@ func (as *authStore) RoleAdd(r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse,
putRole(tx, newRole) putRole(tx, newRole)
as.commitRevision(tx)
plog.Noticef("Role %s is created", r.Name) plog.Noticef("Role %s is created", r.Name)
return &pb.AuthRoleAddResponse{}, nil return &pb.AuthRoleAddResponse{}, nil
} }
func (as *authStore) UsernameFromToken(token string) (string, bool) { func (as *authStore) AuthInfoFromToken(token string) (*AuthInfo, bool) {
as.simpleTokensMu.RLock() as.simpleTokensMu.RLock()
defer as.simpleTokensMu.RUnlock() defer as.simpleTokensMu.RUnlock()
t, ok := as.simpleTokens[token] t, ok := as.simpleTokens[token]
return t, ok return &AuthInfo{Username: t, Revision: as.revision}, ok
} }
type permSlice []*authpb.Permission type permSlice []*authpb.Permission
@ -594,15 +639,21 @@ func (as *authStore) RoleGrantPermission(r *pb.AuthRoleGrantPermissionRequest) (
// It should be optimized. // It should be optimized.
as.clearCachedPerm() as.clearCachedPerm()
as.commitRevision(tx)
plog.Noticef("role %s's permission of key %s is updated as %s", r.Name, r.Perm.Key, authpb.Permission_Type_name[int32(r.Perm.PermType)]) plog.Noticef("role %s's permission of key %s is updated as %s", r.Name, r.Perm.Key, authpb.Permission_Type_name[int32(r.Perm.PermType)])
return &pb.AuthRoleGrantPermissionResponse{}, nil return &pb.AuthRoleGrantPermissionResponse{}, nil
} }
func (as *authStore) isOpPermitted(userName string, key, rangeEnd []byte, permTyp authpb.Permission_Type) bool { func (as *authStore) isOpPermitted(userName string, revision uint64, key, rangeEnd []byte, permTyp authpb.Permission_Type) error {
// TODO(mitake): this function would be costly so we need a caching mechanism // TODO(mitake): this function would be costly so we need a caching mechanism
if !as.isAuthEnabled() { if !as.isAuthEnabled() {
return true return nil
}
if revision < as.revision {
return ErrAuthOldRevision
} }
tx := as.be.BatchTx() tx := as.be.BatchTx()
@ -612,43 +663,47 @@ func (as *authStore) isOpPermitted(userName string, key, rangeEnd []byte, permTy
user := getUser(tx, userName) user := getUser(tx, userName)
if user == nil { if user == nil {
plog.Errorf("invalid user name %s for permission checking", userName) plog.Errorf("invalid user name %s for permission checking", userName)
return false return ErrPermissionDenied
} }
if as.isRangeOpPermitted(tx, userName, key, rangeEnd, permTyp) { if as.isRangeOpPermitted(tx, userName, key, rangeEnd, permTyp) {
return true return nil
} }
return false return ErrPermissionDenied
} }
func (as *authStore) IsPutPermitted(username string, key []byte) bool { func (as *authStore) IsPutPermitted(authInfo *AuthInfo, key []byte) error {
return as.isOpPermitted(username, key, nil, authpb.WRITE) return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, nil, authpb.WRITE)
} }
func (as *authStore) IsRangePermitted(username string, key, rangeEnd []byte) bool { func (as *authStore) IsRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error {
return as.isOpPermitted(username, key, rangeEnd, authpb.READ) return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, rangeEnd, authpb.READ)
} }
func (as *authStore) IsDeleteRangePermitted(username string, key, rangeEnd []byte) bool { func (as *authStore) IsDeleteRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error {
return as.isOpPermitted(username, key, rangeEnd, authpb.WRITE) return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, rangeEnd, authpb.WRITE)
} }
func (as *authStore) IsAdminPermitted(username string) bool { func (as *authStore) IsAdminPermitted(authInfo *AuthInfo) error {
if !as.isAuthEnabled() { if !as.isAuthEnabled() {
return true return nil
} }
tx := as.be.BatchTx() tx := as.be.BatchTx()
tx.Lock() tx.Lock()
defer tx.Unlock() defer tx.Unlock()
u := getUser(tx, username) u := getUser(tx, authInfo.Username)
if u == nil { if u == nil {
return false return ErrUserNotFound
} }
return hasRootRole(u) if !hasRootRole(u) {
return ErrPermissionDenied
}
return nil
} }
func getUser(tx backend.BatchTx, username string) *authpb.User { func getUser(tx backend.BatchTx, username string) *authpb.User {
@ -760,13 +815,18 @@ func NewAuthStore(be backend.Backend) *authStore {
tx.UnsafeCreateBucket(authUsersBucketName) tx.UnsafeCreateBucket(authUsersBucketName)
tx.UnsafeCreateBucket(authRolesBucketName) tx.UnsafeCreateBucket(authRolesBucketName)
as := &authStore{
be: be,
simpleTokens: make(map[string]string),
revision: 0,
}
as.commitRevision(tx)
tx.Unlock() tx.Unlock()
be.ForceCommit() be.ForceCommit()
return &authStore{ return as
be: be,
simpleTokens: make(map[string]string),
}
} }
func hasRootRole(u *authpb.User) bool { func hasRootRole(u *authpb.User) bool {
@ -777,3 +837,23 @@ func hasRootRole(u *authpb.User) bool {
} }
return false return false
} }
func (as *authStore) commitRevision(tx backend.BatchTx) {
as.revision++
revBytes := make([]byte, revBytesLen)
binary.BigEndian.PutUint64(revBytes, as.revision)
tx.UnsafePut(authBucketName, revisionKey, revBytes)
}
func getRevision(tx backend.BatchTx) uint64 {
_, vs := tx.UnsafeRange(authBucketName, []byte(revisionKey), nil, 0)
if len(vs) != 1 {
plog.Panicf("failed to get the key of auth store revision")
}
return binary.BigEndian.Uint64(vs[0])
}
func (as *authStore) Revision() uint64 {
return as.revision
}

View File

@ -28,7 +28,8 @@ type authApplierV3 struct {
// mu serializes Apply so that user isn't corrupted and so that // mu serializes Apply so that user isn't corrupted and so that
// serialized requests don't leak data from TOCTOU errors // serialized requests don't leak data from TOCTOU errors
mu sync.Mutex mu sync.Mutex
user string
authInfo auth.AuthInfo
} }
func newAuthApplierV3(as auth.AuthStore, base applierV3) *authApplierV3 { func newAuthApplierV3(as auth.AuthStore, base applierV3) *authApplierV3 {
@ -41,46 +42,57 @@ func (aa *authApplierV3) Apply(r *pb.InternalRaftRequest) *applyResult {
if r.Header != nil { if r.Header != nil {
// backward-compatible with pre-3.0 releases when internalRaftRequest // backward-compatible with pre-3.0 releases when internalRaftRequest
// does not have header field // does not have header field
aa.user = r.Header.Username aa.authInfo.Username = r.Header.Username
aa.authInfo.Revision = r.Header.AuthRevision
}
if needAdminPermission(r) {
if err := aa.as.IsAdminPermitted(&aa.authInfo); err != nil {
aa.authInfo.Username = ""
aa.authInfo.Revision = 0
return &applyResult{err: err}
} }
if needAdminPermission(r) && !aa.as.IsAdminPermitted(aa.user) {
aa.user = ""
return &applyResult{err: auth.ErrPermissionDenied}
} }
ret := aa.applierV3.Apply(r) ret := aa.applierV3.Apply(r)
aa.user = "" aa.authInfo.Username = ""
aa.authInfo.Revision = 0
return ret return ret
} }
func (aa *authApplierV3) Put(txnID int64, r *pb.PutRequest) (*pb.PutResponse, error) { func (aa *authApplierV3) Put(txnID int64, r *pb.PutRequest) (*pb.PutResponse, error) {
if !aa.as.IsPutPermitted(aa.user, r.Key) { if err := aa.as.IsPutPermitted(&aa.authInfo, r.Key); err != nil {
return nil, auth.ErrPermissionDenied return nil, err
}
if r.PrevKv {
err := aa.as.IsRangePermitted(&aa.authInfo, r.Key, nil)
if err != nil {
return nil, err
} }
if r.PrevKv && !aa.as.IsRangePermitted(aa.user, r.Key, nil) {
return nil, auth.ErrPermissionDenied
} }
return aa.applierV3.Put(txnID, r) return aa.applierV3.Put(txnID, r)
} }
func (aa *authApplierV3) Range(txnID int64, r *pb.RangeRequest) (*pb.RangeResponse, error) { func (aa *authApplierV3) Range(txnID int64, r *pb.RangeRequest) (*pb.RangeResponse, error) {
if !aa.as.IsRangePermitted(aa.user, r.Key, r.RangeEnd) { if err := aa.as.IsRangePermitted(&aa.authInfo, r.Key, r.RangeEnd); err != nil {
return nil, auth.ErrPermissionDenied return nil, err
} }
return aa.applierV3.Range(txnID, r) return aa.applierV3.Range(txnID, r)
} }
func (aa *authApplierV3) DeleteRange(txnID int64, r *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) { func (aa *authApplierV3) DeleteRange(txnID int64, r *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
if !aa.as.IsDeleteRangePermitted(aa.user, r.Key, r.RangeEnd) { if err := aa.as.IsDeleteRangePermitted(&aa.authInfo, r.Key, r.RangeEnd); err != nil {
return nil, auth.ErrPermissionDenied return nil, err
}
if r.PrevKv {
err := aa.as.IsRangePermitted(&aa.authInfo, r.Key, r.RangeEnd)
if err != nil {
return nil, err
} }
if r.PrevKv && !aa.as.IsRangePermitted(aa.user, r.Key, r.RangeEnd) {
return nil, auth.ErrPermissionDenied
} }
return aa.applierV3.DeleteRange(txnID, r) return aa.applierV3.DeleteRange(txnID, r)
} }
func (aa *authApplierV3) checkTxnReqsPermission(reqs []*pb.RequestOp) bool { func (aa *authApplierV3) checkTxnReqsPermission(reqs []*pb.RequestOp) error {
for _, requ := range reqs { for _, requ := range reqs {
switch tv := requ.Request.(type) { switch tv := requ.Request.(type) {
case *pb.RequestOp_RequestRange: case *pb.RequestOp_RequestRange:
@ -88,8 +100,8 @@ func (aa *authApplierV3) checkTxnReqsPermission(reqs []*pb.RequestOp) bool {
continue continue
} }
if !aa.as.IsRangePermitted(aa.user, tv.RequestRange.Key, tv.RequestRange.RangeEnd) { if err := aa.as.IsRangePermitted(&aa.authInfo, tv.RequestRange.Key, tv.RequestRange.RangeEnd); err != nil {
return false return err
} }
case *pb.RequestOp_RequestPut: case *pb.RequestOp_RequestPut:
@ -97,8 +109,8 @@ func (aa *authApplierV3) checkTxnReqsPermission(reqs []*pb.RequestOp) bool {
continue continue
} }
if !aa.as.IsPutPermitted(aa.user, tv.RequestPut.Key) { if err := aa.as.IsPutPermitted(&aa.authInfo, tv.RequestPut.Key); err != nil {
return false return err
} }
case *pb.RequestOp_RequestDeleteRange: case *pb.RequestOp_RequestDeleteRange:
@ -106,31 +118,35 @@ func (aa *authApplierV3) checkTxnReqsPermission(reqs []*pb.RequestOp) bool {
continue continue
} }
if tv.RequestDeleteRange.PrevKv && !aa.as.IsRangePermitted(aa.user, tv.RequestDeleteRange.Key, tv.RequestDeleteRange.RangeEnd) { if tv.RequestDeleteRange.PrevKv {
return false err := aa.as.IsRangePermitted(&aa.authInfo, tv.RequestDeleteRange.Key, tv.RequestDeleteRange.RangeEnd)
if err != nil {
return err
}
} }
if !aa.as.IsDeleteRangePermitted(aa.user, tv.RequestDeleteRange.Key, tv.RequestDeleteRange.RangeEnd) { err := aa.as.IsDeleteRangePermitted(&aa.authInfo, tv.RequestDeleteRange.Key, tv.RequestDeleteRange.RangeEnd)
return false if err != nil {
return err
} }
} }
} }
return true return nil
} }
func (aa *authApplierV3) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) { func (aa *authApplierV3) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
for _, c := range rt.Compare { for _, c := range rt.Compare {
if !aa.as.IsRangePermitted(aa.user, c.Key, nil) { if err := aa.as.IsRangePermitted(&aa.authInfo, c.Key, nil); err != nil {
return nil, auth.ErrPermissionDenied return nil, err
} }
} }
if !aa.checkTxnReqsPermission(rt.Success) { if err := aa.checkTxnReqsPermission(rt.Success); err != nil {
return nil, auth.ErrPermissionDenied return nil, err
} }
if !aa.checkTxnReqsPermission(rt.Failure) { if err := aa.checkTxnReqsPermission(rt.Failure); err != nil {
return nil, auth.ErrPermissionDenied return nil, err
} }
return aa.applierV3.Txn(rt) return aa.applierV3.Txn(rt)

View File

@ -23,6 +23,8 @@ type RequestHeader struct {
ID uint64 `protobuf:"varint,1,opt,name=ID,json=iD,proto3" json:"ID,omitempty"` ID uint64 `protobuf:"varint,1,opt,name=ID,json=iD,proto3" json:"ID,omitempty"`
// username is a username that is associated with an auth token of gRPC connection // username is a username that is associated with an auth token of gRPC connection
Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"` Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"`
// auth_revision is a revision number of auth.authStore. It is not related to mvcc
AuthRevision uint64 `protobuf:"varint,3,opt,name=auth_revision,json=authRevision,proto3" json:"auth_revision,omitempty"`
} }
func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) Reset() { *m = RequestHeader{} }
@ -124,6 +126,11 @@ func (m *RequestHeader) MarshalTo(data []byte) (int, error) {
i = encodeVarintRaftInternal(data, i, uint64(len(m.Username))) i = encodeVarintRaftInternal(data, i, uint64(len(m.Username)))
i += copy(data[i:], m.Username) i += copy(data[i:], m.Username)
} }
if m.AuthRevision != 0 {
data[i] = 0x18
i++
i = encodeVarintRaftInternal(data, i, uint64(m.AuthRevision))
}
return i, nil return i, nil
} }
@ -535,6 +542,9 @@ func (m *RequestHeader) Size() (n int) {
if l > 0 { if l > 0 {
n += 1 + l + sovRaftInternal(uint64(l)) n += 1 + l + sovRaftInternal(uint64(l))
} }
if m.AuthRevision != 0 {
n += 1 + sovRaftInternal(uint64(m.AuthRevision))
}
return n return n
} }
@ -765,6 +775,25 @@ func (m *RequestHeader) Unmarshal(data []byte) error {
} }
m.Username = string(data[iNdEx:postIndex]) m.Username = string(data[iNdEx:postIndex])
iNdEx = postIndex iNdEx = postIndex
case 3:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field AuthRevision", wireType)
}
m.AuthRevision = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowRaftInternal
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := data[iNdEx]
iNdEx++
m.AuthRevision |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
default: default:
iNdEx = preIndex iNdEx = preIndex
skippy, err := skipRaftInternal(data[iNdEx:]) skippy, err := skipRaftInternal(data[iNdEx:])
@ -2006,57 +2035,57 @@ var (
) )
var fileDescriptorRaftInternal = []byte{ var fileDescriptorRaftInternal = []byte{
// 824 bytes of a gzipped FileDescriptorProto // 831 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x7c, 0x96, 0xdd, 0x4e, 0xdb, 0x48, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x7c, 0x56, 0xdb, 0x4e, 0xdb, 0x4a,
0x14, 0xc7, 0x49, 0xf8, 0xcc, 0x24, 0x64, 0xd9, 0x01, 0x76, 0x67, 0x83, 0x94, 0x85, 0xac, 0x76, 0x14, 0x25, 0xe1, 0x9a, 0x49, 0xc8, 0xe1, 0x0c, 0x70, 0xce, 0x9c, 0x20, 0x71, 0x20, 0xa8, 0xf7,
0x97, 0x7e, 0xd1, 0x0a, 0x2e, 0x7b, 0xd1, 0xa6, 0x04, 0x01, 0x12, 0x42, 0xc8, 0xa2, 0x52, 0xa5, 0x96, 0x56, 0xf0, 0x01, 0x6d, 0x4a, 0x10, 0x20, 0xa1, 0x0a, 0x59, 0x54, 0xaa, 0xd4, 0x07, 0x77,
0x5e, 0x58, 0x43, 0x7c, 0x48, 0x5c, 0x1c, 0xdb, 0xb5, 0x27, 0x29, 0x7d, 0x93, 0x3e, 0x46, 0xbf, 0x88, 0xa7, 0xc1, 0xad, 0x63, 0xbb, 0xf6, 0x84, 0xd2, 0x3f, 0xe9, 0x67, 0xf4, 0xf6, 0x11, 0x3c,
0x1e, 0x82, 0x8b, 0x7e, 0xd0, 0xf6, 0x05, 0x5a, 0x7a, 0xd3, 0xab, 0xde, 0xb4, 0x0f, 0x50, 0xcd, 0xf4, 0x42, 0xdb, 0x1f, 0xe8, 0xe5, 0xa5, 0x4f, 0x7d, 0x69, 0x3f, 0xa0, 0x73, 0xf3, 0xd8, 0x4e,
0x87, 0xc7, 0x71, 0x32, 0xe1, 0xce, 0x3e, 0xe7, 0x7f, 0x7e, 0xe7, 0x4c, 0xe6, 0x3f, 0x9e, 0xa0, 0xc6, 0x3c, 0x44, 0xb2, 0xd7, 0xac, 0xbd, 0xd6, 0x1e, 0xef, 0x35, 0x0c, 0x60, 0x36, 0xc2, 0x0f,
0xf9, 0x88, 0x1e, 0x33, 0xdb, 0xf5, 0x19, 0x44, 0x3e, 0xf5, 0xd6, 0xc2, 0x28, 0x60, 0x01, 0x2e, 0xa9, 0xed, 0xfa, 0x94, 0x44, 0x3e, 0xf6, 0x56, 0xc3, 0x28, 0xa0, 0x01, 0xac, 0x11, 0xda, 0x71,
0x01, 0x6b, 0x3a, 0x31, 0x44, 0x3d, 0x88, 0xc2, 0xa3, 0xca, 0x42, 0x2b, 0x68, 0x05, 0x22, 0x71, 0x62, 0x12, 0x1d, 0x91, 0x28, 0x3c, 0x68, 0xcc, 0x75, 0x83, 0x6e, 0x20, 0x16, 0xae, 0xf3, 0x27,
0x93, 0x3f, 0x49, 0x4d, 0x65, 0x2e, 0xd5, 0xa8, 0x48, 0x21, 0x0a, 0x9b, 0xf2, 0xb1, 0x76, 0x1b, 0xc9, 0x69, 0xcc, 0xa4, 0x1c, 0x85, 0x54, 0xa2, 0xb0, 0x23, 0x1f, 0x9b, 0x0f, 0xc0, 0xb4, 0x45,
0xcd, 0x5a, 0xf0, 0xb8, 0x0b, 0x31, 0xdb, 0x01, 0xea, 0x40, 0x84, 0xcb, 0x28, 0xbf, 0xdb, 0x20, 0x9e, 0xf4, 0x49, 0x4c, 0xb7, 0x09, 0x76, 0x48, 0x04, 0xeb, 0xa0, 0xbc, 0xd3, 0x46, 0xa5, 0xa5,
0xb9, 0xe5, 0xdc, 0xea, 0x84, 0x95, 0x77, 0x1b, 0xb8, 0x82, 0x66, 0xba, 0x31, 0x6f, 0xd9, 0x01, 0xd2, 0xc5, 0x31, 0xab, 0xec, 0xb6, 0x61, 0x03, 0x4c, 0xf5, 0x63, 0x6e, 0xd9, 0x23, 0xa8, 0xcc,
0x92, 0x5f, 0xce, 0xad, 0x16, 0x2c, 0xfd, 0x5e, 0xfb, 0x5e, 0x46, 0xf3, 0xbb, 0x6a, 0x20, 0x8b, 0xd0, 0x8a, 0xa5, 0xdf, 0xe1, 0x0a, 0x98, 0xc6, 0x7d, 0x7a, 0x68, 0x47, 0xe4, 0xc8, 0x8d, 0xdd,
0x1e, 0x33, 0x45, 0x1a, 0x62, 0xfc, 0x8b, 0xf2, 0xbd, 0x75, 0x51, 0x5d, 0x5c, 0x5f, 0x5c, 0xeb, 0xc0, 0x47, 0xa3, 0xa2, 0xac, 0xc6, 0x41, 0x4b, 0x61, 0xcd, 0x9f, 0x75, 0x30, 0xbb, 0xa3, 0xba,
0x1f, 0x79, 0x4d, 0x95, 0x58, 0xf9, 0xde, 0x3a, 0xbe, 0x85, 0x26, 0x23, 0xea, 0xb7, 0x80, 0x8c, 0xb6, 0xd8, 0x16, 0x94, 0xdd, 0x90, 0xd1, 0x39, 0x50, 0x3e, 0x5a, 0x13, 0x16, 0xd5, 0xb5, 0xf9,
0x0b, 0x65, 0x65, 0x40, 0xc9, 0x53, 0x89, 0x5c, 0x0a, 0xf1, 0x55, 0x34, 0x1e, 0x76, 0x19, 0x99, 0xd5, 0xec, 0xbe, 0x56, 0x55, 0x89, 0xc5, 0x08, 0xf0, 0x06, 0x18, 0x8f, 0xb0, 0xdf, 0x25, 0xc2,
0x10, 0x7a, 0x92, 0xd5, 0x1f, 0x74, 0x93, 0x79, 0x2c, 0x2e, 0xc2, 0x9b, 0xa8, 0xe4, 0x80, 0x07, 0xab, 0xba, 0xd6, 0x18, 0x60, 0xf2, 0xa5, 0x84, 0x2e, 0x89, 0xf0, 0x32, 0x18, 0x0d, 0xfb, 0x14,
0x0c, 0x6c, 0xd9, 0x64, 0x52, 0x14, 0x2d, 0x67, 0x8b, 0x1a, 0x42, 0x91, 0x69, 0x55, 0x74, 0xd2, 0x8d, 0x09, 0x3e, 0xca, 0xf3, 0xf7, 0xfa, 0x49, 0x3f, 0x16, 0x27, 0xc1, 0x0d, 0x50, 0x73, 0x88,
0x18, 0x6f, 0xc8, 0x4e, 0x7d, 0x32, 0x65, 0x6a, 0x78, 0x78, 0xea, 0xeb, 0x86, 0xec, 0xd4, 0xc7, 0x47, 0x28, 0xb1, 0xa5, 0xc9, 0xb8, 0x28, 0x5a, 0xca, 0x17, 0xb5, 0x05, 0x23, 0x67, 0x55, 0x75,
0x77, 0x10, 0x6a, 0x06, 0x9d, 0x90, 0x36, 0x99, 0x1b, 0xf8, 0x64, 0x5a, 0x94, 0xfc, 0x9d, 0x2d, 0x52, 0x8c, 0x1b, 0xd2, 0x63, 0x1f, 0x4d, 0x98, 0x0c, 0xf7, 0x8f, 0x7d, 0x6d, 0xc8, 0x48, 0xf0,
0xd9, 0xd4, 0xf9, 0xa4, 0xb2, 0xaf, 0x04, 0xdf, 0x45, 0x45, 0x0f, 0x68, 0x0c, 0x76, 0x2b, 0xa2, 0x26, 0x00, 0x9d, 0xa0, 0x17, 0xe2, 0x0e, 0xe5, 0xdf, 0x6f, 0x52, 0x94, 0xfc, 0x9f, 0x2f, 0xd9,
0x3e, 0x23, 0x33, 0x26, 0xc2, 0x1e, 0x17, 0x6c, 0xf3, 0xbc, 0x26, 0x78, 0x3a, 0xc4, 0xd7, 0x2c, 0xd0, 0xeb, 0x49, 0x65, 0xa6, 0x04, 0xde, 0x02, 0x55, 0x8f, 0xe0, 0x98, 0xd8, 0x5d, 0xd6, 0x31,
0x09, 0x11, 0xf4, 0x82, 0x13, 0x20, 0x05, 0xd3, 0x9a, 0x05, 0xc2, 0x12, 0x02, 0xbd, 0x66, 0x2f, 0x45, 0x53, 0x26, 0x85, 0x5d, 0x4e, 0xd8, 0xe2, 0xeb, 0x5a, 0xc1, 0xd3, 0x10, 0xdf, 0xb3, 0x54,
0x8d, 0xf1, 0x6d, 0xa1, 0x1e, 0x8d, 0x3a, 0x04, 0x99, 0xb6, 0xa5, 0xce, 0x53, 0x7a, 0x5b, 0x84, 0x60, 0x63, 0x0c, 0x1e, 0x13, 0x54, 0x31, 0xed, 0x59, 0x48, 0x58, 0x82, 0xa0, 0xf7, 0xec, 0xa5,
0x10, 0x6f, 0xa0, 0xa9, 0xb6, 0x70, 0x13, 0x71, 0x44, 0xc9, 0x92, 0x71, 0xcf, 0xa5, 0xe1, 0x2c, 0x18, 0x1f, 0x0b, 0xf6, 0x70, 0xd4, 0x43, 0xc0, 0x34, 0x96, 0x16, 0x5f, 0xd2, 0x63, 0x11, 0x44,
0x25, 0xc5, 0x75, 0x54, 0xa4, 0x5d, 0xd6, 0xb6, 0xc1, 0xa7, 0x47, 0x1e, 0x90, 0x6f, 0xc6, 0x1f, 0xb8, 0x0e, 0x26, 0x0e, 0x45, 0xe4, 0x90, 0x23, 0x4a, 0x16, 0x8c, 0x33, 0x97, 0xa9, 0xb4, 0x14,
0xac, 0xde, 0x65, 0xed, 0x2d, 0x21, 0xd0, 0xcb, 0xa5, 0x3a, 0x84, 0x1b, 0xa8, 0x24, 0x10, 0x8e, 0x15, 0xb6, 0x40, 0x55, 0x24, 0x8e, 0xf8, 0xf8, 0xc0, 0x23, 0xe8, 0x87, 0xf1, 0x83, 0xb5, 0x18,
0x1b, 0x0b, 0xc6, 0x8f, 0x69, 0xd3, 0x7a, 0x39, 0xa3, 0x21, 0x15, 0x7a, 0xbd, 0x34, 0x8d, 0xe1, 0x63, 0x53, 0x10, 0xf4, 0x76, 0xb1, 0x86, 0x60, 0x1b, 0x88, 0x7c, 0xda, 0x8e, 0x1b, 0x0b, 0x8d,
0x7d, 0x49, 0x01, 0x9f, 0xb9, 0x4d, 0xca, 0x80, 0xfc, 0x94, 0x94, 0x2b, 0x59, 0x4a, 0xe2, 0xfb, 0x5f, 0x93, 0xa6, 0xfd, 0x72, 0x8d, 0xb6, 0x64, 0xe8, 0xfd, 0xe2, 0x14, 0x83, 0x77, 0xa4, 0x0a,
0x7a, 0x9f, 0x34, 0xc1, 0x65, 0xea, 0xf1, 0x16, 0x9a, 0x15, 0x53, 0xf1, 0x63, 0x63, 0x53, 0xc7, 0xf1, 0xa9, 0xdb, 0xc1, 0x94, 0xa0, 0xdf, 0x52, 0xe5, 0x52, 0x5e, 0x25, 0xc9, 0x7d, 0x2b, 0x43,
0x21, 0x6f, 0x66, 0x46, 0x8d, 0x75, 0x3f, 0x86, 0xa8, 0xee, 0x38, 0x99, 0xb1, 0x54, 0x0c, 0xef, 0x4d, 0xe4, 0x72, 0xf5, 0x70, 0x53, 0x1d, 0x25, 0x7e, 0xb6, 0x6c, 0xec, 0x38, 0xe8, 0xed, 0x54,
0xa3, 0xb9, 0x14, 0x23, 0x3d, 0x49, 0xde, 0x4a, 0xd2, 0x3f, 0x66, 0x92, 0x32, 0xb3, 0x82, 0x95, 0x51, 0x5b, 0x77, 0xd9, 0x5b, 0xcb, 0x71, 0x72, 0x6d, 0x29, 0x8c, 0xb5, 0x35, 0x93, 0xca, 0xc8,
0x69, 0x26, 0x9c, 0x1d, 0xab, 0x05, 0x8c, 0xbc, 0xbb, 0x74, 0xac, 0x6d, 0x60, 0x43, 0x63, 0x6d, 0x4c, 0xa2, 0x77, 0x52, 0x69, 0xc5, 0xac, 0xa4, 0xc2, 0xac, 0xc4, 0xea, 0x38, 0x07, 0xe7, 0xdb,
0x03, 0xc3, 0x2d, 0xf4, 0x57, 0x8a, 0x69, 0xb6, 0xf9, 0x29, 0xb1, 0x43, 0x1a, 0xc7, 0x4f, 0x82, 0xea, 0x12, 0x8a, 0xde, 0x9f, 0xd9, 0xd6, 0x16, 0xa1, 0x43, 0x6d, 0x31, 0x0c, 0x76, 0xc1, 0x7f,
0xc8, 0x21, 0xef, 0x25, 0xf2, 0x9a, 0x19, 0xb9, 0x29, 0xd4, 0x07, 0x4a, 0x9c, 0xd0, 0xff, 0xa0, 0xa9, 0x4c, 0xe7, 0x90, 0x9f, 0x12, 0x3b, 0xc4, 0x71, 0xfc, 0x34, 0x88, 0x1c, 0xf4, 0x41, 0x4a,
0xc6, 0x34, 0x7e, 0x80, 0x16, 0xfa, 0xe6, 0xe5, 0xf6, 0xb6, 0xa3, 0xc0, 0x03, 0x72, 0x2e, 0x7b, 0x5e, 0x31, 0x4b, 0x6e, 0x08, 0xf6, 0x9e, 0x22, 0x27, 0xea, 0xff, 0x60, 0xe3, 0x32, 0xbc, 0x07,
0xfc, 0x37, 0x62, 0x6c, 0x71, 0x34, 0x82, 0x74, 0xab, 0x7f, 0xa7, 0x83, 0x19, 0xfc, 0x10, 0x2d, 0xe6, 0x32, 0xfd, 0xf2, 0x78, 0xdb, 0x51, 0xc0, 0x86, 0x7c, 0x2a, 0x3d, 0xce, 0x17, 0xb4, 0x2d,
0xa6, 0x64, 0x79, 0x52, 0x24, 0xfa, 0x83, 0x44, 0xff, 0x6f, 0x46, 0xab, 0x23, 0xd3, 0xc7, 0xc6, 0x8e, 0x46, 0x90, 0x8e, 0xfa, 0x6f, 0x3c, 0xb8, 0x02, 0xef, 0x83, 0xf9, 0x54, 0x59, 0x9e, 0x14,
0x74, 0x28, 0x85, 0x77, 0x50, 0x39, 0x85, 0x7b, 0x6e, 0xcc, 0xc8, 0x47, 0x49, 0x5d, 0x31, 0x53, 0x29, 0xfd, 0x51, 0x4a, 0x5f, 0x30, 0x4b, 0xab, 0x23, 0x93, 0xd1, 0x86, 0x78, 0x68, 0x09, 0x6e,
0xf7, 0xdc, 0x98, 0x65, 0x7c, 0x94, 0x04, 0x35, 0x89, 0x8f, 0x26, 0x49, 0x9f, 0x46, 0x92, 0x78, 0x83, 0x7a, 0x2a, 0xee, 0xb9, 0x31, 0x45, 0x9f, 0xa4, 0xea, 0xb2, 0x59, 0x75, 0x97, 0x51, 0x72,
0xeb, 0x21, 0x52, 0x12, 0xd4, 0x5b, 0x2f, 0x48, 0xdc, 0x91, 0xcf, 0x0b, 0xa3, 0xb6, 0x9e, 0xd7, 0x39, 0x4a, 0x40, 0xad, 0xc4, 0x5b, 0x93, 0x4a, 0x9f, 0x0b, 0x95, 0xb8, 0xf5, 0x90, 0x52, 0x02,
0x0c, 0x3a, 0x52, 0xc5, 0xb4, 0x23, 0x05, 0x46, 0x39, 0xf2, 0x45, 0x61, 0x94, 0x23, 0x79, 0x95, 0xea, 0xd1, 0x0b, 0x25, 0x9e, 0xc8, 0x17, 0x95, 0xa2, 0xd1, 0xf3, 0x9a, 0xc1, 0x44, 0x2a, 0x4c,
0xc1, 0x91, 0x69, 0x38, 0x3b, 0x16, 0x77, 0xe4, 0xcb, 0x4b, 0xc7, 0x1a, 0x74, 0xa4, 0x8a, 0xe1, 0x27, 0x52, 0xc8, 0xa8, 0x44, 0xbe, 0xac, 0x14, 0x25, 0x92, 0x57, 0x19, 0x12, 0x99, 0xc2, 0xf9,
0x47, 0xa8, 0xd2, 0x87, 0x11, 0x46, 0x09, 0x21, 0xea, 0xb8, 0x71, 0xcc, 0xbf, 0xc3, 0xaf, 0x24, 0xb6, 0x78, 0x22, 0x5f, 0x9d, 0xd9, 0xd6, 0x60, 0x22, 0x15, 0x06, 0x1f, 0x81, 0x46, 0x46, 0x46,
0xf3, 0xfa, 0x08, 0x26, 0x97, 0x1f, 0x68, 0x75, 0xc2, 0xff, 0x93, 0x9a, 0xf3, 0xb8, 0x83, 0x96, 0x04, 0x25, 0x24, 0x51, 0xcf, 0x8d, 0xc5, 0x3d, 0xf6, 0x5a, 0x6a, 0x5e, 0x2d, 0xd0, 0xe4, 0xf4,
0xd2, 0x5e, 0xca, 0x3a, 0x7d, 0xcd, 0x5e, 0xcb, 0x66, 0x37, 0xcc, 0xcd, 0xa4, 0x4b, 0x86, 0xbb, 0x3d, 0xcd, 0x4e, 0xf4, 0xff, 0xc5, 0xe6, 0x75, 0xd8, 0x03, 0x0b, 0xa9, 0x97, 0x8a, 0x4e, 0xc6,
0x11, 0x3a, 0x42, 0x50, 0xfb, 0x0d, 0xcd, 0x6e, 0x75, 0x42, 0xf6, 0xd4, 0x82, 0x38, 0x0c, 0xfc, 0xec, 0x8d, 0x34, 0xbb, 0x66, 0x36, 0x93, 0x29, 0x19, 0x76, 0x43, 0xb8, 0x80, 0xd0, 0xfc, 0x0b,
0x18, 0x6a, 0x21, 0x5a, 0xba, 0xe4, 0x43, 0x84, 0x31, 0x9a, 0x10, 0x17, 0x77, 0x4e, 0x5c, 0xdc, 0x4c, 0x6f, 0xf6, 0x42, 0xfa, 0xcc, 0x22, 0x71, 0x18, 0xf8, 0x31, 0x69, 0x86, 0x60, 0xe1, 0x8c,
0xe2, 0x99, 0x5f, 0xe8, 0xfa, 0x7c, 0xaa, 0x0b, 0x3d, 0x79, 0xc7, 0x2b, 0xa8, 0x14, 0xbb, 0x9d, 0x3f, 0x44, 0x10, 0x82, 0x31, 0x71, 0xbb, 0x97, 0xc4, 0xed, 0x2e, 0x9e, 0xf9, 0xad, 0xaf, 0xcf,
0xd0, 0x03, 0x9b, 0x05, 0x27, 0xe0, 0x8b, 0x8b, 0xb8, 0x60, 0x15, 0x65, 0xec, 0x90, 0x87, 0xee, 0xa7, 0xba, 0xf5, 0x93, 0x77, 0xb8, 0x0c, 0x6a, 0xb1, 0xdb, 0x0b, 0xd9, 0x5e, 0x28, 0x33, 0x96,
0x2d, 0x9c, 0x7d, 0xa9, 0x8e, 0x9d, 0x5d, 0x54, 0x73, 0xe7, 0x17, 0xd5, 0xdc, 0xe7, 0x8b, 0x6a, 0x97, 0x7e, 0xc5, 0xaa, 0x4a, 0x6c, 0x9f, 0x43, 0xb7, 0xe7, 0x4e, 0xbe, 0x2e, 0x8e, 0x9c, 0x7c,
0xee, 0xd9, 0xd7, 0xea, 0xd8, 0xd1, 0x94, 0xf8, 0x37, 0xb1, 0xf1, 0x2b, 0x00, 0x00, 0xff, 0xff, 0x5b, 0x2c, 0x9d, 0xb2, 0xdf, 0x17, 0xf6, 0x7b, 0xfe, 0x7d, 0x71, 0xe4, 0x60, 0x42, 0xfc, 0xcb,
0x54, 0x8c, 0x4a, 0x7f, 0xa5, 0x08, 0x00, 0x00, 0xb1, 0xfe, 0x27, 0x00, 0x00, 0xff, 0xff, 0x02, 0x23, 0xd2, 0x00, 0xca, 0x08, 0x00, 0x00,
} }

View File

@ -14,6 +14,8 @@ message RequestHeader {
uint64 ID = 1; uint64 ID = 1;
// username is a username that is associated with an auth token of gRPC connection // username is a username that is associated with an auth token of gRPC connection
string username = 2; string username = 2;
// auth_revision is a revision number of auth.authStore. It is not related to mvcc
uint64 auth_revision = 3;
} }
// An InternalRaftRequest is the union of all requests which can be // An InternalRaftRequest is the union of all requests which can be

View File

@ -19,6 +19,7 @@ import (
"strings" "strings"
"time" "time"
"github.com/coreos/etcd/auth"
pb "github.com/coreos/etcd/etcdserver/etcdserverpb" pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
"github.com/coreos/etcd/lease" "github.com/coreos/etcd/lease"
"github.com/coreos/etcd/lease/leasehttp" "github.com/coreos/etcd/lease/leasehttp"
@ -87,15 +88,34 @@ func (s *EtcdServer) Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeRe
var err error var err error
if r.Serializable { if r.Serializable {
var user string for {
user, err = s.usernameFromCtx(ctx) authInfo, err := s.authInfoFromCtx(ctx)
if err != nil { if err != nil {
return nil, err return nil, err
} }
result = s.applyV3.Apply(
&pb.InternalRaftRequest{ hdr := &pb.RequestHeader{}
Header: &pb.RequestHeader{Username: user}, if authInfo != nil {
Range: r}) hdr.Username = authInfo.Username
hdr.AuthRevision = authInfo.Revision
}
result = s.applyV3.Apply(&pb.InternalRaftRequest{Header: hdr, Range: r})
if result.err != nil {
if result.err == auth.ErrAuthOldRevision {
continue
}
break
}
if authInfo == nil || authInfo.Revision == s.authStore.Revision() {
break
}
// The revision that authorized this request is obsolete.
// For avoiding TOCTOU problem, retry of the request is required.
}
} else { } else {
result, err = s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{Range: r}) result, err = s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{Range: r})
} }
@ -135,14 +155,34 @@ func (s *EtcdServer) Txn(ctx context.Context, r *pb.TxnRequest) (*pb.TxnResponse
var err error var err error
if isTxnSerializable(r) { if isTxnSerializable(r) {
user, err := s.usernameFromCtx(ctx) for {
authInfo, err := s.authInfoFromCtx(ctx)
if err != nil { if err != nil {
return nil, err return nil, err
} }
result = s.applyV3.Apply(
&pb.InternalRaftRequest{ hdr := &pb.RequestHeader{}
Header: &pb.RequestHeader{Username: user}, if authInfo != nil {
Txn: r}) hdr.Username = authInfo.Username
hdr.AuthRevision = authInfo.Revision
}
result = s.applyV3.Apply(&pb.InternalRaftRequest{Header: hdr, Txn: r})
if result.err != nil {
if result.err == auth.ErrAuthOldRevision {
continue
}
break
}
if authInfo == nil || authInfo.Revision == s.authStore.Revision() {
break
}
// The revision that authorized this request is obsolete.
// For avoiding TOCTOU problem, retry of this request is required.
}
} else { } else {
result, err = s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{Txn: r}) result, err = s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{Txn: r})
} }
@ -170,7 +210,7 @@ func isTxnSerializable(r *pb.TxnRequest) bool {
} }
func (s *EtcdServer) Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error) { func (s *EtcdServer) Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error) {
result, err := s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{Compaction: r}) result, err := s.processInternalRaftRequestOnce(ctx, pb.InternalRaftRequest{Compaction: r})
if r.Physical && result != nil && result.physc != nil { if r.Physical && result != nil && result.physc != nil {
<-result.physc <-result.physc
// The compaction is done deleting keys; the hash is now settled // The compaction is done deleting keys; the hash is now settled
@ -203,7 +243,7 @@ func (s *EtcdServer) LeaseGrant(ctx context.Context, r *pb.LeaseGrantRequest) (*
// only use positive int64 id's // only use positive int64 id's
r.ID = int64(s.reqIDGen.Next() & ((1 << 63) - 1)) r.ID = int64(s.reqIDGen.Next() & ((1 << 63) - 1))
} }
result, err := s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{LeaseGrant: r}) result, err := s.processInternalRaftRequestOnce(ctx, pb.InternalRaftRequest{LeaseGrant: r})
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -214,7 +254,7 @@ func (s *EtcdServer) LeaseGrant(ctx context.Context, r *pb.LeaseGrantRequest) (*
} }
func (s *EtcdServer) LeaseRevoke(ctx context.Context, r *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) { func (s *EtcdServer) LeaseRevoke(ctx context.Context, r *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) {
result, err := s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{LeaseRevoke: r}) result, err := s.processInternalRaftRequestOnce(ctx, pb.InternalRaftRequest{LeaseRevoke: r})
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -260,7 +300,7 @@ func (s *EtcdServer) LeaseRenew(id lease.LeaseID) (int64, error) {
} }
func (s *EtcdServer) Alarm(ctx context.Context, r *pb.AlarmRequest) (*pb.AlarmResponse, error) { func (s *EtcdServer) Alarm(ctx context.Context, r *pb.AlarmRequest) (*pb.AlarmResponse, error) {
result, err := s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{Alarm: r}) result, err := s.processInternalRaftRequestOnce(ctx, pb.InternalRaftRequest{Alarm: r})
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -271,7 +311,7 @@ func (s *EtcdServer) Alarm(ctx context.Context, r *pb.AlarmRequest) (*pb.AlarmRe
} }
func (s *EtcdServer) AuthEnable(ctx context.Context, r *pb.AuthEnableRequest) (*pb.AuthEnableResponse, error) { func (s *EtcdServer) AuthEnable(ctx context.Context, r *pb.AuthEnableRequest) (*pb.AuthEnableResponse, error) {
result, err := s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{AuthEnable: r}) result, err := s.processInternalRaftRequestOnce(ctx, pb.InternalRaftRequest{AuthEnable: r})
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -304,7 +344,7 @@ func (s *EtcdServer) Authenticate(ctx context.Context, r *pb.AuthenticateRequest
SimpleToken: st, SimpleToken: st,
} }
result, err := s.processInternalRaftRequest(ctx, pb.InternalRaftRequest{Authenticate: internalReq}) result, err := s.processInternalRaftRequestOnce(ctx, pb.InternalRaftRequest{Authenticate: internalReq})
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -487,31 +527,31 @@ func (s *EtcdServer) isValidSimpleToken(token string) bool {
return true return true
} }
func (s *EtcdServer) usernameFromCtx(ctx context.Context) (string, error) { func (s *EtcdServer) authInfoFromCtx(ctx context.Context) (*auth.AuthInfo, error) {
md, ok := metadata.FromContext(ctx) md, ok := metadata.FromContext(ctx)
if !ok { if !ok {
return "", nil return nil, nil
} }
ts, tok := md["token"] ts, tok := md["token"]
if !tok { if !tok {
return "", nil return nil, nil
} }
token := ts[0] token := ts[0]
if !s.isValidSimpleToken(token) { if !s.isValidSimpleToken(token) {
return "", ErrInvalidAuthToken return nil, ErrInvalidAuthToken
} }
username, uok := s.AuthStore().UsernameFromToken(token) authInfo, uok := s.AuthStore().AuthInfoFromToken(token)
if !uok { if !uok {
plog.Warningf("invalid auth token: %s", token) plog.Warningf("invalid auth token: %s", token)
return "", ErrInvalidAuthToken return nil, ErrInvalidAuthToken
} }
return username, nil return authInfo, nil
} }
func (s *EtcdServer) processInternalRaftRequest(ctx context.Context, r pb.InternalRaftRequest) (*applyResult, error) { func (s *EtcdServer) processInternalRaftRequestOnce(ctx context.Context, r pb.InternalRaftRequest) (*applyResult, error) {
ai := s.getAppliedIndex() ai := s.getAppliedIndex()
ci := s.getCommittedIndex() ci := s.getCommittedIndex()
if ci > ai+maxGapBetweenApplyAndCommitIndex { if ci > ai+maxGapBetweenApplyAndCommitIndex {
@ -521,11 +561,15 @@ func (s *EtcdServer) processInternalRaftRequest(ctx context.Context, r pb.Intern
r.Header = &pb.RequestHeader{ r.Header = &pb.RequestHeader{
ID: s.reqIDGen.Next(), ID: s.reqIDGen.Next(),
} }
username, err := s.usernameFromCtx(ctx)
authInfo, err := s.authInfoFromCtx(ctx)
if err != nil { if err != nil {
return nil, err return nil, err
} }
r.Header.Username = username if authInfo != nil {
r.Header.Username = authInfo.Username
r.Header.AuthRevision = authInfo.Revision
}
data, err := r.Marshal() data, err := r.Marshal()
if err != nil { if err != nil {
@ -562,5 +606,18 @@ func (s *EtcdServer) processInternalRaftRequest(ctx context.Context, r pb.Intern
} }
} }
func (s *EtcdServer) processInternalRaftRequest(ctx context.Context, r pb.InternalRaftRequest) (*applyResult, error) {
var result *applyResult
var err error
for {
result, err = s.processInternalRaftRequestOnce(ctx, r)
if err != auth.ErrAuthOldRevision {
break
}
}
return result, err
}
// Watchable returns a watchable interface attached to the etcdserver. // Watchable returns a watchable interface attached to the etcdserver.
func (s *EtcdServer) Watchable() mvcc.WatchableKV { return s.KV() } func (s *EtcdServer) Watchable() mvcc.WatchableKV { return s.KV() }