allow rollout even if revision not change (#98)

Signed-off-by: mingzhou.swx <mingzhou.swx@alibaba-inc.com>

Signed-off-by: mingzhou.swx <mingzhou.swx@alibaba-inc.com>
Co-authored-by: mingzhou.swx <mingzhou.swx@alibaba-inc.com>
This commit is contained in:
Wei-Xiang Sun 2022-11-01 14:48:24 +08:00 committed by GitHub
parent b7315e1658
commit 5924c727a7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 276 additions and 139 deletions

View File

@ -68,6 +68,10 @@ type BatchReleaseStatus struct {
// It corresponds to this BatchRelease's generation, which is updated on mutation
// by the API Server, and only if BatchRelease Spec was changed, its generation will increase 1.
ObservedGeneration int64 `json:"observedGeneration,omitempty"`
// ObservedRolloutID is the most recent rollout-id observed for this BatchRelease.
// If RolloutID was changed, we will restart to roll out from batch 0,
// to ensure the batch-id and rollout-id labels of Pods are correct.
ObservedRolloutID string `json:"observedRolloutID,omitempty"`
// ObservedWorkloadReplicas is observed replicas of target referenced workload.
// This field is designed to deal with scaling event during rollout, if this field changed,
// it means that the workload is scaling during rollout.

View File

@ -216,6 +216,12 @@ spec:
description: ObservedReleasePlanHash is a hash code of observed itself
spec.releasePlan.
type: string
observedRolloutID:
description: ObservedRolloutID is the most recent rollout-id observed
for this BatchRelease. If RolloutID was changed, we will restart
to roll out from batch 0, to ensure the batch-id and rollout-id
labels of Pods are correct.
type: string
observedWorkloadReplicas:
description: ObservedWorkloadReplicas is observed replicas of target
referenced workload. This field is designed to deal with scaling

View File

@ -71,7 +71,7 @@ func (p podEventHandler) Update(evt event.UpdateEvent, q workqueue.RateLimitingI
if !oldOK || !newOK {
return
}
if oldPod.ResourceVersion == newPod.ResourceVersion || util.IsPodReady(oldPod) == util.IsPodReady(newPod) {
if oldPod.ResourceVersion == newPod.ResourceVersion || (util.IsEqualRevision(oldPod, newPod) && util.IsPodReady(oldPod) == util.IsPodReady(newPod)) {
return
}

View File

@ -74,15 +74,19 @@ func signalRecalculate(release *v1alpha1.BatchRelease, newStatus *v1alpha1.Batch
// When BatchRelease plan was changed, rollout controller will update this batchRelease cr,
// and rollout controller will set BatchPartition as its expected current batch index.
currentBatch := int32(0)
if release.Spec.ReleasePlan.BatchPartition != nil {
// if rollout-id is not changed, just use batchPartition;
// if rollout-id is changed, we should patch pod batch id from batch 0.
observedRolloutID := release.Status.ObservedRolloutID
if release.Spec.ReleasePlan.BatchPartition != nil && release.Spec.ReleasePlan.RolloutID == observedRolloutID {
// ensure current batch upper bound
currentBatch = integer.Int32Min(*release.Spec.ReleasePlan.BatchPartition, int32(len(release.Spec.ReleasePlan.Batches)-1))
}
klog.Infof("BatchRelease(%v) canary batch changed from %v to %v when the release plan changed",
client.ObjectKeyFromObject(release), newStatus.CanaryStatus.CurrentBatch, currentBatch)
klog.Infof("BatchRelease(%v) canary batch changed from %v to %v when the release plan changed, observed-rollout-id: %s, current-rollout-id: %s",
client.ObjectKeyFromObject(release), newStatus.CanaryStatus.CurrentBatch, currentBatch, observedRolloutID, release.Spec.ReleasePlan.RolloutID)
newStatus.CanaryStatus.BatchReadyTime = nil
newStatus.CanaryStatus.CurrentBatch = currentBatch
newStatus.ObservedRolloutID = release.Spec.ReleasePlan.RolloutID
newStatus.CanaryStatus.CurrentBatchState = v1alpha1.UpgradingBatchState
newStatus.ObservedReleasePlanHash = util.HashReleasePlanBatches(&release.Spec.ReleasePlan)
}

View File

@ -57,40 +57,6 @@ func NewCloneSetRolloutController(cli client.Client, recorder record.EventRecord
// VerifyWorkload verifies that the workload is ready to execute release plan
func (c *CloneSetRolloutController) VerifyWorkload() (bool, error) {
var err error
var message string
defer func() {
if err != nil {
c.recorder.Event(c.release, v1.EventTypeWarning, "VerifyFailed", err.Error())
} else if message != "" {
klog.Warningf(message)
}
}()
if err = c.fetchCloneSet(); err != nil {
message = err.Error()
return false, err
}
// if the workload status is untrustworthy
if c.clone.Status.ObservedGeneration != c.clone.Generation {
message = fmt.Sprintf("CloneSet(%v) is still reconciling, wait for it to be done", c.targetNamespacedName)
return false, nil
}
// if the cloneSet has been promoted, no need to go on
if c.clone.Status.UpdatedReplicas == *c.clone.Spec.Replicas {
message = fmt.Sprintf("CloneSet(%v) update revision has been promoted, no need to reconcile", c.targetNamespacedName)
return false, nil
}
// if the cloneSet is not paused and is not under our control
if !(c.clone.Spec.UpdateStrategy.Paused || c.clone.Spec.UpdateStrategy.Partition.IntVal > *c.clone.Spec.Replicas || c.clone.Spec.UpdateStrategy.Partition.StrVal == "100%") {
message = fmt.Sprintf("CloneSet(%v) should be paused before execute the release plan", c.targetNamespacedName)
return false, nil
}
c.recorder.Event(c.release, v1.EventTypeNormal, "VerifiedSuccessfully", "ReleasePlan and the CloneSet resource are verified")
return true, nil
}
@ -238,10 +204,8 @@ func (c *CloneSetRolloutController) UpgradeOneBatch() (bool, error) {
"expectedBatchStableReplicas", expectedBatchStableReplicas,
"expectedPartition", expectedPartition)
// 1. the number of upgrade pod satisfied; 2. partition has been satisfied
IsWorkloadUpgraded := updatedReplicas >= expectedBatchCanaryReplicas && int32(partitionedStableReplicas) <= expectedBatchStableReplicas
if !IsWorkloadUpgraded {
return false, c.patchCloneSetPartition(c.clone, &expectedPartition)
if err := c.patchCloneSetPartition(c.clone, &expectedPartition); err != nil {
return false, err
}
patchDone, err := c.patchPodBatchLabel(pods, plannedBatchCanaryReplicas, expectedBatchStableReplicas)

View File

@ -20,6 +20,7 @@ import (
"context"
"encoding/json"
"fmt"
"reflect"
kruiseappsv1alpha1 "github.com/openkruise/kruise-api/apps/v1alpha1"
"github.com/openkruise/rollouts/pkg/util"
@ -138,6 +139,10 @@ func (c *cloneSetController) releaseCloneSet(clone *kruiseappsv1alpha1.CloneSet,
// scale the deployment
func (c *cloneSetController) patchCloneSetPartition(clone *kruiseappsv1alpha1.CloneSet, partition *intstr.IntOrString) error {
if reflect.DeepEqual(clone.Spec.UpdateStrategy.Partition, partition) {
return nil
}
patch := map[string]interface{}{
"spec": map[string]interface{}{
"updateStrategy": map[string]interface{}{

View File

@ -62,46 +62,10 @@ func NewDeploymentRolloutController(cli client.Client, recorder record.EventReco
// VerifyWorkload verifies that the workload is ready to execute release plan
func (c *DeploymentsRolloutController) VerifyWorkload() (bool, error) {
var err error
var message string
defer func() {
if err != nil {
c.recorder.Event(c.release, v1.EventTypeWarning, "VerifyFailed", err.Error())
} else if message != "" {
klog.Warningf(message)
}
}()
if err = c.fetchStableDeployment(); err != nil {
return false, err
}
if err = c.fetchCanaryDeployment(); client.IgnoreNotFound(err) != nil {
return false, err
}
// if the workload status is untrustworthy, return and retry
if c.stable.Status.ObservedGeneration != c.stable.Generation {
message = fmt.Sprintf("deployment(%v) is still reconciling, wait for it to be done", c.stableNamespacedName)
return false, nil
}
// if the workload has been promoted, return and not retry
if c.stable.Status.UpdatedReplicas == *c.stable.Spec.Replicas {
message = fmt.Sprintf("deployment(%v) update revision has been promoted, no need to rollout", c.stableNamespacedName)
return false, nil
}
// if the workload is not paused, no need to progress it
if !c.stable.Spec.Paused {
message = fmt.Sprintf("deployment(%v) should be paused before execute the release plan", c.stableNamespacedName)
return false, nil
}
// claim the deployment is under our control, and create canary deployment if it needs.
// Do not move this function to Preparing phase, otherwise multi canary deployments
// will be repeatedly created due to informer cache latency.
if _, err = c.claimDeployment(c.stable, c.canary); err != nil {
if _, err := c.claimDeployment(c.stable, c.canary); err != nil {
return false, err
}
@ -145,11 +109,8 @@ func (c *DeploymentsRolloutController) UpgradeOneBatch() (bool, error) {
"current-canary-replicas", currentCanaryReplicas,
"current-canary-status-replicas", c.canary.Status.UpdatedReplicas)
// upgrade pods if it needs
if currentCanaryReplicas < canaryGoal {
if err := c.patchDeploymentReplicas(c.canary, canaryGoal); err != nil {
return false, err
}
if err := c.patchDeploymentReplicas(c.canary, canaryGoal); err != nil {
return false, err
}
// patch current batch label to pods

View File

@ -203,6 +203,10 @@ func (c *deploymentController) releaseDeployment(stableDeploy *apps.Deployment,
// scale the deployment
func (c *deploymentController) patchDeploymentReplicas(deploy *apps.Deployment, replicas int32) error {
if *deploy.Spec.Replicas >= replicas {
return nil
}
patch := map[string]interface{}{
"spec": map[string]interface{}{
"replicas": pointer.Int32Ptr(replicas),

View File

@ -66,40 +66,6 @@ func NewUnifiedWorkloadRolloutControlPlane(f NewUnifiedControllerFunc, c client.
// VerifyWorkload verifies that the workload is ready to execute release plan
func (c *UnifiedWorkloadRolloutControlPlane) VerifyWorkload() (bool, error) {
var err error
var message string
defer func() {
if err != nil {
c.recorder.Event(c.release, v1.EventTypeWarning, "VerifyFailed", err.Error())
} else if message != "" {
klog.Warningf(message)
}
}()
workloadInfo, err := c.GetWorkloadInfo()
if err != nil {
return false, err
}
// If the workload status is untrustworthy
if workloadInfo.Status.ObservedGeneration != workloadInfo.Generation {
message = fmt.Sprintf("%v is still reconciling, wait for it to be done", workloadInfo.GVKWithName)
return false, nil
}
// If the workload has been promoted, no need to go on
if workloadInfo.Status.UpdatedReplicas == *workloadInfo.Replicas {
message = fmt.Sprintf("%v update revision has been promoted, no need to reconcile", workloadInfo.GVKWithName)
return false, nil
}
// If the workload is not paused and is not under our control
if !workloadInfo.Paused {
message = fmt.Sprintf("%v should be paused before execute the release plan", workloadInfo.GVKWithName)
return false, nil
}
c.recorder.Event(c.release, v1.EventTypeNormal, "VerifiedSuccessfully", "ReleasePlan and the workload resource are verified")
return true, nil
}

View File

@ -25,6 +25,9 @@ type BatchRelease interface {
// return whether the batchRelease configuration is consistent with the rollout step
Verify(index int32) (bool, error)
// SyncRolloutID will sync rollout id from Rollout to BatchRelease
SyncRolloutID(currentID string) error
// 1. Promote release workload in step(index), 1<=index<=len(step)
// 2. Promote will resume stable workload if the last batch(index=-1) is finished
Promote(index int32, isRollback, checkReady bool) (bool, error)

View File

@ -63,6 +63,30 @@ func NewInnerBatchController(c client.Client, rollout *rolloutv1alpha1.Rollout,
return r
}
func (r *innerBatchRelease) SyncRolloutID(currentID string) error {
if err := retry.RetryOnConflict(retry.DefaultBackoff, func() error {
batch := &rolloutv1alpha1.BatchRelease{}
if err := r.Get(context.TODO(), client.ObjectKey{Namespace: r.rollout.Namespace, Name: r.batchName}, batch); err != nil {
if errors.IsNotFound(err) {
return nil // just return nil if batchRelease not exist
}
return err
}
if batch.Spec.ReleasePlan.RolloutID == currentID {
return nil
}
batch.Spec.ReleasePlan.RolloutID = r.rolloutID
if err := r.Client.Update(context.TODO(), batch); err != nil {
return err
}
return nil
}); err != nil {
klog.Errorf("rollout(%s/%s) update batchRelease rolloutID %s failed: %s", r.rollout.Namespace, r.rollout.Name, currentID, err.Error())
return err
}
return nil
}
func (r *innerBatchRelease) Verify(index int32) (bool, error) {
index = index - 1
batch := &rolloutv1alpha1.BatchRelease{}
@ -84,7 +108,7 @@ func (r *innerBatchRelease) Verify(index int32) (bool, error) {
// check whether batchRelease configuration is the latest
newBr := createBatchRelease(r.rollout, r.batchName, r.rolloutID)
if reflect.DeepEqual(batch.Spec.ReleasePlan.Batches, newBr.Spec.ReleasePlan.Batches) {
if batchPlanDeepEqual(batch, newBr, index) {
klog.Infof("rollout(%s/%s) batchRelease(generation:%d) configuration is the latest", r.rollout.Namespace, r.rollout.Name, batch.Generation)
return true, nil
}
@ -95,6 +119,7 @@ func (r *innerBatchRelease) Verify(index int32) (bool, error) {
klog.Errorf("error getting updated BatchRelease(%s/%s) from client", batch.Namespace, batch.Name)
return err
}
batch.Spec.ReleasePlan.RolloutID = r.rolloutID
batch.Spec.ReleasePlan.Batches = newBr.Spec.ReleasePlan.Batches
batch.Spec.ReleasePlan.BatchPartition = utilpointer.Int32Ptr(index)
if err = r.Client.Update(context.TODO(), batch); err != nil {
@ -363,3 +388,13 @@ func IsPromoted(rollout *rolloutv1alpha1.Rollout, batch *rolloutv1alpha1.BatchRe
}
return true
}
func batchPlanDeepEqual(old, new *rolloutv1alpha1.BatchRelease, currentBatch int32) bool {
if old.Spec.ReleasePlan.BatchPartition == nil || *old.Spec.ReleasePlan.BatchPartition != currentBatch {
return false
}
if old.Spec.ReleasePlan.RolloutID != new.Spec.ReleasePlan.RolloutID {
return false
}
return reflect.DeepEqual(old.Spec.ReleasePlan.Batches, new.Spec.ReleasePlan.Batches)
}

View File

@ -38,6 +38,7 @@ func (r *rolloutContext) runCanary() error {
if canaryStatus.CanaryRevision == "" {
canaryStatus.CurrentStepState = rolloutv1alpha1.CanaryStepStateUpgrade
canaryStatus.CanaryRevision = r.workload.CanaryRevision
canaryStatus.ObservedRolloutID = getRolloutID(r.workload, r.rollout)
canaryStatus.CurrentStepIndex = 1
canaryStatus.RolloutHash = r.rollout.Annotations[util.RolloutHashAnnotation]
}
@ -45,13 +46,21 @@ func (r *rolloutContext) runCanary() error {
// update canary status
batch, err := r.batchControl.FetchBatchRelease()
if err != nil {
canaryStatus.Message = "BatchRelease not found"
canaryStatus.CanaryReplicas = r.workload.CanaryReplicas
canaryStatus.CanaryReadyReplicas = r.workload.CanaryReadyReplicas
} else {
canaryStatus.Message = fmt.Sprintf("BatchRelease at state %s, id %s, step %d",
batch.Status.CanaryStatus.CurrentBatchState, batch.Status.ObservedRolloutID, batch.Status.CanaryStatus.CurrentBatch+1)
canaryStatus.CanaryReplicas = batch.Status.CanaryStatus.UpdatedReplicas
canaryStatus.CanaryReadyReplicas = batch.Status.CanaryStatus.UpdatedReadyReplicas
}
// sync rollout-id to batchRelease if we need
if err := r.batchControl.SyncRolloutID(r.newStatus.CanaryStatus.ObservedRolloutID); err != nil {
return err
}
switch canaryStatus.CurrentStepState {
case rolloutv1alpha1.CanaryStepStateUpgrade:
klog.Infof("rollout(%s/%s) run canary strategy, and state(%s)", r.rollout.Namespace, r.rollout.Name, rolloutv1alpha1.CanaryStepStateUpgrade)

View File

@ -99,7 +99,7 @@ func (r *RolloutReconciler) updateRolloutStatus(rollout *rolloutv1alpha1.Rollout
newStatus.Phase = rolloutv1alpha1.RolloutPhaseProgressing
cond := util.NewRolloutCondition(rolloutv1alpha1.RolloutConditionProgressing, corev1.ConditionFalse, rolloutv1alpha1.ProgressingReasonInitializing, "Rollout is in Progressing")
util.SetRolloutCondition(&newStatus, *cond)
} else if workload.IsInStable && newStatus.CanaryStatus == nil {
} else if newStatus.CanaryStatus == nil {
// The following logic is to make PaaS be able to judge whether the rollout is ready
// at the first deployment of the Rollout/Workload. For example: generally, a PaaS
// platform can use the following code to judge whether the rollout progression is completed:

View File

@ -54,8 +54,6 @@ type Workload struct {
// Revision hash key
RevisionLabelKey string
// Is it in stable and no need to publish
IsInStable bool
// Is it in rollback phase
IsInRollback bool
// indicate whether the workload can enter the rollout process
@ -143,12 +141,6 @@ func (r *ControllerFinder) getKruiseCloneSet(namespace string, ref *rolloutv1alp
IsStatusConsistent: true,
}
// no need to progress
if cloneSet.Status.Replicas == cloneSet.Status.UpdatedReplicas && cloneSet.Status.CurrentRevision == cloneSet.Status.UpdateRevision {
workload.IsInStable = true
return workload, nil
}
// not in rollout progressing
if _, ok = workload.Annotations[InRolloutProgressingAnnotation]; !ok {
return workload, nil
@ -197,11 +189,6 @@ func (r *ControllerFinder) getDeployment(namespace string, ref *rolloutv1alpha1.
RevisionLabelKey: apps.DefaultDeploymentUniqueLabelKey,
}
// deployment is stable
if stable.Status.Replicas == stable.Status.UpdatedReplicas {
workload.IsInStable = true
}
// not in rollout progressing
if _, ok = workload.Annotations[InRolloutProgressingAnnotation]; !ok {
return workload, nil
@ -267,12 +254,6 @@ func (r *ControllerFinder) getStatefulSetLikeWorkload(namespace string, ref *rol
IsStatusConsistent: true,
}
// no need to progress
if workloadInfo.Status.Replicas == workloadInfo.Status.UpdatedReplicas && workloadInfo.Status.StableRevision == workloadInfo.Status.UpdateRevision {
workload.IsInStable = true
return workload, nil
}
// not in rollout progressing
if _, ok := workload.Annotations[InRolloutProgressingAnnotation]; !ok {
return workload, nil

View File

@ -82,6 +82,19 @@ func IsConsistentWithRevision(pod *v1.Pod, revision string) bool {
return false
}
func IsEqualRevision(a, b *v1.Pod) bool {
if a.Labels[appsv1.DefaultDeploymentUniqueLabelKey] != "" &&
a.Labels[appsv1.DefaultDeploymentUniqueLabelKey] == b.Labels[appsv1.DefaultDeploymentUniqueLabelKey] {
return true
}
if a.Labels[appsv1.ControllerRevisionHashLabelKey] != "" &&
a.Labels[appsv1.ControllerRevisionHashLabelKey] == b.Labels[appsv1.ControllerRevisionHashLabelKey] {
return true
}
return false
}
// FilterActivePods will filter out terminating pods
func FilterActivePods(pods []*v1.Pod) []*v1.Pod {
var activePods []*v1.Pod

View File

@ -3663,6 +3663,186 @@ var _ = SIGDescribe("Rollout", func() {
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "4", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "5", 1)
})
It("patch batch id to pods: only rollout-id changes", func() {
By("Creating Rollout...")
rollout := &rolloutsv1alpha1.Rollout{}
Expect(ReadYamlToObject("./test_data/rollout/rollout_canary_base.yaml", rollout)).ToNot(HaveOccurred())
rollout.Spec.ObjectRef.WorkloadRef = &rolloutsv1alpha1.WorkloadRef{
APIVersion: "apps.kruise.io/v1alpha1",
Kind: "CloneSet",
Name: "echoserver",
}
rollout.Spec.Strategy.Canary.TrafficRoutings = nil
rollout.Annotations = map[string]string{
util.RollbackInBatchAnnotation: "true",
}
rollout.Spec.Strategy.Canary.Steps = []rolloutsv1alpha1.CanaryStep{
{
Weight: utilpointer.Int32(20),
Pause: rolloutsv1alpha1.RolloutPause{},
},
{
Weight: utilpointer.Int32(40),
Pause: rolloutsv1alpha1.RolloutPause{},
},
{
Weight: utilpointer.Int32(60),
Pause: rolloutsv1alpha1.RolloutPause{},
},
{
Weight: utilpointer.Int32(80),
Pause: rolloutsv1alpha1.RolloutPause{},
},
{
Weight: utilpointer.Int32(100),
Pause: rolloutsv1alpha1.RolloutPause{
Duration: utilpointer.Int32(0),
},
},
}
CreateObject(rollout)
By("Creating workload and waiting for all pods ready...")
workload := &appsv1alpha1.CloneSet{}
Expect(ReadYamlToObject("./test_data/rollout/cloneset.yaml", workload)).ToNot(HaveOccurred())
CreateObject(workload)
WaitCloneSetAllPodsReady(workload)
By("Only update rollout id = '1', and start rollout")
workload.Labels[util.RolloutIDLabel] = "1"
workload.Annotations[util.InRolloutProgressingAnnotation] = "true"
UpdateCloneSet(workload)
By("wait step(1) pause")
WaitRolloutCanaryStepPaused(rollout.Name, 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "1", "1", 1)
By("wait step(2) pause")
ResumeRolloutCanary(rollout.Name)
WaitRolloutCanaryStepPaused(rollout.Name, 2)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "1", "2", 1)
By("wait step(3) pause")
ResumeRolloutCanary(rollout.Name)
WaitRolloutCanaryStepPaused(rollout.Name, 3)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "1", "3", 1)
By("Only update rollout id = '2', and check batch label again")
workload.Labels[util.RolloutIDLabel] = "2"
UpdateCloneSet(workload)
By("wait step(3) pause again")
WaitRolloutCanaryStepPaused(rollout.Name, 3)
time.Sleep(30 * time.Second)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "1", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "2", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "3", 1)
By("wait step(4) pause")
ResumeRolloutCanary(rollout.Name)
WaitRolloutCanaryStepPaused(rollout.Name, 4)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "4", 1)
By("Wait rollout complete")
ResumeRolloutCanary(rollout.Name)
WaitRolloutStatusPhase(rollout.Name, rolloutsv1alpha1.RolloutPhaseHealthy)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "1", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "2", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "3", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "4", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "5", 1)
})
It("patch batch id to pods: only change rollout-id after rolling the first step", func() {
By("Creating Rollout...")
rollout := &rolloutsv1alpha1.Rollout{}
Expect(ReadYamlToObject("./test_data/rollout/rollout_canary_base.yaml", rollout)).ToNot(HaveOccurred())
rollout.Spec.ObjectRef.WorkloadRef = &rolloutsv1alpha1.WorkloadRef{
APIVersion: "apps.kruise.io/v1alpha1",
Kind: "CloneSet",
Name: "echoserver",
}
rollout.Spec.Strategy.Canary.TrafficRoutings = nil
rollout.Annotations = map[string]string{
util.RollbackInBatchAnnotation: "true",
}
rollout.Spec.Strategy.Canary.Steps = []rolloutsv1alpha1.CanaryStep{
{
Weight: utilpointer.Int32(20),
Pause: rolloutsv1alpha1.RolloutPause{},
},
{
Weight: utilpointer.Int32(40),
Pause: rolloutsv1alpha1.RolloutPause{},
},
{
Weight: utilpointer.Int32(60),
Pause: rolloutsv1alpha1.RolloutPause{},
},
{
Weight: utilpointer.Int32(80),
Pause: rolloutsv1alpha1.RolloutPause{},
},
{
Weight: utilpointer.Int32(100),
Pause: rolloutsv1alpha1.RolloutPause{
Duration: utilpointer.Int32(0),
},
},
}
CreateObject(rollout)
By("Creating workload and waiting for all pods ready...")
workload := &appsv1alpha1.CloneSet{}
Expect(ReadYamlToObject("./test_data/rollout/cloneset.yaml", workload)).ToNot(HaveOccurred())
CreateObject(workload)
WaitCloneSetAllPodsReady(workload)
By("Only update rollout id = '1', and start rollout")
workload.Labels[util.RolloutIDLabel] = "1"
workload.Annotations[util.InRolloutProgressingAnnotation] = "true"
UpdateCloneSet(workload)
By("wait step(1) pause")
WaitRolloutCanaryStepPaused(rollout.Name, 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "1", "1", 1)
By("Only update rollout id = '2', and check batch label again")
workload.Labels[util.RolloutIDLabel] = "2"
UpdateCloneSet(workload)
By("wait 30s")
time.Sleep(30 * time.Second)
By("wait step(1) pause")
WaitRolloutCanaryStepPaused(rollout.Name, 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "1", 1)
By("wait step(2) pause")
ResumeRolloutCanary(rollout.Name)
WaitRolloutCanaryStepPaused(rollout.Name, 2)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "2", 1)
By("wait step(3) pause")
ResumeRolloutCanary(rollout.Name)
WaitRolloutCanaryStepPaused(rollout.Name, 3)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "3", 1)
By("wait step(4) pause")
ResumeRolloutCanary(rollout.Name)
WaitRolloutCanaryStepPaused(rollout.Name, 4)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "4", 1)
By("Wait rollout complete")
ResumeRolloutCanary(rollout.Name)
WaitRolloutStatusPhase(rollout.Name, rolloutsv1alpha1.RolloutPhaseHealthy)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "1", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "2", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "3", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "4", 1)
CheckPodBatchLabel(workload.Namespace, workload.Spec.Selector, "2", "5", 1)
})
})
})

View File

@ -4,6 +4,8 @@ metadata:
name: echoserver
labels:
app: echoserver
annotations:
rollouts.kruise.io/e2e-test-sample: "true"
spec:
replicas: 5
updateStrategy: