Handle out of disk situation on kubelets.

Kubelet will stop accepting new pods if it detects low disk space on root fs or fs holding docker images.
Running pods are not affected. low-diskspace-threshold-mb is used to configure the low diskspace threshold.
This commit is contained in:
Rohit Jnagal 2015-05-12 08:24:08 +00:00
parent 0d16f43475
commit 2cf0dfb79d
10 changed files with 440 additions and 4 deletions

View File

@ -92,6 +92,7 @@ type KubeletServer struct {
StreamingConnectionIdleTimeout time.Duration
ImageGCHighThresholdPercent int
ImageGCLowThresholdPercent int
LowDiskSpaceThresholdMB int
NetworkPluginName string
CloudProvider string
CloudConfigFile string
@ -151,6 +152,7 @@ func NewKubeletServer() *KubeletServer {
MasterServiceNamespace: api.NamespaceDefault,
ImageGCHighThresholdPercent: 90,
ImageGCLowThresholdPercent: 80,
LowDiskSpaceThresholdMB: 256,
NetworkPluginName: "",
HostNetworkSources: kubelet.FileSource,
CertDirectory: "/var/run/kubernetes",
@ -205,6 +207,7 @@ func (s *KubeletServer) AddFlags(fs *pflag.FlagSet) {
fs.DurationVar(&s.NodeStatusUpdateFrequency, "node-status-update-frequency", s.NodeStatusUpdateFrequency, "Specifies how often kubelet posts node status to master. Note: be cautious when changing the constant, it must work with nodeMonitorGracePeriod in nodecontroller. Default: 10s")
fs.IntVar(&s.ImageGCHighThresholdPercent, "image-gc-high-threshold", s.ImageGCHighThresholdPercent, "The percent of disk usage after which image garbage collection is always run. Default: 90%%")
fs.IntVar(&s.ImageGCLowThresholdPercent, "image-gc-low-threshold", s.ImageGCLowThresholdPercent, "The percent of disk usage before which image garbage collection is never run. Lowest disk usage to garbage collect to. Default: 80%%")
fs.IntVar(&s.LowDiskSpaceThresholdMB, "low-diskspace-threshold-mb", s.LowDiskSpaceThresholdMB, "The absolute free disk space, in MB, to maintain. When disk space falls below this threshold, new pods would be rejected. Default: 256")
fs.StringVar(&s.NetworkPluginName, "network-plugin", s.NetworkPluginName, "<Warning: Alpha feature> The name of the network plugin to be invoked for various events in kubelet/pod lifecycle")
fs.StringVar(&s.CloudProvider, "cloud-provider", s.CloudProvider, "The provider for cloud services. Empty string for no provider.")
fs.StringVar(&s.CloudConfigFile, "cloud-config", s.CloudConfigFile, "The path to the cloud provider configuration file. Empty string for no configuration file.")
@ -247,6 +250,10 @@ func (s *KubeletServer) Run(_ []string) error {
LowThresholdPercent: s.ImageGCLowThresholdPercent,
}
diskSpacePolicy := kubelet.DiskSpacePolicy{
DockerFreeDiskMB: s.LowDiskSpaceThresholdMB,
RootFreeDiskMB: s.LowDiskSpaceThresholdMB,
}
cloud := cloudprovider.InitCloudProvider(s.CloudProvider, s.CloudConfigFile)
glog.Infof("Successfully initialized cloud provider: %q from the config file: %q\n", s.CloudProvider, s.CloudConfigFile)
@ -314,6 +321,7 @@ func (s *KubeletServer) Run(_ []string) error {
StreamingConnectionIdleTimeout: s.StreamingConnectionIdleTimeout,
TLSOptions: tlsOptions,
ImageGCPolicy: imageGCPolicy,
DiskSpacePolicy: diskSpacePolicy,
Cloud: cloud,
NodeStatusUpdateFrequency: s.NodeStatusUpdateFrequency,
ResourceContainer: s.ResourceContainer,
@ -401,6 +409,10 @@ func SimpleKubelet(client *client.Client,
HighThresholdPercent: 90,
LowThresholdPercent: 80,
}
diskSpacePolicy := kubelet.DiskSpacePolicy{
DockerFreeDiskMB: 256,
RootFreeDiskMB: 256,
}
kcfg := KubeletConfig{
KubeClient: client,
DockerClient: dockerClient,
@ -424,6 +436,7 @@ func SimpleKubelet(client *client.Client,
CadvisorInterface: cadvisorInterface,
ConfigFile: configFilePath,
ImageGCPolicy: imageGCPolicy,
DiskSpacePolicy: diskSpacePolicy,
Cloud: cloud,
NodeStatusUpdateFrequency: 10 * time.Second,
ResourceContainer: "/kubelet",
@ -554,6 +567,7 @@ type KubeletConfig struct {
Recorder record.EventRecorder
TLSOptions *kubelet.TLSOptions
ImageGCPolicy kubelet.ImageGCPolicy
DiskSpacePolicy kubelet.DiskSpacePolicy
Cloud cloudprovider.Interface
NodeStatusUpdateFrequency time.Duration
ResourceContainer string
@ -602,6 +616,7 @@ func createAndInitKubelet(kc *KubeletConfig) (k KubeletBootstrap, pc *config.Pod
kc.Recorder,
kc.CadvisorInterface,
kc.ImageGCPolicy,
kc.DiskSpacePolicy,
kc.Cloud,
kc.NodeStatusUpdateFrequency,
kc.ResourceContainer,

View File

@ -52,6 +52,10 @@ func (c *Fake) DockerImagesFsInfo() (cadvisorApiV2.FsInfo, error) {
return cadvisorApiV2.FsInfo{}, nil
}
func (c *Fake) RootFsInfo() (cadvisorApiV2.FsInfo, error) {
return cadvisorApiV2.FsInfo{}, nil
}
func (c *Fake) WatchEvents(request *events.Request) (*events.EventChannel, error) {
return new(events.EventChannel), nil
}

View File

@ -132,16 +132,24 @@ func (cc *cadvisorClient) MachineInfo() (*cadvisorApi.MachineInfo, error) {
}
func (cc *cadvisorClient) DockerImagesFsInfo() (cadvisorApiV2.FsInfo, error) {
res, err := cc.GetFsInfo(cadvisorFs.LabelDockerImages)
return cc.getFsInfo(cadvisorFs.LabelDockerImages)
}
func (cc *cadvisorClient) RootFsInfo() (cadvisorApiV2.FsInfo, error) {
return cc.getFsInfo(cadvisorFs.LabelSystemRoot)
}
func (cc *cadvisorClient) getFsInfo(label string) (cadvisorApiV2.FsInfo, error) {
res, err := cc.GetFsInfo(label)
if err != nil {
return cadvisorApiV2.FsInfo{}, err
}
if len(res) == 0 {
return cadvisorApiV2.FsInfo{}, fmt.Errorf("failed to find information for the filesystem containing Docker images")
return cadvisorApiV2.FsInfo{}, fmt.Errorf("failed to find information for the filesystem labeled %q", label)
}
// TODO(vmarmol): Handle this better when Docker has more than one image filesystem.
// TODO(vmarmol): Handle this better when a label has more than one image filesystem.
if len(res) > 1 {
glog.Warningf("More than one Docker images filesystem: %#v. Only using the first one", res)
glog.Warningf("More than one filesystem labeled %q: %#v. Only using the first one", label, res)
}
return res[0], nil

View File

@ -62,6 +62,11 @@ func (c *Mock) DockerImagesFsInfo() (cadvisorApiV2.FsInfo, error) {
return args.Get(0).(cadvisorApiV2.FsInfo), args.Error(1)
}
func (c *Mock) RootFsInfo() (cadvisorApiV2.FsInfo, error) {
args := c.Called()
return args.Get(0).(cadvisorApiV2.FsInfo), args.Error(1)
}
func (c *Mock) WatchEvents(request *events.Request) (*events.EventChannel, error) {
args := c.Called()
return args.Get(0).(*events.EventChannel), args.Error(1)

View File

@ -61,6 +61,10 @@ func (cu *cadvisorUnsupported) DockerImagesFsInfo() (cadvisorApiV2.FsInfo, error
return cadvisorApiV2.FsInfo{}, unsupportedErr
}
func (cu *cadvisorUnsupported) RootFsInfo() (cadvisorApiV2.FsInfo, error) {
return cadvisorApiV2.FsInfo{}, unsupportedErr
}
func (cu *cadvisorUnsupported) WatchEvents(request *events.Request) (*events.EventChannel, error) {
return nil, unsupportedErr
}

View File

@ -34,6 +34,9 @@ type Interface interface {
// Returns usage information about the filesystem holding Docker images.
DockerImagesFsInfo() (cadvisorApiV2.FsInfo, error)
// Returns usage information about the root filesystem.
RootFsInfo() (cadvisorApiV2.FsInfo, error)
// Get events streamed through passedChannel that fit the request.
WatchEvents(request *events.Request) (*events.EventChannel, error)
}

149
pkg/kubelet/disk_manager.go Normal file
View File

@ -0,0 +1,149 @@
/*
Copyright 2015 The Kubernetes Authors All rights reserved.
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 kubelet
import (
"fmt"
"sync"
"time"
"github.com/GoogleCloudPlatform/kubernetes/pkg/kubelet/cadvisor"
"github.com/golang/glog"
cadvisorApi "github.com/google/cadvisor/info/v2"
)
// Manages policy for diskspace management for disks holding docker images and root fs.
// Implementation is thread-safe.
type diskSpaceManager interface {
// Checks the available disk space
IsRootDiskSpaceAvailable() (bool, error)
IsDockerDiskSpaceAvailable() (bool, error)
// Always returns sufficient space till Unfreeze() is called.
// This is a signal from caller that its internal initialization is done.
Unfreeze()
}
type DiskSpacePolicy struct {
// free disk space threshold for filesystem holding docker images.
DockerFreeDiskMB int
// free disk space threshold for root filesystem. Host volumes are created on root fs.
RootFreeDiskMB int
}
type fsInfo struct {
Usage int64
Capacity int64
Timestamp time.Time
}
type realDiskSpaceManager struct {
cadvisor cadvisor.Interface
cachedInfo map[string]fsInfo // cache of filesystem info.
lock sync.Mutex // protecting cachedInfo and frozen.
policy DiskSpacePolicy // thresholds. Set at creation time.
frozen bool // space checks always return ok when frozen is set. True on creation.
}
func (dm *realDiskSpaceManager) getFsInfo(fsType string, f func() (cadvisorApi.FsInfo, error)) (fsInfo, error) {
dm.lock.Lock()
defer dm.lock.Unlock()
fsi := fsInfo{}
if info, ok := dm.cachedInfo[fsType]; ok {
timeLimit := time.Now().Add(-2 * time.Second)
if info.Timestamp.After(timeLimit) {
fsi = info
}
}
if fsi.Timestamp.IsZero() {
fs, err := f()
if err != nil {
return fsInfo{}, err
}
fsi.Timestamp = time.Now()
fsi.Usage = int64(fs.Usage)
fsi.Capacity = int64(fs.Capacity)
dm.cachedInfo[fsType] = fsi
}
return fsi, nil
}
func (dm *realDiskSpaceManager) IsDockerDiskSpaceAvailable() (bool, error) {
return dm.isSpaceAvailable("docker", dm.policy.DockerFreeDiskMB, dm.cadvisor.DockerImagesFsInfo)
}
func (dm *realDiskSpaceManager) IsRootDiskSpaceAvailable() (bool, error) {
return dm.isSpaceAvailable("root", dm.policy.RootFreeDiskMB, dm.cadvisor.RootFsInfo)
}
func (dm *realDiskSpaceManager) isSpaceAvailable(fsType string, threshold int, f func() (cadvisorApi.FsInfo, error)) (bool, error) {
if dm.frozen {
return true, nil
}
fsInfo, err := dm.getFsInfo(fsType, f)
if err != nil {
return true, fmt.Errorf("failed to get fs info for %q: %v", fsType, err)
}
if fsInfo.Capacity == 0 {
return true, fmt.Errorf("could not determine capacity for %q fs.", fsType)
}
free := fsInfo.Capacity - fsInfo.Usage
if free < 0 {
return true, fmt.Errorf("wrong fs usage for %q: capacity %d, usage %d", fsType, fsInfo.Capacity, fsInfo.Usage)
}
const mb = int64(1024 * 1024)
if free < int64(threshold)*mb {
glog.Infof("Running out of space on disk for %q: free %d MB, threshold %d MB", fsType, free/mb, threshold)
return false, nil
}
return true, nil
}
func (dm *realDiskSpaceManager) Unfreeze() {
dm.lock.Lock()
defer dm.lock.Unlock()
dm.frozen = false
}
func validatePolicy(policy DiskSpacePolicy) error {
if policy.DockerFreeDiskMB < 0 {
return fmt.Errorf("free disk space should be non-negative. Invalid value %d for docker disk space threshold.", policy.DockerFreeDiskMB)
}
if policy.RootFreeDiskMB < 0 {
return fmt.Errorf("free disk space should be non-negative. Invalid value %d for root disk space threshold.", policy.RootFreeDiskMB)
}
return nil
}
func newDiskSpaceManager(cadvisorInterface cadvisor.Interface, policy DiskSpacePolicy) (diskSpaceManager, error) {
// validate policy
err := validatePolicy(policy)
if err != nil {
return nil, err
}
dm := &realDiskSpaceManager{
cadvisor: cadvisorInterface,
policy: policy,
cachedInfo: map[string]fsInfo{},
frozen: true,
}
return dm, nil
}

View File

@ -0,0 +1,144 @@
/*
Copyright 2015 The Kubernetes Authors All rights reserved.
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 kubelet
import (
"fmt"
"testing"
"github.com/GoogleCloudPlatform/kubernetes/pkg/kubelet/cadvisor"
cadvisorApi "github.com/google/cadvisor/info/v2"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func testPolicy() DiskSpacePolicy {
return DiskSpacePolicy{
DockerFreeDiskMB: 250,
RootFreeDiskMB: 250,
}
}
func testValidPolicy(t *testing.T) {
assert := assert.New(t)
policy := testPolicy()
c := new(cadvisor.Mock)
_, err := newDiskSpaceManager(c, policy)
require.NoError(t, err)
policy = testPolicy()
policy.DockerFreeDiskMB = -1
_, err = newDiskSpaceManager(c, policy)
assert.Error(err)
policy = testPolicy()
policy.RootFreeDiskMB = -1
_, err = newDiskSpaceManager(c, policy)
assert.Error(err)
}
func testSpaceAvailable(t *testing.T) {
policy := testPolicy()
mockCadvisor := new(cadvisor.Mock)
dm, err := newDiskSpaceManager(mockCadvisor, policy)
require.NoError(t, err)
const mb = 1024 * 1024
mockCadvisor.On("DockerImagesFsInfo").Return(cadvisorApi.FsInfo{
Usage: 400 * mb,
Capacity: 1000 * mb,
}, nil)
mockCadvisor.On("RootFsInfo").Return(cadvisorApi.FsInfo{
Usage: 9 * mb,
Capacity: 10 * mb,
}, nil)
ok, err := dm.IsDockerDiskSpaceAvailable()
require.NoError(t, err)
require.True(t, ok)
ok, err = dm.IsRootDiskSpaceAvailable()
require.NoError(t, err)
require.True(t, ok)
}
func testRootFsAvailable(t *testing.T) {
policy := testPolicy()
policy.RootFreeDiskMB = 10
mockCadvisor := new(cadvisor.Mock)
dm, err := newDiskSpaceManager(mockCadvisor, policy)
require.NoError(t, err)
const mb = 1024 * 1024
// 500MB free
mockCadvisor.On("DockerImagesFsInfo").Return(cadvisorApi.FsInfo{
Usage: 9500 * mb,
Capacity: 10000 * mb,
}, nil)
// 10MB free
mockCadvisor.On("RootFsInfo").Return(cadvisorApi.FsInfo{
Usage: 990 * mb,
Capacity: 1000 * mb,
}, nil)
ok, err := dm.IsDockerDiskSpaceAvailable()
require.NoError(t, err)
require.True(t, ok)
ok, err = dm.IsRootDiskSpaceAvailable()
require.NoError(t, err)
require.False(t, ok)
}
func testFsInfoError(t *testing.T) {
assert := assert.New(t)
policy := testPolicy()
policy.RootFreeDiskMB = 10
mockCadvisor := new(cadvisor.Mock)
dm, err := newDiskSpaceManager(mockCadvisor, policy)
require.NoError(t, err)
mockCadvisor.On("DockerImagesFsInfo").Return(cadvisorApi.FsInfo{}, fmt.Errorf("can't find fs"))
mockCadvisor.On("RootFsInfo").Return(cadvisorApi.FsInfo{}, fmt.Errorf("EBUSY"))
ok, err := dm.IsDockerDiskSpaceAvailable()
assert.Error(err)
require.True(t, ok)
ok, err = dm.IsRootDiskSpaceAvailable()
assert.Error(err)
require.True(t, ok)
}
func testCache(t *testing.T) {
policy := testPolicy()
mockCadvisor := new(cadvisor.Mock)
dm, err := newDiskSpaceManager(mockCadvisor, policy)
require.NoError(t, err)
const mb = 1024 * 1024
mockCadvisor.On("DockerImagesFsInfo").Return(cadvisorApi.FsInfo{
Usage: 400 * mb,
Capacity: 1000 * mb,
}, nil).Once()
mockCadvisor.On("RootFsInfo").Return(cadvisorApi.FsInfo{
Usage: 9 * mb,
Capacity: 10 * mb,
}, nil).Once()
ok, err := dm.IsDockerDiskSpaceAvailable()
ok, err = dm.IsRootDiskSpaceAvailable()
// Checking for space again shouldn't need to mock as cache would serve it.
ok, err = dm.IsDockerDiskSpaceAvailable()
require.NoError(t, err)
require.True(t, ok)
ok, err = dm.IsRootDiskSpaceAvailable()
require.NoError(t, err)
require.True(t, ok)
}

View File

@ -131,6 +131,7 @@ func NewMainKubelet(
recorder record.EventRecorder,
cadvisorInterface cadvisor.Interface,
imageGCPolicy ImageGCPolicy,
diskSpacePolicy DiskSpacePolicy,
cloud cloudprovider.Interface,
nodeStatusUpdateFrequency time.Duration,
resourceContainer string,
@ -197,6 +198,10 @@ func NewMainKubelet(
if err != nil {
return nil, fmt.Errorf("failed to initialize image manager: %v", err)
}
diskSpaceManager, err := newDiskSpaceManager(cadvisorInterface, diskSpacePolicy)
if err != nil {
return nil, fmt.Errorf("failed to initialize disk manager: %v", err)
}
statusManager := newStatusManager(kubeClient)
readinessManager := kubecontainer.NewReadinessManager()
containerRefManager := kubecontainer.NewRefManager()
@ -228,6 +233,7 @@ func NewMainKubelet(
cadvisor: cadvisorInterface,
containerGC: containerGC,
imageManager: imageManager,
diskSpaceManager: diskSpaceManager,
statusManager: statusManager,
volumeManager: volumeManager,
cloud: cloud,
@ -390,6 +396,9 @@ type Kubelet struct {
// Manager for images.
imageManager imageManager
// Diskspace manager.
diskSpaceManager diskSpaceManager
// Cached MachineInfo returned by cadvisor.
machineInfo *cadvisorApi.MachineInfo
@ -1118,6 +1127,9 @@ func (kl *Kubelet) SyncPods(allPods []*api.Pod, podSyncTypes map[types.UID]metri
// Reject pods that we cannot run.
kl.handleNotFittingPods(allPods)
// Reject new creation requests if diskspace is running low.
kl.handleOutOfDisk(allPods, podSyncTypes)
// Pod phase progresses monotonically. Once a pod has reached a final state,
// it should never leave irregardless of the restart policy. The statuses
// of such pods should not be changed, and there is no need to sync them.
@ -1288,6 +1300,44 @@ func (kl *Kubelet) checkCapacityExceeded(pods []*api.Pod) (fitting []*api.Pod, n
return scheduler.CheckPodsExceedingCapacity(pods, capacity)
}
// handleOutOfDisk detects if pods can't fit due to lack of disk space.
func (kl *Kubelet) handleOutOfDisk(pods []*api.Pod, podSyncTypes map[types.UID]metrics.SyncPodType) {
if len(podSyncTypes) == 0 {
// regular sync. no new pods
return
}
outOfDockerDisk := false
outOfRootDisk := false
// Check disk space once globally and reject or accept all new pods.
withinBounds, err := kl.diskSpaceManager.IsDockerDiskSpaceAvailable()
// Assume enough space in case of errors.
if err == nil && !withinBounds {
outOfDockerDisk = true
}
withinBounds, err = kl.diskSpaceManager.IsRootDiskSpaceAvailable()
// Assume enough space in case of errors.
if err == nil && !withinBounds {
outOfRootDisk = true
}
// Kubelet would indicate all pods as newly created on the first run after restart.
// We ignore the first disk check to ensure that running pods are not killed.
// Disk manager will only declare out of disk problems if unfreeze has been called.
kl.diskSpaceManager.Unfreeze()
if outOfDockerDisk || outOfRootDisk {
for _, pod := range pods {
// Only reject pods that didn't start yet.
if podSyncTypes[pod.UID] == metrics.SyncPodCreate {
kl.recorder.Eventf(pod, "OutOfDisk", "Cannot start the pod due to lack of disk space.")
kl.statusManager.SetPodStatus(pod, api.PodStatus{
Phase: api.PodFailed,
Message: "Pod cannot be started due to lack of disk space."})
continue
}
}
}
}
// checkNodeSelectorMatching detects pods that do not match node's labels.
func (kl *Kubelet) checkNodeSelectorMatching(pods []*api.Pod) (fitting []*api.Pod, notFitting []*api.Pod) {
node, err := kl.GetNode()

View File

@ -52,6 +52,7 @@ import (
_ "github.com/GoogleCloudPlatform/kubernetes/pkg/volume/host_path"
"github.com/fsouza/go-dockerclient"
cadvisorApi "github.com/google/cadvisor/info/v1"
cadvisorApiv2 "github.com/google/cadvisor/info/v2"
)
func init() {
@ -108,6 +109,11 @@ func newTestKubelet(t *testing.T) *TestKubelet {
kubelet.podManager = podManager
kubelet.containerRefManager = kubecontainer.NewRefManager()
runtimeHooks := newKubeletRuntimeHooks(kubelet.recorder)
diskSpaceManager, err := newDiskSpaceManager(mockCadvisor, DiskSpacePolicy{})
if err != nil {
t.Fatalf("can't initialize disk space manager: %v", err)
}
kubelet.diskSpaceManager = diskSpaceManager
kubelet.containerRuntime = dockertools.NewFakeDockerManager(fakeDocker, fakeRecorder, kubelet.readinessManager, kubelet.containerRefManager, dockertools.PodInfraContainerImage, 0, 0, "", kubelet.os, kubelet.networkPlugin, kubelet, &fakeHTTP{}, runtimeHooks)
kubelet.runtimeCache = kubecontainer.NewFakeRuntimeCache(kubelet.containerRuntime)
@ -371,6 +377,8 @@ func generatePodInfraContainerHash(pod *api.Pod) uint64 {
func TestSyncPodsDoesNothing(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -436,6 +444,8 @@ func TestSyncPodsDoesNothing(t *testing.T) {
func TestSyncPodsWithTerminationLog(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -498,6 +508,8 @@ func matchString(t *testing.T, pattern, str string) bool {
func TestSyncPodsCreatesNetAndContainer(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -562,6 +574,8 @@ func TestSyncPodsCreatesNetAndContainer(t *testing.T) {
func TestSyncPodsCreatesNetAndContainerPullsImage(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -622,6 +636,8 @@ func TestSyncPodsCreatesNetAndContainerPullsImage(t *testing.T) {
func TestSyncPodsWithPodInfraCreatesContainer(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -693,6 +709,8 @@ func (f *fakeHTTP) Get(url string) (*http.Response, error) {
func TestSyncPodsWithPodInfraCreatesContainerCallsHandler(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -775,6 +793,8 @@ func TestSyncPodsWithPodInfraCreatesContainerCallsHandler(t *testing.T) {
func TestSyncPodsDeletesWithNoPodInfraContainer(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -889,6 +909,8 @@ func TestSyncPodsDeletesWhenSourcesAreReady(t *testing.T) {
ready := false
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
kubelet.sourcesReady = func() bool { return ready }
@ -934,6 +956,8 @@ func TestSyncPodsDeletesWhenSourcesAreReady(t *testing.T) {
func TestSyncPodsDeletes(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
fakeDocker.ContainerList = []docker.APIContainers{
@ -975,6 +999,8 @@ func TestSyncPodsDeletes(t *testing.T) {
func TestSyncPodsDeletesDuplicate(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -1056,6 +1082,8 @@ func TestSyncPodsDeletesDuplicate(t *testing.T) {
func TestSyncPodsBadHash(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -1127,6 +1155,8 @@ func TestSyncPodsBadHash(t *testing.T) {
func TestSyncPodsUnhealthy(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -1644,6 +1674,8 @@ func TestRunInContainer(t *testing.T) {
func TestSyncPodEventHandlerFails(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -1731,6 +1763,8 @@ func TestSyncPodEventHandlerFails(t *testing.T) {
func TestSyncPodsWithPullPolicy(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -2925,6 +2959,8 @@ func TestHandlePortConflicts(t *testing.T) {
testKubelet := newTestKubelet(t)
kl := testKubelet.kubelet
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
spec := api.PodSpec{Containers: []api.Container{{Ports: []api.ContainerPort{{HostPort: 80}}}}}
pods := []*api.Pod{
@ -2980,6 +3016,8 @@ func TestHandleNodeSelector(t *testing.T) {
{ObjectMeta: api.ObjectMeta{Name: "testnode", Labels: map[string]string{"key": "B"}}},
}}
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
pods := []*api.Pod{
{
ObjectMeta: api.ObjectMeta{
@ -3027,6 +3065,8 @@ func TestHandleMemExceeded(t *testing.T) {
testKubelet := newTestKubelet(t)
kl := testKubelet.kubelet
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{MemoryCapacity: 100}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
spec := api.PodSpec{Containers: []api.Container{{Resources: api.ResourceRequirements{
Limits: api.ResourceList{
@ -3082,6 +3122,8 @@ func TestHandleMemExceeded(t *testing.T) {
func TestPurgingObsoleteStatusMapEntries(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kl := testKubelet.kubelet
pods := []*api.Pod{
@ -3490,6 +3532,8 @@ func TestCreateMirrorPod(t *testing.T) {
func TestDeleteOutdatedMirrorPod(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kl := testKubelet.kubelet
manager := testKubelet.fakeMirrorClient
pod := &api.Pod{
@ -3541,6 +3585,8 @@ func TestDeleteOutdatedMirrorPod(t *testing.T) {
func TestDeleteOrphanedMirrorPods(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kl := testKubelet.kubelet
manager := testKubelet.fakeMirrorClient
orphanPods := []*api.Pod{
@ -3661,6 +3707,8 @@ func TestGetContainerInfoForMirrorPods(t *testing.T) {
func TestDoNotCacheStatusForStaticPods(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
waitGroup := testKubelet.waitGroup
@ -3757,6 +3805,8 @@ func TestHostNetworkDisallowed(t *testing.T) {
func TestSyncPodsWithRestartPolicy(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -3910,6 +3960,8 @@ func TestSyncPodsWithRestartPolicy(t *testing.T) {
func TestGetPodStatusWithLastTermination(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker
waitGroup := testKubelet.waitGroup
@ -4105,6 +4157,8 @@ func TestGetPodCreationFailureReason(t *testing.T) {
func TestGetRestartCount(t *testing.T) {
testKubelet := newTestKubelet(t)
testKubelet.fakeCadvisor.On("MachineInfo").Return(&cadvisorApi.MachineInfo{}, nil)
testKubelet.fakeCadvisor.On("DockerImagesFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
testKubelet.fakeCadvisor.On("RootFsInfo").Return(cadvisorApiv2.FsInfo{}, nil)
kubelet := testKubelet.kubelet
fakeDocker := testKubelet.fakeDocker