store.go 34 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366
  1. // Copyright 2016 The etcd Authors
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package auth
  15. import (
  16. "bytes"
  17. "context"
  18. "encoding/binary"
  19. "errors"
  20. "sort"
  21. "strings"
  22. "sync"
  23. "sync/atomic"
  24. "github.com/coreos/etcd/auth/authpb"
  25. "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
  26. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  27. "github.com/coreos/etcd/mvcc/backend"
  28. "github.com/coreos/pkg/capnslog"
  29. "go.uber.org/zap"
  30. "golang.org/x/crypto/bcrypt"
  31. "google.golang.org/grpc/credentials"
  32. "google.golang.org/grpc/metadata"
  33. "google.golang.org/grpc/peer"
  34. )
  35. var (
  36. enableFlagKey = []byte("authEnabled")
  37. authEnabled = []byte{1}
  38. authDisabled = []byte{0}
  39. revisionKey = []byte("authRevision")
  40. authBucketName = []byte("auth")
  41. authUsersBucketName = []byte("authUsers")
  42. authRolesBucketName = []byte("authRoles")
  43. plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "auth")
  44. ErrRootUserNotExist = errors.New("auth: root user does not exist")
  45. ErrRootRoleNotExist = errors.New("auth: root user does not have root role")
  46. ErrUserAlreadyExist = errors.New("auth: user already exists")
  47. ErrUserEmpty = errors.New("auth: user name is empty")
  48. ErrUserNotFound = errors.New("auth: user not found")
  49. ErrRoleAlreadyExist = errors.New("auth: role already exists")
  50. ErrRoleNotFound = errors.New("auth: role not found")
  51. ErrAuthFailed = errors.New("auth: authentication failed, invalid user ID or password")
  52. ErrPermissionDenied = errors.New("auth: permission denied")
  53. ErrRoleNotGranted = errors.New("auth: role is not granted to the user")
  54. ErrPermissionNotGranted = errors.New("auth: permission is not granted to the role")
  55. ErrAuthNotEnabled = errors.New("auth: authentication is not enabled")
  56. ErrAuthOldRevision = errors.New("auth: revision in header is old")
  57. ErrInvalidAuthToken = errors.New("auth: invalid auth token")
  58. ErrInvalidAuthOpts = errors.New("auth: invalid auth options")
  59. ErrInvalidAuthMgmt = errors.New("auth: invalid auth management")
  60. )
  61. const (
  62. rootUser = "root"
  63. rootRole = "root"
  64. tokenTypeSimple = "simple"
  65. tokenTypeJWT = "jwt"
  66. revBytesLen = 8
  67. )
  68. type AuthInfo struct {
  69. Username string
  70. Revision uint64
  71. }
  72. // AuthenticateParamIndex is used for a key of context in the parameters of Authenticate()
  73. type AuthenticateParamIndex struct{}
  74. // AuthenticateParamSimpleTokenPrefix is used for a key of context in the parameters of Authenticate()
  75. type AuthenticateParamSimpleTokenPrefix struct{}
  76. // AuthStore defines auth storage interface.
  77. type AuthStore interface {
  78. // AuthEnable turns on the authentication feature
  79. AuthEnable() error
  80. // AuthDisable turns off the authentication feature
  81. AuthDisable()
  82. // IsAuthEnabled returns true if the authentication feature is enabled.
  83. IsAuthEnabled() bool
  84. // Authenticate does authentication based on given user name and password
  85. Authenticate(ctx context.Context, username, password string) (*pb.AuthenticateResponse, error)
  86. // Recover recovers the state of auth store from the given backend
  87. Recover(b backend.Backend)
  88. // UserAdd adds a new user
  89. UserAdd(r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error)
  90. // UserDelete deletes a user
  91. UserDelete(r *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error)
  92. // UserChangePassword changes a password of a user
  93. UserChangePassword(r *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error)
  94. // UserGrantRole grants a role to the user
  95. UserGrantRole(r *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error)
  96. // UserGet gets the detailed information of a users
  97. UserGet(r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error)
  98. // UserRevokeRole revokes a role of a user
  99. UserRevokeRole(r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error)
  100. // RoleAdd adds a new role
  101. RoleAdd(r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error)
  102. // RoleGrantPermission grants a permission to a role
  103. RoleGrantPermission(r *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error)
  104. // RoleGet gets the detailed information of a role
  105. RoleGet(r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error)
  106. // RoleRevokePermission gets the detailed information of a role
  107. RoleRevokePermission(r *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error)
  108. // RoleDelete gets the detailed information of a role
  109. RoleDelete(r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error)
  110. // UserList gets a list of all users
  111. UserList(r *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error)
  112. // RoleList gets a list of all roles
  113. RoleList(r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error)
  114. // IsPutPermitted checks put permission of the user
  115. IsPutPermitted(authInfo *AuthInfo, key []byte) error
  116. // IsRangePermitted checks range permission of the user
  117. IsRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error
  118. // IsDeleteRangePermitted checks delete-range permission of the user
  119. IsDeleteRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error
  120. // IsAdminPermitted checks admin permission of the user
  121. IsAdminPermitted(authInfo *AuthInfo) error
  122. // GenTokenPrefix produces a random string in a case of simple token
  123. // in a case of JWT, it produces an empty string
  124. GenTokenPrefix() (string, error)
  125. // Revision gets current revision of authStore
  126. Revision() uint64
  127. // CheckPassword checks a given pair of username and password is correct
  128. CheckPassword(username, password string) (uint64, error)
  129. // Close does cleanup of AuthStore
  130. Close() error
  131. // AuthInfoFromCtx gets AuthInfo from gRPC's context
  132. AuthInfoFromCtx(ctx context.Context) (*AuthInfo, error)
  133. // AuthInfoFromTLS gets AuthInfo from TLS info of gRPC's context
  134. AuthInfoFromTLS(ctx context.Context) *AuthInfo
  135. // WithRoot generates and installs a token that can be used as a root credential
  136. WithRoot(ctx context.Context) context.Context
  137. // HasRole checks that user has role
  138. HasRole(user, role string) bool
  139. }
  140. type TokenProvider interface {
  141. info(ctx context.Context, token string, revision uint64) (*AuthInfo, bool)
  142. assign(ctx context.Context, username string, revision uint64) (string, error)
  143. enable()
  144. disable()
  145. invalidateUser(string)
  146. genTokenPrefix() (string, error)
  147. }
  148. type authStore struct {
  149. // atomic operations; need 64-bit align, or 32-bit tests will crash
  150. revision uint64
  151. lg *zap.Logger
  152. be backend.Backend
  153. enabled bool
  154. enabledMu sync.RWMutex
  155. rangePermCache map[string]*unifiedRangePermissions // username -> unifiedRangePermissions
  156. tokenProvider TokenProvider
  157. bcryptCost int // the algorithm cost / strength for hashing auth passwords
  158. }
  159. func (as *authStore) AuthEnable() error {
  160. as.enabledMu.Lock()
  161. defer as.enabledMu.Unlock()
  162. if as.enabled {
  163. if as.lg != nil {
  164. as.lg.Info("authentication is already enabled; ignored auth enable request")
  165. } else {
  166. plog.Noticef("Authentication already enabled")
  167. }
  168. return nil
  169. }
  170. b := as.be
  171. tx := b.BatchTx()
  172. tx.Lock()
  173. defer func() {
  174. tx.Unlock()
  175. b.ForceCommit()
  176. }()
  177. u := getUser(as.lg, tx, rootUser)
  178. if u == nil {
  179. return ErrRootUserNotExist
  180. }
  181. if !hasRootRole(u) {
  182. return ErrRootRoleNotExist
  183. }
  184. tx.UnsafePut(authBucketName, enableFlagKey, authEnabled)
  185. as.enabled = true
  186. as.tokenProvider.enable()
  187. as.rangePermCache = make(map[string]*unifiedRangePermissions)
  188. as.setRevision(getRevision(tx))
  189. if as.lg != nil {
  190. as.lg.Info("enabled authentication")
  191. } else {
  192. plog.Noticef("Authentication enabled")
  193. }
  194. return nil
  195. }
  196. func (as *authStore) AuthDisable() {
  197. as.enabledMu.Lock()
  198. defer as.enabledMu.Unlock()
  199. if !as.enabled {
  200. return
  201. }
  202. b := as.be
  203. tx := b.BatchTx()
  204. tx.Lock()
  205. tx.UnsafePut(authBucketName, enableFlagKey, authDisabled)
  206. as.commitRevision(tx)
  207. tx.Unlock()
  208. b.ForceCommit()
  209. as.enabled = false
  210. as.tokenProvider.disable()
  211. if as.lg != nil {
  212. as.lg.Info("disabled authentication")
  213. } else {
  214. plog.Noticef("Authentication disabled")
  215. }
  216. }
  217. func (as *authStore) Close() error {
  218. as.enabledMu.Lock()
  219. defer as.enabledMu.Unlock()
  220. if !as.enabled {
  221. return nil
  222. }
  223. as.tokenProvider.disable()
  224. return nil
  225. }
  226. func (as *authStore) Authenticate(ctx context.Context, username, password string) (*pb.AuthenticateResponse, error) {
  227. if !as.IsAuthEnabled() {
  228. return nil, ErrAuthNotEnabled
  229. }
  230. tx := as.be.BatchTx()
  231. tx.Lock()
  232. defer tx.Unlock()
  233. user := getUser(as.lg, tx, username)
  234. if user == nil {
  235. return nil, ErrAuthFailed
  236. }
  237. // Password checking is already performed in the API layer, so we don't need to check for now.
  238. // Staleness of password can be detected with OCC in the API layer, too.
  239. token, err := as.tokenProvider.assign(ctx, username, as.Revision())
  240. if err != nil {
  241. return nil, err
  242. }
  243. if as.lg != nil {
  244. as.lg.Debug(
  245. "authenticated a user",
  246. zap.String("user-name", username),
  247. zap.String("token", token),
  248. )
  249. } else {
  250. plog.Debugf("authorized %s, token is %s", username, token)
  251. }
  252. return &pb.AuthenticateResponse{Token: token}, nil
  253. }
  254. func (as *authStore) CheckPassword(username, password string) (uint64, error) {
  255. if !as.IsAuthEnabled() {
  256. return 0, ErrAuthNotEnabled
  257. }
  258. tx := as.be.BatchTx()
  259. tx.Lock()
  260. defer tx.Unlock()
  261. user := getUser(as.lg, tx, username)
  262. if user == nil {
  263. return 0, ErrAuthFailed
  264. }
  265. if bcrypt.CompareHashAndPassword(user.Password, []byte(password)) != nil {
  266. if as.lg != nil {
  267. as.lg.Info("invalid password", zap.String("user-name", username))
  268. } else {
  269. plog.Noticef("authentication failed, invalid password for user %s", username)
  270. }
  271. return 0, ErrAuthFailed
  272. }
  273. return getRevision(tx), nil
  274. }
  275. func (as *authStore) Recover(be backend.Backend) {
  276. enabled := false
  277. as.be = be
  278. tx := be.BatchTx()
  279. tx.Lock()
  280. _, vs := tx.UnsafeRange(authBucketName, enableFlagKey, nil, 0)
  281. if len(vs) == 1 {
  282. if bytes.Equal(vs[0], authEnabled) {
  283. enabled = true
  284. }
  285. }
  286. as.setRevision(getRevision(tx))
  287. tx.Unlock()
  288. as.enabledMu.Lock()
  289. as.enabled = enabled
  290. as.enabledMu.Unlock()
  291. }
  292. func (as *authStore) UserAdd(r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error) {
  293. if len(r.Name) == 0 {
  294. return nil, ErrUserEmpty
  295. }
  296. hashed, err := bcrypt.GenerateFromPassword([]byte(r.Password), as.bcryptCost)
  297. if err != nil {
  298. if as.lg != nil {
  299. as.lg.Warn(
  300. "failed to bcrypt hash password",
  301. zap.String("user-name", r.Name),
  302. zap.Error(err),
  303. )
  304. } else {
  305. plog.Errorf("failed to hash password: %s", err)
  306. }
  307. return nil, err
  308. }
  309. tx := as.be.BatchTx()
  310. tx.Lock()
  311. defer tx.Unlock()
  312. user := getUser(as.lg, tx, r.Name)
  313. if user != nil {
  314. return nil, ErrUserAlreadyExist
  315. }
  316. newUser := &authpb.User{
  317. Name: []byte(r.Name),
  318. Password: hashed,
  319. }
  320. putUser(as.lg, tx, newUser)
  321. as.commitRevision(tx)
  322. if as.lg != nil {
  323. as.lg.Info("added a user", zap.String("user-name", r.Name))
  324. } else {
  325. plog.Noticef("added a new user: %s", r.Name)
  326. }
  327. return &pb.AuthUserAddResponse{}, nil
  328. }
  329. func (as *authStore) UserDelete(r *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error) {
  330. if as.enabled && r.Name == rootUser {
  331. if as.lg != nil {
  332. as.lg.Warn("cannot delete 'root' user", zap.String("user-name", r.Name))
  333. } else {
  334. plog.Errorf("the user root must not be deleted")
  335. }
  336. return nil, ErrInvalidAuthMgmt
  337. }
  338. tx := as.be.BatchTx()
  339. tx.Lock()
  340. defer tx.Unlock()
  341. user := getUser(as.lg, tx, r.Name)
  342. if user == nil {
  343. return nil, ErrUserNotFound
  344. }
  345. delUser(tx, r.Name)
  346. as.commitRevision(tx)
  347. as.invalidateCachedPerm(r.Name)
  348. as.tokenProvider.invalidateUser(r.Name)
  349. if as.lg != nil {
  350. as.lg.Info(
  351. "deleted a user",
  352. zap.String("user-name", r.Name),
  353. zap.Strings("user-roles", user.Roles),
  354. )
  355. } else {
  356. plog.Noticef("deleted a user: %s", r.Name)
  357. }
  358. return &pb.AuthUserDeleteResponse{}, nil
  359. }
  360. func (as *authStore) UserChangePassword(r *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error) {
  361. // TODO(mitake): measure the cost of bcrypt.GenerateFromPassword()
  362. // If the cost is too high, we should move the encryption to outside of the raft
  363. hashed, err := bcrypt.GenerateFromPassword([]byte(r.Password), as.bcryptCost)
  364. if err != nil {
  365. if as.lg != nil {
  366. as.lg.Warn(
  367. "failed to bcrypt hash password",
  368. zap.String("user-name", r.Name),
  369. zap.Error(err),
  370. )
  371. } else {
  372. plog.Errorf("failed to hash password: %s", err)
  373. }
  374. return nil, err
  375. }
  376. tx := as.be.BatchTx()
  377. tx.Lock()
  378. defer tx.Unlock()
  379. user := getUser(as.lg, tx, r.Name)
  380. if user == nil {
  381. return nil, ErrUserNotFound
  382. }
  383. updatedUser := &authpb.User{
  384. Name: []byte(r.Name),
  385. Roles: user.Roles,
  386. Password: hashed,
  387. }
  388. putUser(as.lg, tx, updatedUser)
  389. as.commitRevision(tx)
  390. as.invalidateCachedPerm(r.Name)
  391. as.tokenProvider.invalidateUser(r.Name)
  392. if as.lg != nil {
  393. as.lg.Info(
  394. "changed a password of a user",
  395. zap.String("user-name", r.Name),
  396. zap.Strings("user-roles", user.Roles),
  397. )
  398. } else {
  399. plog.Noticef("changed a password of a user: %s", r.Name)
  400. }
  401. return &pb.AuthUserChangePasswordResponse{}, nil
  402. }
  403. func (as *authStore) UserGrantRole(r *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error) {
  404. tx := as.be.BatchTx()
  405. tx.Lock()
  406. defer tx.Unlock()
  407. user := getUser(as.lg, tx, r.User)
  408. if user == nil {
  409. return nil, ErrUserNotFound
  410. }
  411. if r.Role != rootRole {
  412. role := getRole(tx, r.Role)
  413. if role == nil {
  414. return nil, ErrRoleNotFound
  415. }
  416. }
  417. idx := sort.SearchStrings(user.Roles, r.Role)
  418. if idx < len(user.Roles) && user.Roles[idx] == r.Role {
  419. if as.lg != nil {
  420. as.lg.Warn(
  421. "ignored grant role request to a user",
  422. zap.String("user-name", r.User),
  423. zap.Strings("user-roles", user.Roles),
  424. zap.String("duplicate-role-name", r.Role),
  425. )
  426. } else {
  427. plog.Warningf("user %s is already granted role %s", r.User, r.Role)
  428. }
  429. return &pb.AuthUserGrantRoleResponse{}, nil
  430. }
  431. user.Roles = append(user.Roles, r.Role)
  432. sort.Strings(user.Roles)
  433. putUser(as.lg, tx, user)
  434. as.invalidateCachedPerm(r.User)
  435. as.commitRevision(tx)
  436. if as.lg != nil {
  437. as.lg.Info(
  438. "granted a role to a user",
  439. zap.String("user-name", r.User),
  440. zap.Strings("user-roles", user.Roles),
  441. zap.String("added-role-name", r.Role),
  442. )
  443. } else {
  444. plog.Noticef("granted role %s to user %s", r.Role, r.User)
  445. }
  446. return &pb.AuthUserGrantRoleResponse{}, nil
  447. }
  448. func (as *authStore) UserGet(r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error) {
  449. tx := as.be.BatchTx()
  450. tx.Lock()
  451. user := getUser(as.lg, tx, r.Name)
  452. tx.Unlock()
  453. if user == nil {
  454. return nil, ErrUserNotFound
  455. }
  456. var resp pb.AuthUserGetResponse
  457. resp.Roles = append(resp.Roles, user.Roles...)
  458. return &resp, nil
  459. }
  460. func (as *authStore) UserList(r *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error) {
  461. tx := as.be.BatchTx()
  462. tx.Lock()
  463. users := getAllUsers(as.lg, tx)
  464. tx.Unlock()
  465. resp := &pb.AuthUserListResponse{Users: make([]string, len(users))}
  466. for i := range users {
  467. resp.Users[i] = string(users[i].Name)
  468. }
  469. return resp, nil
  470. }
  471. func (as *authStore) UserRevokeRole(r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error) {
  472. if as.enabled && r.Name == rootUser && r.Role == rootRole {
  473. if as.lg != nil {
  474. as.lg.Warn(
  475. "'root' user cannot revoke 'root' role",
  476. zap.String("user-name", r.Name),
  477. zap.String("role-name", r.Role),
  478. )
  479. } else {
  480. plog.Errorf("the role root must not be revoked from the user root")
  481. }
  482. return nil, ErrInvalidAuthMgmt
  483. }
  484. tx := as.be.BatchTx()
  485. tx.Lock()
  486. defer tx.Unlock()
  487. user := getUser(as.lg, tx, r.Name)
  488. if user == nil {
  489. return nil, ErrUserNotFound
  490. }
  491. updatedUser := &authpb.User{
  492. Name: user.Name,
  493. Password: user.Password,
  494. }
  495. for _, role := range user.Roles {
  496. if role != r.Role {
  497. updatedUser.Roles = append(updatedUser.Roles, role)
  498. }
  499. }
  500. if len(updatedUser.Roles) == len(user.Roles) {
  501. return nil, ErrRoleNotGranted
  502. }
  503. putUser(as.lg, tx, updatedUser)
  504. as.invalidateCachedPerm(r.Name)
  505. as.commitRevision(tx)
  506. if as.lg != nil {
  507. as.lg.Info(
  508. "revoked a role from a user",
  509. zap.String("user-name", r.Name),
  510. zap.Strings("old-user-roles", user.Roles),
  511. zap.Strings("new-user-roles", updatedUser.Roles),
  512. zap.String("revoked-role-name", r.Role),
  513. )
  514. } else {
  515. plog.Noticef("revoked role %s from user %s", r.Role, r.Name)
  516. }
  517. return &pb.AuthUserRevokeRoleResponse{}, nil
  518. }
  519. func (as *authStore) RoleGet(r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error) {
  520. tx := as.be.BatchTx()
  521. tx.Lock()
  522. defer tx.Unlock()
  523. var resp pb.AuthRoleGetResponse
  524. role := getRole(tx, r.Role)
  525. if role == nil {
  526. return nil, ErrRoleNotFound
  527. }
  528. resp.Perm = append(resp.Perm, role.KeyPermission...)
  529. return &resp, nil
  530. }
  531. func (as *authStore) RoleList(r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error) {
  532. tx := as.be.BatchTx()
  533. tx.Lock()
  534. roles := getAllRoles(as.lg, tx)
  535. tx.Unlock()
  536. resp := &pb.AuthRoleListResponse{Roles: make([]string, len(roles))}
  537. for i := range roles {
  538. resp.Roles[i] = string(roles[i].Name)
  539. }
  540. return resp, nil
  541. }
  542. func (as *authStore) RoleRevokePermission(r *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error) {
  543. tx := as.be.BatchTx()
  544. tx.Lock()
  545. defer tx.Unlock()
  546. role := getRole(tx, r.Role)
  547. if role == nil {
  548. return nil, ErrRoleNotFound
  549. }
  550. updatedRole := &authpb.Role{
  551. Name: role.Name,
  552. }
  553. for _, perm := range role.KeyPermission {
  554. if !bytes.Equal(perm.Key, r.Key) || !bytes.Equal(perm.RangeEnd, r.RangeEnd) {
  555. updatedRole.KeyPermission = append(updatedRole.KeyPermission, perm)
  556. }
  557. }
  558. if len(role.KeyPermission) == len(updatedRole.KeyPermission) {
  559. return nil, ErrPermissionNotGranted
  560. }
  561. putRole(as.lg, tx, updatedRole)
  562. // TODO(mitake): currently single role update invalidates every cache
  563. // It should be optimized.
  564. as.clearCachedPerm()
  565. as.commitRevision(tx)
  566. if as.lg != nil {
  567. as.lg.Info(
  568. "revoked a permission on range",
  569. zap.String("role-name", r.Role),
  570. zap.String("key", string(r.Key)),
  571. zap.String("range-end", string(r.RangeEnd)),
  572. )
  573. } else {
  574. plog.Noticef("revoked key %s from role %s", r.Key, r.Role)
  575. }
  576. return &pb.AuthRoleRevokePermissionResponse{}, nil
  577. }
  578. func (as *authStore) RoleDelete(r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error) {
  579. if as.enabled && r.Role == rootRole {
  580. if as.lg != nil {
  581. as.lg.Warn("cannot delete 'root' role", zap.String("role-name", r.Role))
  582. } else {
  583. plog.Errorf("the role root must not be deleted")
  584. }
  585. return nil, ErrInvalidAuthMgmt
  586. }
  587. tx := as.be.BatchTx()
  588. tx.Lock()
  589. defer tx.Unlock()
  590. role := getRole(tx, r.Role)
  591. if role == nil {
  592. return nil, ErrRoleNotFound
  593. }
  594. delRole(tx, r.Role)
  595. users := getAllUsers(as.lg, tx)
  596. for _, user := range users {
  597. updatedUser := &authpb.User{
  598. Name: user.Name,
  599. Password: user.Password,
  600. }
  601. for _, role := range user.Roles {
  602. if role != r.Role {
  603. updatedUser.Roles = append(updatedUser.Roles, role)
  604. }
  605. }
  606. if len(updatedUser.Roles) == len(user.Roles) {
  607. continue
  608. }
  609. putUser(as.lg, tx, updatedUser)
  610. as.invalidateCachedPerm(string(user.Name))
  611. }
  612. as.commitRevision(tx)
  613. if as.lg != nil {
  614. as.lg.Info("deleted a role", zap.String("role-name", r.Role))
  615. } else {
  616. plog.Noticef("deleted role %s", r.Role)
  617. }
  618. return &pb.AuthRoleDeleteResponse{}, nil
  619. }
  620. func (as *authStore) RoleAdd(r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error) {
  621. tx := as.be.BatchTx()
  622. tx.Lock()
  623. defer tx.Unlock()
  624. role := getRole(tx, r.Name)
  625. if role != nil {
  626. return nil, ErrRoleAlreadyExist
  627. }
  628. newRole := &authpb.Role{
  629. Name: []byte(r.Name),
  630. }
  631. putRole(as.lg, tx, newRole)
  632. as.commitRevision(tx)
  633. if as.lg != nil {
  634. as.lg.Info("created a role", zap.String("role-name", r.Name))
  635. } else {
  636. plog.Noticef("Role %s is created", r.Name)
  637. }
  638. return &pb.AuthRoleAddResponse{}, nil
  639. }
  640. func (as *authStore) authInfoFromToken(ctx context.Context, token string) (*AuthInfo, bool) {
  641. return as.tokenProvider.info(ctx, token, as.Revision())
  642. }
  643. type permSlice []*authpb.Permission
  644. func (perms permSlice) Len() int {
  645. return len(perms)
  646. }
  647. func (perms permSlice) Less(i, j int) bool {
  648. return bytes.Compare(perms[i].Key, perms[j].Key) < 0
  649. }
  650. func (perms permSlice) Swap(i, j int) {
  651. perms[i], perms[j] = perms[j], perms[i]
  652. }
  653. func (as *authStore) RoleGrantPermission(r *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error) {
  654. tx := as.be.BatchTx()
  655. tx.Lock()
  656. defer tx.Unlock()
  657. role := getRole(tx, r.Name)
  658. if role == nil {
  659. return nil, ErrRoleNotFound
  660. }
  661. idx := sort.Search(len(role.KeyPermission), func(i int) bool {
  662. return bytes.Compare(role.KeyPermission[i].Key, r.Perm.Key) >= 0
  663. })
  664. if idx < len(role.KeyPermission) && bytes.Equal(role.KeyPermission[idx].Key, r.Perm.Key) && bytes.Equal(role.KeyPermission[idx].RangeEnd, r.Perm.RangeEnd) {
  665. // update existing permission
  666. role.KeyPermission[idx].PermType = r.Perm.PermType
  667. } else {
  668. // append new permission to the role
  669. newPerm := &authpb.Permission{
  670. Key: r.Perm.Key,
  671. RangeEnd: r.Perm.RangeEnd,
  672. PermType: r.Perm.PermType,
  673. }
  674. role.KeyPermission = append(role.KeyPermission, newPerm)
  675. sort.Sort(permSlice(role.KeyPermission))
  676. }
  677. putRole(as.lg, tx, role)
  678. // TODO(mitake): currently single role update invalidates every cache
  679. // It should be optimized.
  680. as.clearCachedPerm()
  681. as.commitRevision(tx)
  682. if as.lg != nil {
  683. as.lg.Info(
  684. "granted/updated a permission to a user",
  685. zap.String("user-name", r.Name),
  686. zap.String("permission-name", authpb.Permission_Type_name[int32(r.Perm.PermType)]),
  687. )
  688. } else {
  689. 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)])
  690. }
  691. return &pb.AuthRoleGrantPermissionResponse{}, nil
  692. }
  693. func (as *authStore) isOpPermitted(userName string, revision uint64, key, rangeEnd []byte, permTyp authpb.Permission_Type) error {
  694. // TODO(mitake): this function would be costly so we need a caching mechanism
  695. if !as.IsAuthEnabled() {
  696. return nil
  697. }
  698. // only gets rev == 0 when passed AuthInfo{}; no user given
  699. if revision == 0 {
  700. return ErrUserEmpty
  701. }
  702. if revision < as.Revision() {
  703. return ErrAuthOldRevision
  704. }
  705. tx := as.be.BatchTx()
  706. tx.Lock()
  707. defer tx.Unlock()
  708. user := getUser(as.lg, tx, userName)
  709. if user == nil {
  710. if as.lg != nil {
  711. as.lg.Warn("cannot find a user for permission check", zap.String("user-name", userName))
  712. } else {
  713. plog.Errorf("invalid user name %s for permission checking", userName)
  714. }
  715. return ErrPermissionDenied
  716. }
  717. // root role should have permission on all ranges
  718. if hasRootRole(user) {
  719. return nil
  720. }
  721. if as.isRangeOpPermitted(tx, userName, key, rangeEnd, permTyp) {
  722. return nil
  723. }
  724. return ErrPermissionDenied
  725. }
  726. func (as *authStore) IsPutPermitted(authInfo *AuthInfo, key []byte) error {
  727. return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, nil, authpb.WRITE)
  728. }
  729. func (as *authStore) IsRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error {
  730. return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, rangeEnd, authpb.READ)
  731. }
  732. func (as *authStore) IsDeleteRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error {
  733. return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, rangeEnd, authpb.WRITE)
  734. }
  735. func (as *authStore) IsAdminPermitted(authInfo *AuthInfo) error {
  736. if !as.IsAuthEnabled() {
  737. return nil
  738. }
  739. if authInfo == nil {
  740. return ErrUserEmpty
  741. }
  742. tx := as.be.BatchTx()
  743. tx.Lock()
  744. u := getUser(as.lg, tx, authInfo.Username)
  745. tx.Unlock()
  746. if u == nil {
  747. return ErrUserNotFound
  748. }
  749. if !hasRootRole(u) {
  750. return ErrPermissionDenied
  751. }
  752. return nil
  753. }
  754. func getUser(lg *zap.Logger, tx backend.BatchTx, username string) *authpb.User {
  755. _, vs := tx.UnsafeRange(authUsersBucketName, []byte(username), nil, 0)
  756. if len(vs) == 0 {
  757. return nil
  758. }
  759. user := &authpb.User{}
  760. err := user.Unmarshal(vs[0])
  761. if err != nil {
  762. if lg != nil {
  763. lg.Panic(
  764. "failed to unmarshal 'authpb.User'",
  765. zap.String("user-name", username),
  766. zap.Error(err),
  767. )
  768. } else {
  769. plog.Panicf("failed to unmarshal user struct (name: %s): %s", username, err)
  770. }
  771. }
  772. return user
  773. }
  774. func getAllUsers(lg *zap.Logger, tx backend.BatchTx) []*authpb.User {
  775. _, vs := tx.UnsafeRange(authUsersBucketName, []byte{0}, []byte{0xff}, -1)
  776. if len(vs) == 0 {
  777. return nil
  778. }
  779. users := make([]*authpb.User, len(vs))
  780. for i := range vs {
  781. user := &authpb.User{}
  782. err := user.Unmarshal(vs[i])
  783. if err != nil {
  784. if lg != nil {
  785. lg.Panic("failed to unmarshal 'authpb.User'", zap.Error(err))
  786. } else {
  787. plog.Panicf("failed to unmarshal user struct: %s", err)
  788. }
  789. }
  790. users[i] = user
  791. }
  792. return users
  793. }
  794. func putUser(lg *zap.Logger, tx backend.BatchTx, user *authpb.User) {
  795. b, err := user.Marshal()
  796. if err != nil {
  797. if lg != nil {
  798. lg.Panic("failed to unmarshal 'authpb.User'", zap.Error(err))
  799. } else {
  800. plog.Panicf("failed to marshal user struct (name: %s): %s", user.Name, err)
  801. }
  802. }
  803. tx.UnsafePut(authUsersBucketName, user.Name, b)
  804. }
  805. func delUser(tx backend.BatchTx, username string) {
  806. tx.UnsafeDelete(authUsersBucketName, []byte(username))
  807. }
  808. func getRole(tx backend.BatchTx, rolename string) *authpb.Role {
  809. _, vs := tx.UnsafeRange(authRolesBucketName, []byte(rolename), nil, 0)
  810. if len(vs) == 0 {
  811. return nil
  812. }
  813. role := &authpb.Role{}
  814. err := role.Unmarshal(vs[0])
  815. if err != nil {
  816. plog.Panicf("failed to unmarshal role struct (name: %s): %s", rolename, err)
  817. }
  818. return role
  819. }
  820. func getAllRoles(lg *zap.Logger, tx backend.BatchTx) []*authpb.Role {
  821. _, vs := tx.UnsafeRange(authRolesBucketName, []byte{0}, []byte{0xff}, -1)
  822. if len(vs) == 0 {
  823. return nil
  824. }
  825. roles := make([]*authpb.Role, len(vs))
  826. for i := range vs {
  827. role := &authpb.Role{}
  828. err := role.Unmarshal(vs[i])
  829. if err != nil {
  830. if lg != nil {
  831. lg.Panic("failed to unmarshal 'authpb.Role'", zap.Error(err))
  832. } else {
  833. plog.Panicf("failed to unmarshal role struct: %s", err)
  834. }
  835. }
  836. roles[i] = role
  837. }
  838. return roles
  839. }
  840. func putRole(lg *zap.Logger, tx backend.BatchTx, role *authpb.Role) {
  841. b, err := role.Marshal()
  842. if err != nil {
  843. if lg != nil {
  844. lg.Panic(
  845. "failed to marshal 'authpb.Role'",
  846. zap.String("role-name", string(role.Name)),
  847. zap.Error(err),
  848. )
  849. } else {
  850. plog.Panicf("failed to marshal role struct (name: %s): %s", role.Name, err)
  851. }
  852. }
  853. tx.UnsafePut(authRolesBucketName, role.Name, b)
  854. }
  855. func delRole(tx backend.BatchTx, rolename string) {
  856. tx.UnsafeDelete(authRolesBucketName, []byte(rolename))
  857. }
  858. func (as *authStore) IsAuthEnabled() bool {
  859. as.enabledMu.RLock()
  860. defer as.enabledMu.RUnlock()
  861. return as.enabled
  862. }
  863. // NewAuthStore creates a new AuthStore.
  864. func NewAuthStore(lg *zap.Logger, be backend.Backend, tp TokenProvider, bcryptCost int) *authStore {
  865. if bcryptCost < bcrypt.MinCost || bcryptCost > bcrypt.MaxCost {
  866. if lg != nil {
  867. lg.Warn(
  868. "use default bcrypt cost instead of the invalid given cost",
  869. zap.Int("min-cost", bcrypt.MinCost),
  870. zap.Int("max-cost", bcrypt.MaxCost),
  871. zap.Int("default-cost", bcrypt.DefaultCost),
  872. zap.Int("given-cost", bcryptCost))
  873. } else {
  874. plog.Warningf("Use default bcrypt-cost %d instead of the invalid value %d",
  875. bcrypt.DefaultCost, bcryptCost)
  876. }
  877. bcryptCost = bcrypt.DefaultCost
  878. }
  879. tx := be.BatchTx()
  880. tx.Lock()
  881. tx.UnsafeCreateBucket(authBucketName)
  882. tx.UnsafeCreateBucket(authUsersBucketName)
  883. tx.UnsafeCreateBucket(authRolesBucketName)
  884. enabled := false
  885. _, vs := tx.UnsafeRange(authBucketName, enableFlagKey, nil, 0)
  886. if len(vs) == 1 {
  887. if bytes.Equal(vs[0], authEnabled) {
  888. enabled = true
  889. }
  890. }
  891. as := &authStore{
  892. revision: getRevision(tx),
  893. lg: lg,
  894. be: be,
  895. enabled: enabled,
  896. rangePermCache: make(map[string]*unifiedRangePermissions),
  897. tokenProvider: tp,
  898. bcryptCost: bcryptCost,
  899. }
  900. if enabled {
  901. as.tokenProvider.enable()
  902. }
  903. if as.Revision() == 0 {
  904. as.commitRevision(tx)
  905. }
  906. tx.Unlock()
  907. be.ForceCommit()
  908. return as
  909. }
  910. func hasRootRole(u *authpb.User) bool {
  911. // u.Roles is sorted in UserGrantRole(), so we can use binary search.
  912. idx := sort.SearchStrings(u.Roles, rootRole)
  913. return idx != len(u.Roles) && u.Roles[idx] == rootRole
  914. }
  915. func (as *authStore) commitRevision(tx backend.BatchTx) {
  916. atomic.AddUint64(&as.revision, 1)
  917. revBytes := make([]byte, revBytesLen)
  918. binary.BigEndian.PutUint64(revBytes, as.Revision())
  919. tx.UnsafePut(authBucketName, revisionKey, revBytes)
  920. }
  921. func getRevision(tx backend.BatchTx) uint64 {
  922. _, vs := tx.UnsafeRange(authBucketName, revisionKey, nil, 0)
  923. if len(vs) != 1 {
  924. // this can happen in the initialization phase
  925. return 0
  926. }
  927. return binary.BigEndian.Uint64(vs[0])
  928. }
  929. func (as *authStore) setRevision(rev uint64) {
  930. atomic.StoreUint64(&as.revision, rev)
  931. }
  932. func (as *authStore) Revision() uint64 {
  933. return atomic.LoadUint64(&as.revision)
  934. }
  935. func (as *authStore) AuthInfoFromTLS(ctx context.Context) (ai *AuthInfo) {
  936. peer, ok := peer.FromContext(ctx)
  937. if !ok || peer == nil || peer.AuthInfo == nil {
  938. return nil
  939. }
  940. tlsInfo := peer.AuthInfo.(credentials.TLSInfo)
  941. for _, chains := range tlsInfo.State.VerifiedChains {
  942. if len(chains) < 1 {
  943. continue
  944. }
  945. ai = &AuthInfo{
  946. Username: chains[0].Subject.CommonName,
  947. Revision: as.Revision(),
  948. }
  949. if as.lg != nil {
  950. as.lg.Debug(
  951. "found command name",
  952. zap.String("common-name", ai.Username),
  953. zap.String("user-name", ai.Username),
  954. zap.Uint64("revision", ai.Revision),
  955. )
  956. } else {
  957. plog.Debugf("found common name %s", ai.Username)
  958. }
  959. break
  960. }
  961. return ai
  962. }
  963. func (as *authStore) AuthInfoFromCtx(ctx context.Context) (*AuthInfo, error) {
  964. md, ok := metadata.FromIncomingContext(ctx)
  965. if !ok {
  966. return nil, nil
  967. }
  968. //TODO(mitake|hexfusion) review unifying key names
  969. ts, ok := md[rpctypes.TokenFieldNameGRPC]
  970. if !ok {
  971. ts, ok = md[rpctypes.TokenFieldNameSwagger]
  972. }
  973. if !ok {
  974. return nil, nil
  975. }
  976. token := ts[0]
  977. authInfo, uok := as.authInfoFromToken(ctx, token)
  978. if !uok {
  979. if as.lg != nil {
  980. as.lg.Warn("invalid auth token", zap.String("token", token))
  981. } else {
  982. plog.Warningf("invalid auth token: %s", token)
  983. }
  984. return nil, ErrInvalidAuthToken
  985. }
  986. return authInfo, nil
  987. }
  988. func (as *authStore) GenTokenPrefix() (string, error) {
  989. return as.tokenProvider.genTokenPrefix()
  990. }
  991. func decomposeOpts(lg *zap.Logger, optstr string) (string, map[string]string, error) {
  992. opts := strings.Split(optstr, ",")
  993. tokenType := opts[0]
  994. typeSpecificOpts := make(map[string]string)
  995. for i := 1; i < len(opts); i++ {
  996. pair := strings.Split(opts[i], "=")
  997. if len(pair) != 2 {
  998. if lg != nil {
  999. lg.Warn("invalid token option", zap.String("option", optstr))
  1000. } else {
  1001. plog.Errorf("invalid token specific option: %s", optstr)
  1002. }
  1003. return "", nil, ErrInvalidAuthOpts
  1004. }
  1005. if _, ok := typeSpecificOpts[pair[0]]; ok {
  1006. if lg != nil {
  1007. lg.Warn(
  1008. "invalid token option",
  1009. zap.String("option", optstr),
  1010. zap.String("duplicate-parameter", pair[0]),
  1011. )
  1012. } else {
  1013. plog.Errorf("invalid token specific option, duplicated parameters (%s): %s", pair[0], optstr)
  1014. }
  1015. return "", nil, ErrInvalidAuthOpts
  1016. }
  1017. typeSpecificOpts[pair[0]] = pair[1]
  1018. }
  1019. return tokenType, typeSpecificOpts, nil
  1020. }
  1021. // NewTokenProvider creates a new token provider.
  1022. func NewTokenProvider(
  1023. lg *zap.Logger,
  1024. tokenOpts string,
  1025. indexWaiter func(uint64) <-chan struct{}) (TokenProvider, error) {
  1026. tokenType, typeSpecificOpts, err := decomposeOpts(lg, tokenOpts)
  1027. if err != nil {
  1028. return nil, ErrInvalidAuthOpts
  1029. }
  1030. switch tokenType {
  1031. case tokenTypeSimple:
  1032. if lg != nil {
  1033. lg.Warn("simple token is not cryptographically signed")
  1034. } else {
  1035. plog.Warningf("simple token is not cryptographically signed")
  1036. }
  1037. return newTokenProviderSimple(lg, indexWaiter), nil
  1038. case tokenTypeJWT:
  1039. return newTokenProviderJWT(lg, typeSpecificOpts)
  1040. case "":
  1041. return newTokenProviderNop()
  1042. default:
  1043. if lg != nil {
  1044. lg.Warn(
  1045. "unknown token type",
  1046. zap.String("type", tokenType),
  1047. zap.Error(ErrInvalidAuthOpts),
  1048. )
  1049. } else {
  1050. plog.Errorf("unknown token type: %s", tokenType)
  1051. }
  1052. return nil, ErrInvalidAuthOpts
  1053. }
  1054. }
  1055. func (as *authStore) WithRoot(ctx context.Context) context.Context {
  1056. if !as.IsAuthEnabled() {
  1057. return ctx
  1058. }
  1059. var ctxForAssign context.Context
  1060. if ts, ok := as.tokenProvider.(*tokenSimple); ok && ts != nil {
  1061. ctx1 := context.WithValue(ctx, AuthenticateParamIndex{}, uint64(0))
  1062. prefix, err := ts.genTokenPrefix()
  1063. if err != nil {
  1064. if as.lg != nil {
  1065. as.lg.Warn(
  1066. "failed to generate prefix of internally used token",
  1067. zap.Error(err),
  1068. )
  1069. } else {
  1070. plog.Errorf("failed to generate prefix of internally used token")
  1071. }
  1072. return ctx
  1073. }
  1074. ctxForAssign = context.WithValue(ctx1, AuthenticateParamSimpleTokenPrefix{}, prefix)
  1075. } else {
  1076. ctxForAssign = ctx
  1077. }
  1078. token, err := as.tokenProvider.assign(ctxForAssign, "root", as.Revision())
  1079. if err != nil {
  1080. // this must not happen
  1081. if as.lg != nil {
  1082. as.lg.Warn(
  1083. "failed to assign token for lease revoking",
  1084. zap.Error(err),
  1085. )
  1086. } else {
  1087. plog.Errorf("failed to assign token for lease revoking: %s", err)
  1088. }
  1089. return ctx
  1090. }
  1091. mdMap := map[string]string{
  1092. rpctypes.TokenFieldNameGRPC: token,
  1093. }
  1094. tokenMD := metadata.New(mdMap)
  1095. // use "mdIncomingKey{}" since it's called from local etcdserver
  1096. return metadata.NewIncomingContext(ctx, tokenMD)
  1097. }
  1098. func (as *authStore) HasRole(user, role string) bool {
  1099. tx := as.be.BatchTx()
  1100. tx.Lock()
  1101. u := getUser(as.lg, tx, user)
  1102. tx.Unlock()
  1103. if u == nil {
  1104. if as.lg != nil {
  1105. as.lg.Warn(
  1106. "'has-role' requested for non-existing user",
  1107. zap.String("user-name", user),
  1108. zap.String("role-name", role),
  1109. )
  1110. } else {
  1111. plog.Warningf("tried to check user %s has role %s, but user %s doesn't exist", user, role, user)
  1112. }
  1113. return false
  1114. }
  1115. for _, r := range u.Roles {
  1116. if role == r {
  1117. return true
  1118. }
  1119. }
  1120. return false
  1121. }
  1122. func (as *authStore) BcryptCost() int {
  1123. return as.bcryptCost
  1124. }