diff --git a/Documentation/op-guide/configuration.md b/Documentation/op-guide/configuration.md index d07b2430e..1f7064ea6 100644 --- a/Documentation/op-guide/configuration.md +++ b/Documentation/op-guide/configuration.md @@ -289,6 +289,13 @@ Follow the instructions when using these flags. + Set level of detail for exported metrics, specify 'extensive' to include histogram metrics. + default: basic +## Auth flags + +### --auth-token ++ Specify a token type and token specific options, especially for JWT. Its format is "type,var1=val1,var2=val2,...". Possible type is 'simple' or 'jwt'. Possible variables are 'sign-method' for specifying a sign method of jwt (its possible values are 'ES256', 'ES384', 'ES512', 'HS256', 'HS384', 'HS512', 'RS256', 'RS384', 'RS512', 'PS256', 'PS384', or 'PS512'), 'pub-key' for specifying a path to a public key for verifying jwt, and 'priv-key' for specifying a path to a private key for signing jwt. ++ Example option of JWT: '--auth-token jwt,pub-key=app.rsa.pub,priv-key=app.rsa,sign-method=RS512' ++ default: "simple" + [build-cluster]: clustering.md#static [reconfig]: runtime-configuration.md [discovery]: clustering.md#discovery diff --git a/auth/jwt.go b/auth/jwt.go new file mode 100644 index 000000000..57e774c3b --- /dev/null +++ b/auth/jwt.go @@ -0,0 +1,137 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package auth + +import ( + "crypto/rsa" + "io/ioutil" + + jwt "github.com/dgrijalva/jwt-go" + "golang.org/x/net/context" +) + +type tokenJWT struct { + signMethod string + signKey *rsa.PrivateKey + verifyKey *rsa.PublicKey +} + +func (t *tokenJWT) enable() {} +func (t *tokenJWT) disable() {} +func (t *tokenJWT) invalidateUser(string) {} +func (t *tokenJWT) genTokenPrefix() (string, error) { return "", nil } + +func (t *tokenJWT) info(ctx context.Context, token string, rev uint64) (*AuthInfo, bool) { + // rev isn't used in JWT, it is only used in simple token + var ( + username string + revision uint64 + ) + + parsed, err := jwt.Parse(token, func(token *jwt.Token) (interface{}, error) { + return t.verifyKey, nil + }) + + switch err.(type) { + case nil: + if !parsed.Valid { + plog.Warningf("invalid jwt token: %s", token) + return nil, false + } + + claims := parsed.Claims.(jwt.MapClaims) + + username = claims["username"].(string) + revision = uint64(claims["revision"].(float64)) + default: + plog.Warningf("failed to parse jwt token: %s", err) + return nil, false + } + + return &AuthInfo{Username: username, Revision: revision}, true +} + +func (t *tokenJWT) assign(ctx context.Context, username string, revision uint64) (string, error) { + // Future work: let a jwt token include permission infomation would be useful for + // permission checking in proxy side. + tk := jwt.NewWithClaims(jwt.GetSigningMethod(t.signMethod), + jwt.MapClaims{ + "username": username, + "revision": revision, + }) + + token, err := tk.SignedString(t.signKey) + if err != nil { + plog.Debugf("failed to sign jwt token: %s", err) + return "", err + } + + plog.Debugf("jwt token: %s", token) + + return token, err +} + +func prepareOpts(opts map[string]string) (jwtSignMethod, jwtPubKeyPath, jwtPrivKeyPath string, err error) { + for k, v := range opts { + switch k { + case "sign-method": + jwtSignMethod = v + case "pub-key": + jwtPubKeyPath = v + case "priv-key": + jwtPrivKeyPath = v + default: + plog.Errorf("unknown token specific option: %s", k) + return "", "", "", ErrInvalidAuthOpts + } + } + + return jwtSignMethod, jwtPubKeyPath, jwtPrivKeyPath, nil +} + +func newTokenProviderJWT(opts map[string]string) (*tokenJWT, error) { + jwtSignMethod, jwtPubKeyPath, jwtPrivKeyPath, err := prepareOpts(opts) + if err != nil { + return nil, ErrInvalidAuthOpts + } + + t := &tokenJWT{} + + t.signMethod = jwtSignMethod + + verifyBytes, err := ioutil.ReadFile(jwtPubKeyPath) + if err != nil { + plog.Errorf("failed to read public key (%s) for jwt: %s", jwtPubKeyPath, err) + return nil, err + } + t.verifyKey, err = jwt.ParseRSAPublicKeyFromPEM(verifyBytes) + if err != nil { + plog.Errorf("failed to parse public key (%s): %s", jwtPubKeyPath, err) + return nil, err + } + + signBytes, err := ioutil.ReadFile(jwtPrivKeyPath) + if err != nil { + plog.Errorf("failed to read private key (%s) for jwt: %s", jwtPrivKeyPath, err) + return nil, err + } + t.signKey, err = jwt.ParseRSAPrivateKeyFromPEM(signBytes) + if err != nil { + plog.Errorf("failed to parse private key (%s): %s", jwtPrivKeyPath, err) + return nil, err + } + + return t, nil +} diff --git a/auth/simple_token.go b/auth/simple_token.go index 240defea6..5b608af92 100644 --- a/auth/simple_token.go +++ b/auth/simple_token.go @@ -19,9 +19,14 @@ package auth import ( "crypto/rand" + "fmt" "math/big" + "strconv" "strings" + "sync" "time" + + "golang.org/x/net/context" ) const ( @@ -100,7 +105,14 @@ func (tm *simpleTokenTTLKeeper) run() { } } -func (as *authStore) GenSimpleToken() (string, error) { +type tokenSimple struct { + indexWaiter func(uint64) <-chan struct{} + simpleTokenKeeper *simpleTokenTTLKeeper + simpleTokensMu sync.RWMutex + simpleTokens map[string]string // token -> username +} + +func (t *tokenSimple) genTokenPrefix() (string, error) { ret := make([]byte, defaultSimpleTokenLength) for i := 0; i < defaultSimpleTokenLength; i++ { @@ -115,27 +127,104 @@ func (as *authStore) GenSimpleToken() (string, error) { return string(ret), nil } -func (as *authStore) assignSimpleTokenToUser(username, token string) { - as.simpleTokensMu.Lock() +func (t *tokenSimple) assignSimpleTokenToUser(username, token string) { + t.simpleTokensMu.Lock() - _, ok := as.simpleTokens[token] + _, ok := t.simpleTokens[token] if ok { plog.Panicf("token %s is alredy used", token) } - as.simpleTokens[token] = username - as.simpleTokenKeeper.addSimpleToken(token) - as.simpleTokensMu.Unlock() + t.simpleTokens[token] = username + t.simpleTokenKeeper.addSimpleToken(token) + t.simpleTokensMu.Unlock() } -func (as *authStore) invalidateUser(username string) { - as.simpleTokensMu.Lock() - defer as.simpleTokensMu.Unlock() +func (t *tokenSimple) invalidateUser(username string) { + t.simpleTokensMu.Lock() + defer t.simpleTokensMu.Unlock() - for token, name := range as.simpleTokens { + for token, name := range t.simpleTokens { if strings.Compare(name, username) == 0 { - delete(as.simpleTokens, token) - as.simpleTokenKeeper.deleteSimpleToken(token) + delete(t.simpleTokens, token) + t.simpleTokenKeeper.deleteSimpleToken(token) } } } + +func newDeleterFunc(t *tokenSimple) func(string) { + return func(tk string) { + t.simpleTokensMu.Lock() + defer t.simpleTokensMu.Unlock() + if username, ok := t.simpleTokens[tk]; ok { + plog.Infof("deleting token %s for user %s", tk, username) + delete(t.simpleTokens, tk) + } + } +} + +func (t *tokenSimple) enable() { + t.simpleTokenKeeper = NewSimpleTokenTTLKeeper(newDeleterFunc(t)) +} + +func (t *tokenSimple) disable() { + if t.simpleTokenKeeper != nil { + t.simpleTokenKeeper.stop() + t.simpleTokenKeeper = nil + } + + t.simpleTokensMu.Lock() + t.simpleTokens = make(map[string]string) // invalidate all tokens + t.simpleTokensMu.Unlock() +} + +func (t *tokenSimple) info(ctx context.Context, token string, revision uint64) (*AuthInfo, bool) { + if !t.isValidSimpleToken(ctx, token) { + return nil, false + } + + t.simpleTokensMu.RLock() + defer t.simpleTokensMu.RUnlock() + username, ok := t.simpleTokens[token] + if ok { + t.simpleTokenKeeper.resetSimpleToken(token) + } + + return &AuthInfo{Username: username, Revision: revision}, ok +} + +func (t *tokenSimple) assign(ctx context.Context, username string, rev uint64) (string, error) { + // rev isn't used in simple token, it is only used in JWT + index := ctx.Value("index").(uint64) + simpleToken := ctx.Value("simpleToken").(string) + token := fmt.Sprintf("%s.%d", simpleToken, index) + t.assignSimpleTokenToUser(username, token) + + return token, nil +} + +func (t *tokenSimple) isValidSimpleToken(ctx context.Context, 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 <-t.indexWaiter(uint64(index)): + return true + case <-ctx.Done(): + } + + return false +} + +func newTokenProviderSimple(indexWaiter func(uint64) <-chan struct{}) *tokenSimple { + return &tokenSimple{ + simpleTokens: make(map[string]string), + indexWaiter: indexWaiter, + } +} diff --git a/auth/store.go b/auth/store.go index 255404760..e938cbefd 100644 --- a/auth/store.go +++ b/auth/store.go @@ -18,9 +18,7 @@ import ( "bytes" "encoding/binary" "errors" - "fmt" "sort" - "strconv" "strings" "sync" @@ -62,6 +60,7 @@ var ( ErrAuthNotEnabled = errors.New("auth: authentication is not enabled") ErrAuthOldRevision = errors.New("auth: revision in header is old") ErrInvalidAuthToken = errors.New("auth: invalid auth token") + ErrInvalidAuthOpts = errors.New("auth: invalid auth options") // BcryptCost is the algorithm cost / strength for hashing auth passwords BcryptCost = bcrypt.DefaultCost @@ -131,10 +130,6 @@ type AuthStore interface { // RoleList gets a list of all roles RoleList(r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error) - // AuthInfoFromToken gets a username from the given Token and current revision number - // (The revision number is used for preventing the TOCTOU problem) - AuthInfoFromToken(token string) (*AuthInfo, bool) - // IsPutPermitted checks put permission of the user IsPutPermitted(authInfo *AuthInfo, key []byte) error @@ -147,8 +142,9 @@ type AuthStore interface { // IsAdminPermitted checks admin permission of the user IsAdminPermitted(authInfo *AuthInfo) error - // GenSimpleToken produces a simple random string - GenSimpleToken() (string, error) + // GenTokenPrefix produces a random string in a case of simple token + // in a case of JWT, it produces an empty string + GenTokenPrefix() (string, error) // Revision gets current revision of authStore Revision() uint64 @@ -166,6 +162,16 @@ type AuthStore interface { AuthInfoFromTLS(ctx context.Context) *AuthInfo } +type TokenProvider interface { + info(ctx context.Context, token string, revision uint64) (*AuthInfo, bool) + assign(ctx context.Context, username string, revision uint64) (string, error) + enable() + disable() + + invalidateUser(string) + genTokenPrefix() (string, error) +} + type authStore struct { be backend.Backend enabled bool @@ -173,24 +179,9 @@ type authStore struct { rangePermCache map[string]*unifiedRangePermissions // username -> unifiedRangePermissions - simpleTokensMu sync.RWMutex - simpleTokens map[string]string // token -> username - simpleTokenKeeper *simpleTokenTTLKeeper - revision uint64 - indexWaiter func(uint64) <-chan struct{} -} - -func newDeleterFunc(as *authStore) func(string) { - return func(t string) { - as.simpleTokensMu.Lock() - defer as.simpleTokensMu.Unlock() - if username, ok := as.simpleTokens[t]; ok { - plog.Infof("deleting token %s for user %s", t, username) - delete(as.simpleTokens, t) - } - } + tokenProvider TokenProvider } func (as *authStore) AuthEnable() error { @@ -220,8 +211,7 @@ func (as *authStore) AuthEnable() error { tx.UnsafePut(authBucketName, enableFlagKey, authEnabled) as.enabled = true - - as.simpleTokenKeeper = NewSimpleTokenTTLKeeper(newDeleterFunc(as)) + as.tokenProvider.enable() as.rangePermCache = make(map[string]*unifiedRangePermissions) @@ -247,14 +237,7 @@ func (as *authStore) AuthDisable() { b.ForceCommit() as.enabled = false - - as.simpleTokensMu.Lock() - as.simpleTokens = make(map[string]string) // invalidate all tokens - as.simpleTokensMu.Unlock() - if as.simpleTokenKeeper != nil { - as.simpleTokenKeeper.stop() - as.simpleTokenKeeper = nil - } + as.tokenProvider.disable() plog.Noticef("Authentication disabled") } @@ -265,10 +248,7 @@ func (as *authStore) Close() error { if !as.enabled { return nil } - if as.simpleTokenKeeper != nil { - as.simpleTokenKeeper.stop() - as.simpleTokenKeeper = nil - } + as.tokenProvider.disable() return nil } @@ -277,10 +257,6 @@ func (as *authStore) Authenticate(ctx context.Context, username, password string return nil, ErrAuthNotEnabled } - // TODO(mitake): after adding jwt support, branching based on values of ctx is required - index := ctx.Value("index").(uint64) - simpleToken := ctx.Value("simpleToken").(string) - tx := as.be.BatchTx() tx.Lock() defer tx.Unlock() @@ -290,10 +266,15 @@ func (as *authStore) Authenticate(ctx context.Context, username, password string return nil, ErrAuthFailed } - token := fmt.Sprintf("%s.%d", simpleToken, index) - as.assignSimpleTokenToUser(username, token) + // Password checking is already performed in the API layer, so we don't need to check for now. + // Staleness of password can be detected with OCC in the API layer, too. - plog.Infof("authorized %s, token is %s", username, token) + token, err := as.tokenProvider.assign(ctx, username, as.revision) + if err != nil { + return nil, err + } + + plog.Debugf("authorized %s, token is %s", username, token) return &pb.AuthenticateResponse{Token: token}, nil } @@ -385,7 +366,7 @@ func (as *authStore) UserDelete(r *pb.AuthUserDeleteRequest) (*pb.AuthUserDelete as.commitRevision(tx) as.invalidateCachedPerm(r.Name) - as.invalidateUser(r.Name) + as.tokenProvider.invalidateUser(r.Name) plog.Noticef("deleted a user: %s", r.Name) @@ -421,7 +402,7 @@ func (as *authStore) UserChangePassword(r *pb.AuthUserChangePasswordRequest) (*p as.commitRevision(tx) as.invalidateCachedPerm(r.Name) - as.invalidateUser(r.Name) + as.tokenProvider.invalidateUser(r.Name) plog.Noticef("changed a password of a user: %s", r.Name) @@ -650,14 +631,8 @@ func (as *authStore) RoleAdd(r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, return &pb.AuthRoleAddResponse{}, nil } -func (as *authStore) AuthInfoFromToken(token string) (*AuthInfo, bool) { - as.simpleTokensMu.RLock() - defer as.simpleTokensMu.RUnlock() - t, ok := as.simpleTokens[token] - if ok { - as.simpleTokenKeeper.resetSimpleToken(token) - } - return &AuthInfo{Username: t, Revision: as.revision}, ok +func (as *authStore) authInfoFromToken(ctx context.Context, token string) (*AuthInfo, bool) { + return as.tokenProvider.info(ctx, token, as.revision) } type permSlice []*authpb.Permission @@ -887,7 +862,7 @@ func (as *authStore) isAuthEnabled() bool { return as.enabled } -func NewAuthStore(be backend.Backend, indexWaiter func(uint64) <-chan struct{}) *authStore { +func NewAuthStore(be backend.Backend, tp TokenProvider) *authStore { tx := be.BatchTx() tx.Lock() @@ -905,15 +880,14 @@ func NewAuthStore(be backend.Backend, indexWaiter func(uint64) <-chan struct{}) as := &authStore{ be: be, - simpleTokens: make(map[string]string), revision: getRevision(tx), - indexWaiter: indexWaiter, enabled: enabled, rangePermCache: make(map[string]*unifiedRangePermissions), + tokenProvider: tp, } if enabled { - as.simpleTokenKeeper = NewSimpleTokenTTLKeeper(newDeleterFunc(as)) + as.tokenProvider.enable() } if as.revision == 0 { @@ -956,25 +930,6 @@ func (as *authStore) Revision() uint64 { 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) AuthInfoFromTLS(ctx context.Context) *AuthInfo { peer, ok := peer.FromContext(ctx) if !ok || peer == nil || peer.AuthInfo == nil { @@ -1009,14 +964,57 @@ func (as *authStore) AuthInfoFromCtx(ctx context.Context) (*AuthInfo, error) { } token := ts[0] - if !as.isValidSimpleToken(token, ctx) { - return nil, ErrInvalidAuthToken - } - - authInfo, uok := as.AuthInfoFromToken(token) + authInfo, uok := as.authInfoFromToken(ctx, token) if !uok { plog.Warningf("invalid auth token: %s", token) return nil, ErrInvalidAuthToken } return authInfo, nil } + +func (as *authStore) GenTokenPrefix() (string, error) { + return as.tokenProvider.genTokenPrefix() +} + +func decomposeOpts(optstr string) (string, map[string]string, error) { + opts := strings.Split(optstr, ",") + tokenType := opts[0] + + typeSpecificOpts := make(map[string]string) + for i := 1; i < len(opts); i++ { + pair := strings.Split(opts[i], "=") + + if len(pair) != 2 { + plog.Errorf("invalid token specific option: %s", optstr) + return "", nil, ErrInvalidAuthOpts + } + + if _, ok := typeSpecificOpts[pair[0]]; ok { + plog.Errorf("invalid token specific option, duplicated parameters (%s): %s", pair[0], optstr) + return "", nil, ErrInvalidAuthOpts + } + + typeSpecificOpts[pair[0]] = pair[1] + } + + return tokenType, typeSpecificOpts, nil + +} + +func NewTokenProvider(tokenOpts string, indexWaiter func(uint64) <-chan struct{}) (TokenProvider, error) { + tokenType, typeSpecificOpts, err := decomposeOpts(tokenOpts) + if err != nil { + return nil, ErrInvalidAuthOpts + } + + switch tokenType { + case "simple": + plog.Warningf("simple token is not cryptographically signed") + return newTokenProviderSimple(indexWaiter), nil + case "jwt": + return newTokenProviderJWT(typeSpecificOpts) + default: + plog.Errorf("unknown token type: %s", tokenType) + return nil, ErrInvalidAuthOpts + } +} diff --git a/auth/store_test.go b/auth/store_test.go index 64d864b9a..bbabd6e4d 100644 --- a/auth/store_test.go +++ b/auth/store_test.go @@ -44,8 +44,12 @@ func TestNewAuthStoreRevision(t *testing.T) { b, tPath := backend.NewDefaultTmpBackend() defer os.Remove(tPath) - as := NewAuthStore(b, dummyIndexWaiter) - err := enableAuthAndCreateRoot(as) + tp, err := NewTokenProvider("simple", dummyIndexWaiter) + if err != nil { + t.Fatal(err) + } + as := NewAuthStore(b, tp) + err = enableAuthAndCreateRoot(as) if err != nil { t.Fatal(err) } @@ -55,7 +59,7 @@ func TestNewAuthStoreRevision(t *testing.T) { // no changes to commit b2 := backend.NewDefaultBackend(tPath) - as = NewAuthStore(b2, dummyIndexWaiter) + as = NewAuthStore(b2, tp) new := as.Revision() b2.Close() as.Close() @@ -68,8 +72,12 @@ func TestNewAuthStoreRevision(t *testing.T) { func setupAuthStore(t *testing.T) (store *authStore, teardownfunc func(t *testing.T)) { b, tPath := backend.NewDefaultTmpBackend() - as := NewAuthStore(b, dummyIndexWaiter) - err := enableAuthAndCreateRoot(as) + tp, err := NewTokenProvider("simple", dummyIndexWaiter) + if err != nil { + t.Fatal(err) + } + as := NewAuthStore(b, tp) + err = enableAuthAndCreateRoot(as) if err != nil { t.Fatal(err) } @@ -188,18 +196,18 @@ func TestUserChangePassword(t *testing.T) { defer tearDown(t) ctx1 := context.WithValue(context.WithValue(context.TODO(), "index", uint64(1)), "simpleToken", "dummy") - _, err := as.Authenticate(ctx1, "foo", "") + _, err := as.Authenticate(ctx1, "foo", "bar") if err != nil { t.Fatal(err) } - _, err = as.UserChangePassword(&pb.AuthUserChangePasswordRequest{Name: "foo", Password: "bar"}) + _, err = as.UserChangePassword(&pb.AuthUserChangePasswordRequest{Name: "foo", Password: "baz"}) if err != nil { t.Fatal(err) } ctx2 := context.WithValue(context.WithValue(context.TODO(), "index", uint64(2)), "simpleToken", "dummy") - _, err = as.Authenticate(ctx2, "foo", "bar") + _, err = as.Authenticate(ctx2, "foo", "baz") if err != nil { t.Fatal(err) } @@ -544,7 +552,11 @@ func TestRecoverFromSnapshot(t *testing.T) { as.Close() - as2 := NewAuthStore(as.be, dummyIndexWaiter) + tp, err := NewTokenProvider("simple", dummyIndexWaiter) + if err != nil { + t.Fatal(err) + } + as2 := NewAuthStore(as.be, tp) defer func(a *authStore) { a.Close() }(as2) diff --git a/embed/config.go b/embed/config.go index 6ae19479b..7d0f1bf01 100644 --- a/embed/config.go +++ b/embed/config.go @@ -128,6 +128,10 @@ type Config struct { // } // embed.StartEtcd(cfg) ServiceRegister func(*grpc.Server) `json:"-"` + + // auth + + AuthToken string `json:"auth-token"` } // configYAML holds the config suitable for yaml parsing @@ -179,6 +183,7 @@ func NewConfig() *Config { StrictReconfigCheck: true, Metrics: "basic", EnableV2: true, + AuthToken: "simple", } cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name) return cfg diff --git a/embed/etcd.go b/embed/etcd.go index 746accb23..dce4ffe14 100644 --- a/embed/etcd.go +++ b/embed/etcd.go @@ -118,6 +118,7 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) { QuotaBackendBytes: cfg.QuotaBackendBytes, StrictReconfigCheck: cfg.StrictReconfigCheck, ClientCertAuthEnabled: cfg.ClientTLSInfo.ClientCertAuth, + AuthToken: cfg.AuthToken, } if e.Server, err = etcdserver.NewServer(srvcfg); err != nil { diff --git a/etcdmain/config.go b/etcdmain/config.go index c4c63a942..b8732200a 100644 --- a/etcdmain/config.go +++ b/etcdmain/config.go @@ -205,6 +205,9 @@ func newConfig() *config { // additional metrics fs.StringVar(&cfg.Metrics, "metrics", cfg.Metrics, "Set level of detail for exported metrics, specify 'extensive' to include histogram metrics") + // auth + fs.StringVar(&cfg.AuthToken, "auth-token", cfg.AuthToken, "Specify auth token specific options.") + // ignored for _, f := range cfg.ignored { fs.Var(&flags.IgnoredFlag{Name: f}, f, "") diff --git a/etcdmain/help.go b/etcdmain/help.go index fbdaf737c..cd9282a31 100644 --- a/etcdmain/help.go +++ b/etcdmain/help.go @@ -163,5 +163,9 @@ profiling flags: Enable runtime profiling data via HTTP server. Address is at client URL + "/debug/pprof/" --metrics 'basic' Set level of detail for exported metrics, specify 'extensive' to include histogram metrics. + +auth flags: + --auth-token 'simple' + Specify a v3 authentication token type and its options ('simple' or 'jwt'). ` ) diff --git a/etcdserver/config.go b/etcdserver/config.go index 262c394c7..6200ebb9f 100644 --- a/etcdserver/config.go +++ b/etcdserver/config.go @@ -59,6 +59,8 @@ type ServerConfig struct { // ClientCertAuthEnabled is true when cert has been signed by the client CA. ClientCertAuthEnabled bool + + AuthToken string } // VerifyBootstrap sanity-checks the initial config for bootstrap case diff --git a/etcdserver/server.go b/etcdserver/server.go index a6432bd21..759200a48 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -458,11 +458,16 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) { } } srv.consistIndex.setConsistentIndex(srv.kv.ConsistentIndex()) - - srv.authStore = auth.NewAuthStore(srv.be, + tp, err := auth.NewTokenProvider(cfg.AuthToken, func(index uint64) <-chan struct{} { return srv.applyWait.Wait(index) - }) + }, + ) + if err != nil { + plog.Errorf("failed to create token provider: %s", err) + return nil, err + } + srv.authStore = auth.NewAuthStore(srv.be, tp) if h := cfg.AutoCompactionRetention; h != 0 { srv.compactor = compactor.NewPeriodic(h, srv.kv, srv) srv.compactor.Run() diff --git a/etcdserver/v3_server.go b/etcdserver/v3_server.go index 78a4c0384..1472ac34b 100644 --- a/etcdserver/v3_server.go +++ b/etcdserver/v3_server.go @@ -441,7 +441,7 @@ func (s *EtcdServer) Authenticate(ctx context.Context, r *pb.AuthenticateRequest return nil, err } - st, err := s.AuthStore().GenSimpleToken() + st, err := s.AuthStore().GenTokenPrefix() if err != nil { return nil, err } diff --git a/integration/cluster.go b/integration/cluster.go index e683e2c9f..85608aad0 100644 --- a/integration/cluster.go +++ b/integration/cluster.go @@ -515,6 +515,7 @@ func mustNewMember(t *testing.T, mcfg memberConfig) *member { m.ElectionTicks = electionTicks m.TickMs = uint(tickDuration / time.Millisecond) m.QuotaBackendBytes = mcfg.quotaBackendBytes + m.AuthToken = "simple" // for the purpose of integration testing, simple token is enough return m }