Keep pod worker running until pod is truly complete
A number of race conditions exist when pods are terminated early in their lifecycle because components in the kubelet need to know "no running containers" or "containers can't be started from now on" but were relying on outdated state. Only the pod worker knows whether containers are being started for a given pod, which is required to know when a pod is "terminated" (no running containers, none coming). Move that responsibility and podKiller function into the pod workers, and have everything that was killing the pod go into the UpdatePod loop. Split syncPod into three phases - setup, terminate containers, and cleanup pod - and have transitions between those methods be visible to other components. After this change, to kill a pod you tell the pod worker to UpdatePod({UpdateType: SyncPodKill, Pod: pod}). Several places in the kubelet were incorrect about whether they were handling terminating (should stop running, might have containers) or terminated (no running containers) pods. The pod worker exposes methods that allow other loops to know when to set up or tear down resources based on the state of the pod - these methods remove the possibility of race conditions by ensuring a single component is responsible for knowing each pod's allowed state and other components simply delegate to checking whether they are in the window by UID. Removing containers now no longer blocks final pod deletion in the API server and are handled as background cleanup. Node shutdown no longer marks pods as failed as they can be restarted in the next step. See https://docs.google.com/document/d/1Pic5TPntdJnYfIpBeZndDelM-AbS4FN9H2GTLFhoJ04/edit# for details
This commit is contained in:
parent
01819dd322
commit
3eadd1a9ea
@ -92,7 +92,8 @@ type Runtime interface {
|
||||
// file). In this case, garbage collector should refrain itself from aggressive
|
||||
// behavior such as removing all containers of unrecognized pods (yet).
|
||||
// If evictNonDeletedPods is set to true, containers and sandboxes belonging to pods
|
||||
// that are terminated, but not deleted will be evicted. Otherwise, only deleted pods will be GC'd.
|
||||
// that are terminated, but not deleted will be evicted. Otherwise, only deleted pods
|
||||
// will be GC'd.
|
||||
// TODO: Revisit this method and make it cleaner.
|
||||
GarbageCollect(gcPolicy GCPolicy, allSourcesReady bool, evictNonDeletedPods bool) error
|
||||
// SyncPod syncs the running pod into the desired pod.
|
||||
|
@ -561,15 +561,15 @@ func (m *managerImpl) evictPod(pod *v1.Pod, gracePeriodOverride int64, evictMsg
|
||||
klog.ErrorS(nil, "Eviction manager: cannot evict a critical pod", "pod", klog.KObj(pod))
|
||||
return false
|
||||
}
|
||||
status := v1.PodStatus{
|
||||
Phase: v1.PodFailed,
|
||||
Message: evictMsg,
|
||||
Reason: Reason,
|
||||
}
|
||||
// record that we are evicting the pod
|
||||
m.recorder.AnnotatedEventf(pod, annotations, v1.EventTypeWarning, Reason, evictMsg)
|
||||
// this is a blocking call and should only return when the pod and its containers are killed.
|
||||
err := m.killPodFunc(pod, status, &gracePeriodOverride)
|
||||
klog.V(3).InfoS("Evicting pod", "pod", klog.KObj(pod), "podUID", pod.UID, "message", evictMsg)
|
||||
err := m.killPodFunc(pod, true, &gracePeriodOverride, func(status *v1.PodStatus) {
|
||||
status.Phase = v1.PodFailed
|
||||
status.Reason = Reason
|
||||
status.Message = evictMsg
|
||||
})
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Eviction manager: pod failed to evict", "pod", klog.KObj(pod))
|
||||
} else {
|
||||
|
@ -21,7 +21,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"k8s.io/api/core/v1"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/resource"
|
||||
"k8s.io/apimachinery/pkg/types"
|
||||
"k8s.io/apimachinery/pkg/util/clock"
|
||||
@ -46,14 +46,16 @@ const (
|
||||
// mockPodKiller is used to testing which pod is killed
|
||||
type mockPodKiller struct {
|
||||
pod *v1.Pod
|
||||
status v1.PodStatus
|
||||
evict bool
|
||||
statusFn func(*v1.PodStatus)
|
||||
gracePeriodOverride *int64
|
||||
}
|
||||
|
||||
// killPodNow records the pod that was killed
|
||||
func (m *mockPodKiller) killPodNow(pod *v1.Pod, status v1.PodStatus, gracePeriodOverride *int64) error {
|
||||
func (m *mockPodKiller) killPodNow(pod *v1.Pod, evict bool, gracePeriodOverride *int64, statusFn func(*v1.PodStatus)) error {
|
||||
m.pod = pod
|
||||
m.status = status
|
||||
m.statusFn = statusFn
|
||||
m.evict = evict
|
||||
m.gracePeriodOverride = gracePeriodOverride
|
||||
return nil
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ package eviction
|
||||
import (
|
||||
"time"
|
||||
|
||||
"k8s.io/api/core/v1"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/resource"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
statsapi "k8s.io/kubelet/pkg/apis/stats/v1alpha1"
|
||||
@ -92,7 +92,7 @@ type ContainerGC interface {
|
||||
// pod - the pod to kill
|
||||
// status - the desired status to associate with the pod (i.e. why its killed)
|
||||
// gracePeriodOverride - the grace period override to use instead of what is on the pod spec
|
||||
type KillPodFunc func(pod *v1.Pod, status v1.PodStatus, gracePeriodOverride *int64) error
|
||||
type KillPodFunc func(pod *v1.Pod, isEvicted bool, gracePeriodOverride *int64, fn func(*v1.PodStatus)) error
|
||||
|
||||
// MirrorPodFunc returns the mirror pod for the given static pod and
|
||||
// whether it was known to the pod manager.
|
||||
|
@ -17,6 +17,7 @@ limitations under the License.
|
||||
package kubelet
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"fmt"
|
||||
"math"
|
||||
@ -136,6 +137,11 @@ const (
|
||||
// Period for performing global cleanup tasks.
|
||||
housekeepingPeriod = time.Second * 2
|
||||
|
||||
// Duration at which housekeeping failed to satisfy the invariant that
|
||||
// housekeeping should be fast to avoid blocking pod config (while
|
||||
// housekeeping is running no new pods are started or deleted).
|
||||
housekeepingWarningDuration = time.Second * 15
|
||||
|
||||
// Period for performing eviction monitoring.
|
||||
// ensure this is kept in sync with internal cadvisor housekeeping.
|
||||
evictionMonitoringPeriod = time.Second * 10
|
||||
@ -626,6 +632,20 @@ func NewMainKubelet(kubeCfg *kubeletconfiginternal.KubeletConfiguration,
|
||||
klet.containerLogManager = logs.NewStubContainerLogManager()
|
||||
}
|
||||
|
||||
klet.reasonCache = NewReasonCache()
|
||||
klet.workQueue = queue.NewBasicWorkQueue(klet.clock)
|
||||
klet.podWorkers = newPodWorkers(
|
||||
klet.syncPod,
|
||||
klet.syncTerminatingPod,
|
||||
klet.syncTerminatedPod,
|
||||
|
||||
kubeDeps.Recorder,
|
||||
klet.workQueue,
|
||||
klet.resyncInterval,
|
||||
backOffPeriod,
|
||||
klet.podCache,
|
||||
)
|
||||
|
||||
runtime, err := kuberuntime.NewKubeGenericRuntimeManager(
|
||||
kubecontainer.FilterEventRecorder(kubeDeps.Recorder),
|
||||
klet.livenessManager,
|
||||
@ -633,7 +653,7 @@ func NewMainKubelet(kubeCfg *kubeletconfiginternal.KubeletConfiguration,
|
||||
klet.startupManager,
|
||||
seccompProfileRoot,
|
||||
machineInfo,
|
||||
klet,
|
||||
klet.podWorkers,
|
||||
kubeDeps.OSInterface,
|
||||
klet,
|
||||
httpClient,
|
||||
@ -764,7 +784,7 @@ func NewMainKubelet(kubeCfg *kubeletconfiginternal.KubeletConfiguration,
|
||||
kubeCfg.EnableControllerAttachDetach,
|
||||
nodeName,
|
||||
klet.podManager,
|
||||
klet.statusManager,
|
||||
klet.podWorkers,
|
||||
klet.kubeClient,
|
||||
klet.volumePluginMgr,
|
||||
klet.containerRuntime,
|
||||
@ -776,12 +796,7 @@ func NewMainKubelet(kubeCfg *kubeletconfiginternal.KubeletConfiguration,
|
||||
keepTerminatedPodVolumes,
|
||||
volumepathhandler.NewBlockVolumePathHandler())
|
||||
|
||||
klet.reasonCache = NewReasonCache()
|
||||
klet.workQueue = queue.NewBasicWorkQueue(klet.clock)
|
||||
klet.podWorkers = newPodWorkers(klet.syncPod, kubeDeps.Recorder, klet.workQueue, klet.resyncInterval, backOffPeriod, klet.podCache)
|
||||
|
||||
klet.backOff = flowcontrol.NewBackOff(backOffPeriod, MaxContainerBackOff)
|
||||
klet.podKiller = NewPodKiller(klet)
|
||||
|
||||
// setup eviction manager
|
||||
evictionManager, evictionAdmitHandler := eviction.NewManager(klet.resourceAnalyzer, evictionConfig, killPodNow(klet.podWorkers, kubeDeps.Recorder), klet.podManager.GetMirrorPodByPod, klet.imageManager, klet.containerGC, kubeDeps.Recorder, nodeRef, klet.clock)
|
||||
@ -1087,9 +1102,6 @@ type Kubelet struct {
|
||||
// Container restart Backoff
|
||||
backOff *flowcontrol.Backoff
|
||||
|
||||
// Pod killer handles pods to be killed
|
||||
podKiller PodKiller
|
||||
|
||||
// Information about the ports which are opened by daemons on Node running this Kubelet server.
|
||||
daemonEndpoints *v1.NodeDaemonEndpoints
|
||||
|
||||
@ -1452,10 +1464,6 @@ func (kl *Kubelet) Run(updates <-chan kubetypes.PodUpdate) {
|
||||
kl.initNetworkUtil()
|
||||
}
|
||||
|
||||
// Start a goroutine responsible for killing pods (that are not properly
|
||||
// handled by pod workers).
|
||||
go wait.Until(kl.podKiller.PerformPodKillingWork, 1*time.Second, wait.NeverStop)
|
||||
|
||||
// Start component sync loops.
|
||||
kl.statusManager.Start()
|
||||
|
||||
@ -1469,7 +1477,12 @@ func (kl *Kubelet) Run(updates <-chan kubetypes.PodUpdate) {
|
||||
kl.syncLoop(updates, kl)
|
||||
}
|
||||
|
||||
// syncPod is the transaction script for the sync of a single pod.
|
||||
// syncPod is the transaction script for the sync of a single pod (setting up)
|
||||
// a pod. The reverse (teardown) is handled in syncTerminatingPod and
|
||||
// syncTerminatedPod. If syncPod exits without error, then the pod runtime
|
||||
// state is in sync with the desired configuration state (pod is running).
|
||||
// If syncPod exits with a transient error, the next invocation of syncPod
|
||||
// is expected to make progress towards reaching the runtime state.
|
||||
//
|
||||
// Arguments:
|
||||
//
|
||||
@ -1481,7 +1494,7 @@ func (kl *Kubelet) Run(updates <-chan kubetypes.PodUpdate) {
|
||||
// * If the pod is being seen as running for the first time, record pod
|
||||
// start latency
|
||||
// * Update the status of the pod in the status manager
|
||||
// * Kill the pod if it should not be running
|
||||
// * Kill the pod if it should not be running due to soft admission
|
||||
// * Create a mirror pod if the pod is a static pod, and does not
|
||||
// already have a mirror pod
|
||||
// * Create the data directories for the pod if they do not exist
|
||||
@ -1496,39 +1509,9 @@ func (kl *Kubelet) Run(updates <-chan kubetypes.PodUpdate) {
|
||||
// This operation writes all events that are dispatched in order to provide
|
||||
// the most accurate information possible about an error situation to aid debugging.
|
||||
// Callers should not throw an event if this operation returns an error.
|
||||
func (kl *Kubelet) syncPod(o syncPodOptions) error {
|
||||
// pull out the required options
|
||||
pod := o.pod
|
||||
mirrorPod := o.mirrorPod
|
||||
podStatus := o.podStatus
|
||||
updateType := o.updateType
|
||||
|
||||
// if we want to kill a pod, do it now!
|
||||
if updateType == kubetypes.SyncPodKill {
|
||||
killPodOptions := o.killPodOptions
|
||||
if killPodOptions == nil || killPodOptions.PodStatusFunc == nil {
|
||||
return fmt.Errorf("kill pod options are required if update type is kill")
|
||||
}
|
||||
apiPodStatus := killPodOptions.PodStatusFunc(pod, podStatus)
|
||||
kl.statusManager.SetPodStatus(pod, apiPodStatus)
|
||||
// we kill the pod with the specified grace period since this is a termination
|
||||
if err := kl.killPod(pod, nil, podStatus, killPodOptions.PodTerminationGracePeriodSecondsOverride); err != nil {
|
||||
kl.recorder.Eventf(pod, v1.EventTypeWarning, events.FailedToKillPod, "error killing pod: %v", err)
|
||||
// there was an error killing the pod, so we return that error directly
|
||||
utilruntime.HandleError(err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// If a pod is still gracefully terminating, then we do not want to
|
||||
// take further action. This mitigates static pods and deleted pods
|
||||
// from getting rerun prematurely or their cgroups being deleted before
|
||||
// the runtime cleans up.
|
||||
podFullName := kubecontainer.GetPodFullName(pod)
|
||||
if kl.podKiller.IsPodPendingTerminationByPodName(podFullName) {
|
||||
return fmt.Errorf("pod %q is pending termination", podFullName)
|
||||
}
|
||||
func (kl *Kubelet) syncPod(ctx context.Context, updateType kubetypes.SyncPodType, pod, mirrorPod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
klog.V(4).InfoS("syncPod enter", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
defer klog.V(4).InfoS("syncPod exit", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
|
||||
// Latency measurements for the main workflow are relative to the
|
||||
// first time the pod was seen by the API server.
|
||||
@ -1565,16 +1548,15 @@ func (kl *Kubelet) syncPod(o syncPodOptions) error {
|
||||
podStatus.IPs = []string{apiPodStatus.PodIP}
|
||||
}
|
||||
|
||||
// Record the time it takes for the pod to become running.
|
||||
existingStatus, ok := kl.statusManager.GetPodStatus(pod.UID)
|
||||
if !ok || existingStatus.Phase == v1.PodPending && apiPodStatus.Phase == v1.PodRunning &&
|
||||
!firstSeenTime.IsZero() {
|
||||
metrics.PodStartDuration.Observe(metrics.SinceInSeconds(firstSeenTime))
|
||||
}
|
||||
|
||||
// If the pod should not be running, we request the pod's containers be stopped. This is not the same
|
||||
// as termination (we want to stop the pod, but potentially restart it later if soft admission allows
|
||||
// it later). Set the status and phase appropriately
|
||||
runnable := kl.canRunPod(pod)
|
||||
if !runnable.Admit {
|
||||
// Pod is not runnable; update the Pod and Container statuses to why.
|
||||
// Pod is not runnable; and update the Pod and Container statuses to why.
|
||||
if apiPodStatus.Phase != v1.PodFailed && apiPodStatus.Phase != v1.PodSucceeded {
|
||||
apiPodStatus.Phase = v1.PodPending
|
||||
}
|
||||
apiPodStatus.Reason = runnable.Reason
|
||||
apiPodStatus.Message = runnable.Message
|
||||
// Waiting containers are not creating.
|
||||
@ -1591,22 +1573,28 @@ func (kl *Kubelet) syncPod(o syncPodOptions) error {
|
||||
}
|
||||
}
|
||||
|
||||
// Update status in the status manager
|
||||
// Record the time it takes for the pod to become running.
|
||||
existingStatus, ok := kl.statusManager.GetPodStatus(pod.UID)
|
||||
if !ok || existingStatus.Phase == v1.PodPending && apiPodStatus.Phase == v1.PodRunning &&
|
||||
!firstSeenTime.IsZero() {
|
||||
metrics.PodStartDuration.Observe(metrics.SinceInSeconds(firstSeenTime))
|
||||
}
|
||||
|
||||
kl.statusManager.SetPodStatus(pod, apiPodStatus)
|
||||
|
||||
// Kill pod if it should not be running
|
||||
if !runnable.Admit || pod.DeletionTimestamp != nil || apiPodStatus.Phase == v1.PodFailed {
|
||||
// Pods that are not runnable must be stopped - return a typed error to the pod worker
|
||||
if !runnable.Admit {
|
||||
klog.V(2).InfoS("Pod is not runnable and must have running containers stopped", "pod", klog.KObj(pod), "podUID", pod.UID, "message", runnable.Message)
|
||||
var syncErr error
|
||||
if err := kl.killPod(pod, nil, podStatus, nil); err != nil {
|
||||
p := kubecontainer.ConvertPodStatusToRunningPod(kl.getRuntime().Type(), podStatus)
|
||||
if err := kl.killPod(pod, p, nil); err != nil {
|
||||
kl.recorder.Eventf(pod, v1.EventTypeWarning, events.FailedToKillPod, "error killing pod: %v", err)
|
||||
syncErr = fmt.Errorf("error killing pod: %v", err)
|
||||
utilruntime.HandleError(syncErr)
|
||||
} else {
|
||||
if !runnable.Admit {
|
||||
// There was no error killing the pod, but the pod cannot be run.
|
||||
// Return an error to signal that the sync loop should back off.
|
||||
syncErr = fmt.Errorf("pod cannot be run: %s", runnable.Message)
|
||||
}
|
||||
// There was no error killing the pod, but the pod cannot be run.
|
||||
// Return an error to signal that the sync loop should back off.
|
||||
syncErr = fmt.Errorf("pod cannot be run: %s", runnable.Message)
|
||||
}
|
||||
return syncErr
|
||||
}
|
||||
@ -1622,7 +1610,8 @@ func (kl *Kubelet) syncPod(o syncPodOptions) error {
|
||||
pcm := kl.containerManager.NewPodContainerManager()
|
||||
// If pod has already been terminated then we need not create
|
||||
// or update the pod's cgroup
|
||||
if !kl.podIsTerminated(pod) {
|
||||
// TODO: once context cancellation is added this check can be removed
|
||||
if !kl.podWorkers.IsPodTerminationRequested(pod.UID) {
|
||||
// When the kubelet is restarted with the cgroups-per-qos
|
||||
// flag enabled, all the pod's running containers
|
||||
// should be killed intermittently and brought back up
|
||||
@ -1639,7 +1628,8 @@ func (kl *Kubelet) syncPod(o syncPodOptions) error {
|
||||
// exists or the pod is running for the first time
|
||||
podKilled := false
|
||||
if !pcm.Exists(pod) && !firstSync {
|
||||
if err := kl.killPod(pod, nil, podStatus, nil); err == nil {
|
||||
p := kubecontainer.ConvertPodStatusToRunningPod(kl.getRuntime().Type(), podStatus)
|
||||
if err := kl.killPod(pod, p, nil); err == nil {
|
||||
podKilled = true
|
||||
} else {
|
||||
klog.ErrorS(err, "KillPod failed", "pod", klog.KObj(pod), "podStatus", podStatus)
|
||||
@ -1673,6 +1663,7 @@ func (kl *Kubelet) syncPod(o syncPodOptions) error {
|
||||
// The mirror pod is semantically different from the static pod. Remove
|
||||
// it. The mirror pod will get recreated later.
|
||||
klog.InfoS("Trying to delete pod", "pod", klog.KObj(pod), "podUID", mirrorPod.ObjectMeta.UID)
|
||||
podFullName := kubecontainer.GetPodFullName(pod)
|
||||
var err error
|
||||
deleted, err = kl.podManager.DeleteMirrorPod(podFullName, &mirrorPod.ObjectMeta.UID)
|
||||
if deleted {
|
||||
@ -1702,8 +1693,9 @@ func (kl *Kubelet) syncPod(o syncPodOptions) error {
|
||||
return err
|
||||
}
|
||||
|
||||
// Volume manager will not mount volumes for terminated pods
|
||||
if !kl.podIsTerminated(pod) {
|
||||
// Volume manager will not mount volumes for terminating pods
|
||||
// TODO: once context cancellation is added this check can be removed
|
||||
if !kl.podWorkers.IsPodTerminationRequested(pod.UID) {
|
||||
// Wait for volumes to attach/mount
|
||||
if err := kl.volumeManager.WaitForAttachAndMount(pod); err != nil {
|
||||
kl.recorder.Eventf(pod, v1.EventTypeWarning, events.FailedMountVolume, "Unable to attach or mount volumes: %v", err)
|
||||
@ -1734,6 +1726,125 @@ func (kl *Kubelet) syncPod(o syncPodOptions) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// syncTerminatingPod is expected to terminate all running containers in a pod. Once this method
|
||||
// returns without error, the pod's local state can be safely cleaned up. If runningPod is passed,
|
||||
// we perform no status updates.
|
||||
func (kl *Kubelet) syncTerminatingPod(ctx context.Context, pod *v1.Pod, podStatus *kubecontainer.PodStatus, runningPod *kubecontainer.Pod, gracePeriod *int64, podStatusFn func(*v1.PodStatus)) error {
|
||||
klog.V(4).InfoS("syncTerminatingPod enter", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
defer klog.V(4).InfoS("syncTerminatingPod exit", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
|
||||
// when we receive a runtime only pod (runningPod != nil) we don't need to update the status
|
||||
// manager or refresh the status of the cache, because a successful killPod will ensure we do
|
||||
// not get invoked again
|
||||
if runningPod != nil {
|
||||
// we kill the pod with the specified grace period since this is a termination
|
||||
if gracePeriod != nil {
|
||||
klog.V(4).InfoS("Pod terminating with grace period", "pod", klog.KObj(pod), "podUID", pod.UID, "gracePeriod", *gracePeriod)
|
||||
} else {
|
||||
klog.V(4).InfoS("Pod terminating with grace period", "pod", klog.KObj(pod), "podUID", pod.UID, "gracePeriod", nil)
|
||||
}
|
||||
if err := kl.killPod(pod, *runningPod, gracePeriod); err != nil {
|
||||
kl.recorder.Eventf(pod, v1.EventTypeWarning, events.FailedToKillPod, "error killing pod: %v", err)
|
||||
// there was an error killing the pod, so we return that error directly
|
||||
utilruntime.HandleError(err)
|
||||
return err
|
||||
}
|
||||
klog.V(4).InfoS("Pod termination stopped all running orphan containers", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
return nil
|
||||
}
|
||||
|
||||
apiPodStatus := kl.generateAPIPodStatus(pod, podStatus)
|
||||
if podStatusFn != nil {
|
||||
podStatusFn(&apiPodStatus)
|
||||
}
|
||||
kl.statusManager.SetPodStatus(pod, apiPodStatus)
|
||||
|
||||
if gracePeriod != nil {
|
||||
klog.V(4).InfoS("Pod terminating with grace period", "pod", klog.KObj(pod), "podUID", pod.UID, "gracePeriod", *gracePeriod)
|
||||
} else {
|
||||
klog.V(4).InfoS("Pod terminating with grace period", "pod", klog.KObj(pod), "podUID", pod.UID, "gracePeriod", nil)
|
||||
}
|
||||
p := kubecontainer.ConvertPodStatusToRunningPod(kl.getRuntime().Type(), podStatus)
|
||||
if err := kl.killPod(pod, p, gracePeriod); err != nil {
|
||||
kl.recorder.Eventf(pod, v1.EventTypeWarning, events.FailedToKillPod, "error killing pod: %v", err)
|
||||
// there was an error killing the pod, so we return that error directly
|
||||
utilruntime.HandleError(err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Guard against consistency issues in KillPod implementations by checking that there are no
|
||||
// running containers. This method is invoked infrequently so this is effectively free and can
|
||||
// catch race conditions introduced by callers updating pod status out of order.
|
||||
// TODO: have KillPod return the terminal status of stopped containers and write that into the
|
||||
// cache immediately
|
||||
podStatus, err := kl.containerRuntime.GetPodStatus(pod.UID, pod.Name, pod.Namespace)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Unable to read pod status prior to final pod termination", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
return err
|
||||
}
|
||||
var runningContainers []string
|
||||
var containers []string
|
||||
for _, s := range podStatus.ContainerStatuses {
|
||||
if s.State == kubecontainer.ContainerStateRunning {
|
||||
runningContainers = append(runningContainers, s.ID.String())
|
||||
}
|
||||
containers = append(containers, fmt.Sprintf("(%s state=%s exitCode=%d finishedAt=%s)", s.Name, s.State, s.ExitCode, s.FinishedAt.UTC().Format(time.RFC3339Nano)))
|
||||
}
|
||||
if klog.V(4).Enabled() {
|
||||
sort.Strings(containers)
|
||||
klog.InfoS("Post-termination container state", "pod", klog.KObj(pod), "podUID", pod.UID, "containers", strings.Join(containers, " "))
|
||||
}
|
||||
if len(runningContainers) > 0 {
|
||||
return fmt.Errorf("detected running containers after a successful KillPod, CRI violation: %v", runningContainers)
|
||||
}
|
||||
|
||||
// we have successfully stopped all containers, the pod is terminating, our status is "done"
|
||||
klog.V(4).InfoS("Pod termination stopped all running containers", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// syncTerminatedPod cleans up a pod that has terminated (has no running containers).
|
||||
// The invocations in this call are expected to tear down what PodResourcesAreReclaimed checks (which
|
||||
// gates pod deletion). When this method exits the pod is expected to be ready for cleanup.
|
||||
// TODO: make this method take a context and exit early
|
||||
func (kl *Kubelet) syncTerminatedPod(ctx context.Context, pod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
klog.V(4).InfoS("syncTerminatedPod enter", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
defer klog.V(4).InfoS("syncTerminatedPod exit", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
|
||||
// generate the final status of the pod
|
||||
// TODO: should we simply fold this into TerminatePod? that would give a single pod update
|
||||
apiPodStatus := kl.generateAPIPodStatus(pod, podStatus)
|
||||
kl.statusManager.SetPodStatus(pod, apiPodStatus)
|
||||
|
||||
// volumes are unmounted after the pod worker reports ShouldPodRuntimeBeRemoved (which is satisfied
|
||||
// before syncTerminatedPod is invoked)
|
||||
if err := kl.volumeManager.WaitForUnmount(pod); err != nil {
|
||||
return err
|
||||
}
|
||||
klog.V(4).InfoS("Pod termination unmounted volumes", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
|
||||
// Note: we leave pod containers to be reclaimed in the background since dockershim requires the
|
||||
// container for retrieving logs and we want to make sure logs are available until the pod is
|
||||
// physically deleted.
|
||||
|
||||
// remove any cgroups in the hierarchy for pods that are no longer running.
|
||||
if kl.cgroupsPerQOS {
|
||||
pcm := kl.containerManager.NewPodContainerManager()
|
||||
name, _ := pcm.GetPodContainerName(pod)
|
||||
if err := pcm.Destroy(name); err != nil {
|
||||
return err
|
||||
}
|
||||
klog.V(4).InfoS("Pod termination removed cgroups", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
}
|
||||
|
||||
// mark the final pod status
|
||||
kl.statusManager.TerminatePod(pod)
|
||||
klog.V(4).InfoS("Pod is terminated and will need no more status updates", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get pods which should be resynchronized. Currently, the following pod should be resynchronized:
|
||||
// * pod whose work is ready.
|
||||
// * internal modules that request sync of a pod.
|
||||
@ -1776,28 +1887,11 @@ func (kl *Kubelet) deletePod(pod *v1.Pod) error {
|
||||
// for sources that haven't reported yet.
|
||||
return fmt.Errorf("skipping delete because sources aren't ready yet")
|
||||
}
|
||||
kl.podWorkers.ForgetWorker(pod.UID)
|
||||
|
||||
// make sure our runtimeCache is at least as fresh as the last container started event we observed.
|
||||
// this ensures we correctly send graceful deletion signals to all containers we've reported started.
|
||||
if lastContainerStarted, ok := kl.lastContainerStartedTime.Get(pod.UID); ok {
|
||||
if err := kl.runtimeCache.ForceUpdateIfOlder(lastContainerStarted); err != nil {
|
||||
return fmt.Errorf("error updating containers: %v", err)
|
||||
}
|
||||
}
|
||||
// Runtime cache may not have been updated to with the pod, but it's okay
|
||||
// because the periodic cleanup routine will attempt to delete again later.
|
||||
runningPods, err := kl.runtimeCache.GetPods()
|
||||
if err != nil {
|
||||
return fmt.Errorf("error listing containers: %v", err)
|
||||
}
|
||||
runningPod := kubecontainer.Pods(runningPods).FindPod("", pod.UID)
|
||||
if runningPod.IsEmpty() {
|
||||
return fmt.Errorf("pod not found")
|
||||
}
|
||||
podPair := kubecontainer.PodPair{APIPod: pod, RunningPod: &runningPod}
|
||||
kl.podKiller.KillPod(&podPair)
|
||||
|
||||
klog.V(3).InfoS("Pod has been deleted and must be killed", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
kl.podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: pod,
|
||||
UpdateType: kubetypes.SyncPodKill,
|
||||
})
|
||||
// We leave the volume/directory cleanup to the periodic cleanup routine.
|
||||
return nil
|
||||
}
|
||||
@ -1835,7 +1929,7 @@ func (kl *Kubelet) canAdmitPod(pods []*v1.Pod, pod *v1.Pod) (bool, string, strin
|
||||
func (kl *Kubelet) canRunPod(pod *v1.Pod) lifecycle.PodAdmitResult {
|
||||
attrs := &lifecycle.PodAdmitAttributes{Pod: pod}
|
||||
// Get "OtherPods". Rejected pods are failed, so only include admitted pods that are alive.
|
||||
attrs.OtherPods = kl.filterOutTerminatedPods(kl.podManager.GetPods())
|
||||
attrs.OtherPods = kl.GetActivePods()
|
||||
|
||||
for _, handler := range kl.softAdmitHandlers {
|
||||
if result := handler.Admit(attrs); !result.Admit {
|
||||
@ -2025,10 +2119,16 @@ func (kl *Kubelet) syncLoopIteration(configCh <-chan kubetypes.PodUpdate, handle
|
||||
// skip housekeeping, as we may accidentally delete pods from unready sources.
|
||||
klog.V(4).InfoS("SyncLoop (housekeeping, skipped): sources aren't ready yet")
|
||||
} else {
|
||||
start := time.Now()
|
||||
klog.V(4).InfoS("SyncLoop (housekeeping)")
|
||||
if err := handler.HandlePodCleanups(); err != nil {
|
||||
klog.ErrorS(err, "Failed cleaning pods")
|
||||
}
|
||||
duration := time.Since(start)
|
||||
if duration > housekeepingWarningDuration {
|
||||
klog.ErrorS(fmt.Errorf("housekeeping took too long"), "Housekeeping took longer than 15s", "seconds", duration.Seconds())
|
||||
}
|
||||
klog.V(4).InfoS("SyncLoop (housekeeping) end")
|
||||
}
|
||||
}
|
||||
return true
|
||||
@ -2049,31 +2149,12 @@ func handleProbeSync(kl *Kubelet, update proberesults.Update, handler SyncHandle
|
||||
// dispatchWork starts the asynchronous sync of the pod in a pod worker.
|
||||
// If the pod has completed termination, dispatchWork will perform no action.
|
||||
func (kl *Kubelet) dispatchWork(pod *v1.Pod, syncType kubetypes.SyncPodType, mirrorPod *v1.Pod, start time.Time) {
|
||||
// check whether we are ready to delete the pod from the API server (all status up to date)
|
||||
containersTerminal, podWorkerTerminal := kl.podAndContainersAreTerminal(pod)
|
||||
if pod.DeletionTimestamp != nil && containersTerminal {
|
||||
klog.V(4).InfoS("Pod has completed execution and should be deleted from the API server", "pod", klog.KObj(pod), "syncType", syncType)
|
||||
kl.statusManager.TerminatePod(pod)
|
||||
return
|
||||
}
|
||||
|
||||
// optimization: avoid invoking the pod worker if no further changes are possible to the pod definition
|
||||
// (i.e. the pod has completed and its containers have been terminated)
|
||||
if podWorkerTerminal && containersTerminal {
|
||||
klog.V(4).InfoS("Pod has completed and its containers have been terminated, ignoring remaining sync work", "pod", klog.KObj(pod), "syncType", syncType)
|
||||
return
|
||||
}
|
||||
|
||||
// Run the sync in an async worker.
|
||||
kl.podWorkers.UpdatePod(&UpdatePodOptions{
|
||||
kl.podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: pod,
|
||||
MirrorPod: mirrorPod,
|
||||
UpdateType: syncType,
|
||||
OnCompleteFunc: func(err error) {
|
||||
if err != nil {
|
||||
metrics.PodWorkerDuration.WithLabelValues(syncType.String()).Observe(metrics.SinceInSeconds(start))
|
||||
}
|
||||
},
|
||||
StartTime: start,
|
||||
})
|
||||
// Note the number of containers for new pods.
|
||||
if syncType == kubetypes.SyncPodCreate {
|
||||
@ -2109,10 +2190,13 @@ func (kl *Kubelet) HandlePodAdditions(pods []*v1.Pod) {
|
||||
continue
|
||||
}
|
||||
|
||||
if !kl.podIsTerminated(pod) {
|
||||
// Only go through the admission process if the pod is not
|
||||
// terminated.
|
||||
|
||||
// Only go through the admission process if the pod is not requested
|
||||
// for termination by another part of the kubelet. If the pod is already
|
||||
// using resources (previously admitted), the pod worker is going to be
|
||||
// shutting it down. If the pod hasn't started yet, we know that when
|
||||
// the pod worker is invoked it will also avoid setting up the pod, so
|
||||
// we simply avoid doing any work.
|
||||
if !kl.podWorkers.IsPodTerminationRequested(pod.UID) {
|
||||
// We failed pods that we rejected, so activePods include all admitted
|
||||
// pods that are alive.
|
||||
activePods := kl.filterOutTerminatedPods(existingPods)
|
||||
@ -2286,13 +2370,8 @@ func (kl *Kubelet) ListenAndServePodResources() {
|
||||
// Delete the eligible dead container instances in a pod. Depending on the configuration, the latest dead containers may be kept around.
|
||||
func (kl *Kubelet) cleanUpContainersInPod(podID types.UID, exitedContainerID string) {
|
||||
if podStatus, err := kl.podCache.Get(podID); err == nil {
|
||||
removeAll := false
|
||||
if syncedPod, ok := kl.podManager.GetPodByUID(podID); ok {
|
||||
// generate the api status using the cached runtime status to get up-to-date ContainerStatuses
|
||||
apiPodStatus := kl.generateAPIPodStatus(syncedPod, podStatus)
|
||||
// When an evicted or deleted pod has already synced, all containers can be removed.
|
||||
removeAll = eviction.PodIsEvicted(syncedPod.Status) || (syncedPod.DeletionTimestamp != nil && notRunning(apiPodStatus.ContainerStatuses))
|
||||
}
|
||||
// When an evicted or deleted pod has already synced, all containers can be removed.
|
||||
removeAll := kl.podWorkers.ShouldPodContentBeRemoved(podID)
|
||||
kl.containerDeletor.deleteContainersInPod(exitedContainerID, podStatus, removeAll)
|
||||
}
|
||||
}
|
||||
|
@ -30,7 +30,6 @@ import (
|
||||
"runtime"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/errors"
|
||||
@ -54,7 +53,6 @@ import (
|
||||
"k8s.io/kubernetes/pkg/kubelet/cri/streaming/portforward"
|
||||
remotecommandserver "k8s.io/kubernetes/pkg/kubelet/cri/streaming/remotecommand"
|
||||
"k8s.io/kubernetes/pkg/kubelet/envvars"
|
||||
"k8s.io/kubernetes/pkg/kubelet/eviction"
|
||||
"k8s.io/kubernetes/pkg/kubelet/images"
|
||||
"k8s.io/kubernetes/pkg/kubelet/status"
|
||||
kubetypes "k8s.io/kubernetes/pkg/kubelet/types"
|
||||
@ -71,11 +69,6 @@ import (
|
||||
const (
|
||||
managedHostsHeader = "# Kubernetes-managed hosts file.\n"
|
||||
managedHostsHeaderWithHostNetwork = "# Kubernetes-managed hosts file (host network).\n"
|
||||
|
||||
// Capacity of the channel for storing pods to kill. A small number should
|
||||
// suffice because a goroutine is dedicated to check the channel and does
|
||||
// not block on anything else.
|
||||
podKillingChannelCapacity = 50
|
||||
)
|
||||
|
||||
// Container state reason list
|
||||
@ -99,7 +92,9 @@ func (kl *Kubelet) listPodsFromDisk() ([]types.UID, error) {
|
||||
return pods, nil
|
||||
}
|
||||
|
||||
// GetActivePods returns non-terminal pods
|
||||
// GetActivePods returns pods that may have a running container (a
|
||||
// terminated pod is one that is known to have no running containers and
|
||||
// will not get any more).
|
||||
func (kl *Kubelet) GetActivePods() []*v1.Pod {
|
||||
allPods := kl.podManager.GetPods()
|
||||
activePods := kl.filterOutTerminatedPods(allPods)
|
||||
@ -837,18 +832,11 @@ func containerResourceRuntimeValue(fs *v1.ResourceFieldSelector, pod *v1.Pod, co
|
||||
return resource.ExtractResourceValueByContainerName(fs, pod, containerName)
|
||||
}
|
||||
|
||||
// One of the following arguments must be non-nil: runningPod, status.
|
||||
func (kl *Kubelet) killPod(pod *v1.Pod, runningPod *kubecontainer.Pod, status *kubecontainer.PodStatus, gracePeriodOverride *int64) error {
|
||||
var p kubecontainer.Pod
|
||||
if runningPod != nil {
|
||||
p = *runningPod
|
||||
} else if status != nil {
|
||||
p = kubecontainer.ConvertPodStatusToRunningPod(kl.getRuntime().Type(), status)
|
||||
} else {
|
||||
return fmt.Errorf("one of the two arguments must be non-nil: runningPod, status")
|
||||
}
|
||||
|
||||
// Call the container runtime KillPod method which stops all running containers of the pod
|
||||
// killPod instructs the container runtime to kill the pod. This method requires that
|
||||
// the pod status contains the result of the last syncPod, otherwise it may fail to
|
||||
// terminate newly created containers and sandboxes.
|
||||
func (kl *Kubelet) killPod(pod *v1.Pod, p kubecontainer.Pod, gracePeriodOverride *int64) error {
|
||||
// Call the container runtime KillPod method which stops all known running containers of the pod
|
||||
if err := kl.containerRuntime.KillPod(pod, p, gracePeriodOverride); err != nil {
|
||||
return err
|
||||
}
|
||||
@ -896,83 +884,38 @@ func (kl *Kubelet) getPullSecretsForPod(pod *v1.Pod) []v1.Secret {
|
||||
return pullSecrets
|
||||
}
|
||||
|
||||
// podStatusIsTerminal reports when the specified pod has no running containers or is no longer accepting
|
||||
// spec changes.
|
||||
func (kl *Kubelet) podAndContainersAreTerminal(pod *v1.Pod) (containersTerminal, podWorkerTerminal bool) {
|
||||
// Check the cached pod status which was set after the last sync.
|
||||
status, ok := kl.statusManager.GetPodStatus(pod.UID)
|
||||
if !ok {
|
||||
// If there is no cached status, use the status from the
|
||||
// apiserver. This is useful if kubelet has recently been
|
||||
// restarted.
|
||||
status = pod.Status
|
||||
func countRunningContainerStatus(status v1.PodStatus) int {
|
||||
var runningContainers int
|
||||
for _, c := range status.InitContainerStatuses {
|
||||
if c.State.Running != nil {
|
||||
runningContainers++
|
||||
}
|
||||
}
|
||||
// A pod transitions into failed or succeeded from either container lifecycle (RestartNever container
|
||||
// fails) or due to external events like deletion or eviction. A terminal pod *should* have no running
|
||||
// containers, but to know that the pod has completed its lifecycle you must wait for containers to also
|
||||
// be terminal.
|
||||
containersTerminal = notRunning(status.ContainerStatuses)
|
||||
// The kubelet must accept config changes from the pod spec until it has reached a point where changes would
|
||||
// have no effect on any running container.
|
||||
podWorkerTerminal = status.Phase == v1.PodFailed || status.Phase == v1.PodSucceeded || (pod.DeletionTimestamp != nil && containersTerminal)
|
||||
return
|
||||
}
|
||||
|
||||
// podIsTerminated returns true if the provided pod is in a terminal phase ("Failed", "Succeeded") or
|
||||
// has been deleted and has no running containers. This corresponds to when a pod must accept changes to
|
||||
// its pod spec (e.g. terminating containers allow grace period to be shortened).
|
||||
func (kl *Kubelet) podIsTerminated(pod *v1.Pod) bool {
|
||||
_, podWorkerTerminal := kl.podAndContainersAreTerminal(pod)
|
||||
return podWorkerTerminal
|
||||
}
|
||||
|
||||
// IsPodTerminated returns true if the pod with the provided UID is in a terminal phase ("Failed",
|
||||
// "Succeeded") or has been deleted and has no running containers. This corresponds to when a pod must
|
||||
// accept changes to its pod spec (e.g. terminating containers allow grace period to be shortened)
|
||||
func (kl *Kubelet) IsPodTerminated(uid types.UID) bool {
|
||||
pod, podFound := kl.podManager.GetPodByUID(uid)
|
||||
if !podFound {
|
||||
return true
|
||||
for _, c := range status.ContainerStatuses {
|
||||
if c.State.Running != nil {
|
||||
runningContainers++
|
||||
}
|
||||
}
|
||||
return kl.podIsTerminated(pod)
|
||||
}
|
||||
|
||||
// IsPodDeleted returns true if the pod is deleted. For the pod to be deleted, either:
|
||||
// 1. The pod object is deleted
|
||||
// 2. The pod's status is evicted
|
||||
// 3. The pod's deletion timestamp is set, and containers are not running
|
||||
func (kl *Kubelet) IsPodDeleted(uid types.UID) bool {
|
||||
pod, podFound := kl.podManager.GetPodByUID(uid)
|
||||
if !podFound {
|
||||
return true
|
||||
for _, c := range status.EphemeralContainerStatuses {
|
||||
if c.State.Running != nil {
|
||||
runningContainers++
|
||||
}
|
||||
}
|
||||
status, statusFound := kl.statusManager.GetPodStatus(pod.UID)
|
||||
if !statusFound {
|
||||
status = pod.Status
|
||||
}
|
||||
return eviction.PodIsEvicted(status) || (pod.DeletionTimestamp != nil && notRunning(status.ContainerStatuses))
|
||||
return runningContainers
|
||||
}
|
||||
|
||||
// PodResourcesAreReclaimed returns true if all required node-level resources that a pod was consuming have
|
||||
// been reclaimed by the kubelet. Reclaiming resources is a prerequisite to deleting a pod from the API server.
|
||||
func (kl *Kubelet) PodResourcesAreReclaimed(pod *v1.Pod, status v1.PodStatus) bool {
|
||||
if !notRunning(status.ContainerStatuses) {
|
||||
if kl.podWorkers.CouldHaveRunningContainers(pod.UID) {
|
||||
// We shouldn't delete pods that still have running containers
|
||||
klog.V(3).InfoS("Pod is terminated, but some containers are still running", "pod", klog.KObj(pod))
|
||||
return false
|
||||
}
|
||||
// pod's containers should be deleted
|
||||
runtimeStatus, err := kl.podCache.Get(pod.UID)
|
||||
if err != nil {
|
||||
klog.V(3).InfoS("Pod is terminated, Error getting runtimeStatus from the podCache", "pod", klog.KObj(pod), "err", err)
|
||||
return false
|
||||
}
|
||||
if len(runtimeStatus.ContainerStatuses) > 0 {
|
||||
var statusStr string
|
||||
for _, status := range runtimeStatus.ContainerStatuses {
|
||||
statusStr += fmt.Sprintf("%+v ", *status)
|
||||
}
|
||||
klog.V(3).InfoS("Pod is terminated, but some containers have not been cleaned up", "pod", klog.KObj(pod), "status", statusStr)
|
||||
if count := countRunningContainerStatus(status); count > 0 {
|
||||
// We shouldn't delete pods until the reported pod status contains no more running containers (the previous
|
||||
// check ensures no more status can be generated, this check verifies we have seen enough of the status)
|
||||
klog.V(3).InfoS("Pod is terminated, but some container status has not yet been reported", "pod", klog.KObj(pod), "running", count)
|
||||
return false
|
||||
}
|
||||
if kl.podVolumesExist(pod.UID) && !kl.keepTerminatedPodVolumes {
|
||||
@ -987,6 +930,12 @@ func (kl *Kubelet) PodResourcesAreReclaimed(pod *v1.Pod, status v1.PodStatus) bo
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// Note: we leave pod containers to be reclaimed in the background since dockershim requires the
|
||||
// container for retrieving logs and we want to make sure logs are available until the pod is
|
||||
// physically deleted.
|
||||
|
||||
klog.V(3).InfoS("Pod is terminated and all resources are reclaimed", "pod", klog.KObj(pod))
|
||||
return true
|
||||
}
|
||||
|
||||
@ -999,23 +948,12 @@ func (kl *Kubelet) podResourcesAreReclaimed(pod *v1.Pod) bool {
|
||||
return kl.PodResourcesAreReclaimed(pod, status)
|
||||
}
|
||||
|
||||
// notRunning returns true if every status is terminated or waiting, or the status list
|
||||
// is empty.
|
||||
func notRunning(statuses []v1.ContainerStatus) bool {
|
||||
for _, status := range statuses {
|
||||
if status.State.Terminated == nil && status.State.Waiting == nil {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// filterOutTerminatedPods returns the given pods which the status manager
|
||||
// does not consider failed or succeeded.
|
||||
// filterOutTerminatedPods returns the pods that could still have running
|
||||
// containers
|
||||
func (kl *Kubelet) filterOutTerminatedPods(pods []*v1.Pod) []*v1.Pod {
|
||||
var filteredPods []*v1.Pod
|
||||
for _, p := range pods {
|
||||
if kl.podIsTerminated(p) {
|
||||
if !kl.podWorkers.CouldHaveRunningContainers(p.UID) {
|
||||
continue
|
||||
}
|
||||
filteredPods = append(filteredPods, p)
|
||||
@ -1040,9 +978,9 @@ func (kl *Kubelet) removeOrphanedPodStatuses(pods []*v1.Pod, mirrorPods []*v1.Po
|
||||
// If pod killing is done, podManager.DeleteMirrorPod() is called to delete mirror pod
|
||||
// from the API server
|
||||
func (kl *Kubelet) deleteOrphanedMirrorPods() {
|
||||
podFullNames := kl.podManager.GetOrphanedMirrorPodNames()
|
||||
for _, podFullname := range podFullNames {
|
||||
if !kl.podKiller.IsPodPendingTerminationByPodName(podFullname) {
|
||||
mirrorPods := kl.podManager.GetOrphanedMirrorPodNames()
|
||||
for _, podFullname := range mirrorPods {
|
||||
if !kl.podWorkers.IsPodForMirrorPodTerminatingByFullName(podFullname) {
|
||||
_, err := kl.podManager.DeleteMirrorPod(podFullname, nil)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Encountered error when deleting mirror pod", "podName", podFullname)
|
||||
@ -1055,7 +993,8 @@ func (kl *Kubelet) deleteOrphanedMirrorPods() {
|
||||
|
||||
// HandlePodCleanups performs a series of cleanup work, including terminating
|
||||
// pod workers, killing unwanted pods, and removing orphaned volumes/pod
|
||||
// directories.
|
||||
// directories. No config changes are sent to pod workers while this method
|
||||
// is executing which means no new pods can appear.
|
||||
// NOTE: This function is executed by the main sync loop, so it
|
||||
// should not contain any blocking calls.
|
||||
func (kl *Kubelet) HandlePodCleanups() error {
|
||||
@ -1086,43 +1025,92 @@ func (kl *Kubelet) HandlePodCleanups() error {
|
||||
// to the apiserver, it could still restart the terminated pod (even
|
||||
// though the pod was not considered terminated by the apiserver).
|
||||
// These two conditions could be alleviated by checkpointing kubelet.
|
||||
activePods := kl.filterOutTerminatedPods(allPods)
|
||||
|
||||
desiredPods := make(map[types.UID]sets.Empty)
|
||||
for _, pod := range activePods {
|
||||
desiredPods[pod.UID] = sets.Empty{}
|
||||
// Stop the workers for terminated pods not in the config source
|
||||
klog.V(3).InfoS("Clean up pod workers for terminated pods")
|
||||
workingPods := kl.podWorkers.SyncKnownPods(allPods)
|
||||
|
||||
allPodsByUID := make(map[types.UID]*v1.Pod)
|
||||
for _, pod := range allPods {
|
||||
allPodsByUID[pod.UID] = pod
|
||||
}
|
||||
// Stop the workers for no-longer existing pods.
|
||||
kl.podWorkers.ForgetNonExistingPodWorkers(desiredPods)
|
||||
kl.probeManager.CleanupPods(desiredPods)
|
||||
|
||||
runningPods, err := kl.runtimeCache.GetPods()
|
||||
// Identify the set of pods that have workers, which should be all pods
|
||||
// from config that are not terminated, as well as any terminating pods
|
||||
// that have already been removed from config. Pods that are terminating
|
||||
// will be added to possiblyRunningPods, to prevent overly aggressive
|
||||
// cleanup of pod cgroups.
|
||||
runningPods := make(map[types.UID]sets.Empty)
|
||||
possiblyRunningPods := make(map[types.UID]sets.Empty)
|
||||
for uid, sync := range workingPods {
|
||||
switch sync {
|
||||
case SyncPodWork:
|
||||
runningPods[uid] = struct{}{}
|
||||
possiblyRunningPods[uid] = struct{}{}
|
||||
case TerminatingPodWork:
|
||||
possiblyRunningPods[uid] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
// Stop probing pods that are not running
|
||||
klog.V(3).InfoS("Clean up probes for terminating and terminated pods")
|
||||
kl.probeManager.CleanupPods(runningPods)
|
||||
|
||||
// Terminate any pods that are observed in the runtime but not
|
||||
// present in the list of known running pods from config.
|
||||
runningRuntimePods, err := kl.runtimeCache.GetPods()
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Error listing containers")
|
||||
return err
|
||||
}
|
||||
for _, pod := range runningPods {
|
||||
if _, found := desiredPods[pod.ID]; !found {
|
||||
kl.podKiller.KillPod(&kubecontainer.PodPair{APIPod: nil, RunningPod: pod})
|
||||
for _, runningPod := range runningRuntimePods {
|
||||
switch workType, ok := workingPods[runningPod.ID]; {
|
||||
case ok && workType == SyncPodWork, ok && workType == TerminatingPodWork:
|
||||
// if the pod worker is already in charge of this pod, we don't need to do anything
|
||||
continue
|
||||
default:
|
||||
// If the pod isn't in the set that should be running and isn't already terminating, terminate
|
||||
// now. This termination is aggressive because all known pods should already be in a known state
|
||||
// (i.e. a removed static pod should already be terminating), so these are pods that were
|
||||
// orphaned due to kubelet restart or bugs. Since housekeeping blocks other config changes, we
|
||||
// know that another pod wasn't started in the background so we are safe to terminate the
|
||||
// unknown pods.
|
||||
if _, ok := allPodsByUID[runningPod.ID]; !ok {
|
||||
klog.V(3).InfoS("Clean up orphaned pod containers", "podUID", runningPod.ID)
|
||||
one := int64(1)
|
||||
kl.podWorkers.UpdatePod(UpdatePodOptions{
|
||||
UpdateType: kubetypes.SyncPodKill,
|
||||
RunningPod: runningPod,
|
||||
KillPodOptions: &KillPodOptions{
|
||||
PodTerminationGracePeriodSecondsOverride: &one,
|
||||
},
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Remove orphaned pod statuses not in the total list of known config pods
|
||||
klog.V(3).InfoS("Clean up orphaned pod statuses")
|
||||
kl.removeOrphanedPodStatuses(allPods, mirrorPods)
|
||||
// Note that we just killed the unwanted pods. This may not have reflected
|
||||
// in the cache. We need to bypass the cache to get the latest set of
|
||||
// running pods to clean up the volumes.
|
||||
// TODO: Evaluate the performance impact of bypassing the runtime cache.
|
||||
runningPods, err = kl.containerRuntime.GetPods(false)
|
||||
runningRuntimePods, err = kl.containerRuntime.GetPods(false)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Error listing containers")
|
||||
return err
|
||||
}
|
||||
|
||||
// Remove any orphaned volumes.
|
||||
// Note that we pass all pods (including terminated pods) to the function,
|
||||
// so that we don't remove volumes associated with terminated but not yet
|
||||
// deleted pods.
|
||||
err = kl.cleanupOrphanedPodDirs(allPods, runningPods)
|
||||
// Remove orphaned volumes from pods that are known not to have any
|
||||
// containers. Note that we pass all pods (including terminated pods) to
|
||||
// the function, so that we don't remove volumes associated with terminated
|
||||
// but not yet deleted pods.
|
||||
// TODO: this method could more aggressively cleanup terminated pods
|
||||
// in the future (volumes, mount dirs, logs, and containers could all be
|
||||
// better separated)
|
||||
klog.V(3).InfoS("Clean up orphaned pod directories")
|
||||
err = kl.cleanupOrphanedPodDirs(allPods, runningRuntimePods)
|
||||
if err != nil {
|
||||
// We want all cleanup tasks to be run even if one of them failed. So
|
||||
// we just log an error here and continue other cleanup tasks.
|
||||
@ -1130,162 +1118,23 @@ func (kl *Kubelet) HandlePodCleanups() error {
|
||||
klog.ErrorS(err, "Failed cleaning up orphaned pod directories")
|
||||
}
|
||||
|
||||
// Remove any orphaned mirror pods.
|
||||
// Remove any orphaned mirror pods (mirror pods are tracked by name via the
|
||||
// pod worker)
|
||||
klog.V(3).InfoS("Clean up orphaned mirror pods")
|
||||
kl.deleteOrphanedMirrorPods()
|
||||
|
||||
// Remove any cgroups in the hierarchy for pods that are no longer running.
|
||||
// Remove any cgroups in the hierarchy for pods that are definitely no longer
|
||||
// running (not in the container runtime).
|
||||
if kl.cgroupsPerQOS {
|
||||
pcm := kl.containerManager.NewPodContainerManager()
|
||||
kl.cleanupOrphanedPodCgroups(pcm, cgroupPods, activePods)
|
||||
klog.V(3).InfoS("Clean up orphaned pod cgroups")
|
||||
kl.cleanupOrphanedPodCgroups(pcm, cgroupPods, possiblyRunningPods)
|
||||
}
|
||||
|
||||
kl.backOff.GC()
|
||||
return nil
|
||||
}
|
||||
|
||||
// PodKiller handles requests for killing pods
|
||||
type PodKiller interface {
|
||||
// KillPod receives pod speficier representing the pod to kill
|
||||
KillPod(podPair *kubecontainer.PodPair)
|
||||
// PerformPodKillingWork performs the actual pod killing work via calling CRI
|
||||
// It returns after its Close() func is called and all outstanding pod killing requests are served
|
||||
PerformPodKillingWork()
|
||||
// Close ensures that after it's called, then this pod killer wouldn't accept any more pod killing requests
|
||||
Close()
|
||||
// IsPodPendingTerminationByPodName checks whether any pod for the given full pod name is pending termination (thread safe)
|
||||
IsPodPendingTerminationByPodName(podFullname string) bool
|
||||
// IsPodPendingTerminationByUID checks whether the mirror pod for the given uid is pending termination (thread safe)
|
||||
IsPodPendingTerminationByUID(uid types.UID) bool
|
||||
}
|
||||
|
||||
// podKillerWithChannel is an implementation of PodKiller which receives pod killing requests via channel
|
||||
type podKillerWithChannel struct {
|
||||
// Channel for getting pods to kill.
|
||||
podKillingCh chan *kubecontainer.PodPair
|
||||
// lock for synchronization between HandlePodCleanups and pod killer
|
||||
podKillingLock *sync.RWMutex
|
||||
// mirrorPodTerminationMap keeps track of the progress of mirror pod termination
|
||||
// The key is the UID of the pod and the value is the full name of the pod
|
||||
mirrorPodTerminationMap map[string]string
|
||||
// podTerminationMap keeps track of the progress of pod termination.
|
||||
// The key is the UID of the pod and the value is the full name of the pod
|
||||
podTerminationMap map[string]string
|
||||
// killPod is the func which invokes runtime to kill the pod
|
||||
killPod func(pod *v1.Pod, runningPod *kubecontainer.Pod, status *kubecontainer.PodStatus, gracePeriodOverride *int64) error
|
||||
}
|
||||
|
||||
// NewPodKiller returns a functional PodKiller
|
||||
func NewPodKiller(kl *Kubelet) PodKiller {
|
||||
podKiller := &podKillerWithChannel{
|
||||
podKillingCh: make(chan *kubecontainer.PodPair, podKillingChannelCapacity),
|
||||
podKillingLock: &sync.RWMutex{},
|
||||
mirrorPodTerminationMap: make(map[string]string),
|
||||
podTerminationMap: make(map[string]string),
|
||||
killPod: kl.killPod,
|
||||
}
|
||||
return podKiller
|
||||
}
|
||||
|
||||
// IsPodPendingTerminationByUID checks whether the pod for the given uid is pending termination
|
||||
func (pk *podKillerWithChannel) IsPodPendingTerminationByUID(uid types.UID) bool {
|
||||
pk.podKillingLock.RLock()
|
||||
defer pk.podKillingLock.RUnlock()
|
||||
if _, ok := pk.podTerminationMap[string(uid)]; ok {
|
||||
return ok
|
||||
}
|
||||
if _, ok := pk.mirrorPodTerminationMap[string(uid)]; ok {
|
||||
return ok
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// IsPodPendingTerminationByPodName checks whether the given pod is in grace period of termination
|
||||
func (pk *podKillerWithChannel) IsPodPendingTerminationByPodName(podFullname string) bool {
|
||||
pk.podKillingLock.RLock()
|
||||
defer pk.podKillingLock.RUnlock()
|
||||
for _, name := range pk.mirrorPodTerminationMap {
|
||||
if name == podFullname {
|
||||
return true
|
||||
}
|
||||
}
|
||||
for _, name := range pk.podTerminationMap {
|
||||
if name == podFullname {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (pk *podKillerWithChannel) markPodTerminated(uid string) {
|
||||
klog.V(4).InfoS("Marking pod termination", "podUID", uid)
|
||||
pk.podKillingLock.Lock()
|
||||
defer pk.podKillingLock.Unlock()
|
||||
delete(pk.mirrorPodTerminationMap, uid)
|
||||
delete(pk.podTerminationMap, uid)
|
||||
}
|
||||
|
||||
// KillPod sends pod killing request to the killer after marks the pod
|
||||
// unless the given pod has been marked to be killed
|
||||
func (pk *podKillerWithChannel) KillPod(podPair *kubecontainer.PodPair) {
|
||||
pk.podKillingLock.Lock()
|
||||
defer pk.podKillingLock.Unlock()
|
||||
var apiPodExists bool
|
||||
var runningPodExists bool
|
||||
if podPair.APIPod != nil {
|
||||
uid := string(podPair.APIPod.UID)
|
||||
_, apiPodExists = pk.mirrorPodTerminationMap[uid]
|
||||
if !apiPodExists {
|
||||
fullname := kubecontainer.GetPodFullName(podPair.APIPod)
|
||||
klog.V(4).InfoS("Marking api pod pending termination", "podName", fullname, "podUID", uid)
|
||||
pk.mirrorPodTerminationMap[uid] = fullname
|
||||
}
|
||||
}
|
||||
if podPair.RunningPod != nil {
|
||||
uid := string(podPair.RunningPod.ID)
|
||||
_, runningPodExists = pk.podTerminationMap[uid]
|
||||
if !runningPodExists {
|
||||
fullname := podPair.RunningPod.Name
|
||||
klog.V(4).InfoS("Marking running pod pending termination", "podName", fullname, "podUID", uid)
|
||||
pk.podTerminationMap[uid] = fullname
|
||||
}
|
||||
}
|
||||
if apiPodExists || runningPodExists {
|
||||
if apiPodExists && runningPodExists {
|
||||
klog.V(4).InfoS("Api pod and running pod are pending termination", "apiPodUID", podPair.APIPod.UID, "runningPodUID", podPair.RunningPod.ID)
|
||||
} else if apiPodExists {
|
||||
klog.V(4).InfoS("Api pod is pending termination", "podUID", podPair.APIPod.UID)
|
||||
} else {
|
||||
klog.V(4).InfoS("Running pod is pending termination", "podUID", podPair.RunningPod.ID)
|
||||
}
|
||||
return
|
||||
}
|
||||
// Limit to one request per pod
|
||||
pk.podKillingCh <- podPair
|
||||
}
|
||||
|
||||
// Close closes the channel through which requests are delivered
|
||||
func (pk *podKillerWithChannel) Close() {
|
||||
close(pk.podKillingCh)
|
||||
}
|
||||
|
||||
// PerformPodKillingWork launches a goroutine to kill a pod received from the channel if
|
||||
// another goroutine isn't already in action.
|
||||
func (pk *podKillerWithChannel) PerformPodKillingWork() {
|
||||
for podPair := range pk.podKillingCh {
|
||||
runningPod := podPair.RunningPod
|
||||
apiPod := podPair.APIPod
|
||||
|
||||
go func(apiPod *v1.Pod, runningPod *kubecontainer.Pod) {
|
||||
klog.V(2).InfoS("Killing unwanted pod", "podName", runningPod.Name)
|
||||
err := pk.killPod(apiPod, runningPod, nil, nil)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed killing the pod", "podName", runningPod.Name)
|
||||
}
|
||||
pk.markPodTerminated(string(runningPod.ID))
|
||||
}(apiPod, runningPod)
|
||||
}
|
||||
}
|
||||
|
||||
// validateContainerLogStatus returns the container ID for the desired container to retrieve logs for, based on the state
|
||||
// of the container. The previous flag will only return the logs for the last terminated container, otherwise, the current
|
||||
// running container is preferred over a previous termination. If info about the container is not available then a specific
|
||||
@ -1981,25 +1830,14 @@ func (kl *Kubelet) GetPortForward(podName, podNamespace string, podUID types.UID
|
||||
|
||||
// cleanupOrphanedPodCgroups removes cgroups that should no longer exist.
|
||||
// it reconciles the cached state of cgroupPods with the specified list of runningPods
|
||||
func (kl *Kubelet) cleanupOrphanedPodCgroups(pcm cm.PodContainerManager, cgroupPods map[types.UID]cm.CgroupName, activePods []*v1.Pod) {
|
||||
// Add all running pods to the set that we want to preserve
|
||||
podSet := sets.NewString()
|
||||
for _, pod := range activePods {
|
||||
podSet.Insert(string(pod.UID))
|
||||
}
|
||||
|
||||
func (kl *Kubelet) cleanupOrphanedPodCgroups(pcm cm.PodContainerManager, cgroupPods map[types.UID]cm.CgroupName, possiblyRunningPods map[types.UID]sets.Empty) {
|
||||
// Iterate over all the found pods to verify if they should be running
|
||||
for uid, val := range cgroupPods {
|
||||
// if the pod is in the running set, its not a candidate for cleanup
|
||||
if podSet.Has(string(uid)) {
|
||||
if _, ok := possiblyRunningPods[uid]; ok {
|
||||
continue
|
||||
}
|
||||
|
||||
// if the pod is within termination grace period, we shouldn't cleanup the underlying cgroup
|
||||
if kl.podKiller.IsPodPendingTerminationByUID(uid) {
|
||||
klog.V(3).InfoS("Pod is pending termination", "podUID", uid)
|
||||
continue
|
||||
}
|
||||
// If volumes have not been unmounted/detached, do not delete the cgroup
|
||||
// so any memory backed volumes don't have their charges propagated to the
|
||||
// parent croup. If the volumes still exist, reduce the cpu shares for any
|
||||
|
@ -17,9 +17,11 @@ limitations under the License.
|
||||
package kubelet
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"reflect"
|
||||
"sort"
|
||||
"strconv"
|
||||
"testing"
|
||||
@ -119,7 +121,6 @@ type TestKubelet struct {
|
||||
|
||||
func (tk *TestKubelet) Cleanup() {
|
||||
if tk.kubelet != nil {
|
||||
tk.kubelet.podKiller.Close()
|
||||
os.RemoveAll(tk.kubelet.rootDirectory)
|
||||
tk.kubelet = nil
|
||||
}
|
||||
@ -293,12 +294,10 @@ func newTestKubeletWithImageList(
|
||||
fakeClock := clock.NewFakeClock(time.Now())
|
||||
kubelet.backOff = flowcontrol.NewBackOff(time.Second, time.Minute)
|
||||
kubelet.backOff.Clock = fakeClock
|
||||
kubelet.podKiller = NewPodKiller(kubelet)
|
||||
go kubelet.podKiller.PerformPodKillingWork()
|
||||
kubelet.resyncInterval = 10 * time.Second
|
||||
kubelet.workQueue = queue.NewBasicWorkQueue(fakeClock)
|
||||
// Relist period does not affect the tests.
|
||||
kubelet.pleg = pleg.NewGenericPLEG(fakeRuntime, 100, time.Hour, nil, clock.RealClock{})
|
||||
kubelet.pleg = pleg.NewGenericPLEG(fakeRuntime, 100, time.Hour, kubelet.podCache, clock.RealClock{})
|
||||
kubelet.clock = fakeClock
|
||||
|
||||
nodeRef := &v1.ObjectReference{
|
||||
@ -340,7 +339,7 @@ func newTestKubeletWithImageList(
|
||||
controllerAttachDetachEnabled,
|
||||
kubelet.nodeName,
|
||||
kubelet.podManager,
|
||||
kubelet.statusManager,
|
||||
kubelet.podWorkers,
|
||||
fakeKubeClient,
|
||||
kubelet.volumePluginMgr,
|
||||
fakeRuntime,
|
||||
@ -450,10 +449,12 @@ func TestHandlePodCleanupsPerQOS(t *testing.T) {
|
||||
// mark the pod as killed (within this test case).
|
||||
|
||||
kubelet.HandlePodCleanups()
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
// assert that unwanted pods were killed
|
||||
fakeRuntime.AssertKilledPods([]string{"12345678"})
|
||||
if actual, expected := kubelet.podWorkers.(*fakePodWorkers).triggeredDeletion, []types.UID{"12345678"}; !reflect.DeepEqual(actual, expected) {
|
||||
t.Fatalf("expected %v to be deleted, got %v", expected, actual)
|
||||
}
|
||||
fakeRuntime.AssertKilledPods([]string(nil))
|
||||
|
||||
// simulate Runtime.KillPod
|
||||
fakeRuntime.PodList = nil
|
||||
@ -461,7 +462,6 @@ func TestHandlePodCleanupsPerQOS(t *testing.T) {
|
||||
kubelet.HandlePodCleanups()
|
||||
kubelet.HandlePodCleanups()
|
||||
kubelet.HandlePodCleanups()
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
destroyCount := 0
|
||||
err := wait.Poll(100*time.Millisecond, 10*time.Second, func() (bool, error) {
|
||||
@ -488,9 +488,11 @@ func TestDispatchWorkOfCompletedPod(t *testing.T) {
|
||||
testKubelet := newTestKubelet(t, false /* controllerAttachDetachEnabled */)
|
||||
defer testKubelet.Cleanup()
|
||||
kubelet := testKubelet.kubelet
|
||||
var got bool
|
||||
kubelet.podWorkers = &fakePodWorkers{
|
||||
syncPodFn: func(options syncPodOptions) error {
|
||||
return fmt.Errorf("should ignore completed pod %q", options.pod.Name)
|
||||
syncPodFn: func(ctx context.Context, updateType kubetypes.SyncPodType, pod, mirrorPod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
got = true
|
||||
return nil
|
||||
},
|
||||
cache: kubelet.podCache,
|
||||
t: t,
|
||||
@ -554,6 +556,10 @@ func TestDispatchWorkOfCompletedPod(t *testing.T) {
|
||||
}
|
||||
for _, pod := range pods {
|
||||
kubelet.dispatchWork(pod, kubetypes.SyncPodSync, nil, time.Now())
|
||||
if !got {
|
||||
t.Errorf("Should not skip completed pod %q", pod.Name)
|
||||
}
|
||||
got = false
|
||||
}
|
||||
}
|
||||
|
||||
@ -563,7 +569,7 @@ func TestDispatchWorkOfActivePod(t *testing.T) {
|
||||
kubelet := testKubelet.kubelet
|
||||
var got bool
|
||||
kubelet.podWorkers = &fakePodWorkers{
|
||||
syncPodFn: func(options syncPodOptions) error {
|
||||
syncPodFn: func(ctx context.Context, updateType kubetypes.SyncPodType, pod, mirrorPod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
got = true
|
||||
return nil
|
||||
},
|
||||
@ -631,10 +637,12 @@ func TestHandlePodCleanups(t *testing.T) {
|
||||
kubelet := testKubelet.kubelet
|
||||
|
||||
kubelet.HandlePodCleanups()
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
// assert that unwanted pods were killed
|
||||
fakeRuntime.AssertKilledPods([]string{"12345678"})
|
||||
// assert that unwanted pods were queued to kill
|
||||
if actual, expected := kubelet.podWorkers.(*fakePodWorkers).triggeredDeletion, []types.UID{"12345678"}; !reflect.DeepEqual(actual, expected) {
|
||||
t.Fatalf("expected %v to be deleted, got %v", expected, actual)
|
||||
}
|
||||
fakeRuntime.AssertKilledPods([]string(nil))
|
||||
}
|
||||
|
||||
func TestHandlePodRemovesWhenSourcesAreReady(t *testing.T) {
|
||||
@ -667,44 +675,17 @@ func TestHandlePodRemovesWhenSourcesAreReady(t *testing.T) {
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
// Sources are not ready yet. Don't remove any pods.
|
||||
fakeRuntime.AssertKilledPods(nil)
|
||||
if expect, actual := []types.UID(nil), kubelet.podWorkers.(*fakePodWorkers).triggeredDeletion; !reflect.DeepEqual(expect, actual) {
|
||||
t.Fatalf("expected %v kills, got %v", expect, actual)
|
||||
}
|
||||
|
||||
ready = true
|
||||
kubelet.HandlePodRemoves(pods)
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
// Sources are ready. Remove unwanted pods.
|
||||
fakeRuntime.AssertKilledPods([]string{"1"})
|
||||
}
|
||||
|
||||
func TestKillPodFollwedByIsPodPendingTermination(t *testing.T) {
|
||||
testKubelet := newTestKubelet(t, false /* controllerAttachDetachEnabled */)
|
||||
defer testKubelet.Cleanup()
|
||||
|
||||
pod := &kubecontainer.Pod{
|
||||
ID: "12345678",
|
||||
Name: "foo",
|
||||
Namespace: "new",
|
||||
Containers: []*kubecontainer.Container{
|
||||
{Name: "bar"},
|
||||
},
|
||||
}
|
||||
|
||||
fakeRuntime := testKubelet.fakeRuntime
|
||||
fakeContainerManager := testKubelet.fakeContainerManager
|
||||
fakeContainerManager.PodContainerManager.AddPodFromCgroups(pod) // add pod to mock cgroup
|
||||
fakeRuntime.PodList = []*containertest.FakePod{
|
||||
{Pod: pod},
|
||||
}
|
||||
|
||||
kl := testKubelet.kubelet
|
||||
kl.podKiller.KillPod(&kubecontainer.PodPair{
|
||||
APIPod: nil,
|
||||
RunningPod: pod,
|
||||
})
|
||||
|
||||
if !(kl.podKiller.IsPodPendingTerminationByUID(pod.ID) || fakeRuntime.AssertKilledPods([]string{"12345678"})) {
|
||||
t.Fatal("Race condition: When KillPod is complete, the pod should be pending termination or be killed")
|
||||
if expect, actual := []types.UID{"1"}, kubelet.podWorkers.(*fakePodWorkers).triggeredDeletion; !reflect.DeepEqual(expect, actual) {
|
||||
t.Fatalf("expected %v kills, got %v", expect, actual)
|
||||
}
|
||||
}
|
||||
|
||||
@ -726,6 +707,7 @@ func (nl testNodeLister) List(_ labels.Selector) (ret []*v1.Node, err error) {
|
||||
}
|
||||
|
||||
func checkPodStatus(t *testing.T, kl *Kubelet, pod *v1.Pod, phase v1.PodPhase) {
|
||||
t.Helper()
|
||||
status, found := kl.statusManager.GetPodStatus(pod.UID)
|
||||
require.True(t, found, "Status of pod %q is not found in the status map", pod.UID)
|
||||
require.Equal(t, phase, status.Phase)
|
||||
@ -769,6 +751,10 @@ func TestHandlePortConflicts(t *testing.T) {
|
||||
// The newer pod should be rejected.
|
||||
notfittingPod := pods[0]
|
||||
fittingPod := pods[1]
|
||||
kl.podWorkers.(*fakePodWorkers).running = map[types.UID]bool{
|
||||
pods[0].UID: true,
|
||||
pods[1].UID: true,
|
||||
}
|
||||
|
||||
kl.HandlePodAdditions(pods)
|
||||
|
||||
@ -904,6 +890,10 @@ func TestHandleMemExceeded(t *testing.T) {
|
||||
// The newer pod should be rejected.
|
||||
notfittingPod := pods[0]
|
||||
fittingPod := pods[1]
|
||||
kl.podWorkers.(*fakePodWorkers).running = map[types.UID]bool{
|
||||
pods[0].UID: true,
|
||||
pods[1].UID: true,
|
||||
}
|
||||
|
||||
kl.HandlePodAdditions(pods)
|
||||
|
||||
@ -1230,11 +1220,7 @@ func TestCreateMirrorPod(t *testing.T) {
|
||||
pod.Annotations[kubetypes.ConfigSourceAnnotationKey] = "file"
|
||||
pods := []*v1.Pod{pod}
|
||||
kl.podManager.SetPods(pods)
|
||||
err := kl.syncPod(syncPodOptions{
|
||||
pod: pod,
|
||||
podStatus: &kubecontainer.PodStatus{},
|
||||
updateType: updateType,
|
||||
})
|
||||
err := kl.syncPod(context.Background(), updateType, pod, nil, &kubecontainer.PodStatus{})
|
||||
assert.NoError(t, err)
|
||||
podFullName := kubecontainer.GetPodFullName(pod)
|
||||
assert.True(t, manager.HasPod(podFullName), "Expected mirror pod %q to be created", podFullName)
|
||||
@ -1266,12 +1252,7 @@ func TestDeleteOutdatedMirrorPod(t *testing.T) {
|
||||
|
||||
pods := []*v1.Pod{pod, mirrorPod}
|
||||
kl.podManager.SetPods(pods)
|
||||
err := kl.syncPod(syncPodOptions{
|
||||
pod: pod,
|
||||
mirrorPod: mirrorPod,
|
||||
podStatus: &kubecontainer.PodStatus{},
|
||||
updateType: kubetypes.SyncPodUpdate,
|
||||
})
|
||||
err := kl.syncPod(context.Background(), kubetypes.SyncPodUpdate, pod, mirrorPod, &kubecontainer.PodStatus{})
|
||||
assert.NoError(t, err)
|
||||
name := kubecontainer.GetPodFullName(pod)
|
||||
creates, deletes := manager.GetCounts(name)
|
||||
@ -1309,17 +1290,41 @@ func TestDeleteOrphanedMirrorPods(t *testing.T) {
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
UID: "12345670",
|
||||
Name: "pod3",
|
||||
Namespace: "ns",
|
||||
Annotations: map[string]string{
|
||||
kubetypes.ConfigSourceAnnotationKey: "api",
|
||||
kubetypes.ConfigMirrorAnnotationKey: "mirror",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
kl.podManager.SetPods(orphanPods)
|
||||
|
||||
// a static pod that is terminating will not be deleted
|
||||
kl.podWorkers.(*fakePodWorkers).terminatingStaticPods = map[string]bool{
|
||||
kubecontainer.GetPodFullName(orphanPods[2]): true,
|
||||
}
|
||||
|
||||
// Sync with an empty pod list to delete all mirror pods.
|
||||
kl.HandlePodCleanups()
|
||||
assert.Len(t, manager.GetPods(), 0, "Expected 0 mirror pods")
|
||||
for _, pod := range orphanPods {
|
||||
for i, pod := range orphanPods {
|
||||
name := kubecontainer.GetPodFullName(pod)
|
||||
creates, deletes := manager.GetCounts(name)
|
||||
if creates != 0 || deletes != 1 {
|
||||
t.Errorf("expected 0 creation and one deletion of %q, got %d, %d", name, creates, deletes)
|
||||
switch i {
|
||||
case 2:
|
||||
if creates != 0 || deletes != 0 {
|
||||
t.Errorf("expected 0 creation and 0 deletion of %q, got %d, %d", name, creates, deletes)
|
||||
}
|
||||
default:
|
||||
if creates != 0 || deletes != 1 {
|
||||
t.Errorf("expected 0 creation and one deletion of %q, got %d, %d", name, creates, deletes)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1404,20 +1409,12 @@ func TestNetworkErrorsWithoutHostNetwork(t *testing.T) {
|
||||
})
|
||||
|
||||
kubelet.podManager.SetPods([]*v1.Pod{pod})
|
||||
err := kubelet.syncPod(syncPodOptions{
|
||||
pod: pod,
|
||||
podStatus: &kubecontainer.PodStatus{},
|
||||
updateType: kubetypes.SyncPodUpdate,
|
||||
})
|
||||
err := kubelet.syncPod(context.Background(), kubetypes.SyncPodUpdate, pod, nil, &kubecontainer.PodStatus{})
|
||||
assert.Error(t, err, "expected pod with hostNetwork=false to fail when network in error")
|
||||
|
||||
pod.Annotations[kubetypes.ConfigSourceAnnotationKey] = kubetypes.FileSource
|
||||
pod.Spec.HostNetwork = true
|
||||
err = kubelet.syncPod(syncPodOptions{
|
||||
pod: pod,
|
||||
podStatus: &kubecontainer.PodStatus{},
|
||||
updateType: kubetypes.SyncPodUpdate,
|
||||
})
|
||||
err = kubelet.syncPod(context.Background(), kubetypes.SyncPodUpdate, pod, nil, &kubecontainer.PodStatus{})
|
||||
assert.NoError(t, err, "expected pod with hostNetwork=true to succeed when network in error")
|
||||
}
|
||||
|
||||
@ -1442,6 +1439,12 @@ func TestFilterOutTerminatedPods(t *testing.T) {
|
||||
pods[3].Status.Phase = v1.PodPending
|
||||
pods[4].Status.Phase = v1.PodRunning
|
||||
|
||||
kubelet.podWorkers.(*fakePodWorkers).running = map[types.UID]bool{
|
||||
pods[2].UID: true,
|
||||
pods[3].UID: true,
|
||||
pods[4].UID: true,
|
||||
}
|
||||
|
||||
expected := []*v1.Pod{pods[2], pods[3], pods[4]}
|
||||
kubelet.podManager.SetPods(pods)
|
||||
actual := kubelet.filterOutTerminatedPods(pods)
|
||||
@ -1586,17 +1589,12 @@ func TestDeletePodDirsForDeletedPods(t *testing.T) {
|
||||
}
|
||||
|
||||
func syncAndVerifyPodDir(t *testing.T, testKubelet *TestKubelet, pods []*v1.Pod, podsToCheck []*v1.Pod, shouldExist bool) {
|
||||
t.Helper()
|
||||
kl := testKubelet.kubelet
|
||||
|
||||
kl.podManager.SetPods(pods)
|
||||
kl.HandlePodSyncs(pods)
|
||||
kl.HandlePodCleanups()
|
||||
// The first time HandlePodCleanups() is run the pod is placed into the
|
||||
// podKiller, and bypasses the pod directory cleanup. The pod is
|
||||
// already killed in the second run to HandlePodCleanups() and will
|
||||
// cleanup the directories.
|
||||
time.Sleep(2 * time.Second)
|
||||
kl.HandlePodCleanups()
|
||||
for i, pod := range podsToCheck {
|
||||
exist := dirExists(kl.getPodDir(pod.UID))
|
||||
assert.Equal(t, shouldExist, exist, "directory of pod %d", i)
|
||||
@ -1612,7 +1610,6 @@ func TestDoesNotDeletePodDirsForTerminatedPods(t *testing.T) {
|
||||
podWithUIDNameNs("12345679", "pod2", "ns"),
|
||||
podWithUIDNameNs("12345680", "pod3", "ns"),
|
||||
}
|
||||
|
||||
syncAndVerifyPodDir(t, testKubelet, pods, pods, true)
|
||||
// Pod 1 failed, and pod 2 succeeded. None of the pod directories should be
|
||||
// deleted.
|
||||
@ -1634,6 +1631,7 @@ func TestDoesNotDeletePodDirsIfContainerIsRunning(t *testing.T) {
|
||||
// Sync once to create pod directory; confirm that the pod directory has
|
||||
// already been created.
|
||||
pods := []*v1.Pod{apiPod}
|
||||
testKubelet.kubelet.podWorkers.(*fakePodWorkers).running = map[types.UID]bool{apiPod.UID: true}
|
||||
syncAndVerifyPodDir(t, testKubelet, pods, []*v1.Pod{apiPod}, true)
|
||||
|
||||
// Pretend the pod is deleted from apiserver, but is still active on the node.
|
||||
@ -1646,6 +1644,7 @@ func TestDoesNotDeletePodDirsIfContainerIsRunning(t *testing.T) {
|
||||
// should be removed.
|
||||
pods = []*v1.Pod{}
|
||||
testKubelet.fakeRuntime.PodList = []*containertest.FakePod{}
|
||||
testKubelet.kubelet.podWorkers.(*fakePodWorkers).running = nil
|
||||
syncAndVerifyPodDir(t, testKubelet, pods, []*v1.Pod{apiPod}, false)
|
||||
}
|
||||
|
||||
@ -2233,7 +2232,7 @@ func TestGenerateAPIPodStatusInvokesPodSyncHandlers(t *testing.T) {
|
||||
require.Equal(t, "because", apiStatus.Message)
|
||||
}
|
||||
|
||||
func TestSyncPodKillPod(t *testing.T) {
|
||||
func TestSyncTerminatingPodKillPod(t *testing.T) {
|
||||
testKubelet := newTestKubelet(t, false /* controllerAttachDetachEnabled */)
|
||||
defer testKubelet.Cleanup()
|
||||
kl := testKubelet.kubelet
|
||||
@ -2246,21 +2245,12 @@ func TestSyncPodKillPod(t *testing.T) {
|
||||
}
|
||||
pods := []*v1.Pod{pod}
|
||||
kl.podManager.SetPods(pods)
|
||||
podStatus := &kubecontainer.PodStatus{ID: pod.UID}
|
||||
gracePeriodOverride := int64(0)
|
||||
err := kl.syncPod(syncPodOptions{
|
||||
pod: pod,
|
||||
podStatus: &kubecontainer.PodStatus{},
|
||||
updateType: kubetypes.SyncPodKill,
|
||||
killPodOptions: &KillPodOptions{
|
||||
PodStatusFunc: func(p *v1.Pod, podStatus *kubecontainer.PodStatus) v1.PodStatus {
|
||||
return v1.PodStatus{
|
||||
Phase: v1.PodFailed,
|
||||
Reason: "reason",
|
||||
Message: "message",
|
||||
}
|
||||
},
|
||||
PodTerminationGracePeriodSecondsOverride: &gracePeriodOverride,
|
||||
},
|
||||
err := kl.syncTerminatingPod(context.Background(), pod, podStatus, nil, &gracePeriodOverride, func(podStatus *v1.PodStatus) {
|
||||
podStatus.Phase = v1.PodFailed
|
||||
podStatus.Reason = "reason"
|
||||
podStatus.Message = "message"
|
||||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
|
@ -186,11 +186,6 @@ func (kl *Kubelet) cleanupOrphanedPodDirs(pods []*v1.Pod, runningPods []*kubecon
|
||||
if allPods.Has(string(uid)) {
|
||||
continue
|
||||
}
|
||||
// if the pod is within termination grace period, we shouldn't cleanup the underlying volumes
|
||||
if kl.podKiller.IsPodPendingTerminationByUID(uid) {
|
||||
klog.V(3).InfoS("Pod is pending termination", "podUID", uid)
|
||||
continue
|
||||
}
|
||||
// If volumes have not been unmounted/detached, do not delete directory.
|
||||
// Doing so may result in corruption of data.
|
||||
// TODO: getMountedVolumePathListFromDisk() call may be redundant with
|
||||
|
@ -315,9 +315,14 @@ func TestVolumeUnmountAndDetachControllerDisabled(t *testing.T) {
|
||||
1 /* expectedSetUpCallCount */, testKubelet.volumePlugin))
|
||||
|
||||
// Remove pod
|
||||
// TODO: this may not be threadsafe (technically waitForVolumeUnmount)
|
||||
kubelet.podWorkers.(*fakePodWorkers).removeRuntime = map[types.UID]bool{pod.UID: true}
|
||||
kubelet.podManager.SetPods([]*v1.Pod{})
|
||||
|
||||
assert.NoError(t, waitForVolumeUnmount(kubelet.volumeManager, pod))
|
||||
assert.NoError(t, kubelet.volumeManager.WaitForUnmount(pod))
|
||||
if actual := kubelet.volumeManager.GetMountedVolumesForPod(util.GetUniquePodName(pod)); len(actual) > 0 {
|
||||
t.Fatalf("expected volume unmount to wait for no volumes: %v", actual)
|
||||
}
|
||||
|
||||
// Verify volumes unmounted
|
||||
podVolumes = kubelet.volumeManager.GetMountedVolumesForPod(
|
||||
@ -499,6 +504,7 @@ func TestVolumeUnmountAndDetachControllerEnabled(t *testing.T) {
|
||||
1 /* expectedSetUpCallCount */, testKubelet.volumePlugin))
|
||||
|
||||
// Remove pod
|
||||
kubelet.podWorkers.(*fakePodWorkers).removeRuntime = map[types.UID]bool{pod.UID: true}
|
||||
kubelet.podManager.SetPods([]*v1.Pod{})
|
||||
|
||||
assert.NoError(t, waitForVolumeUnmount(kubelet.volumeManager, pod))
|
||||
|
@ -51,25 +51,25 @@ func (f *fakeHTTP) Get(url string) (*http.Response, error) {
|
||||
}
|
||||
|
||||
type fakePodStateProvider struct {
|
||||
existingPods map[types.UID]struct{}
|
||||
runningPods map[types.UID]struct{}
|
||||
terminated map[types.UID]struct{}
|
||||
removed map[types.UID]struct{}
|
||||
}
|
||||
|
||||
func newFakePodStateProvider() *fakePodStateProvider {
|
||||
return &fakePodStateProvider{
|
||||
existingPods: make(map[types.UID]struct{}),
|
||||
runningPods: make(map[types.UID]struct{}),
|
||||
terminated: make(map[types.UID]struct{}),
|
||||
removed: make(map[types.UID]struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
func (f *fakePodStateProvider) IsPodDeleted(uid types.UID) bool {
|
||||
_, found := f.existingPods[uid]
|
||||
return !found
|
||||
func (f *fakePodStateProvider) ShouldPodRuntimeBeRemoved(uid types.UID) bool {
|
||||
_, found := f.terminated[uid]
|
||||
return found
|
||||
}
|
||||
|
||||
func (f *fakePodStateProvider) IsPodTerminated(uid types.UID) bool {
|
||||
_, found := f.runningPods[uid]
|
||||
return !found
|
||||
func (f *fakePodStateProvider) ShouldPodContentBeRemoved(uid types.UID) bool {
|
||||
_, found := f.removed[uid]
|
||||
return found
|
||||
}
|
||||
|
||||
func newFakeKubeRuntimeManager(runtimeService internalapi.RuntimeService, imageService internalapi.ImageManagerService, machineInfo *cadvisorapi.MachineInfo, osInterface kubecontainer.OSInterface, runtimeHelper kubecontainer.RuntimeHelper, keyring credentialprovider.DockerKeyring) (*kubeGenericRuntimeManager, error) {
|
||||
|
@ -220,7 +220,7 @@ func (cgc *containerGC) evictableContainers(minAge time.Duration) (containersByE
|
||||
}
|
||||
|
||||
// evict all containers that are evictable
|
||||
func (cgc *containerGC) evictContainers(gcPolicy kubecontainer.GCPolicy, allSourcesReady bool, evictTerminatedPods bool) error {
|
||||
func (cgc *containerGC) evictContainers(gcPolicy kubecontainer.GCPolicy, allSourcesReady bool, evictNonDeletedPods bool) error {
|
||||
// Separate containers by evict units.
|
||||
evictUnits, err := cgc.evictableContainers(gcPolicy.MinAge)
|
||||
if err != nil {
|
||||
@ -230,7 +230,7 @@ func (cgc *containerGC) evictContainers(gcPolicy kubecontainer.GCPolicy, allSour
|
||||
// Remove deleted pod containers if all sources are ready.
|
||||
if allSourcesReady {
|
||||
for key, unit := range evictUnits {
|
||||
if cgc.podStateProvider.IsPodDeleted(key.uid) || (cgc.podStateProvider.IsPodTerminated(key.uid) && evictTerminatedPods) {
|
||||
if cgc.podStateProvider.ShouldPodContentBeRemoved(key.uid) || (evictNonDeletedPods && cgc.podStateProvider.ShouldPodRuntimeBeRemoved(key.uid)) {
|
||||
cgc.removeOldestN(unit, len(unit)) // Remove all.
|
||||
delete(evictUnits, key)
|
||||
}
|
||||
@ -272,7 +272,7 @@ func (cgc *containerGC) evictContainers(gcPolicy kubecontainer.GCPolicy, allSour
|
||||
// 2. contains no containers.
|
||||
// 3. belong to a non-existent (i.e., already removed) pod, or is not the
|
||||
// most recently created sandbox for the pod.
|
||||
func (cgc *containerGC) evictSandboxes(evictTerminatedPods bool) error {
|
||||
func (cgc *containerGC) evictSandboxes(evictNonDeletedPods bool) error {
|
||||
containers, err := cgc.manager.getKubeletContainers(true)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -311,7 +311,7 @@ func (cgc *containerGC) evictSandboxes(evictTerminatedPods bool) error {
|
||||
}
|
||||
|
||||
for podUID, sandboxes := range sandboxesByPod {
|
||||
if cgc.podStateProvider.IsPodDeleted(podUID) || (cgc.podStateProvider.IsPodTerminated(podUID) && evictTerminatedPods) {
|
||||
if cgc.podStateProvider.ShouldPodContentBeRemoved(podUID) || (evictNonDeletedPods && cgc.podStateProvider.ShouldPodRuntimeBeRemoved(podUID)) {
|
||||
// Remove all evictable sandboxes if the pod has been removed.
|
||||
// Note that the latest dead sandbox is also removed if there is
|
||||
// already an active one.
|
||||
@ -337,9 +337,10 @@ func (cgc *containerGC) evictPodLogsDirectories(allSourcesReady bool) error {
|
||||
for _, dir := range dirs {
|
||||
name := dir.Name()
|
||||
podUID := parsePodUIDFromLogsDirectory(name)
|
||||
if !cgc.podStateProvider.IsPodDeleted(podUID) {
|
||||
if !cgc.podStateProvider.ShouldPodContentBeRemoved(podUID) {
|
||||
continue
|
||||
}
|
||||
klog.V(4).InfoS("Removing pod logs", "podUID", podUID)
|
||||
err := osInterface.RemoveAll(filepath.Join(podLogsRootDirectory, name))
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to remove pod logs directory", "path", name)
|
||||
@ -397,15 +398,15 @@ func (cgc *containerGC) evictPodLogsDirectories(allSourcesReady bool) error {
|
||||
// * removes oldest dead containers by enforcing gcPolicy.MaxContainers.
|
||||
// * gets evictable sandboxes which are not ready and contains no containers.
|
||||
// * removes evictable sandboxes.
|
||||
func (cgc *containerGC) GarbageCollect(gcPolicy kubecontainer.GCPolicy, allSourcesReady bool, evictTerminatedPods bool) error {
|
||||
func (cgc *containerGC) GarbageCollect(gcPolicy kubecontainer.GCPolicy, allSourcesReady bool, evictNonDeletedPods bool) error {
|
||||
errors := []error{}
|
||||
// Remove evictable containers
|
||||
if err := cgc.evictContainers(gcPolicy, allSourcesReady, evictTerminatedPods); err != nil {
|
||||
if err := cgc.evictContainers(gcPolicy, allSourcesReady, evictNonDeletedPods); err != nil {
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
// Remove sandboxes with zero containers
|
||||
if err := cgc.evictSandboxes(evictTerminatedPods); err != nil {
|
||||
if err := cgc.evictSandboxes(evictNonDeletedPods); err != nil {
|
||||
errors = append(errors, err)
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,8 @@ import (
|
||||
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"k8s.io/api/core/v1"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/types"
|
||||
runtimeapi "k8s.io/cri-api/pkg/apis/runtime/v1alpha2"
|
||||
kubecontainer "k8s.io/kubernetes/pkg/kubelet/container"
|
||||
containertest "k8s.io/kubernetes/pkg/kubelet/container/testing"
|
||||
@ -35,16 +36,14 @@ func TestSandboxGC(t *testing.T) {
|
||||
assert.NoError(t, err)
|
||||
|
||||
podStateProvider := m.containerGC.podStateProvider.(*fakePodStateProvider)
|
||||
makeGCSandbox := func(pod *v1.Pod, attempt uint32, state runtimeapi.PodSandboxState, withPodStateProvider bool, createdAt int64) sandboxTemplate {
|
||||
if withPodStateProvider {
|
||||
// initialize the pod getter
|
||||
podStateProvider.existingPods[pod.UID] = struct{}{}
|
||||
}
|
||||
makeGCSandbox := func(pod *v1.Pod, attempt uint32, state runtimeapi.PodSandboxState, hasRunningContainers, isTerminating bool, createdAt int64) sandboxTemplate {
|
||||
return sandboxTemplate{
|
||||
pod: pod,
|
||||
state: state,
|
||||
attempt: attempt,
|
||||
createdAt: createdAt,
|
||||
pod: pod,
|
||||
state: state,
|
||||
attempt: attempt,
|
||||
createdAt: createdAt,
|
||||
running: hasRunningContainers,
|
||||
terminating: isTerminating,
|
||||
}
|
||||
}
|
||||
|
||||
@ -61,133 +60,138 @@ func TestSandboxGC(t *testing.T) {
|
||||
}),
|
||||
}
|
||||
|
||||
for c, test := range []struct {
|
||||
description string // description of the test case
|
||||
sandboxes []sandboxTemplate // templates of sandboxes
|
||||
containers []containerTemplate // templates of containers
|
||||
remain []int // template indexes of remaining sandboxes
|
||||
evictTerminatedPods bool
|
||||
for _, test := range []struct {
|
||||
description string // description of the test case
|
||||
sandboxes []sandboxTemplate // templates of sandboxes
|
||||
containers []containerTemplate // templates of containers
|
||||
remain []int // template indexes of remaining sandboxes
|
||||
evictTerminatingPods bool
|
||||
}{
|
||||
{
|
||||
description: "notready sandboxes without containers for deleted pods should be garbage collected.",
|
||||
sandboxes: []sandboxTemplate{
|
||||
makeGCSandbox(pods[2], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, false, 0),
|
||||
makeGCSandbox(pods[2], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, false, false, 0),
|
||||
},
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{},
|
||||
evictTerminatedPods: false,
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{},
|
||||
evictTerminatingPods: false,
|
||||
},
|
||||
{
|
||||
description: "ready sandboxes without containers for deleted pods should not be garbage collected.",
|
||||
sandboxes: []sandboxTemplate{
|
||||
makeGCSandbox(pods[2], 0, runtimeapi.PodSandboxState_SANDBOX_READY, false, 0),
|
||||
makeGCSandbox(pods[2], 0, runtimeapi.PodSandboxState_SANDBOX_READY, false, false, 0),
|
||||
},
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{0},
|
||||
evictTerminatedPods: false,
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{0},
|
||||
evictTerminatingPods: false,
|
||||
},
|
||||
{
|
||||
description: "sandboxes for existing pods should not be garbage collected.",
|
||||
sandboxes: []sandboxTemplate{
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_READY, true, 0),
|
||||
makeGCSandbox(pods[1], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 0),
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_READY, true, false, 0),
|
||||
makeGCSandbox(pods[1], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, false, 0),
|
||||
},
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{0, 1},
|
||||
evictTerminatedPods: false,
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{0, 1},
|
||||
evictTerminatingPods: false,
|
||||
},
|
||||
{
|
||||
description: "older exited sandboxes without containers for existing pods should be garbage collected if there are more than one exited sandboxes.",
|
||||
sandboxes: []sandboxTemplate{
|
||||
makeGCSandbox(pods[0], 1, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 1),
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 0),
|
||||
makeGCSandbox(pods[0], 1, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, false, 1),
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, false, 0),
|
||||
},
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{0},
|
||||
evictTerminatedPods: false,
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{0},
|
||||
evictTerminatingPods: false,
|
||||
},
|
||||
{
|
||||
description: "older exited sandboxes with containers for existing pods should not be garbage collected even if there are more than one exited sandboxes.",
|
||||
sandboxes: []sandboxTemplate{
|
||||
makeGCSandbox(pods[0], 1, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 1),
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 0),
|
||||
makeGCSandbox(pods[0], 1, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, false, 1),
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, false, 0),
|
||||
},
|
||||
containers: []containerTemplate{
|
||||
{pod: pods[0], container: &pods[0].Spec.Containers[0], sandboxAttempt: 0, state: runtimeapi.ContainerState_CONTAINER_EXITED},
|
||||
},
|
||||
remain: []int{0, 1},
|
||||
evictTerminatedPods: false,
|
||||
remain: []int{0, 1},
|
||||
evictTerminatingPods: false,
|
||||
},
|
||||
{
|
||||
description: "non-running sandboxes for existing pods should be garbage collected if evictTerminatedPods is set.",
|
||||
description: "non-running sandboxes for existing pods should be garbage collected if evictTerminatingPods is set.",
|
||||
sandboxes: []sandboxTemplate{
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_READY, true, 0),
|
||||
makeGCSandbox(pods[1], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 0),
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_READY, true, true, 0),
|
||||
makeGCSandbox(pods[1], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, true, 0),
|
||||
},
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{0},
|
||||
evictTerminatedPods: true,
|
||||
containers: []containerTemplate{},
|
||||
remain: []int{0},
|
||||
evictTerminatingPods: true,
|
||||
},
|
||||
{
|
||||
description: "sandbox with containers should not be garbage collected.",
|
||||
sandboxes: []sandboxTemplate{
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, false, 0),
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, false, false, 0),
|
||||
},
|
||||
containers: []containerTemplate{
|
||||
{pod: pods[0], container: &pods[0].Spec.Containers[0], state: runtimeapi.ContainerState_CONTAINER_EXITED},
|
||||
},
|
||||
remain: []int{0},
|
||||
evictTerminatedPods: false,
|
||||
remain: []int{0},
|
||||
evictTerminatingPods: false,
|
||||
},
|
||||
{
|
||||
description: "multiple sandboxes should be handled properly.",
|
||||
sandboxes: []sandboxTemplate{
|
||||
// running sandbox.
|
||||
makeGCSandbox(pods[0], 1, runtimeapi.PodSandboxState_SANDBOX_READY, true, 1),
|
||||
makeGCSandbox(pods[0], 1, runtimeapi.PodSandboxState_SANDBOX_READY, true, false, 1),
|
||||
// exited sandbox without containers.
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 0),
|
||||
makeGCSandbox(pods[0], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, false, 0),
|
||||
// exited sandbox with containers.
|
||||
makeGCSandbox(pods[1], 1, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 1),
|
||||
makeGCSandbox(pods[1], 1, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, false, 1),
|
||||
// exited sandbox without containers.
|
||||
makeGCSandbox(pods[1], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, 0),
|
||||
makeGCSandbox(pods[1], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, true, false, 0),
|
||||
// exited sandbox without containers for deleted pods.
|
||||
makeGCSandbox(pods[2], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, false, 0),
|
||||
makeGCSandbox(pods[2], 0, runtimeapi.PodSandboxState_SANDBOX_NOTREADY, false, true, 0),
|
||||
},
|
||||
containers: []containerTemplate{
|
||||
{pod: pods[1], container: &pods[1].Spec.Containers[0], sandboxAttempt: 1, state: runtimeapi.ContainerState_CONTAINER_EXITED},
|
||||
},
|
||||
remain: []int{0, 2},
|
||||
evictTerminatedPods: false,
|
||||
remain: []int{0, 2},
|
||||
evictTerminatingPods: false,
|
||||
},
|
||||
} {
|
||||
t.Logf("TestCase #%d: %+v", c, test)
|
||||
fakeSandboxes := makeFakePodSandboxes(t, m, test.sandboxes)
|
||||
fakeContainers := makeFakeContainers(t, m, test.containers)
|
||||
fakeRuntime.SetFakeSandboxes(fakeSandboxes)
|
||||
fakeRuntime.SetFakeContainers(fakeContainers)
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
podStateProvider.removed = make(map[types.UID]struct{})
|
||||
podStateProvider.terminated = make(map[types.UID]struct{})
|
||||
fakeSandboxes := makeFakePodSandboxes(t, m, test.sandboxes)
|
||||
fakeContainers := makeFakeContainers(t, m, test.containers)
|
||||
for _, s := range test.sandboxes {
|
||||
if !s.running && s.pod.Name == "deleted" {
|
||||
podStateProvider.removed[s.pod.UID] = struct{}{}
|
||||
}
|
||||
if s.terminating {
|
||||
podStateProvider.terminated[s.pod.UID] = struct{}{}
|
||||
}
|
||||
}
|
||||
fakeRuntime.SetFakeSandboxes(fakeSandboxes)
|
||||
fakeRuntime.SetFakeContainers(fakeContainers)
|
||||
|
||||
err := m.containerGC.evictSandboxes(test.evictTerminatedPods)
|
||||
assert.NoError(t, err)
|
||||
realRemain, err := fakeRuntime.ListPodSandbox(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Len(t, realRemain, len(test.remain))
|
||||
for _, remain := range test.remain {
|
||||
status, err := fakeRuntime.PodSandboxStatus(fakeSandboxes[remain].Id)
|
||||
err := m.containerGC.evictSandboxes(test.evictTerminatingPods)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, &fakeSandboxes[remain].PodSandboxStatus, status)
|
||||
}
|
||||
realRemain, err := fakeRuntime.ListPodSandbox(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Len(t, realRemain, len(test.remain))
|
||||
for _, remain := range test.remain {
|
||||
status, err := fakeRuntime.PodSandboxStatus(fakeSandboxes[remain].Id)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, &fakeSandboxes[remain].PodSandboxStatus, status)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func makeGCContainer(podStateProvider *fakePodStateProvider, podName, containerName string, attempt int, createdAt int64, state runtimeapi.ContainerState) containerTemplate {
|
||||
func makeGCContainer(podName, containerName string, attempt int, createdAt int64, state runtimeapi.ContainerState) containerTemplate {
|
||||
container := makeTestContainer(containerName, "test-image")
|
||||
pod := makeTestPod(podName, "test-ns", podName, []v1.Container{container})
|
||||
if podName == "running" {
|
||||
podStateProvider.runningPods[pod.UID] = struct{}{}
|
||||
}
|
||||
if podName != "deleted" {
|
||||
podStateProvider.existingPods[pod.UID] = struct{}{}
|
||||
}
|
||||
return containerTemplate{
|
||||
pod: pod,
|
||||
container: &container,
|
||||
@ -204,201 +208,212 @@ func TestContainerGC(t *testing.T) {
|
||||
podStateProvider := m.containerGC.podStateProvider.(*fakePodStateProvider)
|
||||
defaultGCPolicy := kubecontainer.GCPolicy{MinAge: time.Hour, MaxPerPodContainer: 2, MaxContainers: 6}
|
||||
|
||||
for c, test := range []struct {
|
||||
description string // description of the test case
|
||||
containers []containerTemplate // templates of containers
|
||||
policy *kubecontainer.GCPolicy // container gc policy
|
||||
remain []int // template indexes of remaining containers
|
||||
evictTerminatedPods bool
|
||||
allSourcesReady bool
|
||||
for _, test := range []struct {
|
||||
description string // description of the test case
|
||||
containers []containerTemplate // templates of containers
|
||||
policy *kubecontainer.GCPolicy // container gc policy
|
||||
remain []int // template indexes of remaining containers
|
||||
evictTerminatingPods bool
|
||||
allSourcesReady bool
|
||||
}{
|
||||
{
|
||||
description: "all containers should be removed when max container limit is 0",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
policy: &kubecontainer.GCPolicy{MinAge: time.Minute, MaxPerPodContainer: 1, MaxContainers: 0},
|
||||
remain: []int{},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
policy: &kubecontainer.GCPolicy{MinAge: time.Minute, MaxPerPodContainer: 1, MaxContainers: 0},
|
||||
remain: []int{},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "max containers should be complied when no max per pod container limit is set",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 4, 4, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 3, 3, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 4, 4, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 3, 3, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
policy: &kubecontainer.GCPolicy{MinAge: time.Minute, MaxPerPodContainer: -1, MaxContainers: 4},
|
||||
remain: []int{0, 1, 2, 3},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
policy: &kubecontainer.GCPolicy{MinAge: time.Minute, MaxPerPodContainer: -1, MaxContainers: 4},
|
||||
remain: []int{0, 1, 2, 3},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "no containers should be removed if both max container and per pod container limits are not set",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
policy: &kubecontainer.GCPolicy{MinAge: time.Minute, MaxPerPodContainer: -1, MaxContainers: -1},
|
||||
remain: []int{0, 1, 2},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
policy: &kubecontainer.GCPolicy{MinAge: time.Minute, MaxPerPodContainer: -1, MaxContainers: -1},
|
||||
remain: []int{0, 1, 2},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "recently started containers should not be removed",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 2, time.Now().UnixNano(), runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, time.Now().UnixNano(), runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, time.Now().UnixNano(), runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 2, time.Now().UnixNano(), runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, time.Now().UnixNano(), runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, time.Now().UnixNano(), runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
remain: []int{0, 1, 2},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
remain: []int{0, 1, 2},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "oldest containers should be removed when per pod container limit exceeded",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
remain: []int{0, 1},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
remain: []int{0, 1},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "running containers should not be removed",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_RUNNING),
|
||||
makeGCContainer("foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_RUNNING),
|
||||
},
|
||||
remain: []int{0, 1, 2},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
remain: []int{0, 1, 2},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "no containers should be removed when limits are not exceeded",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
remain: []int{0, 1},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
remain: []int{0, 1},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "max container count should apply per (UID, container) pair",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "baz", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "baz", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "baz", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo2", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo2", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo2", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "baz", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "baz", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "baz", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo2", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo2", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo2", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
remain: []int{0, 1, 3, 4, 6, 7},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
remain: []int{0, 1, 3, 4, 6, 7},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "max limit should apply and try to keep from every pod",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "bar1", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "bar1", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo2", "bar2", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo2", "bar2", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo3", "bar3", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo3", "bar3", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo4", "bar4", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo4", "bar4", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "bar1", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "bar1", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo2", "bar2", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo2", "bar2", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo3", "bar3", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo3", "bar3", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo4", "bar4", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo4", "bar4", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
remain: []int{0, 2, 4, 6, 8},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
remain: []int{0, 2, 4, 6, 8},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "oldest pods should be removed if limit exceeded",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "bar1", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "bar1", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo2", "bar2", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo3", "bar3", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo4", "bar4", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo5", "bar5", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo6", "bar6", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo7", "bar7", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "bar1", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "bar1", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo2", "bar2", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo3", "bar3", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo4", "bar4", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo5", "bar5", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo6", "bar6", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo7", "bar7", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
remain: []int{0, 2, 4, 6, 8, 9},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
remain: []int{0, 2, 4, 6, 8, 9},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "all non-running containers should be removed when evictTerminatedPods is set",
|
||||
description: "all non-running containers should be removed when evictTerminatingPods is set",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "bar1", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo1", "bar1", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "running", "bar2", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo3", "bar3", 0, 0, runtimeapi.ContainerState_CONTAINER_RUNNING),
|
||||
makeGCContainer("foo", "bar", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "bar1", 2, 2, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo1", "bar1", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("running", "bar2", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo3", "bar3", 0, 0, runtimeapi.ContainerState_CONTAINER_RUNNING),
|
||||
},
|
||||
remain: []int{4, 5},
|
||||
evictTerminatedPods: true,
|
||||
allSourcesReady: true,
|
||||
remain: []int{4, 5},
|
||||
evictTerminatingPods: true,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "containers for deleted pods should be removed",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
// deleted pods still respect MinAge.
|
||||
makeGCContainer(podStateProvider, "deleted", "bar1", 2, time.Now().UnixNano(), runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "deleted", "bar1", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer(podStateProvider, "deleted", "bar1", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("deleted", "bar1", 2, time.Now().UnixNano(), runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("deleted", "bar1", 1, 1, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("deleted", "bar1", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
remain: []int{0, 1, 2},
|
||||
evictTerminatedPods: false,
|
||||
allSourcesReady: true,
|
||||
remain: []int{0, 1, 2},
|
||||
evictTerminatingPods: false,
|
||||
allSourcesReady: true,
|
||||
},
|
||||
{
|
||||
description: "containers for deleted pods may not be removed if allSourcesReady is set false ",
|
||||
containers: []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "deleted", "bar1", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
makeGCContainer("deleted", "bar1", 0, 0, runtimeapi.ContainerState_CONTAINER_EXITED),
|
||||
},
|
||||
remain: []int{0},
|
||||
evictTerminatedPods: true,
|
||||
allSourcesReady: false,
|
||||
remain: []int{0},
|
||||
evictTerminatingPods: true,
|
||||
allSourcesReady: false,
|
||||
},
|
||||
} {
|
||||
t.Logf("TestCase #%d: %+v", c, test)
|
||||
fakeContainers := makeFakeContainers(t, m, test.containers)
|
||||
fakeRuntime.SetFakeContainers(fakeContainers)
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
podStateProvider.removed = make(map[types.UID]struct{})
|
||||
podStateProvider.terminated = make(map[types.UID]struct{})
|
||||
fakeContainers := makeFakeContainers(t, m, test.containers)
|
||||
for _, s := range test.containers {
|
||||
if s.pod.Name == "deleted" {
|
||||
podStateProvider.removed[s.pod.UID] = struct{}{}
|
||||
}
|
||||
if s.pod.Name != "running" {
|
||||
podStateProvider.terminated[s.pod.UID] = struct{}{}
|
||||
}
|
||||
}
|
||||
fakeRuntime.SetFakeContainers(fakeContainers)
|
||||
|
||||
if test.policy == nil {
|
||||
test.policy = &defaultGCPolicy
|
||||
}
|
||||
err := m.containerGC.evictContainers(*test.policy, test.allSourcesReady, test.evictTerminatedPods)
|
||||
assert.NoError(t, err)
|
||||
realRemain, err := fakeRuntime.ListContainers(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Len(t, realRemain, len(test.remain))
|
||||
for _, remain := range test.remain {
|
||||
status, err := fakeRuntime.ContainerStatus(fakeContainers[remain].Id)
|
||||
if test.policy == nil {
|
||||
test.policy = &defaultGCPolicy
|
||||
}
|
||||
err := m.containerGC.evictContainers(*test.policy, test.allSourcesReady, test.evictTerminatingPods)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, &fakeContainers[remain].ContainerStatus, status)
|
||||
}
|
||||
realRemain, err := fakeRuntime.ListContainers(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Len(t, realRemain, len(test.remain))
|
||||
for _, remain := range test.remain {
|
||||
status, err := fakeRuntime.ContainerStatus(fakeContainers[remain].Id)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, &fakeContainers[remain].ContainerStatus, status)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@ -410,18 +425,15 @@ func TestPodLogDirectoryGC(t *testing.T) {
|
||||
podStateProvider := m.containerGC.podStateProvider.(*fakePodStateProvider)
|
||||
|
||||
// pod log directories without corresponding pods should be removed.
|
||||
podStateProvider.existingPods["123"] = struct{}{}
|
||||
podStateProvider.existingPods["456"] = struct{}{}
|
||||
podStateProvider.existingPods["321"] = struct{}{}
|
||||
podStateProvider.runningPods["123"] = struct{}{}
|
||||
podStateProvider.runningPods["456"] = struct{}{}
|
||||
podStateProvider.existingPods["321"] = struct{}{}
|
||||
files := []string{"123", "456", "789", "012", "name_namespace_321", "name_namespace_654"}
|
||||
removed := []string{
|
||||
filepath.Join(podLogsRootDirectory, "789"),
|
||||
filepath.Join(podLogsRootDirectory, "012"),
|
||||
filepath.Join(podLogsRootDirectory, "name_namespace_654"),
|
||||
}
|
||||
podStateProvider.removed["012"] = struct{}{}
|
||||
podStateProvider.removed["789"] = struct{}{}
|
||||
podStateProvider.removed["654"] = struct{}{}
|
||||
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
@ -452,11 +464,11 @@ func TestUnknownStateContainerGC(t *testing.T) {
|
||||
fakeRuntime, _, m, err := createTestRuntimeManager()
|
||||
assert.NoError(t, err)
|
||||
|
||||
podStateProvider := m.containerGC.podStateProvider.(*fakePodStateProvider)
|
||||
// podStateProvider := m.containerGC.podStateProvider.(*fakePodStateProvider)
|
||||
defaultGCPolicy := kubecontainer.GCPolicy{MinAge: time.Hour, MaxPerPodContainer: 0, MaxContainers: 0}
|
||||
|
||||
fakeContainers := makeFakeContainers(t, m, []containerTemplate{
|
||||
makeGCContainer(podStateProvider, "foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_UNKNOWN),
|
||||
makeGCContainer("foo", "bar", 0, 0, runtimeapi.ContainerState_CONTAINER_UNKNOWN),
|
||||
})
|
||||
fakeRuntime.SetFakeContainers(fakeContainers)
|
||||
|
||||
|
@ -74,10 +74,11 @@ var (
|
||||
ErrVersionNotSupported = errors.New("runtime api version is not supported")
|
||||
)
|
||||
|
||||
// podStateProvider can determine if a pod is deleted ir terminated
|
||||
// podStateProvider can determine if none of the elements are necessary to retain (pod content)
|
||||
// or if none of the runtime elements are necessary to retain (containers)
|
||||
type podStateProvider interface {
|
||||
IsPodDeleted(kubetypes.UID) bool
|
||||
IsPodTerminated(kubetypes.UID) bool
|
||||
ShouldPodContentBeRemoved(kubetypes.UID) bool
|
||||
ShouldPodRuntimeBeRemoved(kubetypes.UID) bool
|
||||
}
|
||||
|
||||
type kubeGenericRuntimeManager struct {
|
||||
@ -787,7 +788,8 @@ func (m *kubeGenericRuntimeManager) SyncPod(pod *v1.Pod, podStatus *kubecontaine
|
||||
// or CRI if the Pod has been deleted while the POD is
|
||||
// being created. If the pod has been deleted then it's
|
||||
// not a real error.
|
||||
if m.podStateProvider.IsPodDeleted(pod.UID) {
|
||||
// TODO: this is probably not needed now that termination is part of the sync loop
|
||||
if m.podStateProvider.ShouldPodContentBeRemoved(pod.UID) {
|
||||
klog.V(4).InfoS("Pod was deleted and sandbox failed to be created", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
return
|
||||
}
|
||||
|
@ -66,10 +66,12 @@ func customTestRuntimeManager(keyring *credentialprovider.BasicDockerKeyring) (*
|
||||
|
||||
// sandboxTemplate is a sandbox template to create fake sandbox.
|
||||
type sandboxTemplate struct {
|
||||
pod *v1.Pod
|
||||
attempt uint32
|
||||
createdAt int64
|
||||
state runtimeapi.PodSandboxState
|
||||
pod *v1.Pod
|
||||
attempt uint32
|
||||
createdAt int64
|
||||
state runtimeapi.PodSandboxState
|
||||
running bool
|
||||
terminating bool
|
||||
}
|
||||
|
||||
// containerTemplate is a container template to create fake container.
|
||||
@ -1397,6 +1399,7 @@ func TestSyncPodWithSandboxAndDeletedPod(t *testing.T) {
|
||||
}
|
||||
|
||||
backOff := flowcontrol.NewBackOff(time.Second, time.Minute)
|
||||
m.podStateProvider.(*fakePodStateProvider).removed = map[types.UID]struct{}{pod.UID: {}}
|
||||
|
||||
// GetPodStatus and the following SyncPod will not return errors in the
|
||||
// case where the pod has been deleted. We are not adding any pods into
|
||||
|
@ -62,7 +62,7 @@ type Manager struct {
|
||||
shutdownGracePeriodCriticalPods time.Duration
|
||||
|
||||
getPods eviction.ActivePodsFunc
|
||||
killPod eviction.KillPodFunc
|
||||
killPodFunc eviction.KillPodFunc
|
||||
syncNodeStatus func()
|
||||
|
||||
dbusCon dbusInhibiter
|
||||
@ -78,7 +78,7 @@ type Manager struct {
|
||||
func NewManager(getPodsFunc eviction.ActivePodsFunc, killPodFunc eviction.KillPodFunc, syncNodeStatus func(), shutdownGracePeriodRequested, shutdownGracePeriodCriticalPods time.Duration) (*Manager, lifecycle.PodAdmitHandler) {
|
||||
manager := &Manager{
|
||||
getPods: getPodsFunc,
|
||||
killPod: killPodFunc,
|
||||
killPodFunc: killPodFunc,
|
||||
syncNodeStatus: syncNodeStatus,
|
||||
shutdownGracePeriodRequested: shutdownGracePeriodRequested,
|
||||
shutdownGracePeriodCriticalPods: shutdownGracePeriodCriticalPods,
|
||||
@ -268,15 +268,10 @@ func (m *Manager) processShutdownEvent() error {
|
||||
}
|
||||
|
||||
klog.V(1).InfoS("Shutdown manager killing pod with gracePeriod", "pod", klog.KObj(pod), "gracePeriod", gracePeriodOverride)
|
||||
|
||||
status := v1.PodStatus{
|
||||
Phase: v1.PodFailed,
|
||||
Reason: nodeShutdownReason,
|
||||
Message: nodeShutdownMessage,
|
||||
}
|
||||
|
||||
err := m.killPod(pod, status, &gracePeriodOverride)
|
||||
if err != nil {
|
||||
if err := m.killPodFunc(pod, false, &gracePeriodOverride, func(status *v1.PodStatus) {
|
||||
status.Message = nodeShutdownMessage
|
||||
status.Reason = nodeShutdownReason
|
||||
}); err != nil {
|
||||
klog.V(1).InfoS("Shutdown manager failed killing pod", "pod", klog.KObj(pod), "err", err)
|
||||
} else {
|
||||
klog.V(1).InfoS("Shutdown manager finished killing pod", "pod", klog.KObj(pod))
|
||||
|
@ -212,7 +212,7 @@ func TestManager(t *testing.T) {
|
||||
}
|
||||
|
||||
podKillChan := make(chan PodKillInfo, 1)
|
||||
killPodsFunc := func(pod *v1.Pod, status v1.PodStatus, gracePeriodOverride *int64) error {
|
||||
killPodsFunc := func(pod *v1.Pod, evict bool, gracePeriodOverride *int64, fn func(podStatus *v1.PodStatus)) error {
|
||||
var gracePeriod int64
|
||||
if gracePeriodOverride != nil {
|
||||
gracePeriod = *gracePeriodOverride
|
||||
@ -297,7 +297,7 @@ func TestFeatureEnabled(t *testing.T) {
|
||||
activePodsFunc := func() []*v1.Pod {
|
||||
return nil
|
||||
}
|
||||
killPodsFunc := func(pod *v1.Pod, status v1.PodStatus, gracePeriodOverride *int64) error {
|
||||
killPodsFunc := func(pod *v1.Pod, evict bool, gracePeriodOverride *int64, fn func(*v1.PodStatus)) error {
|
||||
return nil
|
||||
}
|
||||
defer featuregatetesting.SetFeatureGateDuringTest(t, utilfeature.DefaultFeatureGate, pkgfeatures.GracefulNodeShutdown, tc.featureGateEnabled)()
|
||||
@ -323,7 +323,7 @@ func TestRestart(t *testing.T) {
|
||||
activePodsFunc := func() []*v1.Pod {
|
||||
return nil
|
||||
}
|
||||
killPodsFunc := func(pod *v1.Pod, status v1.PodStatus, gracePeriodOverride *int64) error {
|
||||
killPodsFunc := func(pod *v1.Pod, isEvicted bool, gracePeriodOverride *int64, fn func(*v1.PodStatus)) error {
|
||||
return nil
|
||||
}
|
||||
syncNodeStatus := func() {}
|
||||
|
@ -19,7 +19,7 @@ package pod
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"k8s.io/api/core/v1"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/types"
|
||||
"k8s.io/kubernetes/pkg/kubelet/configmap"
|
||||
kubecontainer "k8s.io/kubernetes/pkg/kubelet/container"
|
||||
|
@ -19,11 +19,13 @@ package testing
|
||||
|
||||
import (
|
||||
kubelettypes "k8s.io/kubernetes/pkg/kubelet/types"
|
||||
)
|
||||
import mock "github.com/stretchr/testify/mock"
|
||||
|
||||
import types "k8s.io/apimachinery/pkg/types"
|
||||
import v1 "k8s.io/api/core/v1"
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
|
||||
types "k8s.io/apimachinery/pkg/types"
|
||||
|
||||
v1 "k8s.io/api/core/v1"
|
||||
)
|
||||
|
||||
// MockManager is an autogenerated mock type for the Manager type
|
||||
type MockManager struct {
|
||||
@ -77,6 +79,20 @@ func (_m *MockManager) GetOrphanedMirrorPodNames() []string {
|
||||
return r0
|
||||
}
|
||||
|
||||
func (_m *MockManager) GetOrphanedMirrorPodNamesAndUIDs() map[string]kubelettypes.ResolvedPodUID {
|
||||
ret := _m.Called()
|
||||
|
||||
var r0 map[string]kubelettypes.ResolvedPodUID
|
||||
if rf, ok := ret.Get(0).(func() map[string]kubelettypes.ResolvedPodUID); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(map[string]kubelettypes.ResolvedPodUID)
|
||||
}
|
||||
}
|
||||
return r0
|
||||
}
|
||||
|
||||
// DeletePod provides a mock function with given fields: _a0
|
||||
func (_m *MockManager) DeletePod(_a0 *v1.Pod) {
|
||||
_m.Called(_a0)
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -17,6 +17,7 @@ limitations under the License.
|
||||
package kubelet
|
||||
|
||||
import (
|
||||
"context"
|
||||
"reflect"
|
||||
"strconv"
|
||||
"sync"
|
||||
@ -38,30 +39,82 @@ import (
|
||||
// fakePodWorkers runs sync pod function in serial, so we can have
|
||||
// deterministic behaviour in testing.
|
||||
type fakePodWorkers struct {
|
||||
lock sync.Mutex
|
||||
syncPodFn syncPodFnType
|
||||
cache kubecontainer.Cache
|
||||
t TestingInterface
|
||||
|
||||
triggeredDeletion []types.UID
|
||||
|
||||
statusLock sync.Mutex
|
||||
running map[types.UID]bool
|
||||
terminating map[types.UID]bool
|
||||
terminationRequested map[types.UID]bool
|
||||
removeRuntime map[types.UID]bool
|
||||
removeContent map[types.UID]bool
|
||||
terminatingStaticPods map[string]bool
|
||||
}
|
||||
|
||||
func (f *fakePodWorkers) UpdatePod(options *UpdatePodOptions) {
|
||||
status, err := f.cache.Get(options.Pod.UID)
|
||||
func (f *fakePodWorkers) UpdatePod(options UpdatePodOptions) {
|
||||
f.lock.Lock()
|
||||
defer f.lock.Unlock()
|
||||
var uid types.UID
|
||||
switch {
|
||||
case options.Pod != nil:
|
||||
uid = options.Pod.UID
|
||||
case options.RunningPod != nil:
|
||||
uid = options.RunningPod.ID
|
||||
default:
|
||||
return
|
||||
}
|
||||
status, err := f.cache.Get(uid)
|
||||
if err != nil {
|
||||
f.t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
if err := f.syncPodFn(syncPodOptions{
|
||||
mirrorPod: options.MirrorPod,
|
||||
pod: options.Pod,
|
||||
podStatus: status,
|
||||
updateType: options.UpdateType,
|
||||
killPodOptions: options.KillPodOptions,
|
||||
}); err != nil {
|
||||
f.t.Errorf("Unexpected error: %v", err)
|
||||
switch options.UpdateType {
|
||||
case kubetypes.SyncPodKill:
|
||||
f.triggeredDeletion = append(f.triggeredDeletion, uid)
|
||||
default:
|
||||
if err := f.syncPodFn(context.Background(), options.UpdateType, options.Pod, options.MirrorPod, status); err != nil {
|
||||
f.t.Errorf("Unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (f *fakePodWorkers) ForgetNonExistingPodWorkers(desiredPods map[types.UID]sets.Empty) {}
|
||||
func (f *fakePodWorkers) SyncKnownPods(desiredPods []*v1.Pod) map[types.UID]PodWorkType {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *fakePodWorkers) ForgetWorker(uid types.UID) {}
|
||||
func (f *fakePodWorkers) CouldHaveRunningContainers(uid types.UID) bool {
|
||||
f.statusLock.Lock()
|
||||
defer f.statusLock.Unlock()
|
||||
return f.running[uid]
|
||||
}
|
||||
func (f *fakePodWorkers) IsPodTerminationRequested(uid types.UID) bool {
|
||||
f.statusLock.Lock()
|
||||
defer f.statusLock.Unlock()
|
||||
return f.terminationRequested[uid]
|
||||
}
|
||||
func (f *fakePodWorkers) ShouldPodContainersBeTerminating(uid types.UID) bool {
|
||||
f.statusLock.Lock()
|
||||
defer f.statusLock.Unlock()
|
||||
return f.terminating[uid]
|
||||
}
|
||||
func (f *fakePodWorkers) ShouldPodRuntimeBeRemoved(uid types.UID) bool {
|
||||
f.statusLock.Lock()
|
||||
defer f.statusLock.Unlock()
|
||||
return f.removeRuntime[uid]
|
||||
}
|
||||
func (f *fakePodWorkers) ShouldPodContentBeRemoved(uid types.UID) bool {
|
||||
f.statusLock.Lock()
|
||||
defer f.statusLock.Unlock()
|
||||
return f.removeContent[uid]
|
||||
}
|
||||
func (f *fakePodWorkers) IsPodForMirrorPodTerminatingByFullName(podFullname string) bool {
|
||||
f.statusLock.Lock()
|
||||
defer f.statusLock.Unlock()
|
||||
return f.terminatingStaticPods[podFullname]
|
||||
}
|
||||
|
||||
type TestingInterface interface {
|
||||
Errorf(format string, args ...interface{})
|
||||
@ -80,6 +133,8 @@ func newPod(uid, name string) *v1.Pod {
|
||||
type syncPodRecord struct {
|
||||
name string
|
||||
updateType kubetypes.SyncPodType
|
||||
runningPod *kubecontainer.Pod
|
||||
terminated bool
|
||||
}
|
||||
|
||||
func createPodWorkers() (*podWorkers, map[types.UID][]syncPodRecord) {
|
||||
@ -88,15 +143,38 @@ func createPodWorkers() (*podWorkers, map[types.UID][]syncPodRecord) {
|
||||
fakeRecorder := &record.FakeRecorder{}
|
||||
fakeRuntime := &containertest.FakeRuntime{}
|
||||
fakeCache := containertest.NewFakeCache(fakeRuntime)
|
||||
podWorkers := newPodWorkers(
|
||||
func(options syncPodOptions) error {
|
||||
w := newPodWorkers(
|
||||
func(ctx context.Context, updateType kubetypes.SyncPodType, pod, mirrorPod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
func() {
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
pod := options.pod
|
||||
pod := pod
|
||||
processed[pod.UID] = append(processed[pod.UID], syncPodRecord{
|
||||
name: pod.Name,
|
||||
updateType: options.updateType,
|
||||
updateType: updateType,
|
||||
})
|
||||
}()
|
||||
return nil
|
||||
},
|
||||
func(ctx context.Context, pod *v1.Pod, podStatus *kubecontainer.PodStatus, runningPod *kubecontainer.Pod, gracePeriod *int64, podStatusFn func(*v1.PodStatus)) error {
|
||||
func() {
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
processed[pod.UID] = append(processed[pod.UID], syncPodRecord{
|
||||
name: pod.Name,
|
||||
updateType: kubetypes.SyncPodKill,
|
||||
runningPod: runningPod,
|
||||
})
|
||||
}()
|
||||
return nil
|
||||
},
|
||||
func(ctx context.Context, pod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
func() {
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
processed[pod.UID] = append(processed[pod.UID], syncPodRecord{
|
||||
name: pod.Name,
|
||||
terminated: true,
|
||||
})
|
||||
}()
|
||||
return nil
|
||||
@ -107,7 +185,7 @@ func createPodWorkers() (*podWorkers, map[types.UID][]syncPodRecord) {
|
||||
time.Second,
|
||||
fakeCache,
|
||||
)
|
||||
return podWorkers, processed
|
||||
return w.(*podWorkers), processed
|
||||
}
|
||||
|
||||
func drainWorkers(podWorkers *podWorkers, numPods int) {
|
||||
@ -115,8 +193,27 @@ func drainWorkers(podWorkers *podWorkers, numPods int) {
|
||||
stillWorking := false
|
||||
podWorkers.podLock.Lock()
|
||||
for i := 0; i < numPods; i++ {
|
||||
if podWorkers.isWorking[types.UID(strconv.Itoa(i))] {
|
||||
if s, ok := podWorkers.podSyncStatuses[types.UID(strconv.Itoa(i))]; ok && s.working {
|
||||
stillWorking = true
|
||||
break
|
||||
}
|
||||
}
|
||||
podWorkers.podLock.Unlock()
|
||||
if !stillWorking {
|
||||
break
|
||||
}
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
}
|
||||
}
|
||||
|
||||
func drainAllWorkers(podWorkers *podWorkers) {
|
||||
for {
|
||||
stillWorking := false
|
||||
podWorkers.podLock.Lock()
|
||||
for _, worker := range podWorkers.podSyncStatuses {
|
||||
if worker.working {
|
||||
stillWorking = true
|
||||
break
|
||||
}
|
||||
}
|
||||
podWorkers.podLock.Unlock()
|
||||
@ -133,7 +230,7 @@ func TestUpdatePod(t *testing.T) {
|
||||
numPods := 20
|
||||
for i := 0; i < numPods; i++ {
|
||||
for j := i; j < numPods; j++ {
|
||||
podWorkers.UpdatePod(&UpdatePodOptions{
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: newPod(strconv.Itoa(j), strconv.Itoa(i)),
|
||||
UpdateType: kubetypes.SyncPodCreate,
|
||||
})
|
||||
@ -165,20 +262,51 @@ func TestUpdatePod(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestUpdatePodForRuntimePod(t *testing.T) {
|
||||
podWorkers, processed := createPodWorkers()
|
||||
|
||||
// ignores running pod of wrong sync type
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
UpdateType: kubetypes.SyncPodCreate,
|
||||
RunningPod: &kubecontainer.Pod{ID: "1", Name: "1", Namespace: "test"},
|
||||
})
|
||||
drainAllWorkers(podWorkers)
|
||||
if len(processed) != 0 {
|
||||
t.Fatalf("Not all pods processed: %v", len(processed))
|
||||
}
|
||||
|
||||
// creates synthetic pod
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
UpdateType: kubetypes.SyncPodKill,
|
||||
RunningPod: &kubecontainer.Pod{ID: "1", Name: "1", Namespace: "test"},
|
||||
})
|
||||
drainAllWorkers(podWorkers)
|
||||
if len(processed) != 1 {
|
||||
t.Fatalf("Not all pods processed: %v", processed)
|
||||
}
|
||||
updates := processed["1"]
|
||||
if len(updates) != 1 {
|
||||
t.Fatalf("unexpected updates: %v", updates)
|
||||
}
|
||||
if updates[0].runningPod == nil || updates[0].updateType != kubetypes.SyncPodKill || updates[0].name != "1" {
|
||||
t.Fatalf("unexpected update: %v", updates)
|
||||
}
|
||||
}
|
||||
|
||||
func TestUpdatePodDoesNotForgetSyncPodKill(t *testing.T) {
|
||||
podWorkers, processed := createPodWorkers()
|
||||
numPods := 20
|
||||
for i := 0; i < numPods; i++ {
|
||||
pod := newPod(strconv.Itoa(i), strconv.Itoa(i))
|
||||
podWorkers.UpdatePod(&UpdatePodOptions{
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: pod,
|
||||
UpdateType: kubetypes.SyncPodCreate,
|
||||
})
|
||||
podWorkers.UpdatePod(&UpdatePodOptions{
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: pod,
|
||||
UpdateType: kubetypes.SyncPodKill,
|
||||
})
|
||||
podWorkers.UpdatePod(&UpdatePodOptions{
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: pod,
|
||||
UpdateType: kubetypes.SyncPodUpdate,
|
||||
})
|
||||
@ -205,12 +333,12 @@ func TestUpdatePodDoesNotForgetSyncPodKill(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestForgetNonExistingPodWorkers(t *testing.T) {
|
||||
func TestSyncKnownPods(t *testing.T) {
|
||||
podWorkers, _ := createPodWorkers()
|
||||
|
||||
numPods := 20
|
||||
for i := 0; i < numPods; i++ {
|
||||
podWorkers.UpdatePod(&UpdatePodOptions{
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: newPod(strconv.Itoa(i), "name"),
|
||||
UpdateType: kubetypes.SyncPodUpdate,
|
||||
})
|
||||
@ -224,7 +352,72 @@ func TestForgetNonExistingPodWorkers(t *testing.T) {
|
||||
desiredPods := map[types.UID]sets.Empty{}
|
||||
desiredPods[types.UID("2")] = sets.Empty{}
|
||||
desiredPods[types.UID("14")] = sets.Empty{}
|
||||
podWorkers.ForgetNonExistingPodWorkers(desiredPods)
|
||||
desiredPodList := []*v1.Pod{newPod("2", "name"), newPod("14", "name")}
|
||||
|
||||
// kill all but the requested pods
|
||||
for i := 0; i < numPods; i++ {
|
||||
pod := newPod(strconv.Itoa(i), "name")
|
||||
if _, ok := desiredPods[pod.UID]; ok {
|
||||
continue
|
||||
}
|
||||
if (i % 2) == 0 {
|
||||
now := metav1.Now()
|
||||
pod.DeletionTimestamp = &now
|
||||
}
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: pod,
|
||||
UpdateType: kubetypes.SyncPodKill,
|
||||
})
|
||||
}
|
||||
drainWorkers(podWorkers, numPods)
|
||||
|
||||
if !podWorkers.ShouldPodContainersBeTerminating(types.UID("0")) {
|
||||
t.Errorf("Expected pod to be terminating")
|
||||
}
|
||||
if !podWorkers.ShouldPodContainersBeTerminating(types.UID("1")) {
|
||||
t.Errorf("Expected pod to be terminating")
|
||||
}
|
||||
if podWorkers.ShouldPodContainersBeTerminating(types.UID("2")) {
|
||||
t.Errorf("Expected pod to not be terminating")
|
||||
}
|
||||
if !podWorkers.IsPodTerminationRequested(types.UID("0")) {
|
||||
t.Errorf("Expected pod to be terminating")
|
||||
}
|
||||
if podWorkers.IsPodTerminationRequested(types.UID("2")) {
|
||||
t.Errorf("Expected pod to not be terminating")
|
||||
}
|
||||
|
||||
if podWorkers.CouldHaveRunningContainers(types.UID("0")) {
|
||||
t.Errorf("Expected pod to be terminated (deleted and terminated)")
|
||||
}
|
||||
if podWorkers.CouldHaveRunningContainers(types.UID("1")) {
|
||||
t.Errorf("Expected pod to be terminated")
|
||||
}
|
||||
if !podWorkers.CouldHaveRunningContainers(types.UID("2")) {
|
||||
t.Errorf("Expected pod to not be terminated")
|
||||
}
|
||||
|
||||
if !podWorkers.ShouldPodContentBeRemoved(types.UID("0")) {
|
||||
t.Errorf("Expected pod to be suitable for removal (deleted and terminated)")
|
||||
}
|
||||
if podWorkers.ShouldPodContentBeRemoved(types.UID("1")) {
|
||||
t.Errorf("Expected pod to not be suitable for removal (terminated but not deleted)")
|
||||
}
|
||||
if podWorkers.ShouldPodContentBeRemoved(types.UID("2")) {
|
||||
t.Errorf("Expected pod to not be suitable for removal (not terminated)")
|
||||
}
|
||||
|
||||
if podWorkers.ShouldPodContainersBeTerminating(types.UID("abc")) {
|
||||
t.Errorf("Expected pod to not be known to be terminating (does not exist but not yet synced)")
|
||||
}
|
||||
if !podWorkers.CouldHaveRunningContainers(types.UID("abc")) {
|
||||
t.Errorf("Expected pod to potentially have running containers (does not exist but not yet synced)")
|
||||
}
|
||||
if podWorkers.ShouldPodContentBeRemoved(types.UID("abc")) {
|
||||
t.Errorf("Expected pod to not be suitable for removal (does not exist but not yet synced)")
|
||||
}
|
||||
|
||||
podWorkers.SyncKnownPods(desiredPodList)
|
||||
if len(podWorkers.podUpdates) != 2 {
|
||||
t.Errorf("Incorrect number of open channels %v", len(podWorkers.podUpdates))
|
||||
}
|
||||
@ -234,34 +427,52 @@ func TestForgetNonExistingPodWorkers(t *testing.T) {
|
||||
if _, exists := podWorkers.podUpdates[types.UID("14")]; !exists {
|
||||
t.Errorf("No updates channel for pod 14")
|
||||
}
|
||||
if podWorkers.IsPodTerminationRequested(types.UID("2")) {
|
||||
t.Errorf("Expected pod termination request to be cleared after sync")
|
||||
}
|
||||
|
||||
podWorkers.ForgetNonExistingPodWorkers(map[types.UID]sets.Empty{})
|
||||
if len(podWorkers.podUpdates) != 0 {
|
||||
if !podWorkers.ShouldPodContainersBeTerminating(types.UID("abc")) {
|
||||
t.Errorf("Expected pod to be expected to terminate containers (does not exist and synced at least once)")
|
||||
}
|
||||
if podWorkers.CouldHaveRunningContainers(types.UID("abc")) {
|
||||
t.Errorf("Expected pod to be known not to have running containers (does not exist and synced at least once)")
|
||||
}
|
||||
if !podWorkers.ShouldPodContentBeRemoved(types.UID("abc")) {
|
||||
t.Errorf("Expected pod to be suitable for removal (does not exist and synced at least once)")
|
||||
}
|
||||
|
||||
// verify workers that are not terminated stay open even if config no longer
|
||||
// sees them
|
||||
podWorkers.SyncKnownPods(nil)
|
||||
if len(podWorkers.podUpdates) != 2 {
|
||||
t.Errorf("Incorrect number of open channels %v", len(podWorkers.podUpdates))
|
||||
}
|
||||
}
|
||||
if len(podWorkers.podSyncStatuses) != 2 {
|
||||
t.Errorf("Incorrect number of tracked statuses: %#v", podWorkers.podSyncStatuses)
|
||||
}
|
||||
if len(podWorkers.lastUndeliveredWorkUpdate) != 0 {
|
||||
t.Errorf("Incorrect number of tracked statuses: %#v", podWorkers.lastUndeliveredWorkUpdate)
|
||||
}
|
||||
|
||||
func TestIsWorkingClearedAfterForgetPodWorkers(t *testing.T) {
|
||||
podWorkers, _ := createPodWorkers()
|
||||
|
||||
numPods := 20
|
||||
for i := 0; i < numPods; i++ {
|
||||
podWorkers.UpdatePod(&UpdatePodOptions{
|
||||
Pod: newPod(strconv.Itoa(i), "name"),
|
||||
UpdateType: kubetypes.SyncPodUpdate,
|
||||
for uid := range desiredPods {
|
||||
pod := newPod(string(uid), "name")
|
||||
podWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: pod,
|
||||
UpdateType: kubetypes.SyncPodKill,
|
||||
})
|
||||
}
|
||||
drainWorkers(podWorkers, numPods)
|
||||
|
||||
if len(podWorkers.podUpdates) != numPods {
|
||||
t.Errorf("Incorrect number of open channels %v", len(podWorkers.podUpdates))
|
||||
}
|
||||
podWorkers.ForgetNonExistingPodWorkers(map[types.UID]sets.Empty{})
|
||||
// verify once those pods terminate (via some other flow) the workers are cleared
|
||||
podWorkers.SyncKnownPods(nil)
|
||||
if len(podWorkers.podUpdates) != 0 {
|
||||
t.Errorf("Incorrect number of open channels %v", len(podWorkers.podUpdates))
|
||||
}
|
||||
if len(podWorkers.isWorking) != 0 {
|
||||
t.Errorf("Incorrect number of isWorking %v", len(podWorkers.isWorking))
|
||||
if len(podWorkers.podSyncStatuses) != 0 {
|
||||
t.Errorf("Incorrect number of tracked statuses: %#v", podWorkers.podSyncStatuses)
|
||||
}
|
||||
if len(podWorkers.lastUndeliveredWorkUpdate) != 0 {
|
||||
t.Errorf("Incorrect number of tracked statuses: %#v", podWorkers.lastUndeliveredWorkUpdate)
|
||||
}
|
||||
}
|
||||
|
||||
@ -272,17 +483,25 @@ type simpleFakeKubelet struct {
|
||||
wg sync.WaitGroup
|
||||
}
|
||||
|
||||
func (kl *simpleFakeKubelet) syncPod(options syncPodOptions) error {
|
||||
kl.pod, kl.mirrorPod, kl.podStatus = options.pod, options.mirrorPod, options.podStatus
|
||||
func (kl *simpleFakeKubelet) syncPod(ctx context.Context, updateType kubetypes.SyncPodType, pod, mirrorPod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
kl.pod, kl.mirrorPod, kl.podStatus = pod, mirrorPod, podStatus
|
||||
return nil
|
||||
}
|
||||
|
||||
func (kl *simpleFakeKubelet) syncPodWithWaitGroup(options syncPodOptions) error {
|
||||
kl.pod, kl.mirrorPod, kl.podStatus = options.pod, options.mirrorPod, options.podStatus
|
||||
func (kl *simpleFakeKubelet) syncPodWithWaitGroup(ctx context.Context, updateType kubetypes.SyncPodType, pod, mirrorPod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
kl.pod, kl.mirrorPod, kl.podStatus = pod, mirrorPod, podStatus
|
||||
kl.wg.Done()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (kl *simpleFakeKubelet) syncTerminatingPod(ctx context.Context, pod *v1.Pod, podStatus *kubecontainer.PodStatus, runningPod *kubecontainer.Pod, gracePeriod *int64, podStatusFn func(*v1.PodStatus)) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (kl *simpleFakeKubelet) syncTerminatedPod(ctx context.Context, pod *v1.Pod, podStatus *kubecontainer.PodStatus) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// TestFakePodWorkers verifies that the fakePodWorkers behaves the same way as the real podWorkers
|
||||
// for their invocation of the syncPodFn.
|
||||
func TestFakePodWorkers(t *testing.T) {
|
||||
@ -293,8 +512,16 @@ func TestFakePodWorkers(t *testing.T) {
|
||||
kubeletForRealWorkers := &simpleFakeKubelet{}
|
||||
kubeletForFakeWorkers := &simpleFakeKubelet{}
|
||||
|
||||
realPodWorkers := newPodWorkers(kubeletForRealWorkers.syncPodWithWaitGroup, fakeRecorder, queue.NewBasicWorkQueue(&clock.RealClock{}), time.Second, time.Second, fakeCache)
|
||||
fakePodWorkers := &fakePodWorkers{kubeletForFakeWorkers.syncPod, fakeCache, t}
|
||||
realPodWorkers := newPodWorkers(
|
||||
kubeletForRealWorkers.syncPodWithWaitGroup,
|
||||
kubeletForRealWorkers.syncTerminatingPod,
|
||||
kubeletForRealWorkers.syncTerminatedPod,
|
||||
fakeRecorder, queue.NewBasicWorkQueue(&clock.RealClock{}), time.Second, time.Second, fakeCache)
|
||||
fakePodWorkers := &fakePodWorkers{
|
||||
syncPodFn: kubeletForFakeWorkers.syncPod,
|
||||
cache: fakeCache,
|
||||
t: t,
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
pod *v1.Pod
|
||||
@ -316,12 +543,12 @@ func TestFakePodWorkers(t *testing.T) {
|
||||
|
||||
for i, tt := range tests {
|
||||
kubeletForRealWorkers.wg.Add(1)
|
||||
realPodWorkers.UpdatePod(&UpdatePodOptions{
|
||||
realPodWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: tt.pod,
|
||||
MirrorPod: tt.mirrorPod,
|
||||
UpdateType: kubetypes.SyncPodUpdate,
|
||||
})
|
||||
fakePodWorkers.UpdatePod(&UpdatePodOptions{
|
||||
fakePodWorkers.UpdatePod(UpdatePodOptions{
|
||||
Pod: tt.pod,
|
||||
MirrorPod: tt.mirrorPod,
|
||||
UpdateType: kubetypes.SyncPodUpdate,
|
||||
@ -350,19 +577,26 @@ func TestKillPodNowFunc(t *testing.T) {
|
||||
killPodFunc := killPodNow(podWorkers, fakeRecorder)
|
||||
pod := newPod("test", "test")
|
||||
gracePeriodOverride := int64(0)
|
||||
err := killPodFunc(pod, v1.PodStatus{Phase: v1.PodFailed, Reason: "reason", Message: "message"}, &gracePeriodOverride)
|
||||
err := killPodFunc(pod, false, &gracePeriodOverride, func(status *v1.PodStatus) {
|
||||
status.Phase = v1.PodFailed
|
||||
status.Reason = "reason"
|
||||
status.Message = "message"
|
||||
})
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %v", err)
|
||||
t.Fatalf("Unexpected error: %v", err)
|
||||
}
|
||||
drainAllWorkers(podWorkers)
|
||||
if len(processed) != 1 {
|
||||
t.Errorf("len(processed) expected: %v, actual: %v", 1, len(processed))
|
||||
return
|
||||
t.Fatalf("len(processed) expected: %v, actual: %#v", 1, processed)
|
||||
}
|
||||
syncPodRecords := processed[pod.UID]
|
||||
if len(syncPodRecords) != 1 {
|
||||
t.Errorf("Pod processed %v times, but expected %v", len(syncPodRecords), 1)
|
||||
if len(syncPodRecords) != 2 {
|
||||
t.Fatalf("Pod processed expected %v times, got %#v", 1, syncPodRecords)
|
||||
}
|
||||
if syncPodRecords[0].updateType != kubetypes.SyncPodKill {
|
||||
t.Errorf("Pod update type was %v, but expected %v", syncPodRecords[0].updateType, kubetypes.SyncPodKill)
|
||||
}
|
||||
if !syncPodRecords[1].terminated {
|
||||
t.Errorf("Pod terminated %v, but expected %v", syncPodRecords[1].terminated, true)
|
||||
}
|
||||
}
|
||||
|
@ -20,7 +20,7 @@ import (
|
||||
"fmt"
|
||||
"math"
|
||||
|
||||
"k8s.io/api/core/v1"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/client-go/tools/record"
|
||||
"k8s.io/klog/v2"
|
||||
"k8s.io/kubernetes/pkg/api/v1/resource"
|
||||
@ -94,18 +94,16 @@ func (c *CriticalPodAdmissionHandler) evictPodsToFreeRequests(admitPod *v1.Pod,
|
||||
if err != nil {
|
||||
return fmt.Errorf("preemption: error finding a set of pods to preempt: %v", err)
|
||||
}
|
||||
klog.InfoS("Attempting to evict pods in order to free up resources", "pods", podsToPreempt,
|
||||
"insufficientResources", insufficientResources.toString())
|
||||
for _, pod := range podsToPreempt {
|
||||
status := v1.PodStatus{
|
||||
Phase: v1.PodFailed,
|
||||
Message: message,
|
||||
Reason: events.PreemptContainer,
|
||||
}
|
||||
// record that we are evicting the pod
|
||||
c.recorder.Eventf(pod, v1.EventTypeWarning, events.PreemptContainer, message)
|
||||
// this is a blocking call and should only return when the pod and its containers are killed.
|
||||
err := c.killPodFunc(pod, status, nil)
|
||||
klog.V(3).InfoS("Preempting pod to free up resources", "pod", klog.KObj(pod), "podUID", pod.UID, "insufficientResources", insufficientResources.toString())
|
||||
err := c.killPodFunc(pod, true, nil, func(status *v1.PodStatus) {
|
||||
status.Phase = v1.PodFailed
|
||||
status.Reason = events.PreemptContainer
|
||||
status.Message = message
|
||||
})
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to evict pod", "pod", klog.KObj(pod))
|
||||
// In future syncPod loops, the kubelet will retry the pod deletion steps that it was stuck on.
|
||||
|
@ -20,7 +20,7 @@ import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"k8s.io/api/core/v1"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/resource"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/client-go/tools/record"
|
||||
@ -57,7 +57,7 @@ func (f *fakePodKiller) getKilledPods() []*v1.Pod {
|
||||
return f.killedPods
|
||||
}
|
||||
|
||||
func (f *fakePodKiller) killPodNow(pod *v1.Pod, status v1.PodStatus, gracePeriodOverride *int64) error {
|
||||
func (f *fakePodKiller) killPodNow(pod *v1.Pod, evict bool, gracePeriodOverride *int64, fn func(status *v1.PodStatus)) error {
|
||||
if f.errDuringPodKilling {
|
||||
f.killedPods = []*v1.Pod{}
|
||||
return fmt.Errorf("problem killing pod %v", pod)
|
||||
|
@ -17,11 +17,12 @@ limitations under the License.
|
||||
package kubelet
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"k8s.io/api/core/v1"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/klog/v2"
|
||||
kubecontainer "k8s.io/kubernetes/pkg/kubelet/container"
|
||||
kubetypes "k8s.io/kubernetes/pkg/kubelet/types"
|
||||
@ -130,12 +131,7 @@ func (kl *Kubelet) runPod(pod *v1.Pod, retryDelay time.Duration) error {
|
||||
klog.ErrorS(err, "Failed creating a mirror pod", "pod", klog.KObj(pod))
|
||||
}
|
||||
mirrorPod, _ := kl.podManager.GetMirrorPodByPod(pod)
|
||||
if err = kl.syncPod(syncPodOptions{
|
||||
pod: pod,
|
||||
mirrorPod: mirrorPod,
|
||||
podStatus: status,
|
||||
updateType: kubetypes.SyncPodUpdate,
|
||||
}); err != nil {
|
||||
if err = kl.syncPod(context.Background(), kubetypes.SyncPodUpdate, pod, mirrorPod, status); err != nil {
|
||||
return fmt.Errorf("error syncing pod %q: %v", format.Pod(pod), err)
|
||||
}
|
||||
if retry >= runOnceMaxRetries {
|
||||
|
@ -79,6 +79,7 @@ func TestRunOnce(t *testing.T) {
|
||||
nodeLister: testNodeLister{},
|
||||
statusManager: status.NewManager(nil, podManager, &statustest.FakePodDeletionSafetyProvider{}),
|
||||
podManager: podManager,
|
||||
podWorkers: &fakePodWorkers{},
|
||||
os: &containertest.FakeOS{},
|
||||
containerRuntime: fakeRuntime,
|
||||
reasonCache: NewReasonCache(),
|
||||
@ -101,7 +102,7 @@ func TestRunOnce(t *testing.T) {
|
||||
true,
|
||||
kb.nodeName,
|
||||
kb.podManager,
|
||||
kb.statusManager,
|
||||
kb.podWorkers,
|
||||
kb.kubeClient,
|
||||
kb.volumePluginMgr,
|
||||
fakeRuntime,
|
||||
@ -122,7 +123,7 @@ func TestRunOnce(t *testing.T) {
|
||||
UID: types.UID(kb.nodeName),
|
||||
Namespace: "",
|
||||
}
|
||||
fakeKillPodFunc := func(pod *v1.Pod, podStatus v1.PodStatus, gracePeriodOverride *int64) error {
|
||||
fakeKillPodFunc := func(pod *v1.Pod, evict bool, gracePeriodOverride *int64, fn func(*v1.PodStatus)) error {
|
||||
return nil
|
||||
}
|
||||
fakeMirrodPodFunc := func(*v1.Pod) (*v1.Pod, bool) { return nil, false }
|
||||
|
@ -20,6 +20,7 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -356,6 +357,7 @@ func (m *manager) TerminatePod(pod *v1.Pod) {
|
||||
}
|
||||
}
|
||||
|
||||
klog.V(5).InfoS("TerminatePod calling updateStatusInternal", "pod", klog.KObj(pod), "podUID", pod.UID)
|
||||
m.updateStatusInternal(pod, status, true)
|
||||
}
|
||||
|
||||
@ -431,10 +433,43 @@ func (m *manager) updateStatusInternal(pod *v1.Pod, status v1.PodStatus, forceUp
|
||||
}
|
||||
|
||||
normalizeStatus(pod, &status)
|
||||
|
||||
// Perform some more extensive logging of container termination state to assist in
|
||||
// debugging production races (generally not needed).
|
||||
if klog.V(5).Enabled() {
|
||||
var containers []string
|
||||
for _, s := range append(append([]v1.ContainerStatus(nil), status.InitContainerStatuses...), status.ContainerStatuses...) {
|
||||
var current, previous string
|
||||
switch {
|
||||
case s.State.Running != nil:
|
||||
current = "running"
|
||||
case s.State.Waiting != nil:
|
||||
current = "waiting"
|
||||
case s.State.Terminated != nil:
|
||||
current = fmt.Sprintf("terminated=%d", s.State.Terminated.ExitCode)
|
||||
default:
|
||||
current = "unknown"
|
||||
}
|
||||
switch {
|
||||
case s.LastTerminationState.Running != nil:
|
||||
previous = "running"
|
||||
case s.LastTerminationState.Waiting != nil:
|
||||
previous = "waiting"
|
||||
case s.LastTerminationState.Terminated != nil:
|
||||
previous = fmt.Sprintf("terminated=%d", s.LastTerminationState.Terminated.ExitCode)
|
||||
default:
|
||||
previous = "<none>"
|
||||
}
|
||||
containers = append(containers, fmt.Sprintf("(%s state=%s previous=%s)", s.Name, current, previous))
|
||||
}
|
||||
sort.Strings(containers)
|
||||
klog.InfoS("updateStatusInternal", "version", cachedStatus.version+1, "pod", klog.KObj(pod), "podUID", pod.UID, "containers", strings.Join(containers, " "))
|
||||
}
|
||||
|
||||
// The intent here is to prevent concurrent updates to a pod's status from
|
||||
// clobbering each other so the phase of a pod progresses monotonically.
|
||||
if isCached && isPodStatusByKubeletEqual(&cachedStatus.status, &status) && !forceUpdate {
|
||||
klog.V(5).InfoS("Ignoring same status for pod", "pod", klog.KObj(pod), "status", status)
|
||||
klog.V(3).InfoS("Ignoring same status for pod", "pod", klog.KObj(pod), "status", status)
|
||||
return false // No new status.
|
||||
}
|
||||
|
||||
@ -583,7 +618,7 @@ func (m *manager) syncPod(uid types.UID, status versionedPodStatus) {
|
||||
|
||||
oldStatus := pod.Status.DeepCopy()
|
||||
newPod, patchBytes, unchanged, err := statusutil.PatchPodStatus(m.kubeClient, pod.Namespace, pod.Name, pod.UID, *oldStatus, mergePodStatus(*oldStatus, status.status))
|
||||
klog.V(3).InfoS("Patch status for pod", "pod", klog.KObj(pod), "patchBytes", patchBytes)
|
||||
klog.V(3).InfoS("Patch status for pod", "pod", klog.KObj(pod), "patch", string(patchBytes))
|
||||
|
||||
if err != nil {
|
||||
klog.InfoS("Failed to update status for pod", "pod", klog.KObj(pod), "err", err)
|
||||
|
@ -120,8 +120,8 @@ const (
|
||||
SyncPodUpdate
|
||||
// SyncPodCreate is when the pod is created from source
|
||||
SyncPodCreate
|
||||
// SyncPodKill is when the pod is killed based on a trigger internal to the kubelet for eviction.
|
||||
// If a SyncPodKill request is made to pod workers, the request is never dropped, and will always be processed.
|
||||
// SyncPodKill is when the pod should have no running containers. A pod stopped in this way could be
|
||||
// restarted in the future due config changes.
|
||||
SyncPodKill
|
||||
)
|
||||
|
||||
|
@ -40,7 +40,6 @@ import (
|
||||
"k8s.io/kubernetes/pkg/kubelet/config"
|
||||
kubecontainer "k8s.io/kubernetes/pkg/kubelet/container"
|
||||
"k8s.io/kubernetes/pkg/kubelet/pod"
|
||||
"k8s.io/kubernetes/pkg/kubelet/status"
|
||||
"k8s.io/kubernetes/pkg/kubelet/volumemanager/cache"
|
||||
"k8s.io/kubernetes/pkg/volume"
|
||||
"k8s.io/kubernetes/pkg/volume/csimigration"
|
||||
@ -70,6 +69,12 @@ type DesiredStateOfWorldPopulator interface {
|
||||
HasAddedPods() bool
|
||||
}
|
||||
|
||||
// podStateProvider can determine if a pod is going to be terminated.
|
||||
type podStateProvider interface {
|
||||
ShouldPodContainersBeTerminating(types.UID) bool
|
||||
ShouldPodRuntimeBeRemoved(types.UID) bool
|
||||
}
|
||||
|
||||
// NewDesiredStateOfWorldPopulator returns a new instance of
|
||||
// DesiredStateOfWorldPopulator.
|
||||
//
|
||||
@ -84,7 +89,7 @@ func NewDesiredStateOfWorldPopulator(
|
||||
loopSleepDuration time.Duration,
|
||||
getPodStatusRetryDuration time.Duration,
|
||||
podManager pod.Manager,
|
||||
podStatusProvider status.PodStatusProvider,
|
||||
podStateProvider podStateProvider,
|
||||
desiredStateOfWorld cache.DesiredStateOfWorld,
|
||||
actualStateOfWorld cache.ActualStateOfWorld,
|
||||
kubeContainerRuntime kubecontainer.Runtime,
|
||||
@ -97,7 +102,7 @@ func NewDesiredStateOfWorldPopulator(
|
||||
loopSleepDuration: loopSleepDuration,
|
||||
getPodStatusRetryDuration: getPodStatusRetryDuration,
|
||||
podManager: podManager,
|
||||
podStatusProvider: podStatusProvider,
|
||||
podStateProvider: podStateProvider,
|
||||
desiredStateOfWorld: desiredStateOfWorld,
|
||||
actualStateOfWorld: actualStateOfWorld,
|
||||
pods: processedPods{
|
||||
@ -117,7 +122,7 @@ type desiredStateOfWorldPopulator struct {
|
||||
loopSleepDuration time.Duration
|
||||
getPodStatusRetryDuration time.Duration
|
||||
podManager pod.Manager
|
||||
podStatusProvider status.PodStatusProvider
|
||||
podStateProvider podStateProvider
|
||||
desiredStateOfWorld cache.DesiredStateOfWorld
|
||||
actualStateOfWorld cache.ActualStateOfWorld
|
||||
pods processedPods
|
||||
@ -177,14 +182,6 @@ func (dswp *desiredStateOfWorldPopulator) populatorLoop() {
|
||||
dswp.findAndRemoveDeletedPods()
|
||||
}
|
||||
|
||||
func (dswp *desiredStateOfWorldPopulator) isPodTerminated(pod *v1.Pod) bool {
|
||||
podStatus, found := dswp.podStatusProvider.GetPodStatus(pod.UID)
|
||||
if !found {
|
||||
podStatus = pod.Status
|
||||
}
|
||||
return util.IsPodTerminated(pod, podStatus)
|
||||
}
|
||||
|
||||
// Iterate through all pods and add to desired state of world if they don't
|
||||
// exist but should
|
||||
func (dswp *desiredStateOfWorldPopulator) findAndAddNewPods() {
|
||||
@ -203,8 +200,8 @@ func (dswp *desiredStateOfWorldPopulator) findAndAddNewPods() {
|
||||
|
||||
processedVolumesForFSResize := sets.NewString()
|
||||
for _, pod := range dswp.podManager.GetPods() {
|
||||
if dswp.isPodTerminated(pod) {
|
||||
// Do not (re)add volumes for terminated pods
|
||||
if dswp.podStateProvider.ShouldPodContainersBeTerminating(pod.UID) {
|
||||
// Do not (re)add volumes for pods that can't also be starting containers
|
||||
continue
|
||||
}
|
||||
dswp.processPodVolumes(pod, mountedVolumesForPod, processedVolumesForFSResize)
|
||||
@ -214,9 +211,6 @@ func (dswp *desiredStateOfWorldPopulator) findAndAddNewPods() {
|
||||
// Iterate through all pods in desired state of world, and remove if they no
|
||||
// longer exist
|
||||
func (dswp *desiredStateOfWorldPopulator) findAndRemoveDeletedPods() {
|
||||
var runningPods []*kubecontainer.Pod
|
||||
|
||||
runningPodsFetched := false
|
||||
for _, volumeToMount := range dswp.desiredStateOfWorld.GetVolumesToMount() {
|
||||
pod, podExists := dswp.podManager.GetPodByUID(volumeToMount.Pod.UID)
|
||||
if podExists {
|
||||
@ -234,8 +228,8 @@ func (dswp *desiredStateOfWorldPopulator) findAndRemoveDeletedPods() {
|
||||
}
|
||||
}
|
||||
|
||||
// Skip running pods
|
||||
if !dswp.isPodTerminated(pod) {
|
||||
// Exclude known pods that we expect to be running
|
||||
if !dswp.podStateProvider.ShouldPodRuntimeBeRemoved(pod.UID) {
|
||||
continue
|
||||
}
|
||||
if dswp.keepTerminatedPodVolumes {
|
||||
@ -245,36 +239,9 @@ func (dswp *desiredStateOfWorldPopulator) findAndRemoveDeletedPods() {
|
||||
|
||||
// Once a pod has been deleted from kubelet pod manager, do not delete
|
||||
// it immediately from volume manager. Instead, check the kubelet
|
||||
// containerRuntime to verify that all containers in the pod have been
|
||||
// pod state provider to verify that all containers in the pod have been
|
||||
// terminated.
|
||||
if !runningPodsFetched {
|
||||
var getPodsErr error
|
||||
runningPods, getPodsErr = dswp.kubeContainerRuntime.GetPods(false)
|
||||
if getPodsErr != nil {
|
||||
klog.ErrorS(getPodsErr, "kubeContainerRuntime.findAndRemoveDeletedPods returned error")
|
||||
continue
|
||||
}
|
||||
|
||||
runningPodsFetched = true
|
||||
dswp.timeOfLastGetPodStatus = time.Now()
|
||||
}
|
||||
|
||||
runningContainers := false
|
||||
for _, runningPod := range runningPods {
|
||||
if runningPod.ID == volumeToMount.Pod.UID {
|
||||
// runningPod.Containers only include containers in the running state,
|
||||
// excluding containers in the creating process.
|
||||
// By adding a non-empty judgment for runningPod.Sandboxes,
|
||||
// ensure that all containers of the pod have been terminated.
|
||||
if len(runningPod.Sandboxes) > 0 || len(runningPod.Containers) > 0 {
|
||||
runningContainers = true
|
||||
}
|
||||
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if runningContainers {
|
||||
if !dswp.podStateProvider.ShouldPodRuntimeBeRemoved(volumeToMount.Pod.UID) {
|
||||
klog.V(4).InfoS("Pod still has one or more containers in the non-exited state and will not be removed from desired state", "pod", klog.KObj(volumeToMount.Pod))
|
||||
continue
|
||||
}
|
||||
|
@ -27,6 +27,7 @@ import (
|
||||
"k8s.io/apimachinery/pkg/api/resource"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/runtime"
|
||||
kubetypes "k8s.io/apimachinery/pkg/types"
|
||||
utilfeature "k8s.io/apiserver/pkg/util/feature"
|
||||
"k8s.io/client-go/kubernetes/fake"
|
||||
core "k8s.io/client-go/testing"
|
||||
@ -34,13 +35,10 @@ import (
|
||||
csitrans "k8s.io/csi-translation-lib"
|
||||
"k8s.io/kubernetes/pkg/features"
|
||||
"k8s.io/kubernetes/pkg/kubelet/configmap"
|
||||
kubecontainer "k8s.io/kubernetes/pkg/kubelet/container"
|
||||
containertest "k8s.io/kubernetes/pkg/kubelet/container/testing"
|
||||
kubepod "k8s.io/kubernetes/pkg/kubelet/pod"
|
||||
podtest "k8s.io/kubernetes/pkg/kubelet/pod/testing"
|
||||
"k8s.io/kubernetes/pkg/kubelet/secret"
|
||||
"k8s.io/kubernetes/pkg/kubelet/status"
|
||||
statustest "k8s.io/kubernetes/pkg/kubelet/status/testing"
|
||||
"k8s.io/kubernetes/pkg/kubelet/volumemanager/cache"
|
||||
"k8s.io/kubernetes/pkg/volume"
|
||||
"k8s.io/kubernetes/pkg/volume/csimigration"
|
||||
@ -81,7 +79,7 @@ func prepareDswpWithVolume(t *testing.T) (*desiredStateOfWorldPopulator, kubepod
|
||||
Phase: v1.ClaimBound,
|
||||
},
|
||||
}
|
||||
dswp, fakePodManager, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, _, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
return dswp, fakePodManager
|
||||
}
|
||||
|
||||
@ -156,7 +154,7 @@ func TestFindAndAddNewPods_WithVolumeRetrievalError(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestFindAndAddNewPods_FindAndRemoveDeletedPods(t *testing.T) {
|
||||
dswp, fakeRuntime, pod, expectedVolumeName, _ := prepareDSWPWithPodPV(t)
|
||||
dswp, fakePodState, pod, expectedVolumeName, _ := prepareDSWPWithPodPV(t)
|
||||
podName := util.GetUniquePodName(pod)
|
||||
|
||||
//let the pod be terminated
|
||||
@ -167,29 +165,15 @@ func TestFindAndAddNewPods_FindAndRemoveDeletedPods(t *testing.T) {
|
||||
podGet.Status.Phase = v1.PodFailed
|
||||
dswp.podManager.DeletePod(pod)
|
||||
|
||||
fakeRuntime.PodList = []*containertest.FakePod{
|
||||
{
|
||||
Pod: &kubecontainer.Pod{
|
||||
Name: pod.Name,
|
||||
ID: pod.UID,
|
||||
Sandboxes: []*kubecontainer.Container{
|
||||
{
|
||||
Name: "dswp-test-pod-sandbox",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
dswp.findAndRemoveDeletedPods()
|
||||
|
||||
if !dswp.pods.processedPods[podName] {
|
||||
t.Fatalf("Pod should not been removed from desired state of world since sandbox exist")
|
||||
t.Fatalf("Pod should not been removed from desired state of world since pod state still thinks it exists")
|
||||
}
|
||||
|
||||
fakeRuntime.PodList = nil
|
||||
fakePodState.removed = map[kubetypes.UID]struct{}{pod.UID: {}}
|
||||
|
||||
// fakeRuntime can not get the pod,so here findAndRemoveDeletedPods() will remove the pod and volumes it is mounted
|
||||
// the pod state is marked as removed, so here findAndRemoveDeletedPods() will remove the pod and volumes it is mounted
|
||||
dswp.findAndRemoveDeletedPods()
|
||||
if dswp.pods.processedPods[podName] {
|
||||
t.Fatalf("Failed to remove pods from desired state of world since they no longer exist")
|
||||
@ -221,7 +205,8 @@ func TestFindAndAddNewPods_FindAndRemoveDeletedPods(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestFindAndRemoveDeletedPodsWithActualState(t *testing.T) {
|
||||
dswp, _, pod, expectedVolumeName, _ := prepareDSWPWithPodPV(t)
|
||||
dswp, fakePodState, pod, expectedVolumeName, _ := prepareDSWPWithPodPV(t)
|
||||
fakeASW := dswp.actualStateOfWorld
|
||||
podName := util.GetUniquePodName(pod)
|
||||
|
||||
//let the pod be terminated
|
||||
@ -251,7 +236,19 @@ func TestFindAndRemoveDeletedPodsWithActualState(t *testing.T) {
|
||||
|
||||
// reconcile with actual state so that volume is added into the actual state
|
||||
// desired state populator now can successfully delete the pod and volume
|
||||
fakeASW := dswp.actualStateOfWorld
|
||||
reconcileASW(fakeASW, dswp.desiredStateOfWorld, t)
|
||||
dswp.findAndRemoveDeletedPods()
|
||||
if !dswp.desiredStateOfWorld.VolumeExists(expectedVolumeName) {
|
||||
t.Fatalf(
|
||||
"VolumeExists(%q) failed. Expected: <false> Actual: <%v>",
|
||||
expectedVolumeName,
|
||||
volumeExists)
|
||||
}
|
||||
|
||||
fakePodState.removed = map[kubetypes.UID]struct{}{pod.UID: {}}
|
||||
|
||||
// reconcile with actual state so that volume is added into the actual state
|
||||
// desired state populator now can successfully delete the pod and volume
|
||||
reconcileASW(fakeASW, dswp.desiredStateOfWorld, t)
|
||||
dswp.findAndRemoveDeletedPods()
|
||||
volumeExists = dswp.desiredStateOfWorld.VolumeExists(expectedVolumeName)
|
||||
@ -271,7 +268,7 @@ func TestFindAndRemoveDeletedPodsWithActualState(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestFindAndRemoveDeletedPodsWithUncertain(t *testing.T) {
|
||||
dswp, fakeRuntime, pod, expectedVolumeName, pv := prepareDSWPWithPodPV(t)
|
||||
dswp, fakePodState, pod, expectedVolumeName, pv := prepareDSWPWithPodPV(t)
|
||||
podName := util.GetUniquePodName(pod)
|
||||
|
||||
//let the pod be terminated
|
||||
@ -281,7 +278,7 @@ func TestFindAndRemoveDeletedPodsWithUncertain(t *testing.T) {
|
||||
}
|
||||
podGet.Status.Phase = v1.PodFailed
|
||||
dswp.podManager.DeletePod(pod)
|
||||
fakeRuntime.PodList = nil
|
||||
fakePodState.removed = map[kubetypes.UID]struct{}{pod.UID: {}}
|
||||
|
||||
// Add the volume to ASW by reconciling.
|
||||
fakeASW := dswp.actualStateOfWorld
|
||||
@ -302,7 +299,7 @@ func TestFindAndRemoveDeletedPodsWithUncertain(t *testing.T) {
|
||||
t.Fatalf("Failed to set the volume as uncertain: %s", err)
|
||||
}
|
||||
|
||||
// fakeRuntime can not get the pod,so here findAndRemoveDeletedPods() will remove the pod and volumes it is mounted
|
||||
// the pod state now lists the pod as removed, so here findAndRemoveDeletedPods() will remove the pod and volumes it is mounted
|
||||
dswp.findAndRemoveDeletedPods()
|
||||
if dswp.pods.processedPods[podName] {
|
||||
t.Fatalf("Failed to remove pods from desired state of world since they no longer exist")
|
||||
@ -333,7 +330,7 @@ func TestFindAndRemoveDeletedPodsWithUncertain(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func prepareDSWPWithPodPV(t *testing.T) (*desiredStateOfWorldPopulator, *containertest.FakeRuntime, *v1.Pod, v1.UniqueVolumeName, *v1.PersistentVolume) {
|
||||
func prepareDSWPWithPodPV(t *testing.T) (*desiredStateOfWorldPopulator, *fakePodStateProvider, *v1.Pod, v1.UniqueVolumeName, *v1.PersistentVolume) {
|
||||
// create dswp
|
||||
mode := v1.PersistentVolumeFilesystem
|
||||
pv := &v1.PersistentVolume{
|
||||
@ -353,7 +350,7 @@ func prepareDSWPWithPodPV(t *testing.T) (*desiredStateOfWorldPopulator, *contain
|
||||
Phase: v1.ClaimBound,
|
||||
},
|
||||
}
|
||||
dswp, fakePodManager, fakesDSW, fakeRuntime := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, fakesDSW, _, fakePodState := createDswpWithVolume(t, pv, pvc)
|
||||
|
||||
// create pod
|
||||
containers := []v1.Container{
|
||||
@ -399,7 +396,7 @@ func prepareDSWPWithPodPV(t *testing.T) (*desiredStateOfWorldPopulator, *contain
|
||||
|
||||
verifyVolumeExistsInVolumesToMount(
|
||||
t, v1.UniqueVolumeName(generatedVolumeName), false /* expectReportedInUse */, fakesDSW)
|
||||
return dswp, fakeRuntime, pod, expectedVolumeName, pv
|
||||
return dswp, fakePodState, pod, expectedVolumeName, pv
|
||||
}
|
||||
|
||||
func TestFindAndRemoveNonattachableVolumes(t *testing.T) {
|
||||
@ -424,7 +421,7 @@ func TestFindAndRemoveNonattachableVolumes(t *testing.T) {
|
||||
}
|
||||
|
||||
fakeVolumePluginMgr, fakeVolumePlugin := volumetesting.GetTestKubeletVolumePluginMgr(t)
|
||||
dswp, fakePodManager, fakesDSW, _ := createDswpWithVolumeWithCustomPluginMgr(t, pv, pvc, fakeVolumePluginMgr)
|
||||
dswp, fakePodManager, fakesDSW, _, _ := createDswpWithVolumeWithCustomPluginMgr(t, pv, pvc, fakeVolumePluginMgr)
|
||||
|
||||
// create pod
|
||||
containers := []v1.Container{
|
||||
@ -487,7 +484,7 @@ func TestEphemeralVolumeOwnerCheck(t *testing.T) {
|
||||
|
||||
// create dswp
|
||||
pod, pv, pvc := createEphemeralVolumeObjects("dswp-test-pod", "dswp-test-volume-name", false /* not owned */)
|
||||
dswp, fakePodManager, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, _, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
fakePodManager.AddPod(pod)
|
||||
|
||||
podName := util.GetUniquePodName(pod)
|
||||
@ -505,7 +502,7 @@ func TestEphemeralVolumeOwnerCheck(t *testing.T) {
|
||||
func TestEphemeralVolumeEnablement(t *testing.T) {
|
||||
// create dswp
|
||||
pod, pv, pvc := createEphemeralVolumeObjects("dswp-test-pod", "dswp-test-volume-name", true /* owned */)
|
||||
dswp, fakePodManager, fakesDSW, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, fakesDSW, _, fakePodState := createDswpWithVolume(t, pv, pvc)
|
||||
fakePodManager.AddPod(pod)
|
||||
|
||||
podName := util.GetUniquePodName(pod)
|
||||
@ -555,7 +552,7 @@ func TestEphemeralVolumeEnablement(t *testing.T) {
|
||||
if !exist {
|
||||
t.Fatalf("Failed to get pod by pod name: %s and namespace: %s", pod.Name, pod.Namespace)
|
||||
}
|
||||
podGet.Status.Phase = v1.PodFailed
|
||||
fakePodState.removed = map[kubetypes.UID]struct{}{podGet.UID: {}}
|
||||
|
||||
// Pretend again that the feature is disabled.
|
||||
// Removal of the pod and volumes is expected to work.
|
||||
@ -620,7 +617,7 @@ func TestFindAndAddNewPods_FindAndRemoveDeletedPods_Valid_Block_VolumeDevices(t
|
||||
Phase: v1.ClaimBound,
|
||||
},
|
||||
}
|
||||
dswp, fakePodManager, fakesDSW, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, fakesDSW, _, fakePodState := createDswpWithVolume(t, pv, pvc)
|
||||
|
||||
// create pod
|
||||
containers := []v1.Container{
|
||||
@ -674,9 +671,10 @@ func TestFindAndAddNewPods_FindAndRemoveDeletedPods_Valid_Block_VolumeDevices(t
|
||||
}
|
||||
podGet.Status.Phase = v1.PodFailed
|
||||
fakePodManager.DeletePod(pod)
|
||||
//pod is added to fakePodManager but fakeRuntime can not get the pod,so here findAndRemoveDeletedPods() will remove the pod and volumes it is mounted
|
||||
dswp.findAndRemoveDeletedPods()
|
||||
fakePodState.removed = map[kubetypes.UID]struct{}{pod.UID: {}}
|
||||
|
||||
//pod is added to fakePodManager but pod state knows the pod is removed, so here findAndRemoveDeletedPods() will remove the pod and volumes it is mounted
|
||||
dswp.findAndRemoveDeletedPods()
|
||||
if dswp.pods.processedPods[podName] {
|
||||
t.Fatalf("Failed to remove pods from desired state of world since they no longer exist")
|
||||
}
|
||||
@ -727,7 +725,7 @@ func TestCreateVolumeSpec_Valid_File_VolumeMounts(t *testing.T) {
|
||||
Phase: v1.ClaimBound,
|
||||
},
|
||||
}
|
||||
dswp, fakePodManager, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, _, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
|
||||
// create pod
|
||||
containers := []v1.Container{
|
||||
@ -773,7 +771,7 @@ func TestCreateVolumeSpec_Valid_Nil_VolumeMounts(t *testing.T) {
|
||||
Phase: v1.ClaimBound,
|
||||
},
|
||||
}
|
||||
dswp, fakePodManager, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, _, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
|
||||
// create pod
|
||||
containers := []v1.Container{
|
||||
@ -819,7 +817,7 @@ func TestCreateVolumeSpec_Valid_Block_VolumeDevices(t *testing.T) {
|
||||
Phase: v1.ClaimBound,
|
||||
},
|
||||
}
|
||||
dswp, fakePodManager, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, _, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
|
||||
// create pod
|
||||
containers := []v1.Container{
|
||||
@ -865,7 +863,7 @@ func TestCreateVolumeSpec_Invalid_File_VolumeDevices(t *testing.T) {
|
||||
Phase: v1.ClaimBound,
|
||||
},
|
||||
}
|
||||
dswp, fakePodManager, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, _, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
|
||||
// create pod
|
||||
containers := []v1.Container{
|
||||
@ -911,7 +909,7 @@ func TestCreateVolumeSpec_Invalid_Block_VolumeMounts(t *testing.T) {
|
||||
Phase: v1.ClaimBound,
|
||||
},
|
||||
}
|
||||
dswp, fakePodManager, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, _, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
|
||||
// create pod
|
||||
containers := []v1.Container{
|
||||
@ -1068,7 +1066,7 @@ func TestCheckVolumeFSResize(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
dswp, fakePodManager, fakeDSW, _ := createDswpWithVolume(t, pv, pvc)
|
||||
dswp, fakePodManager, fakeDSW, _, _ := createDswpWithVolume(t, pv, pvc)
|
||||
fakeASW := dswp.actualStateOfWorld
|
||||
containers := []v1.Container{}
|
||||
|
||||
@ -1290,14 +1288,29 @@ func createEphemeralVolumeObjects(podName, volumeName string, owned bool) (pod *
|
||||
return
|
||||
}
|
||||
|
||||
func createDswpWithVolume(t *testing.T, pv *v1.PersistentVolume, pvc *v1.PersistentVolumeClaim) (*desiredStateOfWorldPopulator, kubepod.Manager, cache.DesiredStateOfWorld, *containertest.FakeRuntime) {
|
||||
func createDswpWithVolume(t *testing.T, pv *v1.PersistentVolume, pvc *v1.PersistentVolumeClaim) (*desiredStateOfWorldPopulator, kubepod.Manager, cache.DesiredStateOfWorld, *containertest.FakeRuntime, *fakePodStateProvider) {
|
||||
fakeVolumePluginMgr, _ := volumetesting.GetTestKubeletVolumePluginMgr(t)
|
||||
dswp, fakePodManager, fakesDSW, fakeRuntime := createDswpWithVolumeWithCustomPluginMgr(t, pv, pvc, fakeVolumePluginMgr)
|
||||
return dswp, fakePodManager, fakesDSW, fakeRuntime
|
||||
dswp, fakePodManager, fakesDSW, fakeRuntime, fakeStateProvider := createDswpWithVolumeWithCustomPluginMgr(t, pv, pvc, fakeVolumePluginMgr)
|
||||
return dswp, fakePodManager, fakesDSW, fakeRuntime, fakeStateProvider
|
||||
}
|
||||
|
||||
type fakePodStateProvider struct {
|
||||
terminating map[kubetypes.UID]struct{}
|
||||
removed map[kubetypes.UID]struct{}
|
||||
}
|
||||
|
||||
func (p *fakePodStateProvider) ShouldPodContainersBeTerminating(uid kubetypes.UID) bool {
|
||||
_, ok := p.terminating[uid]
|
||||
return ok
|
||||
}
|
||||
|
||||
func (p *fakePodStateProvider) ShouldPodRuntimeBeRemoved(uid kubetypes.UID) bool {
|
||||
_, ok := p.removed[uid]
|
||||
return ok
|
||||
}
|
||||
|
||||
func createDswpWithVolumeWithCustomPluginMgr(t *testing.T, pv *v1.PersistentVolume, pvc *v1.PersistentVolumeClaim,
|
||||
fakeVolumePluginMgr *volume.VolumePluginMgr) (*desiredStateOfWorldPopulator, kubepod.Manager, cache.DesiredStateOfWorld, *containertest.FakeRuntime) {
|
||||
fakeVolumePluginMgr *volume.VolumePluginMgr) (*desiredStateOfWorldPopulator, kubepod.Manager, cache.DesiredStateOfWorld, *containertest.FakeRuntime, *fakePodStateProvider) {
|
||||
fakeClient := &fake.Clientset{}
|
||||
fakeClient.AddReactor("get", "persistentvolumeclaims", func(action core.Action) (bool, runtime.Object, error) {
|
||||
return true, pvc, nil
|
||||
@ -1314,8 +1327,7 @@ func createDswpWithVolumeWithCustomPluginMgr(t *testing.T, pv *v1.PersistentVolu
|
||||
fakesDSW := cache.NewDesiredStateOfWorld(fakeVolumePluginMgr)
|
||||
fakeASW := cache.NewActualStateOfWorld("fake", fakeVolumePluginMgr)
|
||||
fakeRuntime := &containertest.FakeRuntime{}
|
||||
|
||||
fakeStatusManager := status.NewManager(fakeClient, fakePodManager, &statustest.FakePodDeletionSafetyProvider{})
|
||||
fakeStateProvider := &fakePodStateProvider{}
|
||||
|
||||
csiTranslator := csitrans.New()
|
||||
dswp := &desiredStateOfWorldPopulator{
|
||||
@ -1323,7 +1335,7 @@ func createDswpWithVolumeWithCustomPluginMgr(t *testing.T, pv *v1.PersistentVolu
|
||||
loopSleepDuration: 100 * time.Millisecond,
|
||||
getPodStatusRetryDuration: 2 * time.Second,
|
||||
podManager: fakePodManager,
|
||||
podStatusProvider: fakeStatusManager,
|
||||
podStateProvider: fakeStateProvider,
|
||||
desiredStateOfWorld: fakesDSW,
|
||||
actualStateOfWorld: fakeASW,
|
||||
pods: processedPods{
|
||||
@ -1334,5 +1346,5 @@ func createDswpWithVolumeWithCustomPluginMgr(t *testing.T, pv *v1.PersistentVolu
|
||||
intreeToCSITranslator: csiTranslator,
|
||||
volumePluginMgr: fakeVolumePluginMgr,
|
||||
}
|
||||
return dswp, fakePodManager, fakesDSW, fakeRuntime
|
||||
return dswp, fakePodManager, fakesDSW, fakeRuntime, fakeStateProvider
|
||||
}
|
||||
|
@ -39,7 +39,6 @@ import (
|
||||
"k8s.io/kubernetes/pkg/kubelet/config"
|
||||
"k8s.io/kubernetes/pkg/kubelet/container"
|
||||
"k8s.io/kubernetes/pkg/kubelet/pod"
|
||||
"k8s.io/kubernetes/pkg/kubelet/status"
|
||||
"k8s.io/kubernetes/pkg/kubelet/volumemanager/cache"
|
||||
"k8s.io/kubernetes/pkg/kubelet/volumemanager/metrics"
|
||||
"k8s.io/kubernetes/pkg/kubelet/volumemanager/populator"
|
||||
@ -106,6 +105,13 @@ type VolumeManager interface {
|
||||
// the duration defined in podAttachAndMountTimeout.
|
||||
WaitForAttachAndMount(pod *v1.Pod) error
|
||||
|
||||
// WaitForUnmount processes the volumes referenced in the specified
|
||||
// pod and blocks until they are all unmounted (reflected in the actual
|
||||
// state of the world).
|
||||
// An error is returned if all volumes are not unmounted within
|
||||
// the duration defined in podAttachAndMountTimeout.
|
||||
WaitForUnmount(pod *v1.Pod) error
|
||||
|
||||
// GetMountedVolumesForPod 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.
|
||||
@ -149,6 +155,12 @@ type VolumeManager interface {
|
||||
MarkVolumesAsReportedInUse(volumesReportedAsInUse []v1.UniqueVolumeName)
|
||||
}
|
||||
|
||||
// podStateProvider can determine if a pod is is going to be terminated
|
||||
type podStateProvider interface {
|
||||
ShouldPodContainersBeTerminating(k8stypes.UID) bool
|
||||
ShouldPodRuntimeBeRemoved(k8stypes.UID) bool
|
||||
}
|
||||
|
||||
// NewVolumeManager returns a new concrete instance implementing the
|
||||
// VolumeManager interface.
|
||||
//
|
||||
@ -160,7 +172,7 @@ func NewVolumeManager(
|
||||
controllerAttachDetachEnabled bool,
|
||||
nodeName k8stypes.NodeName,
|
||||
podManager pod.Manager,
|
||||
podStatusProvider status.PodStatusProvider,
|
||||
podStateProvider podStateProvider,
|
||||
kubeClient clientset.Interface,
|
||||
volumePluginMgr *volume.VolumePluginMgr,
|
||||
kubeContainerRuntime container.Runtime,
|
||||
@ -195,7 +207,7 @@ func NewVolumeManager(
|
||||
desiredStateOfWorldPopulatorLoopSleepPeriod,
|
||||
desiredStateOfWorldPopulatorGetPodStatusRetryDuration,
|
||||
podManager,
|
||||
podStatusProvider,
|
||||
podStateProvider,
|
||||
vm.desiredStateOfWorld,
|
||||
vm.actualStateOfWorld,
|
||||
kubeContainerRuntime,
|
||||
@ -426,6 +438,42 @@ func (vm *volumeManager) WaitForAttachAndMount(pod *v1.Pod) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (vm *volumeManager) WaitForUnmount(pod *v1.Pod) error {
|
||||
if pod == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
klog.V(3).InfoS("Waiting for volumes to unmount for pod", "pod", klog.KObj(pod))
|
||||
uniquePodName := util.GetUniquePodName(pod)
|
||||
|
||||
vm.desiredStateOfWorldPopulator.ReprocessPod(uniquePodName)
|
||||
|
||||
err := wait.PollImmediate(
|
||||
podAttachAndMountRetryInterval,
|
||||
podAttachAndMountTimeout,
|
||||
vm.verifyVolumesUnmountedFunc(uniquePodName))
|
||||
|
||||
if err != nil {
|
||||
var mountedVolumes []string
|
||||
for _, v := range vm.actualStateOfWorld.GetMountedVolumesForPod(uniquePodName) {
|
||||
mountedVolumes = append(mountedVolumes, v.OuterVolumeSpecName)
|
||||
}
|
||||
sort.Strings(mountedVolumes)
|
||||
|
||||
if len(mountedVolumes) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
return fmt.Errorf(
|
||||
"mounted volumes=%v: %s",
|
||||
mountedVolumes,
|
||||
err)
|
||||
}
|
||||
|
||||
klog.V(3).InfoS("All volumes are unmounted for pod", "pod", klog.KObj(pod))
|
||||
return nil
|
||||
}
|
||||
|
||||
// getUnattachedVolumes returns a list of the volumes that are expected to be attached but
|
||||
// are not currently attached to the node
|
||||
func (vm *volumeManager) getUnattachedVolumes(expectedVolumes []string) []string {
|
||||
@ -449,6 +497,17 @@ func (vm *volumeManager) verifyVolumesMountedFunc(podName types.UniquePodName, e
|
||||
}
|
||||
}
|
||||
|
||||
// verifyVolumesUnmountedFunc returns a method that is true when there are no mounted volumes for this
|
||||
// pod.
|
||||
func (vm *volumeManager) verifyVolumesUnmountedFunc(podName types.UniquePodName) wait.ConditionFunc {
|
||||
return func() (done bool, err error) {
|
||||
if errs := vm.desiredStateOfWorld.PopPodErrors(podName); len(errs) > 0 {
|
||||
return true, errors.New(strings.Join(errs, "; "))
|
||||
}
|
||||
return len(vm.actualStateOfWorld.GetMountedVolumesForPod(podName)) == 0, nil
|
||||
}
|
||||
}
|
||||
|
||||
// getUnmountedVolumes fetches the current list of mounted volumes from
|
||||
// the actual state of the world, and uses it to process the list of
|
||||
// expectedVolumes. It returns a list of unmounted volumes.
|
||||
|
@ -50,6 +50,11 @@ func (f *FakeVolumeManager) WaitForAttachAndMount(pod *v1.Pod) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// WaitForUnmount is not implemented
|
||||
func (f *FakeVolumeManager) WaitForUnmount(pod *v1.Pod) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetMountedVolumesForPod is not implemented
|
||||
func (f *FakeVolumeManager) GetMountedVolumesForPod(podName types.UniquePodName) container.VolumeMap {
|
||||
return nil
|
||||
|
@ -28,6 +28,7 @@ import (
|
||||
|
||||
v1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
kubetypes "k8s.io/apimachinery/pkg/types"
|
||||
"k8s.io/apimachinery/pkg/util/sets"
|
||||
"k8s.io/apimachinery/pkg/util/wait"
|
||||
clientset "k8s.io/client-go/kubernetes"
|
||||
@ -40,8 +41,6 @@ import (
|
||||
kubepod "k8s.io/kubernetes/pkg/kubelet/pod"
|
||||
podtest "k8s.io/kubernetes/pkg/kubelet/pod/testing"
|
||||
"k8s.io/kubernetes/pkg/kubelet/secret"
|
||||
"k8s.io/kubernetes/pkg/kubelet/status"
|
||||
statustest "k8s.io/kubernetes/pkg/kubelet/status/testing"
|
||||
"k8s.io/kubernetes/pkg/volume"
|
||||
volumetest "k8s.io/kubernetes/pkg/volume/testing"
|
||||
"k8s.io/kubernetes/pkg/volume/util"
|
||||
@ -269,19 +268,34 @@ func TestGetExtraSupplementalGroupsForPod(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
type fakePodStateProvider struct {
|
||||
shouldRemove map[kubetypes.UID]struct{}
|
||||
terminating map[kubetypes.UID]struct{}
|
||||
}
|
||||
|
||||
func (p *fakePodStateProvider) ShouldPodRuntimeBeRemoved(uid kubetypes.UID) bool {
|
||||
_, ok := p.shouldRemove[uid]
|
||||
return ok
|
||||
}
|
||||
|
||||
func (p *fakePodStateProvider) ShouldPodContainersBeTerminating(uid kubetypes.UID) bool {
|
||||
_, ok := p.terminating[uid]
|
||||
return ok
|
||||
}
|
||||
|
||||
func newTestVolumeManager(t *testing.T, tmpDir string, podManager kubepod.Manager, kubeClient clientset.Interface) VolumeManager {
|
||||
plug := &volumetest.FakeVolumePlugin{PluginName: "fake", Host: nil}
|
||||
fakeRecorder := &record.FakeRecorder{}
|
||||
plugMgr := &volume.VolumePluginMgr{}
|
||||
// TODO (#51147) inject mock prober
|
||||
plugMgr.InitPlugins([]volume.VolumePlugin{plug}, nil /* prober */, volumetest.NewFakeKubeletVolumeHost(t, tmpDir, kubeClient, nil))
|
||||
statusManager := status.NewManager(kubeClient, podManager, &statustest.FakePodDeletionSafetyProvider{})
|
||||
stateProvider := &fakePodStateProvider{}
|
||||
fakePathHandler := volumetest.NewBlockVolumePathHandler()
|
||||
vm := NewVolumeManager(
|
||||
true,
|
||||
testHostname,
|
||||
podManager,
|
||||
statusManager,
|
||||
stateProvider,
|
||||
kubeClient,
|
||||
plugMgr,
|
||||
&containertest.FakeRuntime{},
|
||||
|
@ -381,7 +381,10 @@ func GetUniqueVolumeNameFromSpec(
|
||||
|
||||
// IsPodTerminated checks if pod is terminated
|
||||
func IsPodTerminated(pod *v1.Pod, podStatus v1.PodStatus) bool {
|
||||
return podStatus.Phase == v1.PodFailed || podStatus.Phase == v1.PodSucceeded || (pod.DeletionTimestamp != nil && notRunning(podStatus.ContainerStatuses))
|
||||
// TODO: the guarantees provided by kubelet status are not sufficient to guarantee it's safe to ignore a deleted pod,
|
||||
// even if everything is notRunning (kubelet does not guarantee that when pod status is waiting that it isn't trying
|
||||
// to start a container).
|
||||
return podStatus.Phase == v1.PodFailed || podStatus.Phase == v1.PodSucceeded || (pod.DeletionTimestamp != nil && notRunning(podStatus.InitContainerStatuses) && notRunning(podStatus.ContainerStatuses) && notRunning(podStatus.EphemeralContainerStatuses))
|
||||
}
|
||||
|
||||
// notRunning returns true if every status is terminated or waiting, or the status list
|
||||
|
@ -756,6 +756,10 @@ var _ = SIGDescribe("StatefulSet", func() {
|
||||
}
|
||||
pod, err = f.ClientSet.CoreV1().Pods(f.Namespace.Name).Create(context.TODO(), pod, metav1.CreateOptions{})
|
||||
framework.ExpectNoError(err)
|
||||
ginkgo.By("Waiting until pod " + podName + " will start running in namespace " + f.Namespace.Name)
|
||||
if err := e2epod.WaitForPodNameRunningInNamespace(f.ClientSet, podName, f.Namespace.Name); err != nil {
|
||||
framework.Failf("Pod %v did not start running: %v", podName, err)
|
||||
}
|
||||
|
||||
ginkgo.By("Creating statefulset with conflicting port in namespace " + f.Namespace.Name)
|
||||
ss := e2estatefulset.NewStatefulSet(ssName, f.Namespace.Name, headlessSvcName, 1, nil, nil, labels)
|
||||
@ -765,11 +769,6 @@ var _ = SIGDescribe("StatefulSet", func() {
|
||||
_, err = f.ClientSet.AppsV1().StatefulSets(f.Namespace.Name).Create(context.TODO(), ss, metav1.CreateOptions{})
|
||||
framework.ExpectNoError(err)
|
||||
|
||||
ginkgo.By("Waiting until pod " + podName + " will start running in namespace " + f.Namespace.Name)
|
||||
if err := e2epod.WaitForPodNameRunningInNamespace(f.ClientSet, podName, f.Namespace.Name); err != nil {
|
||||
framework.Failf("Pod %v did not start running: %v", podName, err)
|
||||
}
|
||||
|
||||
var initialStatefulPodUID types.UID
|
||||
ginkgo.By("Waiting until stateful pod " + statefulPodName + " will be recreated and deleted at least once in namespace " + f.Namespace.Name)
|
||||
|
||||
|
@ -387,6 +387,8 @@ var _ = SIGDescribe("Pods Extended", func() {
|
||||
// pod volume teardown races with container start in CRI, which reports a failure
|
||||
framework.Logf("pod %s on node %s failed with the symptoms of https://github.com/kubernetes/kubernetes/issues/88766", pod.Name, pod.Spec.NodeName)
|
||||
default:
|
||||
data, _ := json.MarshalIndent(pod.Status, "", " ")
|
||||
framework.Logf("pod %s on node %s had incorrect final status:\n%s", string(data))
|
||||
return fmt.Errorf("pod %s on node %s container unexpected exit code %d: start=%s end=%s reason=%s message=%s", pod.Name, pod.Spec.NodeName, t.ExitCode, t.StartedAt, t.FinishedAt, t.Reason, t.Message)
|
||||
}
|
||||
switch {
|
||||
|
Loading…
Reference in New Issue
Block a user