From b2738013b9b8552d8f98853af1080e4909bff47b Mon Sep 17 00:00:00 2001 From: Andrea Frittoli Date: Thu, 7 May 2020 15:44:23 +0100 Subject: [PATCH] Sync the pipelinerun status from the informers When we reconcile a pipelinerun, we should ensure that the pipelinerun status is always in sync with the actual list of taskruns that can be provided by the taskrun informer. The only way to filter taskruns is by labels tekton.dev/pipelinerun. In case an orphaned taskrun is found, we can use the other labels on the taskrun to reconstruct the missing entry in the pipelinerun status, whether it's a missing taskrun or a missing condition check. --- pkg/reconciler/pipelinerun/pipelinerun.go | 103 ++++ .../pipelinerun/pipelinerun_test.go | 569 +++++++++++++++++- .../resources/conditionresolution.go | 4 +- .../resources/pipelinerunresolution.go | 6 +- 4 files changed, 673 insertions(+), 9 deletions(-) diff --git a/pkg/reconciler/pipelinerun/pipelinerun.go b/pkg/reconciler/pipelinerun/pipelinerun.go index b7dbb3e8077..f2930ea5019 100644 --- a/pkg/reconciler/pipelinerun/pipelinerun.go +++ b/pkg/reconciler/pipelinerun/pipelinerun.go @@ -21,6 +21,7 @@ import ( "fmt" "path/filepath" "reflect" + "strconv" "strings" "time" @@ -45,6 +46,7 @@ import ( "k8s.io/apimachinery/pkg/api/equality" "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/labels" "k8s.io/client-go/tools/cache" "knative.dev/pkg/apis" "knative.dev/pkg/configmap" @@ -204,6 +206,14 @@ func (c *Reconciler) Reconcile(ctx context.Context, key string) error { return err } + // Make sure that the PipelineRun status is in sync with the actual TaskRuns + err = c.updatePipelineRunStatusFromInformer(pr) + if err != nil { + // This should not fail. Return the error so we can re-try later. + c.Logger.Errorf("Error while syncing the pipelinerun status: %v", err.Error()) + return err + } + // Reconcile this copy of the pipelinerun and then write back any status or label // updates regardless of whether the reconciliation errored out. if err = c.reconcile(ctx, pr); err != nil { @@ -935,3 +945,96 @@ func storePipelineSpec(ctx context.Context, pr *v1beta1.PipelineRun, ps *v1beta1 } return nil } + +func (c *Reconciler) updatePipelineRunStatusFromInformer(pr *v1beta1.PipelineRun) error { + pipelineRunLabels := getTaskrunLabels(pr, "") + taskRuns, err := c.taskRunLister.TaskRuns(pr.Namespace).List(labels.SelectorFromSet(pipelineRunLabels)) + if err != nil { + c.Logger.Errorf("could not list TaskRuns %#v", err) + return err + } + pr.Status = updatePipelineRunStatusFromTaskRuns(c.Logger, pr.Name, pr.Status, taskRuns) + return nil +} + +func updatePipelineRunStatusFromTaskRuns(logger *zap.SugaredLogger, prName string, prStatus v1beta1.PipelineRunStatus, trs []*v1beta1.TaskRun) v1beta1.PipelineRunStatus { + // If no TaskRun was found, nothing to be done. We never remove taskruns from the status + if trs == nil || len(trs) == 0 { + return prStatus + } + // Store a list of Condition TaskRuns for each PipelineTask (by name) + conditionTaskRuns := make(map[string][]*v1beta1.TaskRun) + // Map PipelineTask names to TaskRun names that were already in the status + taskRunByPipelineTask := make(map[string]string) + if prStatus.TaskRuns != nil { + for taskRunName, pipelineRunTaskRunStatus := range prStatus.TaskRuns { + taskRunByPipelineTask[pipelineRunTaskRunStatus.PipelineTaskName] = taskRunName + } + } + // Loop over all the TaskRuns associated to Tasks + for _, taskrun := range trs { + lbls := taskrun.GetLabels() + pipelineTaskName := lbls[pipeline.GroupName+pipeline.PipelineTaskLabelKey] + if _, ok := lbls[pipeline.GroupName+pipeline.ConditionCheckKey]; ok { + // Save condition for looping over them after this + if _, ok := conditionTaskRuns[pipelineTaskName]; !ok { + // If it's the first condition taskrun, initialise the slice + conditionTaskRuns[pipelineTaskName] = []*v1beta1.TaskRun{} + } + conditionTaskRuns[pipelineTaskName] = append(conditionTaskRuns[pipelineTaskName], taskrun) + continue + } + if _, ok := prStatus.TaskRuns[taskrun.Name]; !ok { + // This taskrun was missing from the status. + // Add it without conditions, which are handled in the next loop + prStatus.TaskRuns[taskrun.Name] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: pipelineTaskName, + Status: &taskrun.Status, + ConditionChecks: nil, + } + // Since this was recovered now, add it to the map, or it might be overwritten + taskRunByPipelineTask[pipelineTaskName] = taskrun.Name + } + } + // Then loop by pipelinetask name over all the TaskRuns associated to Conditions + for pipelineTaskName, actualConditionTaskRuns := range conditionTaskRuns { + taskRunName, ok := taskRunByPipelineTask[pipelineTaskName] + if !ok { + // The pipelineTask associated to the conditions was not found in the pipelinerun + // status. This means that the conditions were orphaned, and never added to the + // status. In this case we need to generate a new TaskRun name, that will be used + // to run the TaskRun if the conditions are passed. + taskRunName = resources.GetTaskRunName(prStatus.TaskRuns, pipelineTaskName, prName) + prStatus.TaskRuns[taskRunName] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: pipelineTaskName, + Status: nil, + ConditionChecks: nil, + } + } + // Build the map of condition checks for the taskrun + // If there were no other condition, initialise the map + conditionChecks := prStatus.TaskRuns[taskRunName].ConditionChecks + if conditionChecks == nil { + conditionChecks = make(map[string]*v1beta1.PipelineRunConditionCheckStatus) + } + for i, foundTaskRun := range actualConditionTaskRuns { + lbls := foundTaskRun.GetLabels() + if _, ok := conditionChecks[foundTaskRun.Name]; !ok { + // The condition check was not found, so we need to add it + // We only add the condition name, the status can now be gathered by the + // normal reconcile process + if conditionName, ok := lbls[pipeline.GroupName+pipeline.ConditionNameKey]; ok { + conditionChecks[foundTaskRun.Name] = &v1beta1.PipelineRunConditionCheckStatus{ + ConditionName: fmt.Sprintf("%s-%s", conditionName, strconv.Itoa(i)), + } + } else { + // The condition name label is missing, so we cannot recover this + logger.Warnf("found an orphaned condition taskrun %#v with missing %s label", + foundTaskRun, pipeline.ConditionNameKey) + } + } + } + prStatus.TaskRuns[taskRunName].ConditionChecks = conditionChecks + } + return prStatus +} diff --git a/pkg/reconciler/pipelinerun/pipelinerun_test.go b/pkg/reconciler/pipelinerun/pipelinerun_test.go index a8aca96c10b..ce69696f273 100644 --- a/pkg/reconciler/pipelinerun/pipelinerun_test.go +++ b/pkg/reconciler/pipelinerun/pipelinerun_test.go @@ -19,6 +19,7 @@ package pipelinerun import ( "context" "fmt" + "regexp" "strings" "testing" "time" @@ -38,6 +39,8 @@ import ( test "github.com/tektoncd/pipeline/test" "github.com/tektoncd/pipeline/test/diff" "github.com/tektoncd/pipeline/test/names" + "go.uber.org/zap" + "go.uber.org/zap/zaptest/observer" corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/runtime" @@ -819,8 +822,8 @@ func TestReconcileOnCompletedPipelineRun(t *testing.T) { t.Fatalf("Expected client to have updated the TaskRun status for a completed PipelineRun, but it did not") } - actual := clients.Pipeline.Actions()[1].(ktesting.UpdateAction).GetObject().(*v1beta1.PipelineRun) - if actual == nil { + _, ok := clients.Pipeline.Actions()[1].(ktesting.UpdateAction).GetObject().(*v1beta1.PipelineRun) + if !ok { t.Errorf("Expected a PipelineRun to be updated, but it wasn't.") } t.Log(clients.Pipeline.Actions()) @@ -1986,7 +1989,7 @@ func TestReconcileWithTaskResults(t *testing.T) { ), } trs := []*v1beta1.TaskRun{ - tb.TaskRun("test-pipeline-run-different-service-accs-a-task-9l9zj", + tb.TaskRun("test-pipeline-run-different-service-accs-a-task-xxyyy", tb.TaskRunNamespace("foo"), tb.TaskRunOwnerReference("PipelineRun", "test-pipeline-run-different-service-accs", tb.OwnerReferenceAPIVersion("tekton.dev/v1beta1"), @@ -2029,7 +2032,7 @@ func TestReconcileWithTaskResults(t *testing.T) { if err != nil { t.Fatalf("Somehow had error getting completed reconciled run out of fake client: %s", err) } - expectedTaskRunName := "test-pipeline-run-different-service-accs-b-task-mz4c7" + expectedTaskRunName := "test-pipeline-run-different-service-accs-b-task-9l9zj" expectedTaskRun := tb.TaskRun(expectedTaskRunName, tb.TaskRunNamespace("foo"), tb.TaskRunOwnerReference("PipelineRun", "test-pipeline-run-different-service-accs", @@ -2239,3 +2242,561 @@ func Test_storePipelineSpec(t *testing.T) { t.Fatalf(diff.PrintWantGot(d)) } } + +func TestReconcileOutOfSyncPipelineRun(t *testing.T) { + // It may happen that a PipelineRun creates one or more TaskRuns during reconcile + // but it fails to sync the update on the status back. This test verifies that + // the reconciler is able to coverge back to a consistent state with the orphaned + // TaskRuns back in the PipelineRun status. + // For more details, see https://github.com/tektoncd/pipeline/issues/2558 + prOutOfSyncName := "test-pipeline-run-out-of-sync" + helloWorldTask := tb.Task("hello-world", tb.TaskNamespace("foo")) + + // Condition checks for the third task + prccs3 := make(map[string]*v1beta1.PipelineRunConditionCheckStatus) + conditionCheckName3 := prOutOfSyncName + "-hello-world-3-always-true-xxxyyy" + prccs3[conditionCheckName3] = &v1beta1.PipelineRunConditionCheckStatus{ + ConditionName: "always-true-0", + Status: &v1beta1.ConditionCheckStatus{ + Status: duckv1beta1.Status{ + Conditions: duckv1beta1.Conditions{ + apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }, + }, + }, + }, + } + // Condition checks for the fourth task + prccs4 := make(map[string]*v1beta1.PipelineRunConditionCheckStatus) + conditionCheckName4 := prOutOfSyncName + "-hello-world-4-always-true-xxxyyy" + prccs4[conditionCheckName4] = &v1beta1.PipelineRunConditionCheckStatus{ + ConditionName: "always-true-0", + Status: &v1beta1.ConditionCheckStatus{ + Status: duckv1beta1.Status{ + Conditions: duckv1beta1.Conditions{ + apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }, + }, + }, + }, + } + testPipeline := tb.Pipeline("test-pipeline", tb.PipelineNamespace("foo"), tb.PipelineSpec( + tb.PipelineTask("hello-world-1", helloWorldTask.Name), + tb.PipelineTask("hello-world-2", helloWorldTask.Name), + tb.PipelineTask("hello-world-3", helloWorldTask.Name, tb.PipelineTaskCondition("always-true")), + tb.PipelineTask("hello-world-4", helloWorldTask.Name, tb.PipelineTaskCondition("always-true")))) + + // This taskrun is in the pipelinerun status. It completed successfully. + taskRunDone := tb.TaskRun("test-pipeline-run-out-of-sync-hello-world-1", + tb.TaskRunNamespace("foo"), + tb.TaskRunOwnerReference("PipelineRun", prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineLabelKey, testPipeline.Name), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineRunLabelKey, prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineTaskLabelKey, "hello-world-1"), + tb.TaskRunSpec(tb.TaskRunTaskRef("hello-world")), + tb.TaskRunStatus( + tb.StatusCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionTrue, + }), + ), + ) + + // This taskrun is *not* in the pipelinerun status. It's still running. + taskRunOrphaned := tb.TaskRun("test-pipeline-run-out-of-sync-hello-world-2", + tb.TaskRunNamespace("foo"), + tb.TaskRunOwnerReference("PipelineRun", prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineLabelKey, testPipeline.Name), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineRunLabelKey, prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineTaskLabelKey, "hello-world-2"), + tb.TaskRunSpec(tb.TaskRunTaskRef("hello-world")), + tb.TaskRunStatus( + tb.StatusCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }), + ), + ) + + // This taskrun has a condition attached. The condition is in the pipelinerun, but the taskrun + // itself is *not* in the pipelinerun status. It's still running. + taskRunWithCondition := tb.TaskRun("test-pipeline-run-out-of-sync-hello-world-3", + tb.TaskRunNamespace("foo"), + tb.TaskRunOwnerReference("PipelineRun", prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineLabelKey, testPipeline.Name), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineRunLabelKey, prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineTaskLabelKey, "hello-world-3"), + tb.TaskRunSpec(tb.TaskRunTaskRef("hello-world")), + tb.TaskRunStatus( + tb.StatusCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }), + ), + ) + + taskRunForConditionOfOrphanedTaskRun := tb.TaskRun(conditionCheckName3, + tb.TaskRunNamespace("foo"), + tb.TaskRunOwnerReference("PipelineRun", prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineLabelKey, testPipeline.Name), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineRunLabelKey, prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineTaskLabelKey, "hello-world-3"), + tb.TaskRunLabel(pipeline.GroupName+pipeline.ConditionCheckKey, conditionCheckName3), + tb.TaskRunLabel(pipeline.GroupName+pipeline.ConditionNameKey, "always-true"), + tb.TaskRunSpec(tb.TaskRunTaskRef("always-true-0")), + tb.TaskRunStatus( + tb.StatusCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }), + ), + ) + + // This taskrun has a condition attached. The condition is *not* the in pipelinerun, and it's still + // running. The taskrun itself was not created yet. + taskRunWithOrphanedConditionName := "test-pipeline-run-out-of-sync-hello-world-4" + + taskRunForOrphanedCondition := tb.TaskRun(conditionCheckName4, + tb.TaskRunNamespace("foo"), + tb.TaskRunOwnerReference("PipelineRun", prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineLabelKey, testPipeline.Name), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineRunLabelKey, prOutOfSyncName), + tb.TaskRunLabel(pipeline.GroupName+pipeline.PipelineTaskLabelKey, "hello-world-4"), + tb.TaskRunLabel(pipeline.GroupName+pipeline.ConditionCheckKey, conditionCheckName4), + tb.TaskRunLabel(pipeline.GroupName+pipeline.ConditionNameKey, "always-true"), + tb.TaskRunSpec(tb.TaskRunTaskRef("always-true-0")), + tb.TaskRunStatus( + tb.StatusCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }), + ), + ) + + prOutOfSync := tb.PipelineRun(prOutOfSyncName, + tb.PipelineRunNamespace("foo"), + tb.PipelineRunSpec(testPipeline.Name, tb.PipelineRunServiceAccountName("test-sa")), + tb.PipelineRunStatus(tb.PipelineRunStatusCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + Reason: "", + Message: "", + }), + tb.PipelineRunTaskRunsStatus(taskRunDone.Name, &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "hello-world-1", + Status: &v1beta1.TaskRunStatus{}, + }), + tb.PipelineRunTaskRunsStatus(taskRunWithCondition.Name, &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "hello-world-3", + Status: nil, + ConditionChecks: prccs3, + }), + ), + ) + prs := []*v1beta1.PipelineRun{prOutOfSync} + ps := []*v1beta1.Pipeline{testPipeline} + ts := []*v1beta1.Task{helloWorldTask} + trs := []*v1beta1.TaskRun{taskRunDone, taskRunOrphaned, taskRunWithCondition, + taskRunForOrphanedCondition, taskRunForConditionOfOrphanedTaskRun} + cs := []*v1alpha1.Condition{ + tbv1alpha1.Condition("always-true", tbv1alpha1.ConditionNamespace("foo"), tbv1alpha1.ConditionSpec( + tbv1alpha1.ConditionSpecCheck("", "foo", tbv1alpha1.Args("bar")), + )), + } + d := test.Data{ + PipelineRuns: prs, + Pipelines: ps, + Tasks: ts, + TaskRuns: trs, + Conditions: cs, + } + + testAssets, cancel := getPipelineRunController(t, d) + defer cancel() + c := testAssets.Controller + clients := testAssets.Clients + + if err := c.Reconciler.Reconcile(context.Background(), "foo/"+prOutOfSync.Name); err != nil { + t.Fatalf("Error reconciling: %s", err) + } + + _, ok := clients.Pipeline.Actions()[1].(ktesting.UpdateAction).GetObject().(*v1beta1.PipelineRun) + if !ok { + t.Errorf("Expected a PipelineRun to be updated, but it wasn't.") + } + t.Log(clients.Pipeline.Actions()) + actions := clients.Pipeline.Actions() + pipelineUpdates := 0 + for _, action := range actions { + if action != nil { + switch { + case action.Matches("create", "taskruns"): + t.Errorf("Expected client to not have created a TaskRun, but it did") + case action.Matches("update", "pipelineruns"): + pipelineUpdates++ + default: + continue + } + } + } + if pipelineUpdates != 2 { + // If only the pipelinerun status changed, we expect one update + t.Fatalf("Expected client to have updated the pipelinerun once, but it did %d times", pipelineUpdates) + } + + // Check that the PipelineRun was reconciled correctly + reconciledRun, err := clients.Pipeline.TektonV1beta1().PipelineRuns("foo").Get(prOutOfSync.Name, metav1.GetOptions{}) + if err != nil { + t.Fatalf("Somehow had error getting completed reconciled run out of fake client: %s", err) + } + + // This PipelineRun should still be running and the status should reflect that + if !reconciledRun.Status.GetCondition(apis.ConditionSucceeded).IsUnknown() { + t.Errorf("Expected PipelineRun status to be running, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded)) + } + + expectedTaskRunsStatus := make(map[string]*v1beta1.PipelineRunTaskRunStatus) + // taskRunDone did not change + expectedTaskRunsStatus[taskRunDone.Name] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "hello-world-1", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{ + { + Type: apis.ConditionSucceeded, + Status: corev1.ConditionTrue, + }, + }, + }, + }, + } + // taskRunOrphaned was recovered into the status + expectedTaskRunsStatus[taskRunOrphaned.Name] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "hello-world-2", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{ + { + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }, + }, + }, + }, + } + // taskRunWithCondition was recovered into the status. The condition did not change. + expectedTaskRunsStatus[taskRunWithCondition.Name] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "hello-world-3", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{ + { + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }, + }, + }, + }, + ConditionChecks: prccs3, + } + // taskRunWithOrphanedConditionName had the condition recovered into the status. No taskrun. + expectedTaskRunsStatus[taskRunWithOrphanedConditionName] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "hello-world-4", + ConditionChecks: prccs4, + } + + // We cannot just diff status directly because the taskrun name for the orphaned condition + // is dynamically generated, but we can change the name to allow us to then diff. + for taskRunName, taskRunStatus := range reconciledRun.Status.TaskRuns { + if strings.HasPrefix(taskRunName, taskRunWithOrphanedConditionName) { + reconciledRun.Status.TaskRuns[taskRunWithOrphanedConditionName] = taskRunStatus + delete(reconciledRun.Status.TaskRuns, taskRunName) + break + } + } + if d := cmp.Diff(reconciledRun.Status.TaskRuns, expectedTaskRunsStatus); d != "" { + t.Fatalf("Expected PipelineRun status to match TaskRun(s) status, but got a mismatch: %s", d) + } +} + +func TestUpdatePipelineRunStatusFromTaskRuns(t *testing.T) { + + // PipelineRunConditionCheckStatus recovered by updatePipelineRunStatusFromTaskRuns + // It does not include the status, which is then retrieved via the regular reconcile + prccs2Recovered := map[string]*v1beta1.PipelineRunConditionCheckStatus{ + "pr-task-2-running-condition-check-xxyyy": { + ConditionName: "running-condition-0", + }, + } + prccs3Recovered := map[string]*v1beta1.PipelineRunConditionCheckStatus{ + "pr-task-3-successful-condition-check-xxyyy": { + ConditionName: "successful-condition-0", + }, + } + prccs4Recovered := map[string]*v1beta1.PipelineRunConditionCheckStatus{ + "pr-task-4-failed-condition-check-xxyyy": { + ConditionName: "failed-condition-0", + }, + } + + // PipelineRunConditionCheckStatus full is used to test the behaviour of updatePipelineRunStatusFromTaskRuns + // when no orphan TaskRuns are found, to check we don't alter good ones + prccs2Full := map[string]*v1beta1.PipelineRunConditionCheckStatus{ + "pr-task-2-running-condition-check-xxyyy": { + ConditionName: "running-condition-0", + Status: &v1beta1.ConditionCheckStatus{ + ConditionCheckStatusFields: v1beta1.ConditionCheckStatusFields{ + Check: corev1.ContainerState{ + Running: &corev1.ContainerStateRunning{}, + }, + }, + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionUnknown}}, + }, + }, + }, + } + prccs3Full := map[string]*v1beta1.PipelineRunConditionCheckStatus{ + "pr-task-3-successful-condition-check-xxyyy": { + ConditionName: "successful-condition-0", + Status: &v1beta1.ConditionCheckStatus{ + ConditionCheckStatusFields: v1beta1.ConditionCheckStatusFields{ + Check: corev1.ContainerState{ + Terminated: &corev1.ContainerStateTerminated{ExitCode: 0}, + }, + }, + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionTrue}}, + }, + }, + }, + } + prccs4Full := map[string]*v1beta1.PipelineRunConditionCheckStatus{ + "pr-task-4-failed-condition-check-xxyyy": { + ConditionName: "failed-condition-0", + Status: &v1beta1.ConditionCheckStatus{ + ConditionCheckStatusFields: v1beta1.ConditionCheckStatusFields{ + Check: corev1.ContainerState{ + Terminated: &corev1.ContainerStateTerminated{ExitCode: 127}, + }, + }, + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}, + }, + }, + }, + } + + prRunningStatus := duckv1beta1.Status{ + Conditions: []apis.Condition{ + { + Type: "Succeeded", + Status: "Unknown", + Reason: "Running", + Message: "Not all Tasks in the Pipeline have finished executing", + }, + }, + } + + prStatusWithCondition := v1beta1.PipelineRunStatus{ + Status: prRunningStatus, + PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + TaskRuns: map[string]*v1beta1.PipelineRunTaskRunStatus{ + "pr-task-1-xxyyy": { + PipelineTaskName: "task-1", + Status: &v1beta1.TaskRunStatus{}, + }, + "pr-task-2-xxyyy": { + PipelineTaskName: "task-2", + Status: nil, + ConditionChecks: prccs2Full, + }, + "pr-task-3-xxyyy": { + PipelineTaskName: "task-3", + Status: &v1beta1.TaskRunStatus{}, + ConditionChecks: prccs3Full, + }, + "pr-task-4-xxyyy": { + PipelineTaskName: "task-4", + Status: nil, + ConditionChecks: prccs4Full, + }, + }, + }, + } + + prStatusWithOrphans := v1beta1.PipelineRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{ + { + Type: "Succeeded", + Status: "Unknown", + Reason: "Running", + Message: "Not all Tasks in the Pipeline have finished executing", + }, + }, + }, + PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + TaskRuns: map[string]*v1beta1.PipelineRunTaskRunStatus{}, + }, + } + + prStatusRecovered := v1beta1.PipelineRunStatus{ + Status: prRunningStatus, + PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + TaskRuns: map[string]*v1beta1.PipelineRunTaskRunStatus{ + "pr-task-1-xxyyy": { + PipelineTaskName: "task-1", + Status: &v1beta1.TaskRunStatus{}, + }, + "orphaned-taskruns-pr-task-2-xxyyy": { + PipelineTaskName: "task-2", + Status: nil, + ConditionChecks: prccs2Recovered, + }, + "pr-task-3-xxyyy": { + PipelineTaskName: "task-3", + Status: &v1beta1.TaskRunStatus{}, + ConditionChecks: prccs3Recovered, + }, + "orphaned-taskruns-pr-task-4-xxyyy": { + PipelineTaskName: "task-4", + Status: nil, + ConditionChecks: prccs4Recovered, + }, + }, + }, + } + + allTaskRuns := []*v1beta1.TaskRun{ + { + ObjectMeta: metav1.ObjectMeta{ + Name: "pr-task-1-xxyyy", + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "task-1", + }, + }, + }, + { + ObjectMeta: metav1.ObjectMeta{ + Name: "pr-task-2-running-condition-check-xxyyy", + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "task-2", + pipeline.GroupName + pipeline.ConditionCheckKey: "pr-task-2-running-condition-check-xxyyy", + pipeline.GroupName + pipeline.ConditionNameKey: "running-condition", + }, + }, + }, + { + ObjectMeta: metav1.ObjectMeta{ + Name: "pr-task-3-xxyyy", + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "task-3", + }, + }, + }, + { + ObjectMeta: metav1.ObjectMeta{ + Name: "pr-task-3-successful-condition-check-xxyyy", + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "task-3", + pipeline.GroupName + pipeline.ConditionCheckKey: "pr-task-3-successful-condition-check-xxyyy", + pipeline.GroupName + pipeline.ConditionNameKey: "successful-condition", + }, + }, + }, + { + ObjectMeta: metav1.ObjectMeta{ + Name: "pr-task-4-failed-condition-check-xxyyy", + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "task-4", + pipeline.GroupName + pipeline.ConditionCheckKey: "pr-task-4-failed-condition-check-xxyyy", + pipeline.GroupName + pipeline.ConditionNameKey: "failed-condition", + }, + }, + }, + } + + tcs := []struct { + prName string + prStatus v1beta1.PipelineRunStatus + trs []*v1beta1.TaskRun + expectedPrStatus v1beta1.PipelineRunStatus + }{ + { + prName: "no-status-no-taskruns", + prStatus: v1beta1.PipelineRunStatus{}, + trs: nil, + expectedPrStatus: v1beta1.PipelineRunStatus{}, + }, { + prName: "status-no-taskruns", + prStatus: prStatusWithCondition, + trs: nil, + expectedPrStatus: prStatusWithCondition, + }, { + prName: "status-missing-taskruns", + prStatus: prStatusWithCondition, + trs: []*v1beta1.TaskRun{ + { + ObjectMeta: metav1.ObjectMeta{ + Name: "pr-task-3-xxyyy", + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "task-3", + }, + }, + }, + { + ObjectMeta: metav1.ObjectMeta{ + Name: "pr-task-3-successful-condition-check-xxyyy", + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "task-3", + pipeline.GroupName + pipeline.ConditionCheckKey: "pr-task-3-successful-condition-check-xxyyy", + pipeline.GroupName + pipeline.ConditionNameKey: "successful-condition", + }, + }, + }, + }, + expectedPrStatus: prStatusWithCondition, + }, { + prName: "status-matching-taskruns-pr", + prStatus: prStatusWithCondition, + trs: allTaskRuns, + expectedPrStatus: prStatusWithCondition, + }, { + prName: "orphaned-taskruns-pr", + prStatus: prStatusWithOrphans, + trs: allTaskRuns, + expectedPrStatus: prStatusRecovered, + }, + } + + for _, tc := range tcs { + t.Run(tc.prName, func(t *testing.T) { + observer, _ := observer.New(zap.InfoLevel) + logger := zap.New(observer).Sugar() + + actualPrStatus := updatePipelineRunStatusFromTaskRuns(logger, tc.prName, tc.prStatus, tc.trs) + + // The TaskRun keys for recovered taskruns will contain a new random key, appended to the + // base name that we expect. Replace the random part so we can diff the whole structure + actualTaskRuns := actualPrStatus.PipelineRunStatusFields.TaskRuns + if actualTaskRuns != nil { + fixedTaskRuns := make(map[string]*v1beta1.PipelineRunTaskRunStatus) + re := regexp.MustCompile(`^[a-z\-]*?-task-[0-9]`) + for k, v := range actualTaskRuns { + newK := re.FindString(k) + fixedTaskRuns[newK+"-xxyyy"] = v + } + actualPrStatus.PipelineRunStatusFields.TaskRuns = fixedTaskRuns + } + + if d := cmp.Diff(tc.expectedPrStatus, actualPrStatus); d != "" { + t.Errorf("expected the PipelineRun status to match %#v. Diff %s", tc.expectedPrStatus, diff.PrintWantGot(d)) + } + }) + } +} diff --git a/pkg/reconciler/pipelinerun/resources/conditionresolution.go b/pkg/reconciler/pipelinerun/resources/conditionresolution.go index 421ece88b7f..fe319985e1b 100644 --- a/pkg/reconciler/pipelinerun/resources/conditionresolution.go +++ b/pkg/reconciler/pipelinerun/resources/conditionresolution.go @@ -68,7 +68,7 @@ func (state TaskConditionCheckState) HasStarted() bool { return hasStarted } -// IsComplete returns true if the status for all conditionChecks for a task indicate that they are done +// IsDone returns true if the status for all conditionChecks for a task indicate that they are done func (state TaskConditionCheckState) IsDone() bool { if !state.HasStarted() { return false @@ -134,7 +134,7 @@ func convertParamTemplates(step *v1beta1.Step, params []v1beta1.ParamSpec) { v1beta1.ApplyStepReplacements(step, replacements, map[string][]string{}) } -// ApplyResources applies the substitution from values in resources which are referenced +// ApplyResourceSubstitution applies the substitution from values in resources which are referenced // in spec as subitems of the replacementStr. func ApplyResourceSubstitution(step *v1beta1.Step, resolvedResources map[string]*resourcev1alpha1.PipelineResource, conditionResources []v1beta1.ResourceDeclaration, images pipeline.Images) error { replacements := make(map[string]string) diff --git a/pkg/reconciler/pipelinerun/resources/pipelinerunresolution.go b/pkg/reconciler/pipelinerun/resources/pipelinerunresolution.go index f31b64da2ba..20b0609b597 100644 --- a/pkg/reconciler/pipelinerun/resources/pipelinerunresolution.go +++ b/pkg/reconciler/pipelinerun/resources/pipelinerunresolution.go @@ -317,7 +317,7 @@ func ResolvePipelineRun( rprt := ResolvedPipelineRunTask{ PipelineTask: &pt, - TaskRunName: getTaskRunName(pipelineRun.Status.TaskRuns, pt.Name, pipelineRun.Name), + TaskRunName: GetTaskRunName(pipelineRun.Status.TaskRuns, pt.Name, pipelineRun.Name), } // Find the Task that this PipelineTask is using @@ -394,8 +394,8 @@ func getConditionCheckName(taskRunStatus map[string]*v1beta1.PipelineRunTaskRunS return names.SimpleNameGenerator.RestrictLengthWithRandomSuffix(fmt.Sprintf("%s-%s", trName, conditionRegisterName)) } -// getTaskRunName should return a unique name for a `TaskRun` if one has not already been defined, and the existing one otherwise. -func getTaskRunName(taskRunsStatus map[string]*v1beta1.PipelineRunTaskRunStatus, ptName, prName string) string { +// GetTaskRunName should return a unique name for a `TaskRun` if one has not already been defined, and the existing one otherwise. +func GetTaskRunName(taskRunsStatus map[string]*v1beta1.PipelineRunTaskRunStatus, ptName, prName string) string { for k, v := range taskRunsStatus { if v.PipelineTaskName == ptName { return k