Merge pull request #50258 from liggitt/token-cache

Automatic merge from submit-queue (batch tested with PRs 49488, 50407, 46105, 50456, 50258)

Enable caching successful token authentication

Resolves #50472

To support revocation of service account tokens, an etcd lookup of the token and service account is done by the token authenticator. Controllers that make dozens or hundreds of API calls per second (like the endpoints controller) cause this lookup to be done very frequently on the same objects.

This PR:
* Implements a cached token authenticator that conforms to the authenticator.Token interface
* Implements a union token authenticator (same approach as the union request authenticator, conforming to the authenticator.Token interface)
* Cleans up the auth chain construction to group all token authenticators (means we only do bearer and websocket header parsing once)
* Adds a 10-second TTL cache to successful token authentication

```release-note
API server authentication now caches successful bearer token authentication results for a few seconds.
```
This commit is contained in:
Kubernetes Submit Queue 2017-08-11 14:14:06 -07:00 committed by GitHub
commit 42adb9ef25
17 changed files with 853 additions and 16 deletions

View File

@ -166,7 +166,9 @@ func TestAddFlagsFlag(t *testing.T) {
ServiceAccounts: &kubeoptions.ServiceAccountAuthenticationOptions{
Lookup: true,
},
TokenFile: &kubeoptions.TokenFileAuthenticationOptions{},
TokenFile: &kubeoptions.TokenFileAuthenticationOptions{},
TokenSuccessCacheTTL: 10 * time.Second,
TokenFailureCacheTTL: 0,
},
Authorization: &kubeoptions.BuiltInAuthorizationOptions{
Mode: "AlwaysDeny",

View File

@ -20,7 +20,9 @@ go_library(
"//vendor/k8s.io/apiserver/pkg/authentication/request/union:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/request/websocket:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/request/x509:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/token/cache:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/token/tokenfile:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/token/union:go_default_library",
"//vendor/k8s.io/apiserver/plugin/pkg/authenticator/password/keystone:go_default_library",
"//vendor/k8s.io/apiserver/plugin/pkg/authenticator/password/passwordfile:go_default_library",
"//vendor/k8s.io/apiserver/plugin/pkg/authenticator/request/basicauth:go_default_library",

View File

@ -30,7 +30,9 @@ import (
"k8s.io/apiserver/pkg/authentication/request/union"
"k8s.io/apiserver/pkg/authentication/request/websocket"
"k8s.io/apiserver/pkg/authentication/request/x509"
tokencache "k8s.io/apiserver/pkg/authentication/token/cache"
"k8s.io/apiserver/pkg/authentication/token/tokenfile"
tokenunion "k8s.io/apiserver/pkg/authentication/token/union"
"k8s.io/apiserver/plugin/pkg/authenticator/password/keystone"
"k8s.io/apiserver/plugin/pkg/authenticator/password/passwordfile"
"k8s.io/apiserver/plugin/pkg/authenticator/request/basicauth"
@ -62,6 +64,9 @@ type AuthenticatorConfig struct {
WebhookTokenAuthnConfigFile string
WebhookTokenAuthnCacheTTL time.Duration
TokenSuccessCacheTTL time.Duration
TokenFailureCacheTTL time.Duration
RequestHeaderConfig *authenticatorfactory.RequestHeaderConfig
// TODO, this is the only non-serializable part of the entire config. Factor it out into a clientconfig
@ -73,9 +78,9 @@ type AuthenticatorConfig struct {
// Kubernetes authentication mechanisms.
func (config AuthenticatorConfig) New() (authenticator.Request, *spec.SecurityDefinitions, error) {
var authenticators []authenticator.Request
var tokenAuthenticators []authenticator.Token
securityDefinitions := spec.SecurityDefinitions{}
hasBasicAuth := false
hasTokenAuth := false
// front-proxy, BasicAuth methods, local first, then remote
// Add the front proxy authenticator if requested
@ -125,22 +130,19 @@ func (config AuthenticatorConfig) New() (authenticator.Request, *spec.SecurityDe
if err != nil {
return nil, nil, err
}
authenticators = append(authenticators, bearertoken.New(tokenAuth), websocket.NewProtocolAuthenticator(tokenAuth))
hasTokenAuth = true
tokenAuthenticators = append(tokenAuthenticators, tokenAuth)
}
if len(config.ServiceAccountKeyFiles) > 0 {
serviceAccountAuth, err := newServiceAccountAuthenticator(config.ServiceAccountKeyFiles, config.ServiceAccountLookup, config.ServiceAccountTokenGetter)
if err != nil {
return nil, nil, err
}
authenticators = append(authenticators, bearertoken.New(serviceAccountAuth), websocket.NewProtocolAuthenticator(serviceAccountAuth))
hasTokenAuth = true
tokenAuthenticators = append(tokenAuthenticators, serviceAccountAuth)
}
if config.BootstrapToken {
if config.BootstrapTokenAuthenticator != nil {
// TODO: This can sometimes be nil because of
authenticators = append(authenticators, bearertoken.New(config.BootstrapTokenAuthenticator), websocket.NewProtocolAuthenticator(config.BootstrapTokenAuthenticator))
hasTokenAuth = true
tokenAuthenticators = append(tokenAuthenticators, config.BootstrapTokenAuthenticator)
}
}
// NOTE(ericchiang): Keep the OpenID Connect after Service Accounts.
@ -154,16 +156,14 @@ func (config AuthenticatorConfig) New() (authenticator.Request, *spec.SecurityDe
if err != nil {
return nil, nil, err
}
authenticators = append(authenticators, bearertoken.New(oidcAuth), websocket.NewProtocolAuthenticator(oidcAuth))
hasTokenAuth = true
tokenAuthenticators = append(tokenAuthenticators, oidcAuth)
}
if len(config.WebhookTokenAuthnConfigFile) > 0 {
webhookTokenAuth, err := newWebhookTokenAuthenticator(config.WebhookTokenAuthnConfigFile, config.WebhookTokenAuthnCacheTTL)
if err != nil {
return nil, nil, err
}
authenticators = append(authenticators, bearertoken.New(webhookTokenAuth), websocket.NewProtocolAuthenticator(webhookTokenAuth))
hasTokenAuth = true
tokenAuthenticators = append(tokenAuthenticators, webhookTokenAuth)
}
if hasBasicAuth {
@ -175,7 +175,14 @@ func (config AuthenticatorConfig) New() (authenticator.Request, *spec.SecurityDe
}
}
if hasTokenAuth {
if len(tokenAuthenticators) > 0 {
// Union the token authenticators
tokenAuth := tokenunion.New(tokenAuthenticators...)
// Optionally cache authentication results
if config.TokenSuccessCacheTTL > 0 || config.TokenFailureCacheTTL > 0 {
tokenAuth = tokencache.New(tokenAuth, config.TokenSuccessCacheTTL, config.TokenFailureCacheTTL)
}
authenticators = append(authenticators, bearertoken.New(tokenAuth), websocket.NewProtocolAuthenticator(tokenAuth))
securityDefinitions["BearerToken"] = &spec.SecurityScheme{
SecuritySchemeProps: spec.SecuritySchemeProps{
Type: "apiKey",

View File

@ -41,6 +41,9 @@ type BuiltInAuthenticationOptions struct {
ServiceAccounts *ServiceAccountAuthenticationOptions
TokenFile *TokenFileAuthenticationOptions
WebHook *WebHookAuthenticationOptions
TokenSuccessCacheTTL time.Duration
TokenFailureCacheTTL time.Duration
}
type AnonymousAuthenticationOptions struct {
@ -83,7 +86,10 @@ type WebHookAuthenticationOptions struct {
}
func NewBuiltInAuthenticationOptions() *BuiltInAuthenticationOptions {
return &BuiltInAuthenticationOptions{}
return &BuiltInAuthenticationOptions{
TokenSuccessCacheTTL: 10 * time.Second,
TokenFailureCacheTTL: 0 * time.Second,
}
}
func (s *BuiltInAuthenticationOptions) WithAll() *BuiltInAuthenticationOptions {
@ -250,7 +256,10 @@ func (s *BuiltInAuthenticationOptions) AddFlags(fs *pflag.FlagSet) {
}
func (s *BuiltInAuthenticationOptions) ToAuthenticationConfig() authenticator.AuthenticatorConfig {
ret := authenticator.AuthenticatorConfig{}
ret := authenticator.AuthenticatorConfig{
TokenSuccessCacheTTL: s.TokenSuccessCacheTTL,
TokenFailureCacheTTL: s.TokenFailureCacheTTL,
}
if s.Anonymous != nil {
ret.Anonymous = s.Anonymous.Allow
@ -297,6 +306,15 @@ func (s *BuiltInAuthenticationOptions) ToAuthenticationConfig() authenticator.Au
if s.WebHook != nil {
ret.WebhookTokenAuthnConfigFile = s.WebHook.ConfigFile
ret.WebhookTokenAuthnCacheTTL = s.WebHook.CacheTTL
if len(s.WebHook.ConfigFile) > 0 && s.WebHook.CacheTTL > 0 {
if s.TokenSuccessCacheTTL > 0 && s.WebHook.CacheTTL < s.TokenSuccessCacheTTL {
glog.Warningf("the webhook cache ttl of %s is shorter than the overall cache ttl of %s for successful token authentication attempts.", s.WebHook.CacheTTL, s.TokenSuccessCacheTTL)
}
if s.TokenFailureCacheTTL > 0 && s.WebHook.CacheTTL < s.TokenFailureCacheTTL {
glog.Warningf("the webhook cache ttl of %s is shorter than the overall cache ttl of %s for failed token authentication attempts.", s.WebHook.CacheTTL, s.TokenFailureCacheTTL)
}
}
}
return ret

View File

@ -114,7 +114,9 @@ filegroup(
"//staging/src/k8s.io/apiserver/pkg/authentication/request/websocket:all-srcs",
"//staging/src/k8s.io/apiserver/pkg/authentication/request/x509:all-srcs",
"//staging/src/k8s.io/apiserver/pkg/authentication/serviceaccount:all-srcs",
"//staging/src/k8s.io/apiserver/pkg/authentication/token/cache:all-srcs",
"//staging/src/k8s.io/apiserver/pkg/authentication/token/tokenfile:all-srcs",
"//staging/src/k8s.io/apiserver/pkg/authentication/token/union:all-srcs",
"//staging/src/k8s.io/apiserver/pkg/authentication/user:all-srcs",
"//staging/src/k8s.io/apiserver/pkg/authorization/authorizer:all-srcs",
"//staging/src/k8s.io/apiserver/pkg/authorization/authorizerfactory:all-srcs",

View File

@ -8,7 +8,10 @@ load(
go_test(
name = "go_default_test",
srcs = ["group_adder_test.go"],
srcs = [
"group_adder_test.go",
"token_group_adder_test.go",
],
library = ":go_default_library",
deps = [
"//vendor/k8s.io/apiserver/pkg/authentication/authenticator:go_default_library",
@ -21,6 +24,7 @@ go_library(
srcs = [
"authenticated_group_adder.go",
"group_adder.go",
"token_group_adder.go",
],
deps = [
"//vendor/k8s.io/apiserver/pkg/authentication/authenticator:go_default_library",

View File

@ -0,0 +1,48 @@
/*
Copyright 2017 The Kubernetes 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 group
import (
"k8s.io/apiserver/pkg/authentication/authenticator"
"k8s.io/apiserver/pkg/authentication/user"
)
// TokenGroupAdder adds groups to an authenticated user.Info
type TokenGroupAdder struct {
// Authenticator is delegated to make the authentication decision
Authenticator authenticator.Token
// Groups are additional groups to add to the user.Info from a successful authentication
Groups []string
}
// NewTokenGroupAdder wraps a token authenticator, and adds the specified groups to the returned user when authentication succeeds
func NewTokenGroupAdder(auth authenticator.Token, groups []string) authenticator.Token {
return &TokenGroupAdder{auth, groups}
}
func (g *TokenGroupAdder) AuthenticateToken(token string) (user.Info, bool, error) {
u, ok, err := g.Authenticator.AuthenticateToken(token)
if err != nil || !ok {
return nil, ok, err
}
return &user.DefaultInfo{
Name: u.GetName(),
UID: u.GetUID(),
Groups: append(u.GetGroups(), g.Groups...),
Extra: u.GetExtra(),
}, true, nil
}

View File

@ -0,0 +1,41 @@
/*
Copyright 2017 The Kubernetes 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 group
import (
"reflect"
"testing"
"k8s.io/apiserver/pkg/authentication/authenticator"
"k8s.io/apiserver/pkg/authentication/user"
)
func TestTokenGroupAdder(t *testing.T) {
adder := authenticator.Token(
NewTokenGroupAdder(
authenticator.TokenFunc(func(token string) (user.Info, bool, error) {
return &user.DefaultInfo{Name: "user", Groups: []string{"original"}}, true, nil
}),
[]string{"added"},
),
)
user, _, _ := adder.AuthenticateToken("")
if !reflect.DeepEqual(user.GetGroups(), []string{"original", "added"}) {
t.Errorf("Expected original,added groups, got %#v", user.GetGroups())
}
}

View File

@ -0,0 +1,54 @@
package(default_visibility = ["//visibility:public"])
licenses(["notice"])
load(
"@io_bazel_rules_go//go:def.bzl",
"go_library",
"go_test",
)
go_test(
name = "go_default_test",
srcs = [
"cache_test.go",
"cached_token_authenticator_test.go",
],
library = ":go_default_library",
tags = ["automanaged"],
deps = [
"//vendor/github.com/pborman/uuid:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/clock:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/authenticator:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/user:go_default_library",
],
)
go_library(
name = "go_default_library",
srcs = [
"cache_simple.go",
"cache_striped.go",
"cached_token_authenticator.go",
],
tags = ["automanaged"],
deps = [
"//vendor/k8s.io/apimachinery/pkg/util/cache:go_default_library",
"//vendor/k8s.io/apimachinery/pkg/util/clock:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/authenticator:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/user:go_default_library",
],
)
filegroup(
name = "package-srcs",
srcs = glob(["**"]),
tags = ["automanaged"],
visibility = ["//visibility:private"],
)
filegroup(
name = "all-srcs",
srcs = [":package-srcs"],
tags = ["automanaged"],
)

View File

@ -0,0 +1,49 @@
/*
Copyright 2017 The Kubernetes 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 cache
import (
"time"
lrucache "k8s.io/apimachinery/pkg/util/cache"
"k8s.io/apimachinery/pkg/util/clock"
)
type simpleCache struct {
lru *lrucache.LRUExpireCache
}
func newSimpleCache(size int, clock clock.Clock) cache {
return &simpleCache{lru: lrucache.NewLRUExpireCacheWithClock(size, clock)}
}
func (c *simpleCache) get(key string) (*cacheRecord, bool) {
record, ok := c.lru.Get(key)
if !ok {
return nil, false
}
value, ok := record.(*cacheRecord)
return value, ok
}
func (c *simpleCache) set(key string, value *cacheRecord, ttl time.Duration) {
c.lru.Add(key, value, ttl)
}
func (c *simpleCache) remove(key string) {
c.lru.Remove(key)
}

View File

@ -0,0 +1,60 @@
/*
Copyright 2017 The Kubernetes 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 cache
import (
"hash/fnv"
"time"
)
// split cache lookups across N striped caches
type stripedCache struct {
stripeCount uint32
keyFunc func(string) uint32
caches []cache
}
type keyFunc func(string) uint32
type newCacheFunc func() cache
func newStripedCache(stripeCount int, keyFunc keyFunc, newCacheFunc newCacheFunc) cache {
caches := []cache{}
for i := 0; i < stripeCount; i++ {
caches = append(caches, newCacheFunc())
}
return &stripedCache{
stripeCount: uint32(stripeCount),
keyFunc: keyFunc,
caches: caches,
}
}
func (c *stripedCache) get(key string) (*cacheRecord, bool) {
return c.caches[c.keyFunc(key)%c.stripeCount].get(key)
}
func (c *stripedCache) set(key string, value *cacheRecord, ttl time.Duration) {
c.caches[c.keyFunc(key)%c.stripeCount].set(key, value, ttl)
}
func (c *stripedCache) remove(key string) {
c.caches[c.keyFunc(key)%c.stripeCount].remove(key)
}
func fnvKeyFunc(key string) uint32 {
f := fnv.New32()
f.Write([]byte(key))
return f.Sum32()
}

View File

@ -0,0 +1,94 @@
/*
Copyright 2017 The Kubernetes 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 cache
import (
"math/rand"
"testing"
"time"
"k8s.io/apimachinery/pkg/util/clock"
"k8s.io/apiserver/pkg/authentication/user"
"github.com/pborman/uuid"
)
func TestSimpleCache(t *testing.T) {
testCache(newSimpleCache(4096, clock.RealClock{}), t)
}
func BenchmarkSimpleCache(b *testing.B) {
benchmarkCache(newSimpleCache(4096, clock.RealClock{}), b)
}
func TestStripedCache(t *testing.T) {
testCache(newStripedCache(32, fnvKeyFunc, func() cache { return newSimpleCache(128, clock.RealClock{}) }), t)
}
func BenchmarkStripedCache(b *testing.B) {
benchmarkCache(newStripedCache(32, fnvKeyFunc, func() cache { return newSimpleCache(128, clock.RealClock{}) }), b)
}
func benchmarkCache(cache cache, b *testing.B) {
keys := []string{}
for i := 0; i < b.N; i++ {
key := uuid.NewRandom().String()
keys = append(keys, key)
}
b.ResetTimer()
b.SetParallelism(500)
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
key := keys[rand.Intn(b.N)]
_, ok := cache.get(key)
if ok {
cache.remove(key)
} else {
cache.set(key, &cacheRecord{}, time.Second)
}
}
})
}
func testCache(cache cache, t *testing.T) {
if result, ok := cache.get("foo"); ok || result != nil {
t.Errorf("Expected null, false, got %#v, %v", result, ok)
}
record1 := &cacheRecord{user: &user.DefaultInfo{Name: "bob"}}
record2 := &cacheRecord{user: &user.DefaultInfo{Name: "alice"}}
// when empty, record is stored
cache.set("foo", record1, time.Hour)
if result, ok := cache.get("foo"); !ok || result != record1 {
t.Errorf("Expected %#v, true, got %#v, %v", record1, ok)
}
// newer record overrides
cache.set("foo", record2, time.Hour)
if result, ok := cache.get("foo"); !ok || result != record2 {
t.Errorf("Expected %#v, true, got %#v, %v", record2, ok)
}
// removing the current value removes
cache.remove("foo")
if result, ok := cache.get("foo"); ok || result != nil {
t.Errorf("Expected null, false, got %#v, %v", result, ok)
}
}

View File

@ -0,0 +1,82 @@
/*
Copyright 2017 The Kubernetes 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 cache
import (
"time"
utilclock "k8s.io/apimachinery/pkg/util/clock"
"k8s.io/apiserver/pkg/authentication/authenticator"
"k8s.io/apiserver/pkg/authentication/user"
)
// cacheRecord holds the three return values of the authenticator.Token AuthenticateToken method
type cacheRecord struct {
user user.Info
ok bool
err error
}
type cachedTokenAuthenticator struct {
authenticator authenticator.Token
successTTL time.Duration
failureTTL time.Duration
cache cache
}
type cache interface {
// given a key, return the record, and whether or not it existed
get(key string) (value *cacheRecord, exists bool)
// caches the record for the key
set(key string, value *cacheRecord, ttl time.Duration)
// removes the record for the key
remove(key string)
}
// New returns a token authenticator that caches the results of the specified authenticator. A ttl of 0 bypasses the cache.
func New(authenticator authenticator.Token, successTTL, failureTTL time.Duration) authenticator.Token {
return newWithClock(authenticator, successTTL, failureTTL, utilclock.RealClock{})
}
func newWithClock(authenticator authenticator.Token, successTTL, failureTTL time.Duration, clock utilclock.Clock) authenticator.Token {
return &cachedTokenAuthenticator{
authenticator: authenticator,
successTTL: successTTL,
failureTTL: failureTTL,
cache: newStripedCache(32, fnvKeyFunc, func() cache { return newSimpleCache(128, clock) }),
}
}
// AuthenticateToken implements authenticator.Token
func (a *cachedTokenAuthenticator) AuthenticateToken(token string) (user.Info, bool, error) {
if record, ok := a.cache.get(token); ok {
return record.user, record.ok, record.err
}
user, ok, err := a.authenticator.AuthenticateToken(token)
switch {
case ok && a.successTTL > 0:
a.cache.set(token, &cacheRecord{user: user, ok: ok, err: err}, a.successTTL)
case !ok && a.failureTTL > 0:
a.cache.set(token, &cacheRecord{user: user, ok: ok, err: err}, a.failureTTL)
}
return user, ok, err
}

View File

@ -0,0 +1,105 @@
/*
Copyright 2017 The Kubernetes 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 cache
import (
"reflect"
"testing"
"time"
utilclock "k8s.io/apimachinery/pkg/util/clock"
"k8s.io/apiserver/pkg/authentication/authenticator"
"k8s.io/apiserver/pkg/authentication/user"
)
func TestCachedTokenAuthenticator(t *testing.T) {
var (
calledWithToken []string
resultUsers map[string]user.Info
resultOk bool
resultErr error
)
fakeAuth := authenticator.TokenFunc(func(token string) (user.Info, bool, error) {
calledWithToken = append(calledWithToken, token)
return resultUsers[token], resultOk, resultErr
})
fakeClock := utilclock.NewFakeClock(time.Now())
a := newWithClock(fakeAuth, time.Minute, 0, fakeClock)
calledWithToken, resultUsers, resultOk, resultErr = []string{}, nil, false, nil
a.AuthenticateToken("bad1")
a.AuthenticateToken("bad2")
a.AuthenticateToken("bad3")
a.AuthenticateToken("bad1")
a.AuthenticateToken("bad2")
a.AuthenticateToken("bad3")
if !reflect.DeepEqual(calledWithToken, []string{"bad1", "bad2", "bad3", "bad1", "bad2", "bad3"}) {
t.Errorf("Expected failing calls to bypass cache, got %v", calledWithToken)
}
// reset calls, make the backend return success for three user tokens
calledWithToken = []string{}
resultUsers, resultOk, resultErr = map[string]user.Info{}, true, nil
resultUsers["usertoken1"] = &user.DefaultInfo{Name: "user1"}
resultUsers["usertoken2"] = &user.DefaultInfo{Name: "user2"}
resultUsers["usertoken3"] = &user.DefaultInfo{Name: "user3"}
// populate cache
if user, ok, err := a.AuthenticateToken("usertoken1"); err != nil || !ok || user.GetName() != "user1" {
t.Errorf("Expected user1")
}
if user, ok, err := a.AuthenticateToken("usertoken2"); err != nil || !ok || user.GetName() != "user2" {
t.Errorf("Expected user2")
}
if user, ok, err := a.AuthenticateToken("usertoken3"); err != nil || !ok || user.GetName() != "user3" {
t.Errorf("Expected user3")
}
if !reflect.DeepEqual(calledWithToken, []string{"usertoken1", "usertoken2", "usertoken3"}) {
t.Errorf("Expected token calls, got %v", calledWithToken)
}
// reset calls, make the backend return failures
calledWithToken = []string{}
resultUsers, resultOk, resultErr = nil, false, nil
// authenticate calls still succeed and backend is not hit
if user, ok, err := a.AuthenticateToken("usertoken1"); err != nil || !ok || user.GetName() != "user1" {
t.Errorf("Expected user1")
}
if user, ok, err := a.AuthenticateToken("usertoken2"); err != nil || !ok || user.GetName() != "user2" {
t.Errorf("Expected user2")
}
if user, ok, err := a.AuthenticateToken("usertoken3"); err != nil || !ok || user.GetName() != "user3" {
t.Errorf("Expected user3")
}
if !reflect.DeepEqual(calledWithToken, []string{}) {
t.Errorf("Expected no token calls, got %v", calledWithToken)
}
// skip forward in time
fakeClock.Step(2 * time.Minute)
// backend is consulted again and fails
a.AuthenticateToken("usertoken1")
a.AuthenticateToken("usertoken2")
a.AuthenticateToken("usertoken3")
if !reflect.DeepEqual(calledWithToken, []string{"usertoken1", "usertoken2", "usertoken3"}) {
t.Errorf("Expected token calls, got %v", calledWithToken)
}
}

View File

@ -0,0 +1,41 @@
package(default_visibility = ["//visibility:public"])
licenses(["notice"])
load(
"@io_bazel_rules_go//go:def.bzl",
"go_library",
"go_test",
)
go_test(
name = "go_default_test",
srcs = ["unionauth_test.go"],
library = ":go_default_library",
tags = ["automanaged"],
deps = ["//vendor/k8s.io/apiserver/pkg/authentication/user:go_default_library"],
)
go_library(
name = "go_default_library",
srcs = ["union.go"],
tags = ["automanaged"],
deps = [
"//vendor/k8s.io/apimachinery/pkg/util/errors:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/authenticator:go_default_library",
"//vendor/k8s.io/apiserver/pkg/authentication/user:go_default_library",
],
)
filegroup(
name = "package-srcs",
srcs = glob(["**"]),
tags = ["automanaged"],
visibility = ["//visibility:private"],
)
filegroup(
name = "all-srcs",
srcs = [":package-srcs"],
tags = ["automanaged"],
)

View File

@ -0,0 +1,70 @@
/*
Copyright 2017 The Kubernetes 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 union
import (
utilerrors "k8s.io/apimachinery/pkg/util/errors"
"k8s.io/apiserver/pkg/authentication/authenticator"
"k8s.io/apiserver/pkg/authentication/user"
)
// unionAuthTokenHandler authenticates tokens using a chain of authenticator.Token objects
type unionAuthTokenHandler struct {
// Handlers is a chain of request authenticators to delegate to
Handlers []authenticator.Token
// FailOnError determines whether an error returns short-circuits the chain
FailOnError bool
}
// New returns a token authenticator that validates credentials using a chain of authenticator.Token objects.
// The entire chain is tried until one succeeds. If all fail, an aggregate error is returned.
func New(authTokenHandlers ...authenticator.Token) authenticator.Token {
if len(authTokenHandlers) == 1 {
return authTokenHandlers[0]
}
return &unionAuthTokenHandler{Handlers: authTokenHandlers, FailOnError: false}
}
// NewFailOnError returns a token authenticator that validates credentials using a chain of authenticator.Token objects.
// The first error short-circuits the chain.
func NewFailOnError(authTokenHandlers ...authenticator.Token) authenticator.Token {
if len(authTokenHandlers) == 1 {
return authTokenHandlers[0]
}
return &unionAuthTokenHandler{Handlers: authTokenHandlers, FailOnError: true}
}
// AuthenticateToken authenticates the token using a chain of authenticator.Token objects.
func (authHandler *unionAuthTokenHandler) AuthenticateToken(token string) (user.Info, bool, error) {
var errlist []error
for _, currAuthRequestHandler := range authHandler.Handlers {
info, ok, err := currAuthRequestHandler.AuthenticateToken(token)
if err != nil {
if authHandler.FailOnError {
return info, ok, err
}
errlist = append(errlist, err)
continue
}
if ok {
return info, ok, err
}
}
return nil, false, utilerrors.NewAggregate(errlist)
}

View File

@ -0,0 +1,158 @@
/*
Copyright 2017 The Kubernetes 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 union
import (
"errors"
"reflect"
"strings"
"testing"
"k8s.io/apiserver/pkg/authentication/user"
)
type mockAuthRequestHandler struct {
returnUser user.Info
isAuthenticated bool
err error
}
var (
user1 = &user.DefaultInfo{Name: "fresh_ferret", UID: "alfa"}
user2 = &user.DefaultInfo{Name: "elegant_sheep", UID: "bravo"}
)
func (mock *mockAuthRequestHandler) AuthenticateToken(token string) (user.Info, bool, error) {
return mock.returnUser, mock.isAuthenticated, mock.err
}
func TestAuthenticateTokenSecondPasses(t *testing.T) {
handler1 := &mockAuthRequestHandler{returnUser: user1}
handler2 := &mockAuthRequestHandler{returnUser: user2, isAuthenticated: true}
authRequestHandler := New(handler1, handler2)
authenticatedUser, isAuthenticated, err := authRequestHandler.AuthenticateToken("foo")
if err != nil {
t.Errorf("Unexpected error: %v", err)
}
if !isAuthenticated {
t.Errorf("Unexpectedly unauthenticated: %v", isAuthenticated)
}
if !reflect.DeepEqual(user2, authenticatedUser) {
t.Errorf("Expected %v, got %v", user2, authenticatedUser)
}
}
func TestAuthenticateTokenFirstPasses(t *testing.T) {
handler1 := &mockAuthRequestHandler{returnUser: user1, isAuthenticated: true}
handler2 := &mockAuthRequestHandler{returnUser: user2}
authRequestHandler := New(handler1, handler2)
authenticatedUser, isAuthenticated, err := authRequestHandler.AuthenticateToken("foo")
if err != nil {
t.Errorf("Unexpected error: %v", err)
}
if !isAuthenticated {
t.Errorf("Unexpectedly unauthenticated: %v", isAuthenticated)
}
if !reflect.DeepEqual(user1, authenticatedUser) {
t.Errorf("Expected %v, got %v", user1, authenticatedUser)
}
}
func TestAuthenticateTokenSuppressUnnecessaryErrors(t *testing.T) {
handler1 := &mockAuthRequestHandler{err: errors.New("first")}
handler2 := &mockAuthRequestHandler{isAuthenticated: true}
authRequestHandler := New(handler1, handler2)
_, isAuthenticated, err := authRequestHandler.AuthenticateToken("foo")
if err != nil {
t.Errorf("Unexpected error: %v", err)
}
if !isAuthenticated {
t.Errorf("Unexpectedly unauthenticated: %v", isAuthenticated)
}
}
func TestAuthenticateTokenNoAuthenticators(t *testing.T) {
authRequestHandler := New()
authenticatedUser, isAuthenticated, err := authRequestHandler.AuthenticateToken("foo")
if err != nil {
t.Errorf("Unexpected error: %v", err)
}
if isAuthenticated {
t.Errorf("Unexpectedly authenticated: %v", isAuthenticated)
}
if authenticatedUser != nil {
t.Errorf("Unexpected authenticatedUser: %v", authenticatedUser)
}
}
func TestAuthenticateTokenNonePass(t *testing.T) {
handler1 := &mockAuthRequestHandler{}
handler2 := &mockAuthRequestHandler{}
authRequestHandler := New(handler1, handler2)
_, isAuthenticated, err := authRequestHandler.AuthenticateToken("foo")
if err != nil {
t.Errorf("Unexpected error: %v", err)
}
if isAuthenticated {
t.Errorf("Unexpectedly authenticated: %v", isAuthenticated)
}
}
func TestAuthenticateTokenAdditiveErrors(t *testing.T) {
handler1 := &mockAuthRequestHandler{err: errors.New("first")}
handler2 := &mockAuthRequestHandler{err: errors.New("second")}
authRequestHandler := New(handler1, handler2)
_, isAuthenticated, err := authRequestHandler.AuthenticateToken("foo")
if err == nil {
t.Errorf("Expected an error")
}
if !strings.Contains(err.Error(), "first") {
t.Errorf("Expected error containing %v, got %v", "first", err)
}
if !strings.Contains(err.Error(), "second") {
t.Errorf("Expected error containing %v, got %v", "second", err)
}
if isAuthenticated {
t.Errorf("Unexpectedly authenticated: %v", isAuthenticated)
}
}
func TestAuthenticateTokenFailEarly(t *testing.T) {
handler1 := &mockAuthRequestHandler{err: errors.New("first")}
handler2 := &mockAuthRequestHandler{err: errors.New("second")}
authRequestHandler := NewFailOnError(handler1, handler2)
_, isAuthenticated, err := authRequestHandler.AuthenticateToken("foo")
if err == nil {
t.Errorf("Expected an error")
}
if !strings.Contains(err.Error(), "first") {
t.Errorf("Expected error containing %v, got %v", "first", err)
}
if strings.Contains(err.Error(), "second") {
t.Errorf("Did not expect second error, got %v", err)
}
if isAuthenticated {
t.Errorf("Unexpectedly authenticated: %v", isAuthenticated)
}
}