sidecars: terminate sidecars after main containers

Sidecars should terminate:
- after all main containers have exited
- serialized and in reverse order
This commit is contained in:
Todd Neal 2023-09-12 16:26:27 -05:00
parent 6d70013af5
commit 7bcc98c46b
8 changed files with 928 additions and 17 deletions

View File

@ -289,7 +289,7 @@ func (m *kubeGenericRuntimeManager) startContainer(ctx context.Context, podSandb
"podUID", pod.UID, "containerName", container.Name, "containerID", kubeContainerID.String())
// do not record the message in the event so that secrets won't leak from the server.
m.recordContainerEvent(pod, container, kubeContainerID.ID, v1.EventTypeWarning, events.FailedPostStartHook, "PostStartHook failed")
if err := m.killContainer(ctx, pod, kubeContainerID, container.Name, "FailedPostStartHook", reasonFailedPostStartHook, nil); err != nil {
if err := m.killContainer(ctx, pod, kubeContainerID, container.Name, "FailedPostStartHook", reasonFailedPostStartHook, nil, nil); err != nil {
klog.ErrorS(err, "Failed to kill container", "pod", klog.KObj(pod),
"podUID", pod.UID, "containerName", container.Name, "containerID", kubeContainerID.String())
}
@ -703,7 +703,7 @@ func (m *kubeGenericRuntimeManager) restoreSpecsFromContainerLabels(ctx context.
// killContainer kills a container through the following steps:
// * Run the pre-stop lifecycle hooks (if applicable).
// * Stop the container.
func (m *kubeGenericRuntimeManager) killContainer(ctx context.Context, pod *v1.Pod, containerID kubecontainer.ContainerID, containerName string, message string, reason containerKillReason, gracePeriodOverride *int64) error {
func (m *kubeGenericRuntimeManager) killContainer(ctx context.Context, pod *v1.Pod, containerID kubecontainer.ContainerID, containerName string, message string, reason containerKillReason, gracePeriodOverride *int64, ordering *terminationOrdering) error {
var containerSpec *v1.Container
if pod != nil {
if containerSpec = kubecontainer.GetContainerSpec(pod, containerName); containerSpec == nil {
@ -738,6 +738,13 @@ func (m *kubeGenericRuntimeManager) killContainer(ctx context.Context, pod *v1.P
gracePeriod = gracePeriod - m.executePreStopHook(ctx, pod, containerID, containerSpec, gracePeriod)
}
// if we care about termination ordering, then wait for this container's turn to exit if there is
// time remaining
if ordering != nil && gracePeriod > 0 {
// grace period is only in seconds, so the time we've waited gets truncated downward
gracePeriod -= int64(ordering.waitForTurn(containerName, gracePeriod))
}
// always give containers a minimal shutdown window to avoid unnecessary SIGKILLs
if gracePeriod < minimumGracePeriodInSeconds {
gracePeriod = minimumGracePeriodInSeconds
@ -755,6 +762,10 @@ func (m *kubeGenericRuntimeManager) killContainer(ctx context.Context, pod *v1.P
klog.V(3).InfoS("Container exited normally", "pod", klog.KObj(pod), "podUID", pod.UID,
"containerName", containerName, "containerID", containerID.String())
if ordering != nil {
ordering.containerTerminated(containerName)
}
return nil
}
@ -764,13 +775,22 @@ func (m *kubeGenericRuntimeManager) killContainersWithSyncResult(ctx context.Con
wg := sync.WaitGroup{}
wg.Add(len(runningPod.Containers))
var termOrdering *terminationOrdering
// we only care about container termination ordering if the sidecars feature is enabled
if utilfeature.DefaultFeatureGate.Enabled(features.SidecarContainers) {
var runningContainerNames []string
for _, container := range runningPod.Containers {
runningContainerNames = append(runningContainerNames, container.Name)
}
termOrdering = newTerminationOrdering(pod, runningContainerNames)
}
for _, container := range runningPod.Containers {
go func(container *kubecontainer.Container) {
defer utilruntime.HandleCrash()
defer wg.Done()
killContainerResult := kubecontainer.NewSyncResult(kubecontainer.KillContainer, container.Name)
if err := m.killContainer(ctx, pod, container.ID, container.Name, "", reasonUnknown, gracePeriodOverride); err != nil {
if err := m.killContainer(ctx, pod, container.ID, container.Name, "", reasonUnknown, gracePeriodOverride, termOrdering); err != nil {
killContainerResult.Fail(kubecontainer.ErrKillContainer, err.Error())
// Use runningPod for logging as the pod passed in could be *nil*.
klog.ErrorS(err, "Kill container failed", "pod", klog.KRef(runningPod.Namespace, runningPod.Name), "podUID", runningPod.ID,

View File

@ -38,6 +38,7 @@ import (
v1 "k8s.io/api/core/v1"
runtimeapi "k8s.io/cri-api/pkg/apis/runtime/v1"
"k8s.io/kubernetes/pkg/features"
kubecontainer "k8s.io/kubernetes/pkg/kubelet/container"
containertest "k8s.io/kubernetes/pkg/kubelet/container/testing"
@ -128,7 +129,7 @@ func TestKillContainer(t *testing.T) {
for _, test := range tests {
ctx := context.Background()
err := m.killContainer(ctx, test.pod, test.containerID, test.containerName, test.reason, "", &test.gracePeriodOverride)
err := m.killContainer(ctx, test.pod, test.containerID, test.containerName, test.reason, "", &test.gracePeriodOverride, nil)
if test.succeed != (err == nil) {
t.Errorf("%s: expected %v, got %v (%v)", test.caseName, test.succeed, (err == nil), err)
}
@ -402,7 +403,7 @@ func testLifeCycleHook(t *testing.T, testPod *v1.Pod, testContainer *v1.Containe
t.Run("PreStop-CMDExec", func(t *testing.T) {
ctx := context.Background()
testContainer.Lifecycle = cmdLifeCycle
m.killContainer(ctx, testPod, cID, "foo", "testKill", "", &gracePeriod)
_ = m.killContainer(ctx, testPod, cID, "foo", "testKill", "", &gracePeriod, nil)
if fakeRunner.Cmd[0] != cmdLifeCycle.PreStop.Exec.Command[0] {
t.Errorf("CMD Prestop hook was not invoked")
}
@ -416,8 +417,7 @@ func testLifeCycleHook(t *testing.T, testPod *v1.Pod, testContainer *v1.Containe
defer featuregatetesting.SetFeatureGateDuringTest(t, utilfeature.DefaultFeatureGate, features.ConsistentHTTPGetHandlers, false)()
httpLifeCycle.PreStop.HTTPGet.Port = intstr.IntOrString{}
testContainer.Lifecycle = httpLifeCycle
m.killContainer(ctx, testPod, cID, "foo", "testKill", "", &gracePeriod)
_ = m.killContainer(ctx, testPod, cID, "foo", "testKill", "", &gracePeriod, nil)
if fakeHTTP.req == nil || !strings.Contains(fakeHTTP.req.URL.String(), httpLifeCycle.PreStop.HTTPGet.Host) {
t.Errorf("HTTP Prestop hook was not invoked")
}
@ -427,8 +427,7 @@ func testLifeCycleHook(t *testing.T, testPod *v1.Pod, testContainer *v1.Containe
defer func() { fakeHTTP.req = nil }()
httpLifeCycle.PreStop.HTTPGet.Port = intstr.FromInt32(80)
testContainer.Lifecycle = httpLifeCycle
m.killContainer(ctx, testPod, cID, "foo", "testKill", "", &gracePeriod)
_ = m.killContainer(ctx, testPod, cID, "foo", "testKill", "", &gracePeriod, nil)
if fakeHTTP.req == nil || !strings.Contains(fakeHTTP.req.URL.String(), httpLifeCycle.PreStop.HTTPGet.Host) {
t.Errorf("HTTP Prestop hook was not invoked")
}
@ -443,8 +442,7 @@ func testLifeCycleHook(t *testing.T, testPod *v1.Pod, testContainer *v1.Containe
testPod.DeletionGracePeriodSeconds = &gracePeriodLocal
testPod.Spec.TerminationGracePeriodSeconds = &gracePeriodLocal
m.killContainer(ctx, testPod, cID, "foo", "testKill", "", &gracePeriodLocal)
_ = m.killContainer(ctx, testPod, cID, "foo", "testKill", "", &gracePeriodLocal, nil)
if fakeHTTP.req != nil {
t.Errorf("HTTP Prestop hook Should not execute when gracePeriod is 0")
}

View File

@ -141,7 +141,7 @@ func (cgc *containerGC) removeOldestN(ctx context.Context, containers []containe
ID: containers[i].id,
}
message := "Container is in unknown state, try killing it before removal"
if err := cgc.manager.killContainer(ctx, nil, id, containers[i].name, message, reasonUnknown, nil); err != nil {
if err := cgc.manager.killContainer(ctx, nil, id, containers[i].name, message, reasonUnknown, nil, nil); err != nil {
klog.ErrorS(err, "Failed to stop container", "containerID", containers[i].id)
continue
}

View File

@ -44,6 +44,7 @@ import (
"k8s.io/component-base/logs/logreduction"
internalapi "k8s.io/cri-api/pkg/apis"
runtimeapi "k8s.io/cri-api/pkg/apis/runtime/v1"
"k8s.io/kubernetes/pkg/api/legacyscheme"
podutil "k8s.io/kubernetes/pkg/api/v1/pod"
"k8s.io/kubernetes/pkg/credentialprovider"
@ -1090,7 +1091,7 @@ func (m *kubeGenericRuntimeManager) SyncPod(ctx context.Context, pod *v1.Pod, po
klog.V(3).InfoS("Killing unwanted container for pod", "containerName", containerInfo.name, "containerID", containerID, "pod", klog.KObj(pod))
killContainerResult := kubecontainer.NewSyncResult(kubecontainer.KillContainer, containerInfo.name)
result.AddSyncResult(killContainerResult)
if err := m.killContainer(ctx, pod, containerID, containerInfo.name, containerInfo.message, containerInfo.reason, nil); err != nil {
if err := m.killContainer(ctx, pod, containerID, containerInfo.name, containerInfo.message, containerInfo.reason, nil, nil); err != nil {
killContainerResult.Fail(kubecontainer.ErrKillContainer, err.Error())
klog.ErrorS(err, "killContainer for pod failed", "containerName", containerInfo.name, "containerID", containerID, "pod", klog.KObj(pod))
return

View File

@ -0,0 +1,114 @@
/*
Copyright 2023 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package kuberuntime
import (
"time"
v1 "k8s.io/api/core/v1"
"k8s.io/kubernetes/pkg/kubelet/types"
)
// terminationOrdering is used to enforce a termination ordering for sidecar containers. It sets up
// dependencies between sidecars and allows the pod termination process to wait until the grace period
// expires, or all dependent containers have finished terminating.
type terminationOrdering struct {
// terminated is a map from container name to a channel, that if closed
// indicates that the container with that name was terminated
terminated map[string]chan struct{}
// prereqs is a map from container name to a list of channel that the container
// must wait on to ensure termination ordering
prereqs map[string][]chan struct{}
}
// newTerminationOrdering constructs a terminationOrdering based on the pod spec and the currently running containers.
func newTerminationOrdering(pod *v1.Pod, runningContainerNames []string) *terminationOrdering {
to := &terminationOrdering{
prereqs: map[string][]chan struct{}{},
terminated: map[string]chan struct{}{},
}
runningContainers := map[string]struct{}{}
for _, name := range runningContainerNames {
runningContainers[name] = struct{}{}
}
var mainContainerChannels []chan struct{}
// sidecar containers need to wait on main containers, so we create a channel per main container
// for them to wait on
for _, c := range pod.Spec.Containers {
channel := make(chan struct{})
to.terminated[c.Name] = channel
mainContainerChannels = append(mainContainerChannels, channel)
// if its not a running container, pre-close the channel so nothing waits on it
if _, isRunning := runningContainers[c.Name]; !isRunning {
close(channel)
}
}
var previousSidecarName string
for i := range pod.Spec.InitContainers {
// get the init containers in reverse order
ic := pod.Spec.InitContainers[len(pod.Spec.InitContainers)-i-1]
to.terminated[ic.Name] = make(chan struct{})
if types.IsRestartableInitContainer(&ic) {
// sidecars need to wait for all main containers to exit
to.prereqs[ic.Name] = append(to.prereqs[ic.Name], mainContainerChannels...)
// if there is a later sidecar, this container needs to wait for it to finish
if previousSidecarName != "" {
to.prereqs[ic.Name] = append(to.prereqs[ic.Name], to.terminated[previousSidecarName])
}
previousSidecarName = ic.Name
}
}
return to
}
// waitForTurn waits until it is time for the container with the specified name to begin terminating, up until
// the specified grace period. If gracePeriod = 0, there is no wait.
func (o *terminationOrdering) waitForTurn(name string, gracePeriod int64) float64 {
// if there is no grace period, we don't wait
if gracePeriod <= 0 {
return 0
}
start := time.Now()
remainingGrace := time.NewTimer(time.Duration(gracePeriod) * time.Second)
for _, c := range o.prereqs[name] {
select {
case <-c:
case <-remainingGrace.C:
// grace period expired, so immediately exit
return time.Since(start).Seconds()
}
}
return time.Since(start).Seconds()
}
// containerTerminated should be called once the container with the speecified name has exited.
func (o *terminationOrdering) containerTerminated(name string) {
if ch, ok := o.terminated[name]; ok {
close(ch)
}
}

View File

@ -0,0 +1,292 @@
/*
Copyright 2023 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package kuberuntime
import (
"sync"
"testing"
"time"
v1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
)
func TestTerminationOrderingSidecarStopAfterMain(t *testing.T) {
restartPolicy := v1.ContainerRestartPolicyAlways
pod := &v1.Pod{
ObjectMeta: metav1.ObjectMeta{
UID: "12345678",
Name: "bar",
Namespace: "new",
},
Spec: v1.PodSpec{
InitContainers: []v1.Container{
{
Name: "init",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
RestartPolicy: &restartPolicy,
},
},
Containers: []v1.Container{
{
Name: "main",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
},
},
},
}
to := newTerminationOrdering(pod, getContainerNames(pod))
var wg sync.WaitGroup
wg.Add(1)
var sidecarWaitDelay int64
var mainWaitDelay int64
go func() {
sidecarWaitDelay = int64(to.waitForTurn("init", 30))
to.containerTerminated("init")
wg.Done()
}()
wg.Add(1)
go func() {
mainWaitDelay = int64(to.waitForTurn("main", 0))
time.Sleep(1 * time.Second)
to.containerTerminated("main")
wg.Done()
}()
wg.Wait()
if sidecarWaitDelay != 1 {
t.Errorf("expected sidecar to wait for main container to exit, got delay of %d", sidecarWaitDelay)
}
if mainWaitDelay != 0 {
t.Errorf("expected main container to not wait to exit, got delay of %d", mainWaitDelay)
}
}
func TestTerminationOrderingSidecarsInReverseOrder(t *testing.T) {
restartPolicy := v1.ContainerRestartPolicyAlways
pod := &v1.Pod{
ObjectMeta: metav1.ObjectMeta{
UID: "12345678",
Name: "bar",
Namespace: "new",
},
Spec: v1.PodSpec{
InitContainers: []v1.Container{
{
Name: "sc1",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
RestartPolicy: &restartPolicy,
},
{
Name: "sc2",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
RestartPolicy: &restartPolicy,
},
{
Name: "sc3",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
RestartPolicy: &restartPolicy,
},
},
Containers: []v1.Container{
{
Name: "main",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
},
},
},
}
to := newTerminationOrdering(pod, getContainerNames(pod))
var wg sync.WaitGroup
var delays sync.Map
waitAndExit := func(name string) {
delay := int64(to.waitForTurn(name, 30))
delays.Store(name, delay)
time.Sleep(1 * time.Second)
to.containerTerminated(name)
wg.Done()
}
for _, ic := range pod.Spec.InitContainers {
wg.Add(1)
go waitAndExit(ic.Name)
}
for _, c := range pod.Spec.Containers {
wg.Add(1)
go waitAndExit(c.Name)
}
// wait for our simulated containers to exit
wg.Wait()
getDelay := func(name string) int64 {
delay, ok := delays.Load(name)
if !ok {
t.Errorf("unable to find delay for container %s", name)
}
return delay.(int64)
}
for _, tc := range []struct {
containerName string
expectedDelay int64
}{
// sidecars should exit in reverse order, so
// sc1 = 3 (main container + sc3 + sc2)
{
containerName: "sc1",
expectedDelay: 3,
},
// sc2 = 2 (main container + sc3)
{
containerName: "sc2",
expectedDelay: 2,
},
// sc3 = 1 (main container)
{
containerName: "sc3",
expectedDelay: 1,
},
// main container = 0 delay, nothing to wait on
{
containerName: "main",
expectedDelay: 0,
},
} {
if got := getDelay(tc.containerName); got != tc.expectedDelay {
t.Errorf("expected delay for container %s = %d, got %d", tc.containerName, tc.expectedDelay, got)
}
}
}
func TestTerminationOrderingObeysGrace(t *testing.T) {
restartPolicy := v1.ContainerRestartPolicyAlways
pod := &v1.Pod{
ObjectMeta: metav1.ObjectMeta{
UID: "12345678",
Name: "bar",
Namespace: "new",
},
Spec: v1.PodSpec{
InitContainers: []v1.Container{
{
Name: "sc1",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
RestartPolicy: &restartPolicy,
},
{
Name: "sc2",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
RestartPolicy: &restartPolicy,
},
{
Name: "sc3",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
RestartPolicy: &restartPolicy,
},
},
Containers: []v1.Container{
{
Name: "main",
Image: "busybox",
ImagePullPolicy: v1.PullIfNotPresent,
},
},
},
}
to := newTerminationOrdering(pod, getContainerNames(pod))
var wg sync.WaitGroup
var delays sync.Map
waitAndExit := func(name string) {
// just a two second grace period which is not long enough for all of the waits to finish
delay := int64(to.waitForTurn(name, 2))
delays.Store(name, delay)
time.Sleep(1 * time.Second)
to.containerTerminated(name)
wg.Done()
}
for _, ic := range pod.Spec.InitContainers {
wg.Add(1)
go waitAndExit(ic.Name)
}
for _, c := range pod.Spec.Containers {
wg.Add(1)
go waitAndExit(c.Name)
}
// wait for our simulated containers to exit
wg.Wait()
getDelay := func(name string) int64 {
delay, ok := delays.Load(name)
if !ok {
t.Errorf("unable to find delay for container %s", name)
}
return delay.(int64)
}
for _, tc := range []struct {
containerName string
expectedDelay int64
}{
{
containerName: "sc1",
// overall grace period limits the amount of time waited here
expectedDelay: 2,
},
{
containerName: "sc2",
expectedDelay: 2,
},
{
containerName: "sc3",
expectedDelay: 1,
},
{
containerName: "main",
expectedDelay: 0,
},
} {
if got := getDelay(tc.containerName); got != tc.expectedDelay {
t.Errorf("expected delay for container %s = %d, got %d", tc.containerName, tc.expectedDelay, got)
}
}
}
func getContainerNames(p *v1.Pod) []string {
var running []string
for _, ic := range p.Spec.InitContainers {
running = append(running, ic.Name)
}
for _, c := range p.Spec.Containers {
running = append(running, c.Name)
}
return running
}

View File

@ -40,6 +40,9 @@ type execCommand struct {
StartDelay int
// Delay is how long the container should delay before exiting
Delay int
// LoopForever if set will cause the command to log once per second in a loop until
// terminated
LoopForever bool
// TerminationSeconds is the time it takes for the container before
// terminating if it catches SIGTERM.
TerminationSeconds int
@ -86,6 +89,9 @@ func ExecCommand(name string, c execCommand) []string {
if c.Delay != 0 {
fmt.Fprint(&cmd, sleepCommand(c.Delay))
}
if c.LoopForever {
fmt.Fprintf(&cmd, "while true; do echo %s '%s Looping' | tee -a %s >> /proc/1/fd/1 ; sleep 1 ; done; ", timeCmd, name, containerLog)
}
fmt.Fprintf(&cmd, "echo %s '%s Exiting' | tee -a %s >> /proc/1/fd/1; ", timeCmd, name, containerLog)
fmt.Fprintf(&cmd, "exit %d", c.ExitCode)
return []string{"sh", "-c", cmd.String()}
@ -110,8 +116,8 @@ type containerOutputList []containerOutput
func (o containerOutputList) String() string {
var b bytes.Buffer
for _, v := range o {
fmt.Fprintf(&b, "%s %f %s %s\n", v.timestamp, v.timeSinceBoot, v.containerName, v.command)
for i, v := range o {
fmt.Fprintf(&b, "%d) %s %f %s %s\n", i, v.timestamp, v.timeSinceBoot, v.containerName, v.command)
}
return b.String()
}
@ -186,10 +192,10 @@ func (o containerOutputList) ExitsBefore(lhs, rhs string) error {
}
// this works even for the same names (restart case)
rhsExit := o.findIndex(rhs, "Starting", lhsExit+1)
rhsExit := o.findIndex(rhs, "Exiting", lhsExit+1)
if rhsExit == -1 {
return fmt.Errorf("couldn't find that %s starting before %s exited, got\n%v", rhs, lhs, o)
return fmt.Errorf("couldn't find that %s starting before %s exited (starting at idx %d), got\n%v", rhs, lhs, lhsExit+1, o)
}
return nil
}
@ -278,6 +284,32 @@ func (o containerOutputList) findIndex(name string, command string, startIdx int
}
return -1
}
func (o containerOutputList) findLastIndex(name string, command string) int {
found := -1
for i, v := range o {
if v.containerName == name && v.command == command {
found = i
}
}
return found
}
// TimeOfStart returns the time since the node boot in floating point seconds that the specified container started.
func (o containerOutputList) TimeOfStart(name string) (float64, error) {
idx := o.findIndex(name, "Starting", 0)
if idx == -1 {
return 0.0, fmt.Errorf("couldn't find that %s ever started, got\n%v", name, o)
}
return o[idx].timeSinceBoot, nil
}
func (o containerOutputList) TimeOfLastLoop(name string) (float64, error) {
idx := o.findLastIndex(name, "Looping")
if idx == -1 {
return 0.0, fmt.Errorf("couldn't find that %s ever looped, got\n%v", name, o)
}
return o[idx].timeSinceBoot, nil
}
// parseOutput combines the container log from all of the init and regular
// containers and parses/sorts the outputs to produce an execution log

View File

@ -2505,4 +2505,458 @@ var _ = SIGDescribe("[NodeAlphaFeature:SidecarContainers] Containers Lifecycle",
framework.ExpectNoError(results.Exits(restartableInit1))
framework.ExpectNoError(results.Starts(regular1))
})
ginkgo.It("should terminate sidecars in reverse order after all main containers have exited", func() {
restartableInit1 := "restartable-init-1"
restartableInit2 := "restartable-init-2"
restartableInit3 := "restartable-init-3"
regular1 := "regular-1"
pod := &v1.Pod{
ObjectMeta: metav1.ObjectMeta{
Name: "serialize-termination",
},
Spec: v1.PodSpec{
RestartPolicy: v1.RestartPolicyNever,
InitContainers: []v1.Container{
{
Name: restartableInit1,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit1, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
},
{
Name: restartableInit2,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit2, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
},
{
Name: restartableInit3,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit3, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
},
},
Containers: []v1.Container{
{
Name: regular1,
Image: busyboxImage,
Command: ExecCommand(regular1, execCommand{
Delay: 5,
ExitCode: 0,
}),
},
},
},
}
preparePod(pod)
client := e2epod.NewPodClient(f)
pod = client.Create(context.TODO(), pod)
err := e2epod.WaitTimeoutForPodNoLongerRunningInNamespace(context.TODO(), f.ClientSet, pod.Name, pod.Namespace, 5*time.Minute)
framework.ExpectNoError(err)
pod, err = client.Get(context.TODO(), pod.Name, metav1.GetOptions{})
framework.ExpectNoError(err)
results := parseOutput(context.TODO(), f, pod)
ginkgo.By("Analyzing results")
framework.ExpectNoError(results.StartsBefore(restartableInit1, restartableInit2))
framework.ExpectNoError(results.StartsBefore(restartableInit1, restartableInit3))
framework.ExpectNoError(results.StartsBefore(restartableInit2, restartableInit3))
framework.ExpectNoError(results.StartsBefore(restartableInit1, regular1))
framework.ExpectNoError(results.StartsBefore(restartableInit2, regular1))
framework.ExpectNoError(results.StartsBefore(restartableInit3, regular1))
// main containers exit first
framework.ExpectNoError(results.ExitsBefore(regular1, restartableInit1))
framework.ExpectNoError(results.ExitsBefore(regular1, restartableInit2))
framework.ExpectNoError(results.ExitsBefore(regular1, restartableInit3))
// followed by sidecars in reverse order
framework.ExpectNoError(results.ExitsBefore(restartableInit3, restartableInit2))
framework.ExpectNoError(results.ExitsBefore(restartableInit2, restartableInit1))
})
ginkgo.It("should terminate sidecars simultaneously if prestop doesn't exit", func() {
restartableInit1 := "restartable-init-1"
restartableInit2 := "restartable-init-2"
restartableInit3 := "restartable-init-3"
regular1 := "regular-1"
makePrestop := func(containerName string) *v1.Lifecycle {
return &v1.Lifecycle{
PreStop: &v1.LifecycleHandler{
Exec: &v1.ExecAction{
Command: ExecCommand(prefixedName(PreStopPrefix, containerName), execCommand{
ExitCode: 0,
ContainerName: containerName,
LoopForever: true,
}),
},
},
}
}
pod := &v1.Pod{
ObjectMeta: metav1.ObjectMeta{
Name: "serialize-termination",
},
Spec: v1.PodSpec{
RestartPolicy: v1.RestartPolicyNever,
InitContainers: []v1.Container{
{
Name: restartableInit1,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit1, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit1),
},
{
Name: restartableInit2,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit2, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit2),
},
{
Name: restartableInit3,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit3, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit3),
},
},
Containers: []v1.Container{
{
Name: regular1,
Image: busyboxImage,
Command: ExecCommand(regular1, execCommand{
Delay: 5,
ExitCode: 0,
}),
},
},
},
}
preparePod(pod)
client := e2epod.NewPodClient(f)
pod = client.Create(context.TODO(), pod)
err := e2epod.WaitTimeoutForPodNoLongerRunningInNamespace(context.TODO(), f.ClientSet, pod.Name, pod.Namespace, 5*time.Minute)
framework.ExpectNoError(err)
pod, err = client.Get(context.TODO(), pod.Name, metav1.GetOptions{})
framework.ExpectNoError(err)
results := parseOutput(context.TODO(), f, pod)
ginkgo.By("Analyzing results")
framework.ExpectNoError(results.StartsBefore(restartableInit1, restartableInit2))
framework.ExpectNoError(results.StartsBefore(restartableInit1, restartableInit3))
framework.ExpectNoError(results.StartsBefore(restartableInit2, restartableInit3))
framework.ExpectNoError(results.StartsBefore(restartableInit1, regular1))
framework.ExpectNoError(results.StartsBefore(restartableInit2, regular1))
framework.ExpectNoError(results.StartsBefore(restartableInit3, regular1))
ps1, err := results.TimeOfStart(prefixedName(PreStopPrefix, restartableInit1))
framework.ExpectNoError(err)
ps2, err := results.TimeOfStart(prefixedName(PreStopPrefix, restartableInit2))
framework.ExpectNoError(err)
ps3, err := results.TimeOfStart(prefixedName(PreStopPrefix, restartableInit3))
framework.ExpectNoError(err)
ps1Last, err := results.TimeOfLastLoop(prefixedName(PreStopPrefix, restartableInit1))
framework.ExpectNoError(err)
ps2Last, err := results.TimeOfLastLoop(prefixedName(PreStopPrefix, restartableInit2))
framework.ExpectNoError(err)
ps3Last, err := results.TimeOfLastLoop(prefixedName(PreStopPrefix, restartableInit3))
framework.ExpectNoError(err)
const simulToleration = 0.5
// should all end together since they loop infinitely and exceed their grace period
gomega.Expect(ps1Last-ps2Last).To(gomega.BeNumerically("~", 0, simulToleration),
fmt.Sprintf("expected PostStart 1 & PostStart 2 to be killed at the same time, got %s", results))
gomega.Expect(ps1Last-ps3Last).To(gomega.BeNumerically("~", 0, simulToleration),
fmt.Sprintf("expected PostStart 1 & PostStart 3 to be killed at the same time, got %s", results))
gomega.Expect(ps2Last-ps3Last).To(gomega.BeNumerically("~", 0, simulToleration),
fmt.Sprintf("expected PostStart 2 & PostStart 3 to be killed at the same time, got %s", results))
// 30 seconds + 2 second minimum grace for the SIGKILL
const lifetimeToleration = 1
gomega.Expect(ps1Last-ps1).To(gomega.BeNumerically("~", 32, lifetimeToleration),
fmt.Sprintf("expected PostStart 1 to live for ~32 seconds, got %s", results))
gomega.Expect(ps2Last-ps2).To(gomega.BeNumerically("~", 32, lifetimeToleration),
fmt.Sprintf("expected PostStart 2 to live for ~32 seconds, got %s", results))
gomega.Expect(ps3Last-ps3).To(gomega.BeNumerically("~", 32, lifetimeToleration),
fmt.Sprintf("expected PostStart 3 to live for ~32 seconds, got %s", results))
})
ginkgo.It("should call sidecar container PreStop hook simultaneously", func() {
restartableInit1 := "restartable-init-1"
restartableInit2 := "restartable-init-2"
restartableInit3 := "restartable-init-3"
regular1 := "regular-1"
makePrestop := func(containerName string) *v1.Lifecycle {
return &v1.Lifecycle{
PreStop: &v1.LifecycleHandler{
Exec: &v1.ExecAction{
Command: ExecCommand(prefixedName(PreStopPrefix, containerName), execCommand{
Delay: 1,
ExitCode: 0,
ContainerName: containerName,
}),
},
},
}
}
pod := &v1.Pod{
ObjectMeta: metav1.ObjectMeta{
Name: "serialize-termination-simul-prestop",
},
Spec: v1.PodSpec{
RestartPolicy: v1.RestartPolicyNever,
InitContainers: []v1.Container{
{
Name: restartableInit1,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit1, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit1),
},
{
Name: restartableInit2,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit2, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit2),
},
{
Name: restartableInit3,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit3, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit3),
},
},
Containers: []v1.Container{
{
Name: regular1,
Image: busyboxImage,
Command: ExecCommand(regular1, execCommand{
Delay: 5,
ExitCode: 0,
}),
},
},
},
}
preparePod(pod)
client := e2epod.NewPodClient(f)
pod = client.Create(context.TODO(), pod)
err := e2epod.WaitTimeoutForPodNoLongerRunningInNamespace(context.TODO(), f.ClientSet, pod.Name, pod.Namespace, 5*time.Minute)
framework.ExpectNoError(err)
pod, err = client.Get(context.TODO(), pod.Name, metav1.GetOptions{})
framework.ExpectNoError(err)
results := parseOutput(context.TODO(), f, pod)
ginkgo.By("Analyzing results")
framework.ExpectNoError(results.StartsBefore(restartableInit1, restartableInit2))
framework.ExpectNoError(results.StartsBefore(restartableInit1, restartableInit3))
framework.ExpectNoError(results.StartsBefore(restartableInit2, restartableInit3))
framework.ExpectNoError(results.StartsBefore(restartableInit1, regular1))
framework.ExpectNoError(results.StartsBefore(restartableInit2, regular1))
framework.ExpectNoError(results.StartsBefore(restartableInit3, regular1))
// main containers exit first
framework.ExpectNoError(results.ExitsBefore(regular1, restartableInit1))
framework.ExpectNoError(results.ExitsBefore(regular1, restartableInit2))
framework.ExpectNoError(results.ExitsBefore(regular1, restartableInit3))
// followed by sidecars in reverse order
framework.ExpectNoError(results.ExitsBefore(restartableInit3, restartableInit2))
framework.ExpectNoError(results.ExitsBefore(restartableInit2, restartableInit1))
// and the pre-stop hooks should have been called simultaneously
ps1, err := results.TimeOfStart(prefixedName(PreStopPrefix, restartableInit1))
framework.ExpectNoError(err)
ps2, err := results.TimeOfStart(prefixedName(PreStopPrefix, restartableInit2))
framework.ExpectNoError(err)
ps3, err := results.TimeOfStart(prefixedName(PreStopPrefix, restartableInit3))
framework.ExpectNoError(err)
const toleration = 0.5
gomega.Expect(ps1-ps2).To(gomega.BeNumerically("~", 0, toleration),
fmt.Sprintf("expected PostStart 1 & PostStart 2 to start at the same time, got %s", results))
gomega.Expect(ps1-ps3).To(gomega.BeNumerically("~", 0, toleration),
fmt.Sprintf("expected PostStart 1 & PostStart 3 to start at the same time, got %s", results))
gomega.Expect(ps2-ps3).To(gomega.BeNumerically("~", 0, toleration),
fmt.Sprintf("expected PostStart 2 & PostStart 3 to start at the same time, got %s", results))
})
ginkgo.It("should not hang in termination if terminated during initialization", func() {
startInit := "start-init"
restartableInit1 := "restartable-init-1"
restartableInit2 := "restartable-init-2"
restartableInit3 := "restartable-init-3"
regular1 := "regular-1"
makePrestop := func(containerName string) *v1.Lifecycle {
return &v1.Lifecycle{
PreStop: &v1.LifecycleHandler{
Exec: &v1.ExecAction{
Command: ExecCommand(prefixedName(PreStopPrefix, containerName), execCommand{
Delay: 1,
ExitCode: 0,
ContainerName: containerName,
}),
},
},
}
}
pod := &v1.Pod{
ObjectMeta: metav1.ObjectMeta{
Name: "dont-hang-if-terminated-in-init",
},
Spec: v1.PodSpec{
RestartPolicy: v1.RestartPolicyNever,
InitContainers: []v1.Container{
{
Name: startInit,
Image: busyboxImage,
Command: ExecCommand(startInit, execCommand{
Delay: 300,
TerminationSeconds: 0,
ExitCode: 0,
}),
},
{
Name: restartableInit1,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit1, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit1),
},
{
Name: restartableInit2,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit2, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit2),
},
{
Name: restartableInit3,
Image: busyboxImage,
RestartPolicy: &containerRestartPolicyAlways,
Command: ExecCommand(restartableInit3, execCommand{
Delay: 60,
TerminationSeconds: 5,
ExitCode: 0,
}),
Lifecycle: makePrestop(restartableInit3),
},
},
Containers: []v1.Container{
{
Name: regular1,
Image: busyboxImage,
Command: ExecCommand(regular1, execCommand{
Delay: 5,
ExitCode: 0,
}),
},
},
},
}
preparePod(pod)
client := e2epod.NewPodClient(f)
pod = client.Create(context.TODO(), pod)
err := e2epod.WaitForPodCondition(context.TODO(), f.ClientSet, pod.Namespace, pod.Name, "pod pending and init running", 2*time.Minute, func(pod *v1.Pod) (bool, error) {
if pod.Status.Phase != v1.PodPending {
return false, fmt.Errorf("pod should be in pending phase")
}
if len(pod.Status.InitContainerStatuses) < 1 {
return false, nil
}
containerStatus := pod.Status.InitContainerStatuses[0]
return *containerStatus.Started && containerStatus.State.Running != nil, nil
})
framework.ExpectNoError(err)
// the init container is running, so we stop the pod before the sidecars even start
start := time.Now()
grace := int64(3)
ginkgo.By("deleting the pod")
err = client.Delete(context.TODO(), pod.Name, metav1.DeleteOptions{GracePeriodSeconds: &grace})
framework.ExpectNoError(err)
ginkgo.By("waiting for the pod to disappear")
err = e2epod.WaitForPodNotFoundInNamespace(context.TODO(), f.ClientSet, pod.Name, pod.Namespace, 120*time.Second)
framework.ExpectNoError(err)
buffer := int64(2)
deleteTime := time.Since(start).Seconds()
// should delete quickly and not try to start/wait on any sidecars since they never started
gomega.Expect(deleteTime).To(gomega.BeNumerically("<", grace+buffer), fmt.Sprintf("should delete in < %d seconds, took %f", grace+buffer, deleteTime))
})
})