Merge pull request #29119 from wongma7/pv-gid-squash2

Automatic merge from submit-queue

Allow PVs to specify supplemental GIDs

Retry of https://github.com/kubernetes/kubernetes/pull/28691 . Adds a Kubelet helper function for getting extra supplemental groups
This commit is contained in:
k8s-merge-robot 2016-07-22 22:36:51 -07:00 committed by GitHub
commit d525d5345e
13 changed files with 442 additions and 119 deletions

View File

@ -44,6 +44,10 @@ type RuntimeHelper interface {
GetClusterDNS(pod *api.Pod) (dnsServers []string, dnsSearches []string, err error)
GetPodDir(podUID types.UID) string
GeneratePodHostNameAndDomain(pod *api.Pod) (hostname string, hostDomain string, err error)
// GetExtraSupplementalGroupsForPod returns a list of the extra
// supplemental groups for the Pod. These extra supplemental groups come
// from annotations on persistent volumes that the pod depends on.
GetExtraSupplementalGroupsForPod(pod *api.Pod) []int64
}
// ShouldContainerBeRestarted checks whether a container needs to be restarted.

View File

@ -688,9 +688,10 @@ func (dm *DockerManager) runContainer(
glog.V(3).Infof("Container %v/%v/%v: setting entrypoint \"%v\" and command \"%v\"", pod.Namespace, pod.Name, container.Name, dockerOpts.Config.Entrypoint, dockerOpts.Config.Cmd)
supplementalGids := dm.runtimeHelper.GetExtraSupplementalGroupsForPod(pod)
securityContextProvider := securitycontext.NewSimpleSecurityContextProvider()
securityContextProvider.ModifyContainerConfig(pod, container, dockerOpts.Config)
securityContextProvider.ModifyHostConfig(pod, container, dockerOpts.HostConfig)
securityContextProvider.ModifyHostConfig(pod, container, dockerOpts.HostConfig, supplementalGids)
createResp, err := dm.client.CreateContainer(dockerOpts)
if err != nil {
dm.recorder.Eventf(ref, api.EventTypeWarning, events.FailedToCreateContainer, "Failed to create docker container with error: %v", err)

View File

@ -102,6 +102,10 @@ func (f *fakeRuntimeHelper) GetPodDir(kubetypes.UID) string {
return ""
}
func (f *fakeRuntimeHelper) GetExtraSupplementalGroupsForPod(pod *api.Pod) []int64 {
return nil
}
func createTestDockerManager(fakeHTTPClient *fakeHTTP, fakeDocker *FakeDockerClient) (*DockerManager, *FakeDockerClient) {
if fakeHTTPClient == nil {
fakeHTTPClient = &fakeHTTP{}
@ -111,7 +115,13 @@ func createTestDockerManager(fakeHTTPClient *fakeHTTP, fakeDocker *FakeDockerCli
}
fakeRecorder := &record.FakeRecorder{}
containerRefManager := kubecontainer.NewRefManager()
networkPlugin, _ := network.InitNetworkPlugin([]network.NetworkPlugin{}, "", nettest.NewFakeHost(nil), componentconfig.HairpinNone, "10.0.0.0/8")
networkPlugin, _ := network.InitNetworkPlugin(
[]network.NetworkPlugin{},
"",
nettest.NewFakeHost(nil),
componentconfig.HairpinNone,
"10.0.0.0/8")
dockerManager := NewFakeDockerManager(
fakeDocker,
fakeRecorder,

View File

@ -227,3 +227,10 @@ func (kl *Kubelet) getHostIPAnyWay() (net.IP, error) {
}
return nodeutil.GetNodeHostIP(node)
}
// GetExtraSupplementalGroupsForPod returns a list of the extra
// supplemental groups for the Pod. These extra supplemental groups come
// from annotations on persistent volumes that the pod depends on.
func (kl *Kubelet) GetExtraSupplementalGroupsForPod(pod *api.Pod) []int64 {
return kl.volumeManager.GetExtraSupplementalGroupsForPod(pod)
}

View File

@ -175,6 +175,10 @@ func (f *fakeRuntimeHelper) GetPodDir(podUID types.UID) string {
return "/poddir/" + string(podUID)
}
func (f *fakeRuntimeHelper) GetExtraSupplementalGroupsForPod(pod *api.Pod) []int64 {
return nil
}
type fakeRktCli struct {
sync.Mutex
cmds []string

View File

@ -507,9 +507,11 @@ func verifyNonRoot(app *appctypes.App, ctx *api.SecurityContext) error {
return nil
}
func setSupplementaryGIDs(app *appctypes.App, podCtx *api.PodSecurityContext) {
if podCtx != nil {
func setSupplementalGIDs(app *appctypes.App, podCtx *api.PodSecurityContext, supplementalGids []int64) {
if podCtx != nil || len(supplementalGids) != 0 {
app.SupplementaryGIDs = app.SupplementaryGIDs[:0]
}
if podCtx != nil {
for _, v := range podCtx.SupplementalGroups {
app.SupplementaryGIDs = append(app.SupplementaryGIDs, int(v))
}
@ -517,10 +519,13 @@ func setSupplementaryGIDs(app *appctypes.App, podCtx *api.PodSecurityContext) {
app.SupplementaryGIDs = append(app.SupplementaryGIDs, int(*podCtx.FSGroup))
}
}
for _, v := range supplementalGids {
app.SupplementaryGIDs = append(app.SupplementaryGIDs, int(v))
}
}
// setApp merges the container spec with the image's manifest.
func setApp(imgManifest *appcschema.ImageManifest, c *api.Container, opts *kubecontainer.RunContainerOptions, ctx *api.SecurityContext, podCtx *api.PodSecurityContext) error {
func setApp(imgManifest *appcschema.ImageManifest, c *api.Container, opts *kubecontainer.RunContainerOptions, ctx *api.SecurityContext, podCtx *api.PodSecurityContext, supplementalGids []int64) error {
app := imgManifest.App
// Set up Exec.
@ -561,7 +566,7 @@ func setApp(imgManifest *appcschema.ImageManifest, c *api.Container, opts *kubec
if ctx != nil && ctx.RunAsUser != nil {
app.User = strconv.Itoa(int(*ctx.RunAsUser))
}
setSupplementaryGIDs(app, podCtx)
setSupplementalGIDs(app, podCtx, supplementalGids)
// If 'User' or 'Group' are still empty at this point,
// then apply the root UID and GID.
@ -803,8 +808,9 @@ func (r *Runtime) newAppcRuntimeApp(pod *api.Pod, podIP string, c api.Container,
})
}
supplementalGids := r.runtimeHelper.GetExtraSupplementalGroupsForPod(pod)
ctx := securitycontext.DetermineEffectiveSecurityContext(pod, &c)
if err := setApp(imgManifest, &c, opts, ctx, pod.Spec.SecurityContext); err != nil {
if err := setApp(imgManifest, &c, opts, ctx, pod.Spec.SecurityContext, supplementalGids); err != nil {
return err
}

View File

@ -940,21 +940,23 @@ func TestSetApp(t *testing.T) {
fsgid := int64(3)
tests := []struct {
container *api.Container
opts *kubecontainer.RunContainerOptions
ctx *api.SecurityContext
podCtx *api.PodSecurityContext
expect *appctypes.App
err error
container *api.Container
opts *kubecontainer.RunContainerOptions
ctx *api.SecurityContext
podCtx *api.PodSecurityContext
supplementalGids []int64
expect *appctypes.App
err error
}{
// Nothing should change, but the "User" and "Group" should be filled.
{
container: &api.Container{},
opts: &kubecontainer.RunContainerOptions{},
ctx: nil,
podCtx: nil,
expect: baseAppWithRootUserGroup(t),
err: nil,
container: &api.Container{},
opts: &kubecontainer.RunContainerOptions{},
ctx: nil,
podCtx: nil,
supplementalGids: nil,
expect: baseAppWithRootUserGroup(t),
err: nil,
},
// error verifying non-root.
@ -965,9 +967,10 @@ func TestSetApp(t *testing.T) {
RunAsNonRoot: &runAsNonRootTrue,
RunAsUser: &rootUser,
},
podCtx: nil,
expect: nil,
err: fmt.Errorf("container has no runAsUser and image will run as root"),
podCtx: nil,
supplementalGids: nil,
expect: nil,
err: fmt.Errorf("container has no runAsUser and image will run as root"),
},
// app's args should be changed.
@ -975,9 +978,10 @@ func TestSetApp(t *testing.T) {
container: &api.Container{
Args: []string{"foo"},
},
opts: &kubecontainer.RunContainerOptions{},
ctx: nil,
podCtx: nil,
opts: &kubecontainer.RunContainerOptions{},
ctx: nil,
podCtx: nil,
supplementalGids: nil,
expect: &appctypes.App{
Exec: appctypes.Exec{"/bin/foo", "foo"},
User: "0",
@ -1036,11 +1040,12 @@ func TestSetApp(t *testing.T) {
SupplementalGroups: []int64{1, 2},
FSGroup: &fsgid,
},
supplementalGids: []int64{4},
expect: &appctypes.App{
Exec: appctypes.Exec{"/bin/bar", "foo"},
User: "42",
Group: "0",
SupplementaryGIDs: []int{1, 2, 3},
SupplementaryGIDs: []int{1, 2, 3, 4},
WorkingDirectory: tmpDir,
Environment: []appctypes.EnvironmentVariable{
{"env-foo", "bar"},
@ -1099,11 +1104,12 @@ func TestSetApp(t *testing.T) {
SupplementalGroups: []int64{1, 2},
FSGroup: &fsgid,
},
supplementalGids: []int64{4},
expect: &appctypes.App{
Exec: appctypes.Exec{"/bin/hello", "foo", "hello", "world", "bar"},
User: "42",
Group: "0",
SupplementaryGIDs: []int{1, 2, 3},
SupplementaryGIDs: []int{1, 2, 3, 4},
WorkingDirectory: tmpDir,
Environment: []appctypes.EnvironmentVariable{
{"env-foo", "foo"},
@ -1128,7 +1134,7 @@ func TestSetApp(t *testing.T) {
for i, tt := range tests {
testCaseHint := fmt.Sprintf("test case #%d", i)
img := baseImageManifest(t)
err := setApp(img, tt.container, tt.opts, tt.ctx, tt.podCtx)
err := setApp(img, tt.container, tt.opts, tt.ctx, tt.podCtx, tt.supplementalGids)
if err == nil && tt.err != nil || err != nil && tt.err == nil {
t.Errorf("%s: expect %v, saw %v", testCaseHint, tt.err, err)
}

View File

@ -99,16 +99,10 @@ type VolumeManager interface {
// volumes.
GetMountedVolumesForPod(podName types.UniquePodName) container.VolumeMap
// GetVolumesForPodAndApplySupplementalGroups, like GetVolumesForPod returns
// a VolumeMap containing the volumes referenced by the specified pod that
// are successfully attached and mounted. The key in the map is the
// OuterVolumeSpecName (i.e. pod.Spec.Volumes[x].Name).
// It returns an empty VolumeMap if pod has no volumes.
// In addition for every volume that specifies a VolumeGidValue, it appends
// the SecurityContext.SupplementalGroups for the specified pod.
// XXX: https://github.com/kubernetes/kubernetes/issues/27197 mutating the
// pod object is bad, and should be avoided.
GetVolumesForPodAndAppendSupplementalGroups(pod *api.Pod) container.VolumeMap
// GetExtraSupplementalGroupsForPod returns a list of the extra
// supplemental groups for the Pod. These extra supplemental groups come
// from annotations on persistent volumes that the pod depends on.
GetExtraSupplementalGroupsForPod(pod *api.Pod) []int64
// Returns a list of all volumes that implement the volume.Attacher
// interface and are currently in use according to the actual and desired
@ -227,12 +221,34 @@ func (vm *volumeManager) Run(stopCh <-chan struct{}) {
func (vm *volumeManager) GetMountedVolumesForPod(
podName types.UniquePodName) container.VolumeMap {
return vm.getVolumesForPodHelper(podName, nil /* pod */)
podVolumes := make(container.VolumeMap)
for _, mountedVolume := range vm.actualStateOfWorld.GetMountedVolumesForPod(podName) {
podVolumes[mountedVolume.OuterVolumeSpecName] = container.VolumeInfo{Mounter: mountedVolume.Mounter}
}
return podVolumes
}
func (vm *volumeManager) GetVolumesForPodAndAppendSupplementalGroups(
pod *api.Pod) container.VolumeMap {
return vm.getVolumesForPodHelper("" /* podName */, pod)
func (vm *volumeManager) GetExtraSupplementalGroupsForPod(pod *api.Pod) []int64 {
podName := volumehelper.GetUniquePodName(pod)
supplementalGroups := sets.NewString()
for _, mountedVolume := range vm.actualStateOfWorld.GetMountedVolumesForPod(podName) {
if mountedVolume.VolumeGidValue != "" {
supplementalGroups.Insert(mountedVolume.VolumeGidValue)
}
}
result := make([]int64, 0, supplementalGroups.Len())
for _, group := range supplementalGroups.List() {
iGroup, extra := getExtraSupplementalGid(group, pod)
if !extra {
continue
}
result = append(result, int64(iGroup))
}
return result
}
func (vm *volumeManager) GetVolumesInUse() []api.UniqueVolumeName {
@ -279,33 +295,6 @@ func (vm *volumeManager) MarkVolumesAsReportedInUse(
vm.desiredStateOfWorld.MarkVolumesReportedInUse(volumesReportedAsInUse)
}
// getVolumesForPodHelper is a helper method implements the common logic for
// the GetVolumesForPod methods.
// XXX: https://github.com/kubernetes/kubernetes/issues/27197 mutating the pod
// object is bad, and should be avoided.
func (vm *volumeManager) getVolumesForPodHelper(
podName types.UniquePodName, pod *api.Pod) container.VolumeMap {
if pod != nil {
podName = volumehelper.GetUniquePodName(pod)
}
podVolumes := make(container.VolumeMap)
for _, mountedVolume := range vm.actualStateOfWorld.GetMountedVolumesForPod(podName) {
podVolumes[mountedVolume.OuterVolumeSpecName] =
container.VolumeInfo{Mounter: mountedVolume.Mounter}
if pod != nil {
err := applyPersistentVolumeAnnotations(
mountedVolume.VolumeGidValue, pod)
if err != nil {
glog.Errorf("applyPersistentVolumeAnnotations failed for pod %q volume %q with: %v",
podName,
mountedVolume.VolumeName,
err)
}
}
}
return podVolumes
}
func (vm *volumeManager) WaitForAttachAndMount(pod *api.Pod) error {
expectedVolumes := getExpectedVolumes(pod)
if len(expectedVolumes) == 0 {
@ -395,32 +384,26 @@ func getExpectedVolumes(pod *api.Pod) []string {
return expectedVolumes
}
// applyPersistentVolumeAnnotations appends a pod
// SecurityContext.SupplementalGroups if a GID annotation is provided.
// XXX: https://github.com/kubernetes/kubernetes/issues/27197 mutating the pod
// object is bad, and should be avoided.
func applyPersistentVolumeAnnotations(
volumeGidValue string, pod *api.Pod) error {
if volumeGidValue != "" {
gid, err := strconv.ParseInt(volumeGidValue, 10, 64)
if err != nil {
return fmt.Errorf(
"Invalid value for %s %v",
volumehelper.VolumeGidAnnotationKey,
err)
}
if pod.Spec.SecurityContext == nil {
pod.Spec.SecurityContext = &api.PodSecurityContext{}
}
for _, existingGid := range pod.Spec.SecurityContext.SupplementalGroups {
if gid == existingGid {
return nil
}
}
pod.Spec.SecurityContext.SupplementalGroups =
append(pod.Spec.SecurityContext.SupplementalGroups, gid)
// getExtraSupplementalGid returns the value of an extra supplemental GID as
// defined by an annotation on a volume and a boolean indicating whether the
// volume defined a GID that the pod doesn't already request.
func getExtraSupplementalGid(volumeGidValue string, pod *api.Pod) (int64, bool) {
if volumeGidValue == "" {
return 0, false
}
return nil
gid, err := strconv.ParseInt(volumeGidValue, 10, 64)
if err != nil {
return 0, false
}
if pod.Spec.SecurityContext != nil {
for _, existingGid := range pod.Spec.SecurityContext.SupplementalGroups {
if gid == existingGid {
return 0, false
}
}
}
return gid, true
}

View File

@ -0,0 +1,278 @@
/*
Copyright 2016 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 volumemanager
import (
"os"
"reflect"
"strconv"
"testing"
"time"
"k8s.io/kubernetes/pkg/api"
"k8s.io/kubernetes/pkg/client/clientset_generated/internalclientset"
"k8s.io/kubernetes/pkg/client/clientset_generated/internalclientset/fake"
containertest "k8s.io/kubernetes/pkg/kubelet/container/testing"
"k8s.io/kubernetes/pkg/kubelet/pod"
kubepod "k8s.io/kubernetes/pkg/kubelet/pod"
podtest "k8s.io/kubernetes/pkg/kubelet/pod/testing"
"k8s.io/kubernetes/pkg/util/mount"
utiltesting "k8s.io/kubernetes/pkg/util/testing"
"k8s.io/kubernetes/pkg/volume"
volumetest "k8s.io/kubernetes/pkg/volume/testing"
"k8s.io/kubernetes/pkg/volume/util/types"
"k8s.io/kubernetes/pkg/volume/util/volumehelper"
)
const (
testHostname = "test-hostname"
)
func TestGetMountedVolumesForPodAndGetVolumesInUse(t *testing.T) {
tmpDir, err := utiltesting.MkTmpdir("volumeManagerTest")
if err != nil {
t.Fatalf("can't make a temp dir: %v", err)
}
defer os.RemoveAll(tmpDir)
podManager := kubepod.NewBasicPodManager(podtest.NewFakeMirrorClient())
node, pod, pv, claim := createObjects()
kubeClient := fake.NewSimpleClientset(node, pod, pv, claim)
manager, err := newTestVolumeManager(tmpDir, podManager, kubeClient)
if err != nil {
t.Fatalf("Failed to initialize volume manager: %v", err)
}
stopCh := make(chan struct{})
go manager.Run(stopCh)
defer close(stopCh)
podManager.SetPods([]*api.Pod{pod})
// Fake node status update
go simulateVolumeInUseUpdate(
api.UniqueVolumeName(node.Status.VolumesAttached[0].Name),
stopCh,
manager)
err = manager.WaitForAttachAndMount(pod)
if err != nil {
t.Errorf("Expected success: %v", err)
}
expectedMounted := pod.Spec.Volumes[0].Name
actualMounted := manager.GetMountedVolumesForPod(types.UniquePodName(pod.ObjectMeta.UID))
if _, ok := actualMounted[expectedMounted]; !ok || (len(actualMounted) != 1) {
t.Errorf("Expected %v to be mounted to pod but got %v", expectedMounted, actualMounted)
}
expectedInUse := []api.UniqueVolumeName{api.UniqueVolumeName(node.Status.VolumesAttached[0].Name)}
actualInUse := manager.GetVolumesInUse()
if !reflect.DeepEqual(expectedInUse, actualInUse) {
t.Errorf("Expected %v to be in use but got %v", expectedInUse, actualInUse)
}
}
func TestGetExtraSupplementalGroupsForPod(t *testing.T) {
tmpDir, err := utiltesting.MkTmpdir("volumeManagerTest")
if err != nil {
t.Fatalf("can't make a temp dir: %v", err)
}
defer os.RemoveAll(tmpDir)
podManager := kubepod.NewBasicPodManager(podtest.NewFakeMirrorClient())
node, pod, _, claim := createObjects()
existingGid := pod.Spec.SecurityContext.SupplementalGroups[0]
cases := []struct {
gidAnnotation string
expected []int64
}{
{
gidAnnotation: "777",
expected: []int64{777},
},
{
gidAnnotation: strconv.FormatInt(existingGid, 10),
expected: []int64{},
},
{
gidAnnotation: "a",
expected: []int64{},
},
{
gidAnnotation: "",
expected: []int64{},
},
}
for _, tc := range cases {
pv := &api.PersistentVolume{
ObjectMeta: api.ObjectMeta{
Name: "pvA",
Annotations: map[string]string{
volumehelper.VolumeGidAnnotationKey: tc.gidAnnotation,
},
},
Spec: api.PersistentVolumeSpec{
PersistentVolumeSource: api.PersistentVolumeSource{
GCEPersistentDisk: &api.GCEPersistentDiskVolumeSource{
PDName: "fake-device",
},
},
ClaimRef: &api.ObjectReference{
Name: claim.ObjectMeta.Name,
},
},
}
kubeClient := fake.NewSimpleClientset(node, pod, pv, claim)
manager, err := newTestVolumeManager(tmpDir, podManager, kubeClient)
if err != nil {
t.Errorf("Failed to initialize volume manager: %v", err)
continue
}
stopCh := make(chan struct{})
go manager.Run(stopCh)
podManager.SetPods([]*api.Pod{pod})
// Fake node status update
go simulateVolumeInUseUpdate(
api.UniqueVolumeName(node.Status.VolumesAttached[0].Name),
stopCh,
manager)
err = manager.WaitForAttachAndMount(pod)
if err != nil {
t.Errorf("Expected success: %v", err)
continue
}
actual := manager.GetExtraSupplementalGroupsForPod(pod)
if !reflect.DeepEqual(tc.expected, actual) {
t.Errorf("Expected supplemental groups %v, got %v", tc.expected, actual)
}
close(stopCh)
}
}
func newTestVolumeManager(
tmpDir string,
podManager pod.Manager,
kubeClient internalclientset.Interface) (VolumeManager, error) {
plug := &volumetest.FakeVolumePlugin{PluginName: "fake", Host: nil}
plugMgr := &volume.VolumePluginMgr{}
plugMgr.InitPlugins([]volume.VolumePlugin{plug}, volumetest.NewFakeVolumeHost(tmpDir, kubeClient, nil, "" /* rootContext */))
vm, err := NewVolumeManager(
true,
testHostname,
podManager,
kubeClient,
plugMgr,
&containertest.FakeRuntime{},
&mount.FakeMounter{})
return vm, err
}
// createObjects returns objects for making a fake clientset. The pv is
// already attached to the node and bound to the claim used by the pod.
func createObjects() (*api.Node, *api.Pod, *api.PersistentVolume, *api.PersistentVolumeClaim) {
node := &api.Node{
ObjectMeta: api.ObjectMeta{Name: testHostname},
Status: api.NodeStatus{
VolumesAttached: []api.AttachedVolume{
{
Name: "fake/pvA",
DevicePath: "fake/path",
},
}},
Spec: api.NodeSpec{ExternalID: testHostname},
}
pod := &api.Pod{
ObjectMeta: api.ObjectMeta{
Name: "abc",
Namespace: "nsA",
UID: "1234",
},
Spec: api.PodSpec{
Volumes: []api.Volume{
{
Name: "vol1",
VolumeSource: api.VolumeSource{
PersistentVolumeClaim: &api.PersistentVolumeClaimVolumeSource{
ClaimName: "claimA",
},
},
},
},
SecurityContext: &api.PodSecurityContext{
SupplementalGroups: []int64{555},
},
},
}
pv := &api.PersistentVolume{
ObjectMeta: api.ObjectMeta{
Name: "pvA",
},
Spec: api.PersistentVolumeSpec{
PersistentVolumeSource: api.PersistentVolumeSource{
GCEPersistentDisk: &api.GCEPersistentDiskVolumeSource{
PDName: "fake-device",
},
},
ClaimRef: &api.ObjectReference{
Name: "claimA",
},
},
}
claim := &api.PersistentVolumeClaim{
ObjectMeta: api.ObjectMeta{
Name: "claimA",
Namespace: "nsA",
},
Spec: api.PersistentVolumeClaimSpec{
VolumeName: "pvA",
},
Status: api.PersistentVolumeClaimStatus{
Phase: api.ClaimBound,
},
}
return node, pod, pv, claim
}
func simulateVolumeInUseUpdate(
volumeName api.UniqueVolumeName,
stopCh <-chan struct{},
volumeManager VolumeManager) {
ticker := time.NewTicker(100 * time.Millisecond)
defer ticker.Stop()
for {
select {
case <-ticker.C:
volumeManager.MarkVolumesAsReportedInUse(
[]api.UniqueVolumeName{volumeName})
case <-stopCh:
return
}
}
}

View File

@ -41,5 +41,5 @@ type FakeSecurityContextProvider struct{}
func (p FakeSecurityContextProvider) ModifyContainerConfig(pod *api.Pod, container *api.Container, config *dockercontainer.Config) {
}
func (p FakeSecurityContextProvider) ModifyHostConfig(pod *api.Pod, container *api.Container, hostConfig *dockercontainer.HostConfig) {
func (p FakeSecurityContextProvider) ModifyHostConfig(pod *api.Pod, container *api.Container, hostConfig *dockercontainer.HostConfig, supplementalGids []int64) {
}

View File

@ -47,12 +47,12 @@ func (p SimpleSecurityContextProvider) ModifyContainerConfig(pod *api.Pod, conta
}
}
// ModifyHostConfig is called before the Docker runContainer call.
// The security context provider can make changes to the HostConfig, affecting
// ModifyHostConfig is called before the Docker runContainer call. The
// security context provider can make changes to the HostConfig, affecting
// security options, whether the container is privileged, volume binds, etc.
func (p SimpleSecurityContextProvider) ModifyHostConfig(pod *api.Pod, container *api.Container, hostConfig *dockercontainer.HostConfig) {
// Apply pod security context
if container.Name != leaky.PodInfraContainerName && pod.Spec.SecurityContext != nil {
func (p SimpleSecurityContextProvider) ModifyHostConfig(pod *api.Pod, container *api.Container, hostConfig *dockercontainer.HostConfig, supplementalGids []int64) {
// Apply supplemental groups
if container.Name != leaky.PodInfraContainerName {
// TODO: We skip application of supplemental groups to the
// infra container to work around a runc issue which
// requires containers to have the '/etc/group'. For
@ -60,15 +60,17 @@ func (p SimpleSecurityContextProvider) ModifyHostConfig(pod *api.Pod, container
// https://github.com/opencontainers/runc/pull/313
// This can be removed once the fix makes it into the
// required version of docker.
if pod.Spec.SecurityContext.SupplementalGroups != nil {
hostConfig.GroupAdd = make([]string, len(pod.Spec.SecurityContext.SupplementalGroups))
for i, group := range pod.Spec.SecurityContext.SupplementalGroups {
hostConfig.GroupAdd[i] = strconv.Itoa(int(group))
if pod.Spec.SecurityContext != nil {
for _, group := range pod.Spec.SecurityContext.SupplementalGroups {
hostConfig.GroupAdd = append(hostConfig.GroupAdd, strconv.Itoa(int(group)))
}
if pod.Spec.SecurityContext.FSGroup != nil {
hostConfig.GroupAdd = append(hostConfig.GroupAdd, strconv.Itoa(int(*pod.Spec.SecurityContext.FSGroup)))
}
}
if pod.Spec.SecurityContext.FSGroup != nil {
hostConfig.GroupAdd = append(hostConfig.GroupAdd, strconv.Itoa(int(*pod.Spec.SecurityContext.FSGroup)))
for _, group := range supplementalGids {
hostConfig.GroupAdd = append(hostConfig.GroupAdd, strconv.Itoa(int(group)))
}
}

View File

@ -166,7 +166,7 @@ func TestModifyHostConfig(t *testing.T) {
dummyContainer.SecurityContext = tc.sc
dockerCfg := &dockercontainer.HostConfig{}
provider.ModifyHostConfig(pod, dummyContainer, dockerCfg)
provider.ModifyHostConfig(pod, dummyContainer, dockerCfg, nil)
if e, a := tc.expected, dockerCfg; !reflect.DeepEqual(e, a) {
t.Errorf("%v: unexpected modification of host config\nExpected:\n\n%#v\n\nGot:\n\n%#v", tc.name, e, a)
@ -181,32 +181,50 @@ func TestModifyHostConfigPodSecurityContext(t *testing.T) {
supplementalGroupHC.GroupAdd = []string{"2222"}
fsGroupHC := fullValidHostConfig()
fsGroupHC.GroupAdd = []string{"1234"}
extraSupplementalGroupHC := fullValidHostConfig()
extraSupplementalGroupHC.GroupAdd = []string{"1234"}
bothHC := fullValidHostConfig()
bothHC.GroupAdd = []string{"2222", "1234"}
fsGroup := int64(1234)
extraSupplementalGroup := []int64{1234}
testCases := map[string]struct {
securityContext *api.PodSecurityContext
expected *dockercontainer.HostConfig
securityContext *api.PodSecurityContext
expected *dockercontainer.HostConfig
extraSupplementalGroups []int64
}{
"nil": {
securityContext: nil,
expected: fullValidHostConfig(),
securityContext: nil,
expected: fullValidHostConfig(),
extraSupplementalGroups: nil,
},
"SupplementalGroup": {
securityContext: supplementalGroupsSC,
expected: supplementalGroupHC,
securityContext: supplementalGroupsSC,
expected: supplementalGroupHC,
extraSupplementalGroups: nil,
},
"FSGroup": {
securityContext: &api.PodSecurityContext{FSGroup: &fsGroup},
expected: fsGroupHC,
securityContext: &api.PodSecurityContext{FSGroup: &fsGroup},
expected: fsGroupHC,
extraSupplementalGroups: nil,
},
"FSGroup + SupplementalGroups": {
securityContext: &api.PodSecurityContext{
SupplementalGroups: []int64{2222},
FSGroup: &fsGroup,
},
expected: bothHC,
expected: bothHC,
extraSupplementalGroups: nil,
},
"ExtraSupplementalGroup": {
securityContext: nil,
expected: extraSupplementalGroupHC,
extraSupplementalGroups: extraSupplementalGroup,
},
"ExtraSupplementalGroup + SupplementalGroups": {
securityContext: supplementalGroupsSC,
expected: bothHC,
extraSupplementalGroups: extraSupplementalGroup,
},
}
@ -220,7 +238,7 @@ func TestModifyHostConfigPodSecurityContext(t *testing.T) {
for k, v := range testCases {
dummyPod.Spec.SecurityContext = v.securityContext
dockerCfg := &dockercontainer.HostConfig{}
provider.ModifyHostConfig(dummyPod, dummyContainer, dockerCfg)
provider.ModifyHostConfig(dummyPod, dummyContainer, dockerCfg, v.extraSupplementalGroups)
if !reflect.DeepEqual(v.expected, dockerCfg) {
t.Errorf("unexpected modification of host config for %s. Expected: %#v Got: %#v", k, v.expected, dockerCfg)
}

View File

@ -33,7 +33,11 @@ type SecurityContextProvider interface {
// security options, whether the container is privileged, volume binds, etc.
// An error is returned if it's not possible to secure the container as requested
// with a security context.
ModifyHostConfig(pod *api.Pod, container *api.Container, hostConfig *dockercontainer.HostConfig)
//
// - pod: the pod to modify the docker hostconfig for
// - container: the container to modify the hostconfig for
// - supplementalGids: additional supplemental GIDs associated with the pod's volumes
ModifyHostConfig(pod *api.Pod, container *api.Container, hostConfig *dockercontainer.HostConfig, supplementalGids []int64)
}
const (