Feature: support batch-id labeling for bluegreen strategy (#250)

* Feature: support batch-id labeling for blue-green strategy

---------

Signed-off-by: AiRanthem <zhongtianyun.zty@alibaba-inc.com>
This commit is contained in:
Ai Ranthem 2025-03-20 19:26:46 +08:00 committed by GitHub
parent 7baf47d70e
commit 3e66fa1ad8
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 474 additions and 123 deletions

View File

@ -27,7 +27,7 @@ jobs:
with:
go-version: ${{ env.GO_VERSION }}
- name: Cache Go Dependencies
uses: actions/cache@v2
uses: actions/cache@v4
with:
path: ~/go/pkg/mod
key: ${{ runner.os }}-go-${{ hashFiles('**/go.sum') }}
@ -54,7 +54,7 @@ jobs:
with:
go-version: ${{ env.GO_VERSION }}
- name: Cache Go Dependencies
uses: actions/cache@v2
uses: actions/cache@v4
with:
path: ~/go/pkg/mod
key: ${{ runner.os }}-go-${{ hashFiles('**/go.sum') }}

View File

@ -18,6 +18,7 @@ package bluegreenstyle
import (
"github.com/openkruise/rollouts/api/v1beta1"
"github.com/openkruise/rollouts/pkg/controller/batchrelease/context"
"github.com/openkruise/rollouts/pkg/controller/batchrelease/control"
"github.com/openkruise/rollouts/pkg/controller/batchrelease/labelpatch"
"github.com/openkruise/rollouts/pkg/util"
@ -40,7 +41,7 @@ type realBatchControlPlane struct {
type NewInterfaceFunc func(cli client.Client, key types.NamespacedName, gvk schema.GroupVersionKind) Interface
// NewControlPlane creates a new release controller with bluegreen-style to drive batch release state machine
// NewControlPlane creates a new release controller with blue-green style to drive batch release state machine
func NewControlPlane(f NewInterfaceFunc, cli client.Client, recorder record.EventRecorder, release *v1beta1.BatchRelease, newStatus *v1beta1.BatchReleaseStatus, key types.NamespacedName, gvk schema.GroupVersionKind) *realBatchControlPlane {
return &realBatchControlPlane{
Client: cli,
@ -72,6 +73,15 @@ func (rc *realBatchControlPlane) Initialize() error {
return err
}
func (rc *realBatchControlPlane) patchPodLabels(batchContext *context.BatchContext) error {
pods, err := rc.ListOwnedPods() // add pods to rc for patching pod batch labels
if err != nil {
return err
}
batchContext.Pods = pods
return rc.patcher.PatchPodBatchLabel(batchContext)
}
func (rc *realBatchControlPlane) UpgradeBatch() error {
controller, err := rc.BuildController()
if err != nil {
@ -94,7 +104,7 @@ func (rc *realBatchControlPlane) UpgradeBatch() error {
return err
}
return nil
return rc.patchPodLabels(batchContext)
}
func (rc *realBatchControlPlane) CheckBatchReady() error {

View File

@ -226,7 +226,7 @@ func (rc *realBatchControlPlane) markNoNeedUpdatePodsIfNeeds() (*int32, error) {
if !pods[i].DeletionTimestamp.IsZero() {
continue
}
if !util.IsConsistentWithRevision(pods[i], rc.newStatus.UpdateRevision) {
if !util.IsConsistentWithRevision(pods[i].GetLabels(), rc.newStatus.UpdateRevision) {
continue
}
if pods[i].Labels[util.NoNeedUpdatePodLabel] == rolloutID {
@ -273,7 +273,7 @@ func (rc *realBatchControlPlane) countAndUpdateNoNeedUpdateReplicas() error {
if !pod.DeletionTimestamp.IsZero() {
continue
}
if !util.IsConsistentWithRevision(pod, rc.release.Status.UpdateRevision) {
if !util.IsConsistentWithRevision(pod.GetLabels(), rc.release.Status.UpdateRevision) {
continue
}
id, ok := pod.Labels[util.NoNeedUpdatePodLabel]

View File

@ -67,7 +67,7 @@ func (rc *realController) BuildController() (partitionstyle.Interface, error) {
if !pod.DeletionTimestamp.IsZero() {
return false
}
if !util.IsConsistentWithRevision(pod, rc.WorkloadInfo.Status.UpdateRevision) {
if !util.IsConsistentWithRevision(pod.GetLabels(), rc.WorkloadInfo.Status.UpdateRevision) {
return false
}
return util.IsPodReady(pod)

View File

@ -77,7 +77,7 @@ func (rc *realController) BuildController() (partitionstyle.Interface, error) {
if !pod.DeletionTimestamp.IsZero() {
return false
}
if !util.IsConsistentWithRevision(pod, rc.WorkloadInfo.Status.UpdateRevision) {
if !util.IsConsistentWithRevision(pod.GetLabels(), rc.WorkloadInfo.Status.UpdateRevision) {
return false
}
return util.IsPodReady(pod)

View File

@ -57,7 +57,7 @@ func FilterPodsForUnorderedUpdate(pods []*corev1.Pod, ctx *batchcontext.BatchCon
terminatingPods = append(terminatingPods, pod)
continue
}
if !util.IsConsistentWithRevision(pod, ctx.UpdateRevision) {
if !util.IsConsistentWithRevision(pod.GetLabels(), ctx.UpdateRevision) {
continue
}
if pod.Labels[util.NoNeedUpdatePodLabel] == ctx.RolloutID && pod.Labels[v1beta1.RolloutIDLabel] != ctx.RolloutID {
@ -113,7 +113,7 @@ func FilterPodsForOrderedUpdate(pods []*corev1.Pod, ctx *batchcontext.BatchConte
terminatingPods = append(terminatingPods, pod)
continue
}
if !util.IsConsistentWithRevision(pod, ctx.UpdateRevision) {
if !util.IsConsistentWithRevision(pod.GetLabels(), ctx.UpdateRevision) {
continue
}
if getPodOrdinal(pod) >= partition {

View File

@ -24,7 +24,9 @@ import (
"github.com/openkruise/rollouts/api/v1beta1"
batchcontext "github.com/openkruise/rollouts/pkg/controller/batchrelease/context"
"github.com/openkruise/rollouts/pkg/util"
v1 "k8s.io/api/apps/v1"
corev1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/intstr"
"k8s.io/klog/v2"
@ -61,47 +63,94 @@ func (r *realPatcher) patchPodBatchLabel(pods []*corev1.Pod, ctx *batchcontext.B
plannedUpdatedReplicasForBatches := r.calculatePlannedStepIncrements(r.batches, int(ctx.Replicas), int(ctx.CurrentBatch))
var updatedButUnpatchedPods []*corev1.Pod
revisionHashCache := map[types.UID]string{} // to prevent duplicate computing for revision hash
podsToPatchControllerRevision := map[*corev1.Pod]string{} // to record pods to patch controller-revision-hash
for _, pod := range pods {
if !pod.DeletionTimestamp.IsZero() {
klog.InfoS("Pod is being deleted, skip patching", "pod", klog.KObj(pod), "rollout", r.logKey)
continue
}
labels := make(map[string]string, len(pod.Labels))
for k, v := range pod.Labels {
labels[k] = v
}
if labels[v1.ControllerRevisionHashLabelKey] == "" {
// For native deployment, we need to get the revision hash from ReplicaSet, which is exactly constants with the update revision
// The reason is that, The status of the Deployment that KCM sees may differ from the Deployment seen by
// the Rollouts controller due to some default values not being assigned yet. Therefore, even if both use
// exactly the same algorithm, they cannot compute the same pod-template-hash. The fact that K8S does not
// expose the method for computing the pod-template-hash also confirms that third-party components relying
// on the pod-template-hash is not recommended. Thus, we use the CloneSet algorithm to compute the
// controller-revision-hash: this method is fully defined by OpenKruise and can ensure that the same
// ReplicaSet produces the same value.
owner := metav1.GetControllerOf(pod)
if owner != nil && owner.Kind == "ReplicaSet" {
var hash string
if cache, ok := revisionHashCache[owner.UID]; ok {
hash = cache
} else {
rs := &v1.ReplicaSet{}
if err := r.Get(context.Background(), types.NamespacedName{Namespace: pod.Namespace, Name: owner.Name}, rs); err != nil {
klog.ErrorS(err, "Failed to get ReplicaSet", "pod", klog.KObj(pod), "rollout", r.logKey, "owner", owner.Name, "namespace", pod.Namespace)
return err
}
delete(rs.Spec.Template.ObjectMeta.Labels, v1.DefaultDeploymentUniqueLabelKey)
hash = util.ComputeHash(&rs.Spec.Template, nil)
revisionHashCache[owner.UID] = hash
}
labels[v1.ControllerRevisionHashLabelKey] = hash
podsToPatchControllerRevision[pod] = hash
klog.InfoS("Pod controller-revision-hash updated", "pod", klog.KObj(pod), "rollout", r.logKey, "hash", hash)
}
}
// we don't patch label for the active old revision pod
if !util.IsConsistentWithRevision(pod, ctx.UpdateRevision) {
klog.InfoS("Pod is not consistent with revision, skip patching", "pod", klog.KObj(pod), "rollout", r.logKey)
if !util.IsConsistentWithRevision(labels, ctx.UpdateRevision) {
klog.InfoS("Pod is not consistent with revision, skip patching", "pod", klog.KObj(pod),
"revision", ctx.UpdateRevision, "pod-template-hash", labels[v1.DefaultDeploymentUniqueLabelKey],
"controller-revision-hash", labels[v1.ControllerRevisionHashLabelKey], "rollout", r.logKey)
continue
}
if pod.Labels[v1beta1.RolloutIDLabel] != ctx.RolloutID {
if labels[v1beta1.RolloutIDLabel] != ctx.RolloutID {
// for example: new/recreated pods
updatedButUnpatchedPods = append(updatedButUnpatchedPods, pod)
klog.InfoS("Find a pod to add updatedButUnpatchedPods", "pod", klog.KObj(pod), "rollout", r.logKey)
continue
}
podBatchID, err := strconv.Atoi(pod.Labels[v1beta1.RolloutBatchIDLabel])
podBatchID, err := strconv.Atoi(labels[v1beta1.RolloutBatchIDLabel])
if err != nil {
klog.InfoS("Pod batchID is not a number, skip patching", "pod", klog.KObj(pod), "rollout", r.logKey)
continue
}
plannedUpdatedReplicasForBatches[podBatchID-1]--
}
klog.InfoS("updatedButUnpatchedPods amount calculated", "amount", len(updatedButUnpatchedPods), "rollout", r.logKey)
klog.InfoS("updatedButUnpatchedPods amount calculated", "amount", len(updatedButUnpatchedPods),
"rollout", r.logKey, "plan", plannedUpdatedReplicasForBatches)
// patch the pods
for i := len(plannedUpdatedReplicasForBatches) - 1; i >= 0; i-- {
for ; plannedUpdatedReplicasForBatches[i] > 0; plannedUpdatedReplicasForBatches[i]-- {
if len(updatedButUnpatchedPods) == 0 {
klog.Warningf("no pods to patch for %v, batch %d", r.logKey, i+1)
return nil
i = -1
break
}
// patch the updated but unpatced pod
// patch the updated but unpatched pod
pod := updatedButUnpatchedPods[len(updatedButUnpatchedPods)-1]
clone := util.GetEmptyObjectWithKey(pod)
by := fmt.Sprintf(`{"metadata":{"labels":{"%s":"%s","%s":"%d"}}}`,
v1beta1.RolloutIDLabel, ctx.RolloutID, v1beta1.RolloutBatchIDLabel, i+1)
if err := r.Patch(context.TODO(), clone, client.RawPatch(types.StrategicMergePatchType, []byte(by))); err != nil {
var patchStr string
if hash, ok := podsToPatchControllerRevision[pod]; ok {
patchStr = fmt.Sprintf(`{"metadata":{"labels":{"%s":"%s","%s":"%d","%s":"%s"}}}`,
v1beta1.RolloutIDLabel, ctx.RolloutID, v1beta1.RolloutBatchIDLabel, i+1, v1.ControllerRevisionHashLabelKey, hash)
delete(podsToPatchControllerRevision, pod)
} else {
patchStr = fmt.Sprintf(`{"metadata":{"labels":{"%s":"%s","%s":"%d"}}}`,
v1beta1.RolloutIDLabel, ctx.RolloutID, v1beta1.RolloutBatchIDLabel, i+1)
}
if err := r.Patch(context.TODO(), clone, client.RawPatch(types.StrategicMergePatchType, []byte(patchStr))); err != nil {
return err
}
klog.InfoS("Successfully patch Pod batchID", "batchID", i+1, "pod", klog.KObj(pod), "rollout", r.logKey)
klog.InfoS("Successfully patched Pod batchID", "batchID", i+1, "pod", klog.KObj(pod), "rollout", r.logKey)
// update the counter
updatedButUnpatchedPods = updatedButUnpatchedPods[:len(updatedButUnpatchedPods)-1]
}
@ -112,6 +161,18 @@ func (r *realPatcher) patchPodBatchLabel(pods []*corev1.Pod, ctx *batchcontext.B
if len(updatedButUnpatchedPods) != 0 {
klog.Warningf("still has %d pods to patch for %v", len(updatedButUnpatchedPods), r.logKey)
}
// pods with controller-revision-hash label updated but not in the rollout release need to be patched too
// We must promptly patch the computed controller-revision-hash label to the Pod so that it can be directly read
// during frequent Reconcile processes, avoiding a large amount of redundant computation.
for pod, hash := range podsToPatchControllerRevision {
clone := util.GetEmptyObjectWithKey(pod)
patchStr := fmt.Sprintf(`{"metadata":{"labels":{"%s":"%s"}}}`, v1.ControllerRevisionHashLabelKey, hash)
if err := r.Patch(context.TODO(), clone, client.RawPatch(types.StrategicMergePatchType, []byte(patchStr))); err != nil {
return err
}
klog.InfoS("Successfully patch Pod controller-revision-hash", "pod", klog.KObj(pod), "rollout", r.logKey, "hash", hash)
}
return nil
}

View File

@ -18,6 +18,7 @@ package labelpatch
import (
"context"
"reflect"
"strconv"
"testing"
@ -25,11 +26,15 @@ import (
. "github.com/onsi/gomega"
"github.com/openkruise/rollouts/api/v1beta1"
batchcontext "github.com/openkruise/rollouts/pkg/controller/batchrelease/context"
apps "k8s.io/api/apps/v1"
"github.com/openkruise/rollouts/pkg/util"
appsv1 "k8s.io/api/apps/v1"
corev1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/util/intstr"
"k8s.io/apimachinery/pkg/util/rand"
"k8s.io/klog/v2"
"k8s.io/utils/pointer"
"sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/client/fake"
)
@ -39,7 +44,8 @@ var (
)
func init() {
corev1.AddToScheme(scheme)
_ = corev1.AddToScheme(scheme)
_ = appsv1.AddToScheme(scheme)
}
func TestLabelPatcher(t *testing.T) {
@ -240,7 +246,7 @@ func TestLabelPatcher(t *testing.T) {
}
cli := fake.NewClientBuilder().WithScheme(scheme).WithObjects(objects...).Build()
patcher := NewLabelPatcher(cli, klog.ObjectRef{Name: "test"}, cs.Batches)
patcher.patchPodBatchLabel(ctx.Pods, ctx)
_ = patcher.patchPodBatchLabel(ctx.Pods, ctx)
podList := &corev1.PodList{}
err := cli.List(context.TODO(), podList)
@ -258,16 +264,308 @@ func TestLabelPatcher(t *testing.T) {
}
}
func TestDeploymentPatch(t *testing.T) {
rs := &appsv1.ReplicaSet{
ObjectMeta: metav1.ObjectMeta{
Name: "rs-1",
},
Spec: appsv1.ReplicaSetSpec{
Template: corev1.PodTemplateSpec{
Spec: corev1.PodSpec{
Containers: []corev1.Container{
{
Name: "nginx",
Image: "nginx:1.14.2",
Ports: []corev1.ContainerPort{
{
ContainerPort: 80,
},
},
},
},
},
},
},
}
revision := util.ComputeHash(&rs.Spec.Template, nil)
// randomly inserted to test cases, pods with this revision should be skipped and
// the result should not be influenced
skippedRevision := "should-skip"
cases := map[string]struct {
batchContext func() *batchcontext.BatchContext
Batches []v1beta1.ReleaseBatch
expectedPatched []int
}{
"10 pods, 0 patched, 5 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
Replicas: 10,
}
pods := generateDeploymentPods(1, ctx.Replicas, 0, "", "")
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{
CanaryReplicas: intstr.FromInt(5),
},
},
expectedPatched: []int{5},
},
"10 pods, 2 patched, 3 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
Replicas: 10,
}
pods := generateDeploymentPods(1, ctx.Replicas, 2,
ctx.RolloutID, strconv.Itoa(int(ctx.CurrentBatch+1)))
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{
CanaryReplicas: intstr.FromInt(5),
},
},
expectedPatched: []int{5},
},
"10 pods, 5 patched, 0 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
Replicas: 10,
}
pods := generateDeploymentPods(1, ctx.Replicas, 5,
ctx.RolloutID, strconv.Itoa(int(ctx.CurrentBatch+1)))
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{
CanaryReplicas: intstr.FromInt(5),
},
},
expectedPatched: []int{5},
},
"10 pods, 7 patched, 0 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
Replicas: 10,
}
pods := generateDeploymentPods(1, ctx.Replicas, 7,
ctx.RolloutID, strconv.Itoa(int(ctx.CurrentBatch+1)))
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{
CanaryReplicas: intstr.FromInt(5),
},
},
expectedPatched: []int{7},
},
"2 pods, 0 patched, 2 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
Replicas: 10,
}
pods := generateDeploymentPods(1, 2, 0,
ctx.RolloutID, strconv.Itoa(int(ctx.CurrentBatch+1)))
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{
CanaryReplicas: intstr.FromInt(5),
},
},
expectedPatched: []int{2},
},
"10 pods, 3 patched with old rollout-id, 5 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
Replicas: 10,
}
pods := generateDeploymentPods(1, ctx.Replicas, 3,
"previous-rollout-id", strconv.Itoa(int(ctx.CurrentBatch+1)))
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{
CanaryReplicas: intstr.FromInt(5),
},
},
expectedPatched: []int{5},
},
"10 pods, 2 patched with batch-id:1, 3 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
CurrentBatch: 1,
Replicas: 10,
}
pods := generateDeploymentPods(1, 5, 2,
"rollout-1", strconv.Itoa(1))
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{CanaryReplicas: intstr.FromInt(2)},
{CanaryReplicas: intstr.FromInt(5)},
},
expectedPatched: []int{2, 3},
},
"10 pods, 0 patched with batch-id:1, 5 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
CurrentBatch: 1,
Replicas: 10,
}
pods := generateDeploymentPods(1, 5, 0,
"rollout-1", strconv.Itoa(1))
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{CanaryReplicas: intstr.FromInt(2)},
{CanaryReplicas: intstr.FromInt(5)},
},
expectedPatched: []int{2, 3},
},
"10 pods, 3 patched with batch-id:1, 2 new patched": {
batchContext: func() *batchcontext.BatchContext {
ctx := &batchcontext.BatchContext{
RolloutID: "rollout-1",
UpdateRevision: revision,
PlannedUpdatedReplicas: 5,
CurrentBatch: 1,
Replicas: 10,
}
pods := generateDeploymentPods(1, 5, 3,
"rollout-1", strconv.Itoa(1))
ctx.Pods = pods
return ctx
},
Batches: []v1beta1.ReleaseBatch{
{CanaryReplicas: intstr.FromInt(2)},
{CanaryReplicas: intstr.FromInt(5)},
},
expectedPatched: []int{3, 2},
},
}
for name, cs := range cases {
t.Run(name, func(t *testing.T) {
ctx := cs.batchContext()
insertedSkippedPodNum := int32(rand.Intn(3))
if insertedSkippedPodNum > 0 {
ctx.Pods = append(ctx.Pods, generatePods(
100, 99+insertedSkippedPodNum, 0, "doesn't matter", "1", skippedRevision)...)
}
t.Logf("%d should-skip pods inserted", insertedSkippedPodNum)
if rand.Intn(2) > 0 {
now := metav1.Now()
ctx.Pods = append(ctx.Pods, &corev1.Pod{
ObjectMeta: metav1.ObjectMeta{
DeletionTimestamp: &now,
Labels: map[string]string{
appsv1.ControllerRevisionHashLabelKey: skippedRevision,
},
},
})
t.Logf("deleted pod inserted")
}
var objects []client.Object
for _, pod := range ctx.Pods {
objects = append(objects, pod)
}
objects = append(objects, rs)
cli := fake.NewClientBuilder().WithScheme(scheme).WithObjects(objects...).Build()
patcher := NewLabelPatcher(cli, klog.ObjectRef{Name: "test"}, cs.Batches)
if err := patcher.patchPodBatchLabel(ctx.Pods, ctx); err != nil {
t.Fatalf("failed to patch pods: %v", err)
}
podList := &corev1.PodList{}
if err := cli.List(context.TODO(), podList); err != nil {
t.Fatalf("failed to list pods: %v", err)
}
patched := make([]int, ctx.CurrentBatch+1)
for _, pod := range podList.Items {
if pod.Labels[v1beta1.RolloutIDLabel] == ctx.RolloutID {
if batchID, err := strconv.Atoi(pod.Labels[v1beta1.RolloutBatchIDLabel]); err == nil {
patched[batchID-1]++
}
}
}
if !reflect.DeepEqual(patched, cs.expectedPatched) {
t.Fatalf("expected patched: %v, got: %v", cs.expectedPatched, patched)
}
for _, pod := range podList.Items {
if pod.Labels[appsv1.ControllerRevisionHashLabelKey] != revision &&
pod.Labels[appsv1.ControllerRevisionHashLabelKey] != skippedRevision {
t.Fatalf("expected pod %s/%s to have revision %s, got %s", pod.Namespace, pod.Name, revision, pod.Labels[appsv1.ControllerRevisionHashLabelKey])
}
}
})
}
}
func generateDeploymentPods(ordinalBegin, ordinalEnd, labeled int32, rolloutID, batchID string) []*corev1.Pod {
podsWithLabel := generateLabeledPods(map[string]string{
v1beta1.RolloutIDLabel: rolloutID,
v1beta1.RolloutBatchIDLabel: batchID,
}, int(labeled), int(ordinalBegin))
total := ordinalEnd - ordinalBegin + 1
podsWithoutLabel := generateLabeledPods(map[string]string{}, int(total-labeled), int(labeled+ordinalBegin))
pods := append(podsWithoutLabel, podsWithLabel...)
for _, pod := range pods {
pod.OwnerReferences = []metav1.OwnerReference{
{
APIVersion: "apps/v1",
Kind: "ReplicaSet",
Name: "rs-1",
UID: "123",
Controller: pointer.Bool(true),
},
}
}
return pods
}
func generatePods(ordinalBegin, ordinalEnd, labeled int32, rolloutID, batchID, version string) []*corev1.Pod {
podsWithLabel := generateLabeledPods(map[string]string{
v1beta1.RolloutIDLabel: rolloutID,
v1beta1.RolloutBatchIDLabel: batchID,
apps.ControllerRevisionHashLabelKey: version,
v1beta1.RolloutIDLabel: rolloutID,
v1beta1.RolloutBatchIDLabel: batchID,
appsv1.ControllerRevisionHashLabelKey: version,
}, int(labeled), int(ordinalBegin))
total := ordinalEnd - ordinalBegin + 1
podsWithoutLabel := generateLabeledPods(map[string]string{
apps.ControllerRevisionHashLabelKey: version,
appsv1.ControllerRevisionHashLabelKey: version,
}, int(total-labeled), int(labeled+ordinalBegin))
return append(podsWithoutLabel, podsWithLabel...)
}

View File

@ -1,6 +1,5 @@
/*
Copyright 2022 The Kruise Authors.
Copyright 2015 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.

View File

@ -1,6 +1,5 @@
/*
Copyright 2022 The Kruise Authors.
Copyright 2015 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.
@ -68,15 +67,15 @@ func GetPodConditionFromList(conditions []v1.PodCondition, conditionType v1.PodC
return -1, nil
}
// IsConsistentWithRevision return true iff pod is match the revision
func IsConsistentWithRevision(pod *v1.Pod, revision string) bool {
if pod.Labels[appsv1.DefaultDeploymentUniqueLabelKey] != "" &&
strings.HasSuffix(revision, pod.Labels[appsv1.DefaultDeploymentUniqueLabelKey]) {
// IsConsistentWithRevision return true if pod is match the revision
func IsConsistentWithRevision(labels map[string]string, revision string) bool {
if labels[appsv1.DefaultDeploymentUniqueLabelKey] != "" &&
strings.HasSuffix(revision, labels[appsv1.DefaultDeploymentUniqueLabelKey]) {
return true
}
if pod.Labels[appsv1.ControllerRevisionHashLabelKey] != "" &&
strings.HasSuffix(revision, pod.Labels[appsv1.ControllerRevisionHashLabelKey]) {
if labels[appsv1.ControllerRevisionHashLabelKey] != "" &&
strings.HasSuffix(revision, labels[appsv1.ControllerRevisionHashLabelKey]) {
return true
}
return false

View File

@ -131,25 +131,6 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
return clone
}
// UpdateDaemonSet := func(object *appsv1alpha1.DaemonSet) *appsv1alpha1.DaemonSet {
// var daemon *appsv1alpha1.DaemonSet
// Expect(retry.RetryOnConflict(retry.DefaultRetry, func() error {
// daemon = &appsv1alpha1.DaemonSet{}
// err := GetObject(object.Name, daemon)
// if err != nil {
// return err
// }
// // daemon.Spec.Replicas = utilpointer.Int32(*object.Spec.Replicas)
// daemon.Spec.Template = *object.Spec.Template.DeepCopy()
// daemon.Spec.UpdateStrategy = *object.Spec.UpdateStrategy.DeepCopy()
// daemon.Labels = mergeMap(daemon.Labels, object.Labels)
// daemon.Annotations = mergeMap(daemon.Annotations, object.Annotations)
// return k8sClient.Update(context.TODO(), daemon)
// })).NotTo(HaveOccurred())
// return daemon
// }
UpdateNativeStatefulSet := func(object *apps.StatefulSet) *apps.StatefulSet {
var clone *apps.StatefulSet
Expect(retry.RetryOnConflict(retry.DefaultRetry, func() error {
@ -292,25 +273,6 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
}, 20*time.Minute, 3*time.Second).Should(BeTrue())
}
// WaitDaemonSetAllPodsReady := func(daemonset *appsv1alpha1.DaemonSet) {
// Eventually(func() bool {
// daemon := &appsv1alpha1.DaemonSet{}
// Expect(GetObject(daemonset.Name, daemon)).NotTo(HaveOccurred())
// klog.Infof("DaemonSet updateStrategy(%s) Generation(%d) ObservedGeneration(%d) DesiredNumberScheduled(%d) UpdatedNumberScheduled(%d) NumberReady(%d)",
// util.DumpJSON(daemon.Spec.UpdateStrategy), daemon.Generation, daemon.Status.ObservedGeneration, daemon.Status.DesiredNumberScheduled, daemon.Status.UpdatedNumberScheduled, daemon.Status.NumberReady)
// return daemon.Status.ObservedGeneration == daemon.Generation && daemon.Status.DesiredNumberScheduled == daemon.Status.UpdatedNumberScheduled && daemon.Status.DesiredNumberScheduled == daemon.Status.NumberReady
// }, 5*time.Minute, time.Second).Should(BeTrue())
// }
// WaitDeploymentCanaryReplicas := func(deployment *apps.Deployment) {
// Eventually(func() bool {
// clone := &apps.Deployment{}
// Expect(GetObject(deployment.Name, clone)).NotTo(HaveOccurred())
// return clone.Status.ObservedGeneration == clone.Generation &&
// *clone.Spec.Replicas == clone.Status.ReadyReplicas && *clone.Spec.Replicas == clone.Status.Replicas
// }, 10*time.Minute, time.Second).Should(BeTrue())
// }
WaitDeploymentBlueGreenReplicas := func(deployment *apps.Deployment) {
Eventually(func() bool {
clone := &apps.Deployment{}
@ -319,32 +281,6 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
}, 10*time.Minute, time.Second).Should(BeTrue())
}
// WaitClonesetBlueGreenReplicas := func(cloneset *appsv1alpha1.CloneSet) {
// Eventually(func() bool {
// clone := &appsv1alpha1.CloneSet{}
// Expect(GetObject(cloneset.Name, clone)).NotTo(HaveOccurred())
// return clone.Status.ObservedGeneration == clone.Generation &&
// *clone.Spec.Replicas == clone.Status.AvailableReplicas && clone.Status.ReadyReplicas == clone.Status.Replicas
// }, 10*time.Minute, time.Second).Should(BeTrue())
// }
// WaitRolloutStepUpgrade := func(name string, stepIndex int32) {
// start := time.Now()
// Eventually(func() bool {
// if start.Add(time.Minute * 5).Before(time.Now()) {
// DumpAllResources()
// Expect(true).Should(BeFalse())
// }
// clone := &v1beta1.Rollout{}
// Expect(GetObject(name, clone)).NotTo(HaveOccurred())
// if clone.Status.GetSubStatus() == nil {
// return false
// }
// klog.Infof("current step:%v target step:%v current step state %v", clone.Status.GetSubStatus().CurrentStepIndex, stepIndex, clone.Status.GetSubStatus().CurrentStepState)
// return clone.Status.GetSubStatus().CurrentStepIndex == stepIndex && clone.Status.GetSubStatus().CurrentStepState == v1beta1.CanaryStepStateUpgrade
// }, 20*time.Minute, time.Second).Should(BeTrue())
// }
WaitRolloutStepPaused := func(name string, stepIndex int32) {
start := time.Now()
Eventually(func() bool {
@ -439,6 +375,37 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(count).Should(BeNumerically("==", expected))
}
// CheckPodBatchLabel cannot be located if error occurs. use this as Expect(CheckPodBatchLabelV2(...)).Should(Succeed())
CheckPodBatchLabelV2 := func(namespace string, labelSelector *metav1.LabelSelector, rolloutID, batchID string, expected int) error {
fn := func() error {
pods, err := ListPods(namespace, labelSelector)
if err != nil {
return err
}
count := 0
for _, pod := range pods {
if pod.Labels[v1beta1.RolloutIDLabel] == rolloutID &&
pod.Labels[v1beta1.RolloutBatchIDLabel] == batchID {
count++
}
}
if count != expected {
return fmt.Errorf("expected %d pods with rolloutID %s and batchID %s, got %d", expected, rolloutID, batchID, count)
}
klog.InfoS("check pod batch label success", "count", count, "rolloutID", rolloutID, "batchID", batchID)
return nil
}
var err error
for i := 0; i < 120; i++ {
if err = fn(); err == nil {
return nil
}
time.Sleep(time.Second)
}
return err
}
ListReplicaSet := func(d *apps.Deployment) []*apps.ReplicaSet {
var rss []*apps.ReplicaSet
rsLister := &apps.ReplicaSetList{}
@ -1695,12 +1662,6 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
cService := &v1.Service{}
Expect(GetObject(service.Name+"-canary", cService)).NotTo(HaveOccurred())
Expect(cService.Spec.Selector[apps.DefaultDeploymentUniqueLabelKey]).Should(Equal(canaryRevision))
// canary ingress
// when traffic is 0%, ingress canary won't create and annotation won't be set (for ingress-nginx)
// cIngress := &netv1.Ingress{}
// Expect(GetObject(service.Name+"-canary", cIngress)).NotTo(HaveOccurred())
// Expect(cIngress.Annotations[fmt.Sprintf("%s/canary", nginxIngressAnnotationDefaultPrefix)]).Should(Equal("true"))
// Expect(cIngress.Annotations[fmt.Sprintf("%s/canary-weight", nginxIngressAnnotationDefaultPrefix)]).Should(Equal(removePercentageSign(*rollout.Spec.Strategy.BlueGreen.Steps[0].Traffic)))
// ------ step 2: replicas: 100%, traffic: 0% ------
// resume rollout canary
@ -1727,6 +1688,9 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(rollout.Status.BlueGreenStatus.UpdatedReplicas).Should(BeNumerically("==", 5))
Expect(rollout.Status.BlueGreenStatus.UpdatedReadyReplicas).Should(BeNumerically("==", 5))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 3)).Should(Succeed())
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "2", 2)).Should(Succeed())
// ------ step 3: replicas: 100%, traffic: 50% ------
// resume rollout canary
ResumeRollout(rollout.Name)
@ -1940,12 +1904,6 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
cService := &v1.Service{}
Expect(GetObject(service.Name+"-canary", cService)).NotTo(HaveOccurred())
Expect(cService.Spec.Selector[apps.DefaultDeploymentUniqueLabelKey]).Should(Equal(canaryRevision))
// canary ingress
// when traffic is 0%, ingress canary won't create and annotation won't be set (for ingress-nginx)
// cIngress := &netv1.Ingress{}
// Expect(GetObject(service.Name+"-canary", cIngress)).NotTo(HaveOccurred())
// Expect(cIngress.Annotations[fmt.Sprintf("%s/canary", nginxIngressAnnotationDefaultPrefix)]).Should(Equal("true"))
// Expect(cIngress.Annotations[fmt.Sprintf("%s/canary-weight", nginxIngressAnnotationDefaultPrefix)]).Should(Equal(removePercentageSign(*rollout.Spec.Strategy.BlueGreen.Steps[0].Traffic)))
// ------ step 2: replicas: 100%, traffic: 0% ------
// resume rollout canary
@ -1972,6 +1930,10 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(rollout.Status.BlueGreenStatus.UpdatedReplicas).Should(BeNumerically("==", 5))
Expect(rollout.Status.BlueGreenStatus.UpdatedReadyReplicas).Should(BeNumerically("==", 5))
By("checking pod labels")
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 3)).Should(Succeed())
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "2", 2)).Should(Succeed())
// ------ step 3: replicas: 100%, traffic: 50% ------
// resume rollout canary
ResumeRollout(rollout.Name)
@ -2134,7 +2096,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(GetObject(rollout.Name, rollout)).NotTo(HaveOccurred())
cond := getRolloutCondition(rollout.Status, v1beta1.RolloutConditionProgressing)
Expect(string(cond.Reason)).Should(Equal(string(v1beta1.CanaryStepStateCompleted)))
Expect(cond.Reason).Should(Equal(string(v1beta1.CanaryStepStateCompleted)))
Expect(string(cond.Status)).Should(Equal(string(metav1.ConditionFalse)))
// canary ingress and canary service should be deleted
cIngress = &netv1.Ingress{}
@ -2145,6 +2107,10 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
// check service update
Expect(GetObject(service.Name, service)).NotTo(HaveOccurred())
Expect(service.Spec.Selector[apps.DefaultDeploymentUniqueLabelKey]).Should(Equal(""))
By("checking pod labelsa after rollback")
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 0)).Should(Succeed())
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "2", 0)).Should(Succeed())
})
It("bluegreen deployment continuous rolling case", func() {
@ -2372,6 +2338,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(workload.Status.UpdatedReplicas).Should(BeNumerically("==", 3))
Expect(workload.Status.UnavailableReplicas).Should(BeNumerically("==", 9))
Expect(workload.Status.ReadyReplicas).Should(BeNumerically("==", 9))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 3)).Should(Succeed())
// ------ scale up: from 6 to 7 ------
workload.Spec.Replicas = utilpointer.Int32(7)
@ -2387,6 +2354,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(workload.Status.UpdatedReplicas).Should(BeNumerically("==", 4))
Expect(workload.Status.UnavailableReplicas).Should(BeNumerically("==", 11))
Expect(workload.Status.ReadyReplicas).Should(BeNumerically("==", 11))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 4)).Should(Succeed())
// ------ scale up: from 7 to 8 ------
workload.Spec.Replicas = utilpointer.Int32(8)
@ -2403,6 +2371,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(workload.Status.UpdatedReplicas).Should(BeNumerically("==", 4))
Expect(workload.Status.UnavailableReplicas).Should(BeNumerically("==", 12))
Expect(workload.Status.ReadyReplicas).Should(BeNumerically("==", 12))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 4)).Should(Succeed())
// ------ scale down: from 8 to 4 ------
workload.Spec.Replicas = utilpointer.Int32(4)
@ -2419,6 +2388,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(workload.Status.UpdatedReplicas).Should(BeNumerically("==", 2))
Expect(workload.Status.UnavailableReplicas).Should(BeNumerically("==", 6))
Expect(workload.Status.ReadyReplicas).Should(BeNumerically("==", 6))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 2)).Should(Succeed())
// ------ step 2: replicas: 100%, traffic: 0% ------
// resume rollout canary
@ -2459,6 +2429,8 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(workload.Status.UpdatedReplicas).Should(BeNumerically("==", 7))
Expect(workload.Status.UnavailableReplicas).Should(BeNumerically("==", 14))
Expect(workload.Status.ReadyReplicas).Should(BeNumerically("==", 14))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 4)).Should(Succeed())
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "2", 3)).Should(Succeed())
// ------ scale up: from 7 to 8 ------
workload.Spec.Replicas = utilpointer.Int32(8)
@ -2474,6 +2446,8 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(workload.Status.UpdatedReplicas).Should(BeNumerically("==", 8))
Expect(workload.Status.UnavailableReplicas).Should(BeNumerically("==", 16))
Expect(workload.Status.ReadyReplicas).Should(BeNumerically("==", 16))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 4)).Should(Succeed())
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "2", 4)).Should(Succeed())
// ------ scale down: from 8 to 4 ------
workload.Spec.Replicas = utilpointer.Int32(4)
@ -2489,7 +2463,8 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(workload.Status.UpdatedReplicas).Should(BeNumerically("==", 4))
Expect(workload.Status.UnavailableReplicas).Should(BeNumerically("==", 8))
Expect(workload.Status.ReadyReplicas).Should(BeNumerically("==", 8))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 2)).Should(Succeed())
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "2", 2)).Should(Succeed())
})
It("bluegreen delete rollout case", func() {
@ -2957,10 +2932,13 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
WaitCloneSetAllPodsReady(workload)
// check rollout status
Expect(GetObject(rollout.Name, rollout)).NotTo(HaveOccurred())
Expect(GetObject(workload.Name, workload)).NotTo(HaveOccurred())
Expect(rollout.Status.Phase).Should(Equal(v1beta1.RolloutPhaseHealthy))
Expect(rollout.Status.BlueGreenStatus.StableRevision).Should(Equal(workload.Status.CurrentRevision[strings.LastIndex(workload.Status.CurrentRevision, "-")+1:]))
Eventually(func(g Gomega) {
g.Expect(GetObject(rollout.Name, rollout)).NotTo(HaveOccurred())
g.Expect(GetObject(workload.Name, workload)).NotTo(HaveOccurred())
g.Expect(rollout.Status.Phase).Should(Equal(v1beta1.RolloutPhaseHealthy))
g.Expect(rollout.Status.BlueGreenStatus.StableRevision).Should(Equal(workload.Status.CurrentRevision[strings.LastIndex(workload.Status.CurrentRevision, "-")+1:]))
}).WithTimeout(10 * time.Second).WithPolling(time.Second).Should(Succeed())
stableRevision := rollout.Status.BlueGreenStatus.StableRevision
By("check rollout status & paused success")
@ -2991,6 +2969,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(rollout.Status.BlueGreenStatus.CurrentStepIndex).Should(BeNumerically("==", 1))
Expect(rollout.Status.BlueGreenStatus.NextStepIndex).Should(BeNumerically("==", 2))
Expect(rollout.Status.BlueGreenStatus.RolloutHash).Should(Equal(rollout.Annotations[util.RolloutHashAnnotation]))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 5)).Should(Succeed())
// check stable, canary service & ingress
// stable service
Expect(GetObject(service.Name, service)).NotTo(HaveOccurred())
@ -3118,10 +3097,12 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
WaitCloneSetAllPodsReady(workload)
// check rollout status
Expect(GetObject(rollout.Name, rollout)).NotTo(HaveOccurred())
Expect(GetObject(workload.Name, workload)).NotTo(HaveOccurred())
Expect(rollout.Status.Phase).Should(Equal(v1beta1.RolloutPhaseHealthy))
Expect(rollout.Status.BlueGreenStatus.StableRevision).Should(Equal(workload.Status.CurrentRevision[strings.LastIndex(workload.Status.CurrentRevision, "-")+1:]))
Eventually(func(g Gomega) {
g.Expect(GetObject(rollout.Name, rollout)).NotTo(HaveOccurred())
g.Expect(GetObject(workload.Name, workload)).NotTo(HaveOccurred())
g.Expect(rollout.Status.Phase).Should(Equal(v1beta1.RolloutPhaseHealthy))
g.Expect(rollout.Status.BlueGreenStatus.StableRevision).Should(Equal(workload.Status.CurrentRevision[strings.LastIndex(workload.Status.CurrentRevision, "-")+1:]))
}).WithTimeout(time.Second * 30).WithPolling(time.Second).Should(Succeed())
stableRevision := rollout.Status.BlueGreenStatus.StableRevision
By("check rollout status & paused success")
@ -3152,6 +3133,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(rollout.Status.BlueGreenStatus.CurrentStepIndex).Should(BeNumerically("==", 1))
Expect(rollout.Status.BlueGreenStatus.NextStepIndex).Should(BeNumerically("==", 2))
Expect(rollout.Status.BlueGreenStatus.RolloutHash).Should(Equal(rollout.Annotations[util.RolloutHashAnnotation]))
Expect(CheckPodBatchLabelV2(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 5)).Should(Succeed())
// check stable, canary service & ingress
// stable service
Expect(GetObject(service.Name, service)).NotTo(HaveOccurred())
@ -3259,6 +3241,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(rollout.Status.BlueGreenStatus.CurrentStepIndex).Should(BeNumerically("==", 1))
Expect(rollout.Status.BlueGreenStatus.NextStepIndex).Should(BeNumerically("==", 2))
Expect(rollout.Status.BlueGreenStatus.RolloutHash).Should(Equal(rollout.Annotations[util.RolloutHashAnnotation]))
CheckPodBatchLabel(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 5)
// if network configuration has restored
cIngress = &netv1.Ingress{}
Expect(GetObject(service.Name+"-canary", cIngress)).NotTo(HaveOccurred())
@ -3275,7 +3258,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(GetObject(rollout.Name, rollout)).NotTo(HaveOccurred())
cond := getRolloutCondition(rollout.Status, v1beta1.RolloutConditionProgressing)
Expect(string(cond.Reason)).Should(Equal(string(v1beta1.CanaryStepStateCompleted)))
Expect(cond.Reason).Should(Equal(string(v1beta1.CanaryStepStateCompleted)))
Expect(string(cond.Status)).Should(Equal(string(metav1.ConditionFalse)))
CheckIngressRestored(service.Name)
@ -3285,6 +3268,7 @@ var _ = SIGDescribe("Rollout v1beta1", func() {
Expect(workload.Status.UpdatedReplicas).Should(BeNumerically("==", 5))
Expect(workload.Status.UpdatedReadyReplicas).Should(BeNumerically("==", 5))
Expect(workload.Spec.UpdateStrategy.Paused).Should(BeFalse())
CheckPodBatchLabel(namespace, workload.Spec.Selector, rollout.Status.BlueGreenStatus.ObservedRolloutID, "1", 0)
})
It("bluegreen continuous rolling case for cloneset", func() {