mirror of
https://github.com/k3s-io/kubernetes.git
synced 2025-07-29 14:37:00 +00:00
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:
parent
0d16f43475
commit
2cf0dfb79d
@ -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,
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
149
pkg/kubelet/disk_manager.go
Normal 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
|
||||
}
|
144
pkg/kubelet/disk_manager_test.go
Normal file
144
pkg/kubelet/disk_manager_test.go
Normal 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)
|
||||
}
|
@ -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()
|
||||
|
@ -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
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user