Merge pull request #64149 from sttts/sttts-ctrl-mgr-secure-ports

Automatic merge from submit-queue (batch tested with PRs 67756, 64149, 68076, 68131, 68120). If you want to cherry-pick this change to another branch, please follow the instructions here: https://github.com/kubernetes/community/blob/master/contributors/devel/cherry-picks.md.

kube-ctrl-mgr: enable secure port 10257

This PR enables authn+authz (delegated to the kube-apiserver) and the secure port 10257 for the kube-controller-manager. In addition, the insecure port is disabled.

Moreover, it adds integration test coverage for the `--port` and `--secure-port` flags, plus the testserver infrastructure to tests flags in general inside integration tests.

```release-note
Enable secure serving on port 10257 to kube-controller-manager (configurable via `--secure-port`). Delegated authentication and authorization have to be configured like for aggregated API servers.
```
pull/8/head
Kubernetes Submit Queue 2018-08-31 09:06:31 -07:00 committed by GitHub
commit 5d4b0f81bf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 646 additions and 41 deletions

View File

@ -39,8 +39,13 @@ func BuildHandlerChain(apiHandler http.Handler, authorizationInfo *apiserver.Aut
requestInfoResolver := &apirequest.RequestInfoFactory{}
failedHandler := genericapifilters.Unauthorized(legacyscheme.Codecs, false)
handler := genericapifilters.WithAuthorization(apiHandler, authorizationInfo.Authorizer, legacyscheme.Codecs)
handler = genericapifilters.WithAuthentication(handler, authenticationInfo.Authenticator, failedHandler)
handler := apiHandler
if authorizationInfo != nil {
handler = genericapifilters.WithAuthorization(apiHandler, authorizationInfo.Authorizer, legacyscheme.Codecs)
}
if authenticationInfo != nil {
handler = genericapifilters.WithAuthentication(handler, authenticationInfo.Authenticator, failedHandler)
}
handler = genericapifilters.WithRequestInfo(handler, requestInfoResolver)
handler = genericfilters.WithPanicRecovery(handler)

View File

@ -111,6 +111,7 @@ go_library(
"//staging/src/k8s.io/apimachinery/pkg/util/sets:go_default_library",
"//staging/src/k8s.io/apimachinery/pkg/util/uuid:go_default_library",
"//staging/src/k8s.io/apimachinery/pkg/util/wait:go_default_library",
"//staging/src/k8s.io/apiserver/pkg/server:go_default_library",
"//staging/src/k8s.io/apiserver/pkg/server/mux:go_default_library",
"//staging/src/k8s.io/apiserver/pkg/util/feature:go_default_library",
"//staging/src/k8s.io/apiserver/pkg/util/flag:go_default_library",
@ -157,6 +158,7 @@ filegroup(
":package-srcs",
"//cmd/kube-controller-manager/app/config:all-srcs",
"//cmd/kube-controller-manager/app/options:all-srcs",
"//cmd/kube-controller-manager/app/testing:all-srcs",
],
tags = ["automanaged"],
visibility = ["//visibility:public"],

View File

@ -29,6 +29,9 @@ type Config struct {
ComponentConfig componentconfig.KubeControllerManagerConfiguration
SecureServing *apiserver.SecureServingInfo
// LoopbackClientConfig is a config for a privileged loopback connection
LoopbackClientConfig *restclient.Config
// TODO: remove deprecated insecure serving
InsecureServing *apiserver.DeprecatedInsecureServingInfo
Authentication apiserver.AuthenticationInfo
@ -60,5 +63,8 @@ type CompletedConfig struct {
// Complete fills in any fields not set that are required to have valid data. It's mutating the receiver.
func (c *Config) Complete() *CompletedConfig {
cc := completedConfig{c}
apiserver.AuthorizeClientBearerToken(c.LoopbackClientConfig, &c.Authentication, &c.Authorization)
return &CompletedConfig{&cc}
}

View File

@ -25,19 +25,19 @@ import (
"fmt"
"io/ioutil"
"math/rand"
"net/http"
"os"
"time"
"github.com/golang/glog"
"github.com/spf13/cobra"
"net/http"
"k8s.io/apimachinery/pkg/runtime/schema"
utilruntime "k8s.io/apimachinery/pkg/util/runtime"
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/apimachinery/pkg/util/uuid"
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/apiserver/pkg/server"
"k8s.io/apiserver/pkg/server/mux"
apiserverflag "k8s.io/apiserver/pkg/util/flag"
cacheddiscovery "k8s.io/client-go/discovery/cached"
@ -160,7 +160,8 @@ func Run(c *config.CompletedConfig, stopCh <-chan struct{}) error {
}
if c.InsecureServing != nil {
unsecuredMux = genericcontrollermanager.NewBaseHandler(&c.ComponentConfig.Debugging)
handler := genericcontrollermanager.BuildHandlerChain(unsecuredMux, &c.Authorization, &c.Authentication)
insecureSuperuserAuthn := server.AuthenticationInfo{Authenticator: &server.InsecureSuperuser{}}
handler := genericcontrollermanager.BuildHandlerChain(unsecuredMux, nil, &insecureSuperuserAuthn)
if err := c.InsecureServing.Serve(handler, 0, stopCh); err != nil {
return err
}

View File

@ -83,9 +83,9 @@ type KubeControllerManagerOptions struct {
Controllers []string
ExternalCloudVolumePlugin string
SecureServing *apiserveroptions.SecureServingOptions
SecureServing *apiserveroptions.SecureServingOptionsWithLoopback
// TODO: remove insecure serving mode
InsecureServing *apiserveroptions.DeprecatedInsecureServingOptions
InsecureServing *apiserveroptions.DeprecatedInsecureServingOptionsWithLoopback
Authentication *apiserveroptions.DelegatingAuthenticationOptions
Authorization *apiserveroptions.DelegatingAuthorizationOptions
@ -179,22 +179,23 @@ func NewKubeControllerManagerOptions() (*KubeControllerManagerOptions, error) {
ConcurrentServiceSyncs: componentConfig.ServiceController.ConcurrentServiceSyncs,
},
Controllers: componentConfig.Controllers,
SecureServing: apiserveroptions.NewSecureServingOptions(),
InsecureServing: &apiserveroptions.DeprecatedInsecureServingOptions{
SecureServing: apiserveroptions.NewSecureServingOptions().WithLoopback(),
InsecureServing: (&apiserveroptions.DeprecatedInsecureServingOptions{
BindAddress: net.ParseIP(componentConfig.KubeCloudShared.Address),
BindPort: int(componentConfig.KubeCloudShared.Port),
BindNetwork: "tcp",
},
Authentication: nil, // TODO: enable with apiserveroptions.NewDelegatingAuthenticationOptions()
Authorization: nil, // TODO: enable with apiserveroptions.NewDelegatingAuthorizationOptions()
}).WithLoopback(),
Authentication: apiserveroptions.NewDelegatingAuthenticationOptions(),
Authorization: apiserveroptions.NewDelegatingAuthorizationOptions(),
}
s.Authentication.RemoteKubeConfigFileOptional = true
s.Authorization.RemoteKubeConfigFileOptional = true
s.Authorization.AlwaysAllowPaths = []string{"/healthz"}
s.SecureServing.ServerCert.CertDirectory = "/var/run/kubernetes"
s.SecureServing.ServerCert.PairName = "kube-controller-manager"
// disable secure serving for now
// TODO: enable HTTPS by default
s.SecureServing.BindPort = 0
s.SecureServing.BindPort = ports.KubeControllerManagerPort
gcIgnoredResources := make([]componentconfig.GroupResource, 0, len(garbagecollector.DefaultIgnoredResources()))
for r := range garbagecollector.DefaultIgnoredResources() {
@ -346,17 +347,19 @@ func (s *KubeControllerManagerOptions) ApplyTo(c *kubecontrollerconfig.Config) e
if err := s.ServiceController.ApplyTo(&c.ComponentConfig.ServiceController); err != nil {
return err
}
if err := s.InsecureServing.ApplyTo(&c.InsecureServing); err != nil {
if err := s.InsecureServing.ApplyTo(&c.InsecureServing, &c.LoopbackClientConfig); err != nil {
return err
}
if err := s.SecureServing.ApplyTo(&c.SecureServing); err != nil {
if err := s.SecureServing.ApplyTo(&c.SecureServing, &c.LoopbackClientConfig); err != nil {
return err
}
if err := s.Authentication.ApplyTo(&c.Authentication, c.SecureServing, nil); err != nil {
return err
}
if err := s.Authorization.ApplyTo(&c.Authorization); err != nil {
return err
if s.SecureServing.BindPort != 0 || s.SecureServing.Listener != nil {
if err := s.Authentication.ApplyTo(&c.Authentication, c.SecureServing, nil); err != nil {
return err
}
if err := s.Authorization.ApplyTo(&c.Authorization); err != nil {
return err
}
}
// sync back to component config

View File

@ -250,7 +250,7 @@ func TestAddFlags(t *testing.T) {
ConcurrentServiceSyncs: 2,
},
Controllers: []string{"foo", "bar"},
SecureServing: &apiserveroptions.SecureServingOptions{
SecureServing: (&apiserveroptions.SecureServingOptions{
BindPort: 10001,
BindAddress: net.ParseIP("192.168.4.21"),
ServerCert: apiserveroptions.GeneratableKeyCert{
@ -258,11 +258,27 @@ func TestAddFlags(t *testing.T) {
PairName: "kube-controller-manager",
},
HTTP2MaxStreamsPerConnection: 47,
},
InsecureServing: &apiserveroptions.DeprecatedInsecureServingOptions{
}).WithLoopback(),
InsecureServing: (&apiserveroptions.DeprecatedInsecureServingOptions{
BindAddress: net.ParseIP("192.168.4.10"),
BindPort: int(10000),
BindNetwork: "tcp",
}).WithLoopback(),
Authentication: &apiserveroptions.DelegatingAuthenticationOptions{
CacheTTL: 10 * time.Second,
ClientCert: apiserveroptions.ClientCertAuthenticationOptions{},
RequestHeader: apiserveroptions.RequestHeaderAuthenticationOptions{
UsernameHeaders: []string{"x-remote-user"},
GroupHeaders: []string{"x-remote-group"},
ExtraHeaderPrefixes: []string{"x-remote-extra-"},
},
RemoteKubeConfigFileOptional: true,
},
Authorization: &apiserveroptions.DelegatingAuthorizationOptions{
AllowCacheTTL: 10 * time.Second,
DenyCacheTTL: 10 * time.Second,
RemoteKubeConfigFileOptional: true,
AlwaysAllowPaths: []string{"/healthz"}, // note: this does not match /healthz/ or /healthz/*
},
Kubeconfig: "/kubeconfig",
Master: "192.168.4.20",

View File

@ -0,0 +1,31 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")
filegroup(
name = "package-srcs",
srcs = glob(["**"]),
tags = ["automanaged"],
visibility = ["//visibility:private"],
)
filegroup(
name = "all-srcs",
srcs = [":package-srcs"],
tags = ["automanaged"],
visibility = ["//visibility:public"],
)
go_library(
name = "go_default_library",
srcs = ["testserver.go"],
importpath = "k8s.io/kubernetes/cmd/kube-controller-manager/app/testing",
visibility = ["//visibility:public"],
deps = [
"//cmd/kube-controller-manager/app:go_default_library",
"//cmd/kube-controller-manager/app/config:go_default_library",
"//cmd/kube-controller-manager/app/options:go_default_library",
"//staging/src/k8s.io/apimachinery/pkg/util/wait:go_default_library",
"//staging/src/k8s.io/client-go/kubernetes:go_default_library",
"//staging/src/k8s.io/client-go/rest:go_default_library",
"//vendor/github.com/spf13/pflag:go_default_library",
],
)

View File

@ -0,0 +1,175 @@
/*
Copyright 2018 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 testing
import (
"fmt"
"io/ioutil"
"net"
"os"
"time"
"github.com/spf13/pflag"
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/client-go/kubernetes"
restclient "k8s.io/client-go/rest"
"k8s.io/kubernetes/cmd/kube-controller-manager/app"
kubecontrollerconfig "k8s.io/kubernetes/cmd/kube-controller-manager/app/config"
"k8s.io/kubernetes/cmd/kube-controller-manager/app/options"
)
// TearDownFunc is to be called to tear down a test server.
type TearDownFunc func()
// TestServer return values supplied by kube-test-ApiServer
type TestServer struct {
LoopbackClientConfig *restclient.Config // Rest client config using the magic token
Options *options.KubeControllerManagerOptions
Config *kubecontrollerconfig.Config
TearDownFn TearDownFunc // TearDown function
TmpDir string // Temp Dir used, by the apiserver
}
// Logger allows t.Testing and b.Testing to be passed to StartTestServer and StartTestServerOrDie
type Logger interface {
Errorf(format string, args ...interface{})
Fatalf(format string, args ...interface{})
Logf(format string, args ...interface{})
}
// StartTestServer starts a kube-controller-manager. A rest client config and a tear-down func,
// and location of the tmpdir are returned.
//
// Note: we return a tear-down func instead of a stop channel because the later will leak temporary
// files that because Golang testing's call to os.Exit will not give a stop channel go routine
// enough time to remove temporary files.
func StartTestServer(t Logger, customFlags []string) (result TestServer, err error) {
stopCh := make(chan struct{})
tearDown := func() {
close(stopCh)
if len(result.TmpDir) != 0 {
os.RemoveAll(result.TmpDir)
}
}
defer func() {
if result.TearDownFn == nil {
tearDown()
}
}()
result.TmpDir, err = ioutil.TempDir("", "kube-controller-manager")
if err != nil {
return result, fmt.Errorf("failed to create temp dir: %v", err)
}
fs := pflag.NewFlagSet("test", pflag.PanicOnError)
s, err := options.NewKubeControllerManagerOptions()
if err != nil {
return TestServer{}, err
}
all, disabled := app.KnownControllers(), app.ControllersDisabledByDefault.List()
namedFlagSets := s.Flags(all, disabled)
for _, f := range namedFlagSets.FlagSets {
fs.AddFlagSet(f)
}
fs.Parse(customFlags)
if s.SecureServing.BindPort != 0 {
s.SecureServing.Listener, s.SecureServing.BindPort, err = createListenerOnFreePort()
if err != nil {
return result, fmt.Errorf("failed to create listener: %v", err)
}
s.SecureServing.ServerCert.CertDirectory = result.TmpDir
t.Logf("kube-controller-manager will listen securely on port %d...", s.SecureServing.BindPort)
}
if s.InsecureServing.BindPort != 0 {
s.InsecureServing.Listener, s.InsecureServing.BindPort, err = createListenerOnFreePort()
if err != nil {
return result, fmt.Errorf("failed to create listener: %v", err)
}
t.Logf("kube-controller-manager will listen insecurely on port %d...", s.InsecureServing.BindPort)
}
config, err := s.Config(all, disabled)
if err != nil {
return result, fmt.Errorf("failed to create config from options: %v", err)
}
go func(stopCh <-chan struct{}) {
if err := app.Run(config.Complete(), stopCh); err != nil {
t.Errorf("kube-apiserver failed run: %v", err)
}
}(stopCh)
t.Logf("Waiting for /healthz to be ok...")
client, err := kubernetes.NewForConfig(config.LoopbackClientConfig)
if err != nil {
return result, fmt.Errorf("failed to create a client: %v", err)
}
err = wait.Poll(100*time.Millisecond, 30*time.Second, func() (bool, error) {
result := client.CoreV1().RESTClient().Get().AbsPath("/healthz").Do()
status := 0
result.StatusCode(&status)
if status == 200 {
return true, nil
}
return false, nil
})
if err != nil {
return result, fmt.Errorf("failed to wait for /healthz to return ok: %v", err)
}
// from here the caller must call tearDown
result.LoopbackClientConfig = config.LoopbackClientConfig
result.Options = s
result.Config = config
result.TearDownFn = tearDown
return result, nil
}
// StartTestServerOrDie calls StartTestServer t.Fatal if it does not succeed.
func StartTestServerOrDie(t Logger, flags []string) *TestServer {
result, err := StartTestServer(t, flags)
if err == nil {
return &result
}
t.Fatalf("failed to launch server: %v", err)
return nil
}
func createListenerOnFreePort() (net.Listener, int, error) {
ln, err := net.Listen("tcp", ":0")
if err != nil {
return nil, 0, err
}
// get port
tcpAddr, ok := ln.Addr().(*net.TCPAddr)
if !ok {
ln.Close()
return nil, 0, fmt.Errorf("invalid listen address: %q", ln.Addr().String())
}
return ln, tcpAddr.Port, nil
}

View File

@ -10,7 +10,6 @@ go_library(
srcs = ["insecure_handler.go"],
importpath = "k8s.io/kubernetes/pkg/kubeapiserver/server",
deps = [
"//staging/src/k8s.io/apiserver/pkg/authentication/user:go_default_library",
"//staging/src/k8s.io/apiserver/pkg/endpoints/filters:go_default_library",
"//staging/src/k8s.io/apiserver/pkg/server:go_default_library",
"//staging/src/k8s.io/apiserver/pkg/server/filters:go_default_library",

View File

@ -19,7 +19,6 @@ package server
import (
"net/http"
"k8s.io/apiserver/pkg/authentication/user"
genericapifilters "k8s.io/apiserver/pkg/endpoints/filters"
"k8s.io/apiserver/pkg/server"
genericfilters "k8s.io/apiserver/pkg/server/filters"
@ -32,7 +31,7 @@ import (
func BuildInsecureHandlerChain(apiHandler http.Handler, c *server.Config) http.Handler {
handler := apiHandler
handler = genericapifilters.WithAudit(handler, c.AuditBackend, c.AuditPolicyChecker, c.LongRunningFunc)
handler = genericapifilters.WithAuthentication(handler, insecureSuperuser{}, nil)
handler = genericapifilters.WithAuthentication(handler, server.InsecureSuperuser{}, nil)
handler = genericfilters.WithCORS(handler, c.CorsAllowedOriginList, nil, nil, nil, "true")
handler = genericfilters.WithTimeoutForNonLongRunningRequests(handler, c.LongRunningFunc, c.RequestTimeout)
handler = genericfilters.WithMaxInFlightLimit(handler, c.MaxRequestsInFlight, c.MaxMutatingRequestsInFlight, c.LongRunningFunc)
@ -42,15 +41,3 @@ func BuildInsecureHandlerChain(apiHandler http.Handler, c *server.Config) http.H
return handler
}
// insecureSuperuser implements authenticator.Request to always return a superuser.
// This is functionally equivalent to skipping authentication and authorization,
// but allows apiserver code to stop special-casing a nil user to skip authorization checks.
type insecureSuperuser struct{}
func (insecureSuperuser) AuthenticateRequest(req *http.Request) (user.Info, bool, error) {
return &user.DefaultInfo{
Name: "system:unsecured",
Groups: []string{user.SystemPrivilegedGroup, user.AllAuthenticated},
}, true, nil
}

View File

@ -28,6 +28,7 @@ const (
SchedulerPort = 10251
// InsecureKubeControllerManagerPort is the default port for the controller manager status server.
// May be overridden by a flag at startup.
// Deprecated: use the secure KubeControllerManagerPort instead.
InsecureKubeControllerManagerPort = 10252
// InsecureCloudControllerManagerPort is the default port for the cloud controller manager server.
// This value may be overridden by a flag at startup.
@ -41,4 +42,7 @@ const (
// ProxyHealthzPort is the default port for the proxy healthz server.
// May be overridden by a flag at startup.
ProxyHealthzPort = 10256
// KubeControllerManagerPort is the default port for the controller manager status server.
// May be overridden by a flag at startup.
KubeControllerManagerPort = 10257
)

View File

@ -23,6 +23,7 @@ import (
"github.com/golang/glog"
"k8s.io/apiserver/pkg/authentication/user"
"k8s.io/client-go/rest"
)
@ -70,3 +71,15 @@ func (s *DeprecatedInsecureServingInfo) NewLoopbackClientConfig() (*rest.Config,
Burst: 100,
}, nil
}
// InsecureSuperuser implements authenticator.Request to always return a superuser.
// This is functionally equivalent to skipping authentication and authorization,
// but allows apiserver code to stop special-casing a nil user to skip authorization checks.
type InsecureSuperuser struct{}
func (InsecureSuperuser) AuthenticateRequest(req *http.Request) (user.Info, bool, error) {
return &user.DefaultInfo{
Name: "system:unsecured",
Groups: []string{user.SystemPrivilegedGroup, user.AllAuthenticated},
}, true, nil
}

View File

@ -51,6 +51,7 @@ filegroup(
"//test/integration/framework:all-srcs",
"//test/integration/garbagecollector:all-srcs",
"//test/integration/ipamperf:all-srcs",
"//test/integration/kube_controller_manager:all-srcs",
"//test/integration/master:all-srcs",
"//test/integration/metrics:all-srcs",
"//test/integration/objectmeta:all-srcs",

View File

@ -0,0 +1,40 @@
package(default_visibility = ["//visibility:public"])
load(
"@io_bazel_rules_go//go:def.bzl",
"go_test",
)
go_test(
name = "go_default_test",
size = "large",
srcs = [
"main_test.go",
"serving_test.go",
],
tags = [
"etcd",
"integration",
],
deps = [
"//cmd/kube-apiserver/app/testing:go_default_library",
"//cmd/kube-controller-manager/app/testing:go_default_library",
"//staging/src/k8s.io/api/rbac/v1:go_default_library",
"//staging/src/k8s.io/apimachinery/pkg/apis/meta/v1:go_default_library",
"//staging/src/k8s.io/client-go/kubernetes:go_default_library",
"//test/integration/framework: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,27 @@
/*
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 kubecontrollermanager
import (
"testing"
"k8s.io/kubernetes/test/integration/framework"
)
func TestMain(m *testing.M) {
framework.EtcdMain(m.Run)
}

View File

@ -0,0 +1,295 @@
/*
Copyright 2018 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 kubecontrollermanager
import (
"crypto/tls"
"crypto/x509"
"fmt"
"io/ioutil"
"net/http"
"os"
"path"
"strings"
"testing"
rbacv1 "k8s.io/api/rbac/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/client-go/kubernetes"
kubeapiservertesting "k8s.io/kubernetes/cmd/kube-apiserver/app/testing"
ctrlmgrtesting "k8s.io/kubernetes/cmd/kube-controller-manager/app/testing"
"k8s.io/kubernetes/test/integration/framework"
)
func TestStartTestServer(t *testing.T) {
// Insulate this test from picking up in-cluster config when run inside a pod
// We can't assume we have permissions to write to /var/run/secrets/... from a unit test to mock in-cluster config for testing
originalHost := os.Getenv("KUBERNETES_SERVICE_HOST")
if len(originalHost) > 0 {
os.Setenv("KUBERNETES_SERVICE_HOST", "")
defer os.Setenv("KUBERNETES_SERVICE_HOST", originalHost)
}
// authenticate to apiserver via bearer token
token := "flwqkenfjasasdfmwerasd"
tokenFile, err := ioutil.TempFile("", "kubeconfig")
if err != nil {
t.Fatal(err)
}
tokenFile.WriteString(fmt.Sprintf(`
%s,kube-controller-manager,kube-controller-manager,""
`, token))
tokenFile.Close()
// start apiserver
server := kubeapiservertesting.StartTestServerOrDie(t, nil, []string{
"--token-auth-file", tokenFile.Name(),
"--authorization-mode", "RBAC",
}, framework.SharedEtcd())
defer server.TearDownFn()
// allow kube-controller-manager to do SubjectAccessReview
client, err := kubernetes.NewForConfig(server.ClientConfig)
if err != nil {
t.Fatalf("unexpected error creating client config: %v", err)
}
_, err = client.RbacV1().ClusterRoleBindings().Create(&rbacv1.ClusterRoleBinding{
ObjectMeta: metav1.ObjectMeta{Name: "kube-controller-manager:system:auth-delegator"},
Subjects: []rbacv1.Subject{{
Kind: "User",
Name: "kube-controller-manager",
}},
RoleRef: rbacv1.RoleRef{
APIGroup: "rbac.authorization.k8s.io",
Kind: "ClusterRole",
Name: "system:auth-delegator",
},
})
if err != nil {
t.Fatalf("failed to create system:auth-delegator rbac cluster role binding: %v", err)
}
// allow kube-controller-manager to read kube-system/extension-apiserver-authentication
_, err = client.RbacV1().RoleBindings("kube-system").Create(&rbacv1.RoleBinding{
ObjectMeta: metav1.ObjectMeta{Name: "kube-controller-manager:extension-apiserver-authentication-reader"},
Subjects: []rbacv1.Subject{{
Kind: "User",
Name: "kube-controller-manager",
}},
RoleRef: rbacv1.RoleRef{
APIGroup: "rbac.authorization.k8s.io",
Kind: "Role",
Name: "extension-apiserver-authentication-reader",
},
})
if err != nil {
t.Fatalf("failed to create kube-controller-manager:extension-apiserver-authentication-reader rbac role binding: %v", err)
}
// create kubeconfig for the apiserver
apiserverConfig, err := ioutil.TempFile("", "kubeconfig")
if err != nil {
t.Fatal(err)
}
apiserverConfig.WriteString(fmt.Sprintf(`
apiVersion: v1
kind: Config
clusters:
- cluster:
server: %s
certificate-authority: %s
name: integration
contexts:
- context:
cluster: integration
user: kube-controller-manager
name: default-context
current-context: default-context
users:
- name: kube-controller-manager
user:
token: %s
`, server.ClientConfig.Host, server.ServerOpts.SecureServing.ServerCert.CertKey.CertFile, token))
apiserverConfig.Close()
// create BROKEN kubeconfig for the apiserver
brokenApiserverConfig, err := ioutil.TempFile("", "kubeconfig")
if err != nil {
t.Fatal(err)
}
brokenApiserverConfig.WriteString(fmt.Sprintf(`
apiVersion: v1
kind: Config
clusters:
- cluster:
server: %s
certificate-authority: %s
name: integration
contexts:
- context:
cluster: integration
user: kube-controller-manager
name: default-context
current-context: default-context
users:
- name: kube-controller-manager
user:
token: WRONGTOKEN
`, server.ClientConfig.Host, server.ServerOpts.SecureServing.ServerCert.CertKey.CertFile))
brokenApiserverConfig.Close()
tests := []struct {
name string
flags []string
path string
anonymous bool // to use the token or not
wantErr bool
wantSecureCode, wantInsecureCode *int
}{
{"no-flags", nil, "/healthz", false, true, nil, nil},
{"insecurely /healthz", []string{
"--secure-port=0",
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/healthz", true, false, nil, intPtr(http.StatusOK)},
{"insecurely /metrics", []string{
"--secure-port=0",
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/metrics", true, false, nil, intPtr(http.StatusOK)},
{"/healthz without authn/authz", []string{
"--port=0",
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/healthz", true, false, intPtr(http.StatusOK), nil},
{"/metrics without auhn/z", []string{
"--kubeconfig", apiserverConfig.Name(),
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/metrics", true, false, intPtr(http.StatusForbidden), intPtr(http.StatusOK)},
{"authorization skipped for /healthz with authn/authz", []string{
"--port=0",
"--authentication-kubeconfig", apiserverConfig.Name(),
"--authorization-kubeconfig", apiserverConfig.Name(),
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/healthz", false, false, intPtr(http.StatusOK), nil},
{"authorization skipped for /healthz with BROKEN authn/authz", []string{
"--port=0",
"--authentication-skip-lookup", // to survive unaccessible extensions-apiserver-authentication configmap
"--authentication-kubeconfig", brokenApiserverConfig.Name(),
"--authorization-kubeconfig", brokenApiserverConfig.Name(),
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/healthz", false, false, intPtr(http.StatusOK), nil},
{"not authorized /metrics", []string{
"--port=0",
"--authentication-kubeconfig", apiserverConfig.Name(),
"--authorization-kubeconfig", apiserverConfig.Name(),
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/metrics", false, false, intPtr(http.StatusForbidden), nil},
{"not authorized /metrics with BROKEN authn/authz", []string{
"--authentication-kubeconfig", apiserverConfig.Name(),
"--authorization-kubeconfig", brokenApiserverConfig.Name(),
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/metrics", false, false, intPtr(http.StatusInternalServerError), intPtr(http.StatusOK)},
{"always-allowed /metrics with BROKEN authn/authz", []string{
"--port=0",
"--authentication-skip-lookup", // to survive unaccessible extensions-apiserver-authentication configmap
"--authentication-kubeconfig", apiserverConfig.Name(),
"--authorization-kubeconfig", apiserverConfig.Name(),
"--authorization-always-allow-paths", "/healthz,/metrics",
"--kubeconfig", apiserverConfig.Name(),
"--leader-elect=false",
}, "/metrics", false, false, intPtr(http.StatusOK), nil},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
gotResult, err := ctrlmgrtesting.StartTestServer(t, tt.flags)
if gotResult.TearDownFn != nil {
defer gotResult.TearDownFn()
}
if (err != nil) != tt.wantErr {
t.Fatalf("StartTestServer() error = %v, wantErr %v", err, tt.wantErr)
}
if err != nil {
return
}
if want, got := tt.wantSecureCode != nil, gotResult.Config.SecureServing != nil; want != got {
t.Errorf("SecureServing enabled: expected=%v got=%v", want, got)
} else if want {
url := fmt.Sprintf("https://%s%s", gotResult.Config.SecureServing.Listener.Addr().String(), tt.path)
url = strings.Replace(url, "[::]", "127.0.0.1", -1) // switch to IPv4 because the self-signed cert does not support [::]
// read self-signed server cert disk
pool := x509.NewCertPool()
serverCertPath := path.Join(gotResult.Options.SecureServing.ServerCert.CertDirectory, gotResult.Options.SecureServing.ServerCert.PairName+".crt")
serverCert, err := ioutil.ReadFile(serverCertPath)
if err != nil {
t.Fatalf("Failed to read controller-manager server cert %q: %v", serverCertPath, err)
}
pool.AppendCertsFromPEM(serverCert)
tr := &http.Transport{
TLSClientConfig: &tls.Config{
RootCAs: pool,
},
}
client := &http.Client{Transport: tr}
req, err := http.NewRequest("GET", url, nil)
if err != nil {
t.Fatal(err)
}
if !tt.anonymous {
req.Header.Add("Authorization", fmt.Sprintf("Token %s", token))
}
r, err := client.Do(req)
if err != nil {
t.Fatalf("failed to GET %s from controller-manager: %v", tt.path, err)
}
body, err := ioutil.ReadAll(r.Body)
defer r.Body.Close()
if got, expected := r.StatusCode, *tt.wantSecureCode; got != expected {
t.Fatalf("expected http %d at %s of controller-manager, got: %d %q", expected, tt.path, got, string(body))
}
}
if want, got := tt.wantInsecureCode != nil, gotResult.Config.InsecureServing != nil; want != got {
t.Errorf("InsecureServing enabled: expected=%v got=%v", want, got)
} else if want {
url := fmt.Sprintf("http://%s%s", gotResult.Config.InsecureServing.Listener.Addr().String(), tt.path)
r, err := http.Get(url)
if err != nil {
t.Fatalf("failed to GET %s from controller-manager: %v", tt.path, err)
}
body, err := ioutil.ReadAll(r.Body)
defer r.Body.Close()
if got, expected := r.StatusCode, *tt.wantInsecureCode; got != expected {
t.Fatalf("expected http %d at %s of controller-manager, got: %d %q", expected, tt.path, got, string(body))
}
}
})
}
}
func intPtr(x int) *int {
return &x
}