diff --git a/docs/pipelines.md b/docs/pipelines.md index 614806393b3..a5429f7fbd3 100644 --- a/docs/pipelines.md +++ b/docs/pipelines.md @@ -21,6 +21,7 @@ weight: 3 - [Configuring execution results at the `Pipeline` level](#configuring-execution-results-at-the-pipeline-level) - [Configuring the `Task` execution order](#configuring-the-task-execution-order) - [Adding a description](#adding-a-description) + - [Adding `Finally` to the `Pipeline`](#adding-finally-to-the-pipeline) - [Code examples](#code-examples) ## Overview @@ -528,6 +529,198 @@ In particular: The `description` field is an optional field and can be used to provide description of the `Pipeline`. +## Adding `Finally` to the `Pipeline` + +You can specify a list of one or more final tasks under `finally` section. Final tasks are guaranteed to be executed +in parallel after all `PipelineTasks` under `tasks` have completed regardless of success or error. Final tasks are very +similar to `PipelineTasks` under `tasks` section and follow the same syntax. Each final task must have a +[valid](https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names) `name` and a [taskRef or +taskSpec](taskruns.md#specifying-the-target-task). For example: + +```yaml +spec: + tasks: + - name: tests + taskRef: + Name: integration-test + finally: + - name: cleanup-test + taskRef: + Name: cleanup +``` + +### Specifying `Workspaces` in Final Tasks + +Finally tasks may want to use [workspaces](workspaces.md) which `PipelineTasks` might have utilized +e.g. a mount point for credentials held in Secrets. To support that requirement, you can specify one or more +`Workspaces` in the `workspaces` field for the final tasks similar to `tasks`. + +```yaml +spec: + resources: + - name: app-git + type: git + workspaces: + - name: shared-workspace + tasks: + - name: clone-app-source + taskRef: + name: clone-app-repo-to-workspace + workspaces: + - name: shared-workspace + workspace: shared-workspace + resources: + inputs: + - name: app-git + resource: app-git + finally: + - name: cleanup-workspace + taskRef: + name: cleanup-workspace + workspaces: + - name: shared-workspace + workspace: shared-workspace +``` + +### Specifying `Parameters` in Final Tasks + +Again, similar to `tasks`, you can specify [`Parameters`](tasks.md#specifying-parameters): + +```yaml +spec: + tasks: + - name: tests + taskRef: + Name: integration-test + finally: + - name: report-results + taskRef: + Name: report-results + params: + - name: url + value: "someURL" +``` + +### `PipelineRun` Status with `finally` + +With `finally`, `PipelineRun` status is calculated based on `PipelineTasks` under `tasks` section and final tasks. + +Without `finally`: + +| `PipelineTasks` under `tasks` | `PipelineRun` status | Reason | +| ----------------------------- | -------------------- | ------ | +| all `PipelineTasks` successful | `true` | `Succeeded` | +| one or more `PipelineTasks` skipped and rest successful | `true` | `Completed` | +| single failure of `PipelineTask` | `false` | `failed` | + +With `finally`: + +| `PipelineTasks` under `tasks` | Final Tasks | `PipelineRun` status | Reason | +| ----------------------------- | ----------- | -------------------- | ------ | +| all `PipelineTask` successful | all final tasks successful | `true` | `Succeeded` | +| all `PipelineTask` successful | one or more failure of final tasks | `false` | `Failed` | +| one or more `PipelineTask` skipped and rest successful | all final tasks successful | `true` | `Completed` | +| one or more `PipelineTask` skipped and rest successful | one or more failure of final tasks | `false` | `Failed` | +| single failure of `PipelineTask` | all final tasks successful | `false` | `failed` | +| single failure of `PipelineTask` | one or more failure of final tasks | `false` | `failed` | + +### Known Limitations + +### Specifying `Resources` in Final Tasks + +Similar to `tasks`, you can use [PipelineResources](#specifying-resources) as inputs and outputs for +final tasks in the Pipeline. The only difference here is, final tasks with an input resource can not have a `from` clause +like a `PipelineTask` from `tasks` section. For example: + +```yaml +spec: + tasks: + - name: tests + taskRef: + Name: integration-test + resources: + inputs: + - name: source + resource: tektoncd-pipeline-repo + outputs: + - name: workspace + resource: my-repo + finally: + - name: clear-workspace + taskRef: + Name: clear-workspace + resources: + inputs: + - name: workspace + resource: my-repo + from: #invalid + - tests +``` + +### Cannot configure the Final Task execution order + +It's not possible to configure or modify the execution order of the final tasks. Unlike `Tasks` in a `Pipeline`, +all final tasks run simultaneously and starts executing once all `PipelineTasks` under `tasks` have settled which means +no `runAfter` can be specified in final tasks. + +### Cannot specify execution `Conditions` in Final Tasks + +`Tasks` in a `Pipeline` can be configured to run only if some conditions are satisfied using `conditions`. But the +final tasks are guaranteed to be executed after all `PipelineTasks` therefore no `conditions` can be specified in +final tasks. + +#### Cannot configure `Task` execution results with `finally` + +Final tasks can not be configured to consume `Results` of `PipelineTask` from `tasks` section i.e. the following +example is not supported right now but we are working on adding support for the same (tracked in issue +[#2557](https://github.com/tektoncd/pipeline/issues/2557)). + +```yaml +spec: + tasks: + - name: count-comments-before + taskRef: + Name: count-comments + - name: add-comment + taskRef: + Name: add-comment + - name: count-comments-after + taskRef: + Name: count-comments + finally: + - name: check-count + taskRef: + Name: check-count + params: + - name: before-count + value: $(tasks.count-comments-before.results.count) #invalid + - name: after-count + value: $(tasks.count-comments-after.results.count) #invalid +``` + +#### Cannot configure `Pipeline` result with `finally` + +Final tasks can emit `Results` but results emitted from the final tasks can not be configured in the +[Pipeline Results](#configuring-execution-results-at-the-pipeline-level). We are working on adding support for this +(tracked in issue [#2710](https://github.com/tektoncd/pipeline/issues/2710)). + +```yaml + results: + - name: comment-count-validate + value: $(finally.check-count.results.comment-count-validate) +``` + +In this example, `PipelineResults` is set to: + +``` +"pipelineResults": [ + { + "name": "comment-count-validate", + "value": "$(finally.check-count.results.comment-count-validate)" + } +], +``` + ## Code examples For a better understanding of `Pipelines`, study [our code examples](https://github.com/tektoncd/pipeline/tree/master/examples). diff --git a/examples/v1beta1/pipelineruns/pipelinerun-with-final-tasks.yaml b/examples/v1beta1/pipelineruns/pipelinerun-with-final-tasks.yaml new file mode 100644 index 00000000000..3488b057d38 --- /dev/null +++ b/examples/v1beta1/pipelineruns/pipelinerun-with-final-tasks.yaml @@ -0,0 +1,95 @@ +# Task to clone repo into shared workspace +apiVersion: tekton.dev/v1beta1 +kind: Task +metadata: + name: clone-app-repo-to-workspace +spec: + workspaces: + - name: shared-workspace + resources: + inputs: + - name: app-git + type: git + targetPath: application + steps: + - name: clone-app-repo-to-workspace + image: ubuntu + script: | + #!/usr/bin/env bash + set -xe + cp -avr $(resources.inputs.app-git.path)/ $(workspaces.shared-workspace.path)/ +--- + +# Task to cleanup shared workspace +apiVersion: tekton.dev/v1beta1 +kind: Task +metadata: + name: cleanup-workspace +spec: + workspaces: + - name: shared-workspace + steps: + - name: cleanup-workspace + image: ubuntu + script: | + #!/usr/bin/env bash + set -xe + rm -rf $(workspaces.shared-workspace.path)/application/ +--- + +# Pipeline to clone repo into shared workspace and cleanup the workspace after done +apiVersion: tekton.dev/v1beta1 +kind: Pipeline +metadata: + name: clone-into-workspace-and-cleanup-workspace +spec: + resources: + - name: app-git + type: git + workspaces: + - name: shared-workspace + tasks: + - name: clone-app-source + taskRef: + name: clone-app-repo-to-workspace + workspaces: + - name: shared-workspace + workspace: shared-workspace + resources: + inputs: + - name: app-git + resource: app-git + finally: + - name: cleanup-workspace + taskRef: + name: cleanup-workspace + workspaces: + - name: shared-workspace + workspace: shared-workspace +--- + +# PipelineRun to execute pipeline - clone-into-workspace-and-cleanup-workspace +apiVersion: tekton.dev/v1beta1 +kind: PipelineRun +metadata: + name: write-and-cleanup-workspace +spec: + pipelineRef: + name: clone-into-workspace-and-cleanup-workspace + workspaces: + - name: shared-workspace + volumeClaimTemplate: + spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 16Mi + resources: + - name: app-git + resourceSpec: + type: git + params: + - name: url + value: https://github.com/tektoncd/pipeline.git +--- diff --git a/pkg/apis/pipeline/v1beta1/pipeline_types.go b/pkg/apis/pipeline/v1beta1/pipeline_types.go index 6a4234bf2e1..a36249c1a04 100644 --- a/pkg/apis/pipeline/v1beta1/pipeline_types.go +++ b/pkg/apis/pipeline/v1beta1/pipeline_types.go @@ -71,6 +71,10 @@ type PipelineSpec struct { // Results are values that this pipeline can output once run // +optional Results []PipelineResult `json:"results,omitempty"` + // Finally declares the list of Tasks that execute just before leaving the Pipeline + // i.e. either after all Tasks are finished executing successfully + // or after a failure which would result in ending the Pipeline + Finally []PipelineTask `json:"finally,omitempty"` } // PipelineResult used to describe the results of a pipeline diff --git a/pkg/apis/pipeline/v1beta1/zz_generated.deepcopy.go b/pkg/apis/pipeline/v1beta1/zz_generated.deepcopy.go index a8c0313d3b0..d4d9becf209 100644 --- a/pkg/apis/pipeline/v1beta1/zz_generated.deepcopy.go +++ b/pkg/apis/pipeline/v1beta1/zz_generated.deepcopy.go @@ -785,6 +785,13 @@ func (in *PipelineSpec) DeepCopyInto(out *PipelineSpec) { *out = make([]PipelineResult, len(*in)) copy(*out, *in) } + if in.Finally != nil { + in, out := &in.Finally, &out.Finally + *out = make([]PipelineTask, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } return } diff --git a/pkg/reconciler/pipelinerun/pipelinerun.go b/pkg/reconciler/pipelinerun/pipelinerun.go index a1a0912c596..0a62e7ad265 100644 --- a/pkg/reconciler/pipelinerun/pipelinerun.go +++ b/pkg/reconciler/pipelinerun/pipelinerun.go @@ -404,6 +404,23 @@ func (c *Reconciler) reconcile(ctx context.Context, pr *v1beta1.PipelineRun) err return nil } + // build DAG with a list of final tasks, this DAG is used later to identify + // if a task from PipelineRunState is one of the tasks specified under finally section + // the finally section is optional and might not be specified + // in case this section is not specified, dfinally holds an empty Graph + dfinally, err := dag.Build(v1beta1.PipelineTaskList(pipelineSpec.Finally)) + if err != nil { + // This Run has failed, so we need to mark it as failed and stop reconciling it + pr.Status.SetCondition(&apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + Reason: ReasonInvalidGraph, + Message: fmt.Sprintf("PipelineRun %s's Pipeline DAG is invalid: %s", + fmt.Sprintf("%s/%s", pr.Namespace, pr.Name), err), + }) + return nil + } + if err := pipelineSpec.Validate(ctx); err != nil { // This Run has failed, so we need to mark it as failed and stop reconciling it pr.Status.SetCondition(&apis.Condition{ @@ -482,6 +499,9 @@ func (c *Reconciler) reconcile(ctx context.Context, pr *v1beta1.PipelineRun) err // Apply parameter substitution from the PipelineRun pipelineSpec = resources.ApplyParameters(pipelineSpec, pr) + // pipelineState holds a list of pipeline tasks with resolved conditions and pipeline resources + // pipelineState also holds a taskRun for each pipeline task after the taskRun is created + // pipelineState is instantiated on every reconcile cycle pipelineState, err := resources.ResolvePipelineRun(ctx, *pr, func(name string) (v1beta1.TaskInterface, error) { @@ -496,7 +516,7 @@ func (c *Reconciler) reconcile(ctx context.Context, pr *v1beta1.PipelineRun) err func(name string) (*v1alpha1.Condition, error) { return c.conditionLister.Conditions(pr.Namespace).Get(name) }, - pipelineSpec.Tasks, providedResources, + append(pipelineSpec.Tasks, pipelineSpec.Finally...), providedResources, ) if err != nil { @@ -566,12 +586,29 @@ func (c *Reconciler) reconcile(ctx context.Context, pr *v1beta1.PipelineRun) err } } - candidateTasks, err := dag.GetSchedulable(d, pipelineState.SuccessfulPipelineTaskNames()...) + // candidateTasks contains a list of pipeline tasks that can be scheduled next + // This list of candidates is derived based on the successfully finished tasks and skipped tasks + // A task is considered as candidate if its all parents have finished executing successfully + // candidateTasks are first initialized with all root/s of DAG tasks + successOrSkippedTasks := append(pipelineState.SuccessfulPipelineTaskNames(), pipelineState.SkippedPipelineTaskNames(d)...) + candidateTasks, err := dag.GetSchedulable(d, successOrSkippedTasks...) if err != nil { c.Logger.Errorf("Error getting potential next tasks for valid pipelinerun %s: %v", pr.Name, err) } + // GetNextTasks returns a list of tasks which should be executed next and is derived based on candidateTasks + // GetNextTasks returns a list of candidates for which pipelineState does not have any taskRun or a list of + // failed tasks which haven't exhausted their retries + // Pipeline execution continues until GetNextTasks does not return any new pipeline task to execute nextRprts := pipelineState.GetNextTasks(candidateTasks) + + // GetFinalTasks returns a list of final tasks without any taskRun associated with it + // GetFinalTasks returns final tasks only when all DAG tasks have finished executing successfully or + // executing any one DAG task resulted in failure + if len(nextRprts) == 0 { + nextRprts = pipelineState.GetFinalTasks(d, dfinally) + } + resolvedResultRefs, err := resources.ResolveResultRefs(pipelineState, nextRprts) if err != nil { c.Logger.Infof("Failed to resolve all task params for %q with error %v", pr.Name, err) diff --git a/pkg/reconciler/pipelinerun/pipelinerun_test.go b/pkg/reconciler/pipelinerun/pipelinerun_test.go index ad5910b7473..eebbb145401 100644 --- a/pkg/reconciler/pipelinerun/pipelinerun_test.go +++ b/pkg/reconciler/pipelinerun/pipelinerun_test.go @@ -36,7 +36,7 @@ import ( taskrunresources "github.com/tektoncd/pipeline/pkg/reconciler/taskrun/resources" ttesting "github.com/tektoncd/pipeline/pkg/reconciler/testing" "github.com/tektoncd/pipeline/pkg/system" - test "github.com/tektoncd/pipeline/test" + "github.com/tektoncd/pipeline/test" "github.com/tektoncd/pipeline/test/diff" "github.com/tektoncd/pipeline/test/names" "go.uber.org/zap" @@ -430,7 +430,26 @@ func TestReconcile_InvalidPipelineRuns(t *testing.T) { tb.PipelineTask("some-task", "a-task-that-needs-array-params")), tb.PipelineRunParam("some-param", "stringval"), )), + tb.PipelineRun("pipeline-invalid-dag-graph", tb.PipelineRunNamespace("foo"), tb.PipelineRunSpec("", tb.PipelineRunPipelineSpec( + tb.PipelineTask("dag-task-1", "dag-task-1", tb.RunAfter("dag-task-1")), + ))), } + + prs = append(prs, &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "pipeline-invalid-final-graph", Namespace: "foo"}, + Spec: v1beta1.PipelineRunSpec{ + PipelineSpec: &v1beta1.PipelineSpec{ + Tasks: []v1beta1.PipelineTask{ + {Name: "dag-task-1", TaskRef: &v1beta1.TaskRef{Name: "taskName"}}, + }, + Finally: []v1beta1.PipelineTask{ + {Name: "final-task-1", TaskRef: &v1beta1.TaskRef{Name: "taskName"}}, + {Name: "final-task-1", TaskRef: &v1beta1.TaskRef{Name: "taskName"}}, + }, + }, + }, + }) + d := test.Data{ Tasks: ts, Pipelines: ps, @@ -487,6 +506,14 @@ func TestReconcile_InvalidPipelineRuns(t *testing.T) { name: "invalid-embedded-pipeline-mismatching-parameter-types", pipelineRun: prs[10], reason: ReasonParameterTypeMismatch, + }, { + name: "invalid-pipeline-with-invalid-dag-graph", + pipelineRun: prs[11], + reason: ReasonInvalidGraph, + }, { + name: "invalid-pipeline-with-invalid-final-tasks-graph", + pipelineRun: prs[12], + reason: ReasonInvalidGraph, }, } @@ -2806,3 +2833,398 @@ func TestUpdatePipelineRunStatusFromTaskRuns(t *testing.T) { }) } } + +// this test validates pipeline with finally, pipeline run should result in error +// when a dag task is executed and resulted in failure but final task is executed successfully +func TestReconcilePipelineRunWithFinallyWithDAGTaskFailure(t *testing.T) { + pipelineRunName := "final-pipeline-run-with-dag-task-failing" + pipelineName := "final-pipeline-with-dag-task-failing" + taskRunName := pipelineRunName + "-dag-task" + finalTaskRunName := pipelineRunName + "-final-task" + taskName := "hello-world" + + prs := []*v1beta1.PipelineRun{{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineRunName, Namespace: "foo"}, + Spec: v1beta1.PipelineRunSpec{ + ServiceAccountName: "test-sa", + Timeout: &metav1.Duration{Duration: 1 * time.Minute}, + PipelineRef: &v1beta1.PipelineRef{Name: pipelineName}, + }, + }} + + ps := []*v1beta1.Pipeline{{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineName, Namespace: "foo"}, + Spec: v1beta1.PipelineSpec{ + Tasks: []v1beta1.PipelineTask{{Name: "dag-task-1", TaskRef: &v1beta1.TaskRef{Name: taskName}}}, + Finally: []v1beta1.PipelineTask{{Name: "final-task-1", TaskRef: &v1beta1.TaskRef{Name: taskName}}}, + }, + }} + + ts := []*v1beta1.Task{{ObjectMeta: metav1.ObjectMeta{Name: taskName, Namespace: "foo"}}} + + trs := []*v1beta1.TaskRun{{ + ObjectMeta: metav1.ObjectMeta{Name: taskRunName, Namespace: "foo", + OwnerReferences: []metav1.OwnerReference{{Kind: "pipelineRun", Name: pipelineRunName}}, + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineLabelKey: pipelineName, + pipeline.GroupName + pipeline.PipelineRunLabelKey: pipelineRunName, + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "dag-task-1", + }, + }, + Spec: v1beta1.TaskRunSpec{TaskRef: &v1beta1.TaskRef{Name: taskName}}, + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}}, + }, + }, { + ObjectMeta: metav1.ObjectMeta{Name: finalTaskRunName, Namespace: "foo", + OwnerReferences: []metav1.OwnerReference{{Kind: "pipelineRun", Name: pipelineRunName}}, + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineLabelKey: pipelineName, + pipeline.GroupName + pipeline.PipelineRunLabelKey: pipelineRunName, + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "final-task-1", + }, + }, + Spec: v1beta1.TaskRunSpec{TaskRef: &v1beta1.TaskRef{Name: taskName}}, + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded}}}, + }, + }} + d := test.Data{ + PipelineRuns: prs, + Pipelines: ps, + Tasks: ts, + TaskRuns: trs, + } + + testAssets, cancel := getPipelineRunController(t, d) + defer cancel() + c := testAssets.Controller + clients := testAssets.Clients + + if err := c.Reconciler.Reconcile(context.Background(), "foo/"+pipelineRunName); err != nil { + t.Fatalf("Error reconciling: %s", err) + } + + // one for pipelines and two more for each TaskRun + if len(clients.Pipeline.Actions()) != 3 { + t.Fatalf("Expected client to have updated the PipelineRun status (PipelineRunTaskRunStatus), but it did not") + } + + actual := clients.Pipeline.Actions()[1].(ktesting.UpdateAction).GetObject().(*v1beta1.PipelineRun) + if actual == nil { + t.Errorf("Expected a PipelineRun to be updated, but it wasn't.") + } + + actions := clients.Pipeline.Actions() + for _, action := range actions { + if action != nil { + resource := action.GetResource().Resource + if resource == "taskruns" { + t.Fatalf("Expected client to not have created a TaskRun for the PipelineRun, but it did") + } + } + } + + // Check that the PipelineRun was reconciled correctly + reconciledRun, err := clients.Pipeline.TektonV1beta1().PipelineRuns("foo").Get(pipelineRunName, 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 failed and the status should reflect that + if !reconciledRun.Status.GetCondition(apis.ConditionSucceeded).IsFalse() { + t.Errorf("Expected PipelineRun status to be failed, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded)) + } + + // This PipelineRun should still be failed and the reason should be set to Failed + if reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Reason != resources.ReasonFailed { + t.Errorf("Expected PipelineRun status to be failed and reason being failed, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Reason) + } + + // This PipelineRun should still be failed and the message should have "TaskRun dag task has failed" + if !strings.Contains(reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Message, taskRunName) { + t.Errorf("Expected PipelineRun status to be failed and message having dag task, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Message) + } + + expectedTaskRunsStatus := make(map[string]*v1beta1.PipelineRunTaskRunStatus) + expectedTaskRunsStatus[taskRunName] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "dag-task-1", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}}, + }, + } + expectedTaskRunsStatus[finalTaskRunName] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "final-task-1", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded}}}, + }, + } + 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) + } +} + +// this test validates pipeline with finally, pipeline run should result in error +// when a dag task is executed successfully but final task is executed and resulted in failure +func TestReconcilePipelineRunWithFinallyWithFinalTaskFailure(t *testing.T) { + pipelineRunName := "final-pipeline-run-with-dag-passing-but-final-failing" + pipelineName := "final-pipeline-with-dag-passing-but-final-failing" + taskRunName := pipelineRunName + "-dag-task" + finalTaskRunName := pipelineRunName + "-final-task" + taskName := "hello-world" + + prs := []*v1beta1.PipelineRun{{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineRunName, Namespace: "foo"}, + Spec: v1beta1.PipelineRunSpec{ + ServiceAccountName: "test-sa", + PipelineRef: &v1beta1.PipelineRef{Name: pipelineName}}, + }} + + ps := []*v1beta1.Pipeline{{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineName, Namespace: "foo"}, + Spec: v1beta1.PipelineSpec{ + Tasks: []v1beta1.PipelineTask{{Name: "dag-task-1", TaskRef: &v1beta1.TaskRef{Name: taskName}}}, + Finally: []v1beta1.PipelineTask{{Name: "final-task-1", TaskRef: &v1beta1.TaskRef{Name: taskName}}}, + }, + }} + + ts := []*v1beta1.Task{{ObjectMeta: metav1.ObjectMeta{Name: taskName, Namespace: "foo"}}} + + trs := []*v1beta1.TaskRun{{ + ObjectMeta: metav1.ObjectMeta{Name: taskRunName, Namespace: "foo", + OwnerReferences: []metav1.OwnerReference{{Kind: "pipelineRun", Name: pipelineRunName}}, + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineLabelKey: pipelineName, + pipeline.GroupName + pipeline.PipelineRunLabelKey: pipelineRunName, + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "dag-task-1", + }, + }, + Spec: v1beta1.TaskRunSpec{TaskRef: &v1beta1.TaskRef{Name: taskName}}, + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded}}}, + }, + }, { + ObjectMeta: metav1.ObjectMeta{Name: finalTaskRunName, Namespace: "foo", + OwnerReferences: []metav1.OwnerReference{{Kind: "pipelineRun", Name: pipelineRunName}}, + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineLabelKey: pipelineName, + pipeline.GroupName + pipeline.PipelineRunLabelKey: pipelineRunName, + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "final-task-1", + }, + }, + Spec: v1beta1.TaskRunSpec{TaskRef: &v1beta1.TaskRef{Name: taskName}}, + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}}, + }, + }} + + d := test.Data{ + PipelineRuns: prs, + Pipelines: ps, + Tasks: ts, + TaskRuns: trs, + } + + testAssets, cancel := getPipelineRunController(t, d) + defer cancel() + c := testAssets.Controller + clients := testAssets.Clients + + if err := c.Reconciler.Reconcile(context.Background(), "foo/"+pipelineRunName); err != nil { + t.Fatalf("Error reconciling: %s", err) + } + + // one for pipelines and two more for each TaskRun + if len(clients.Pipeline.Actions()) != 3 { + t.Fatalf("Expected client to have updated the PipelineRun status (PipelineRunTaskRunStatus), but it did not") + } + + actual := clients.Pipeline.Actions()[1].(ktesting.UpdateAction).GetObject().(*v1beta1.PipelineRun) + if actual == nil { + t.Errorf("Expected a PipelineRun to be updated, but it wasn't.") + } + + actions := clients.Pipeline.Actions() + for _, action := range actions { + if action != nil { + resource := action.GetResource().Resource + if resource == "taskruns" { + t.Fatalf("Expected client to not have created a TaskRun for the PipelineRun, but it did") + } + } + } + + // Check that the PipelineRun was reconciled correctly + reconciledRun, err := clients.Pipeline.TektonV1beta1().PipelineRuns("foo").Get(pipelineRunName, 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 failed and the status should reflect that + if !reconciledRun.Status.GetCondition(apis.ConditionSucceeded).IsFalse() { + t.Errorf("Expected PipelineRun status to be failed, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded)) + } + + // This PipelineRun should still be failed and the reason should be set to Failed + if reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Reason != resources.ReasonFailed { + t.Errorf("Expected PipelineRun status to be failed and reason being failed, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Reason) + } + + // This PipelineRun should still be failed and the message should have "TaskRun final task has failed" + if !strings.Contains(reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Message, finalTaskRunName) { + t.Errorf("Expected PipelineRun status to be failed and message having final task, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Message) + } + + expectedTaskRunsStatus := make(map[string]*v1beta1.PipelineRunTaskRunStatus) + expectedTaskRunsStatus[taskRunName] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "dag-task-1", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded}}}, + }, + } + expectedTaskRunsStatus[finalTaskRunName] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "final-task-1", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}, + }, + }, + } + 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) + } +} + +// this test validates pipeline with finally, pipeline run should result in error +// when a dag task and final task both are executed and resulted in failure +func TestReconcilePipelineRunWithFinallyWithDAGAndFinalTaskFailure(t *testing.T) { + pipelineRunName := "final-pipeline-run-with-dag-and-final-failing" + pipelineName := "final-pipeline-with-dag-and-final-failing" + taskRunName := pipelineRunName + "-dag-task" + finalTaskRunName := pipelineRunName + "-final-task" + taskName := "hello-world" + + prs := []*v1beta1.PipelineRun{{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineRunName, Namespace: "foo"}, + Spec: v1beta1.PipelineRunSpec{ + ServiceAccountName: "test-sa", + Timeout: &metav1.Duration{Duration: 1 * time.Minute}, + PipelineRef: &v1beta1.PipelineRef{Name: pipelineName}}, + }} + + ps := []*v1beta1.Pipeline{{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineName, Namespace: "foo"}, + Spec: v1beta1.PipelineSpec{ + Tasks: []v1beta1.PipelineTask{{Name: "dag-task-1", TaskRef: &v1beta1.TaskRef{Name: taskName}}}, + Finally: []v1beta1.PipelineTask{{Name: "final-task-1", TaskRef: &v1beta1.TaskRef{Name: taskName}}}, + }, + }} + + ts := []*v1beta1.Task{{ObjectMeta: metav1.ObjectMeta{Name: taskName, Namespace: "foo"}}} + + trs := []*v1beta1.TaskRun{{ + ObjectMeta: metav1.ObjectMeta{Name: taskRunName, Namespace: "foo", + OwnerReferences: []metav1.OwnerReference{{Kind: "pipelineRun", Name: pipelineRunName}}, + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineLabelKey: pipelineName, + pipeline.GroupName + pipeline.PipelineRunLabelKey: pipelineRunName, + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "dag-task-1", + }, + }, + Spec: v1beta1.TaskRunSpec{TaskRef: &v1beta1.TaskRef{Name: taskName}}, + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}}, + }, + }, { + ObjectMeta: metav1.ObjectMeta{Name: finalTaskRunName, Namespace: "foo", + OwnerReferences: []metav1.OwnerReference{{Kind: "pipelineRun", Name: pipelineRunName}}, + Labels: map[string]string{ + pipeline.GroupName + pipeline.PipelineLabelKey: pipelineName, + pipeline.GroupName + pipeline.PipelineRunLabelKey: pipelineRunName, + pipeline.GroupName + pipeline.PipelineTaskLabelKey: "final-task-1", + }, + }, + Spec: v1beta1.TaskRunSpec{TaskRef: &v1beta1.TaskRef{Name: taskName}}, + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}}, + }, + }} + + d := test.Data{ + PipelineRuns: prs, + Pipelines: ps, + Tasks: ts, + TaskRuns: trs, + } + + testAssets, cancel := getPipelineRunController(t, d) + defer cancel() + c := testAssets.Controller + clients := testAssets.Clients + + if err := c.Reconciler.Reconcile(context.Background(), "foo/"+pipelineRunName); err != nil { + t.Fatalf("Error reconciling: %s", err) + } + + // one for pipelines and two more for each TaskRun + if len(clients.Pipeline.Actions()) != 3 { + t.Fatalf("Expected client to have updated the PipelineRun status (PipelineRunTaskRunStatus), but it did not") + } + + actual := clients.Pipeline.Actions()[1].(ktesting.UpdateAction).GetObject().(*v1beta1.PipelineRun) + if actual == nil { + t.Errorf("Expected a PipelineRun to be updated, but it wasn't.") + } + + actions := clients.Pipeline.Actions() + for _, action := range actions { + if action != nil { + resource := action.GetResource().Resource + if resource == "taskruns" { + t.Fatalf("Expected client to not have created a TaskRun for the PipelineRun, but it did") + } + } + } + + // Check that the PipelineRun was reconciled correctly + reconciledRun, err := clients.Pipeline.TektonV1beta1().PipelineRuns("foo").Get(pipelineRunName, 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 failed and the status should reflect that + if !reconciledRun.Status.GetCondition(apis.ConditionSucceeded).IsFalse() { + t.Errorf("Expected PipelineRun status to be failed, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded)) + } + + // This PipelineRun should still be failed and the reason should be set to Failed + if reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Reason != resources.ReasonFailed { + t.Errorf("Expected PipelineRun status to be failed and reason being failed, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Reason) + } + + // This PipelineRun should still be failed and the message should have "TaskRun dag task has failed" + if !strings.Contains(reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Message, taskRunName) { + t.Errorf("Expected PipelineRun status to be failed and message having dag task, but was %v", reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Message) + } + + expectedTaskRunsStatus := make(map[string]*v1beta1.PipelineRunTaskRunStatus) + expectedTaskRunsStatus[taskRunName] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "dag-task-1", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}, + }, + }, + } + expectedTaskRunsStatus[finalTaskRunName] = &v1beta1.PipelineRunTaskRunStatus{ + PipelineTaskName: "final-task-1", + Status: &v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{Type: apis.ConditionSucceeded, Status: corev1.ConditionFalse}}, + }, + }, + } + 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) + } +} diff --git a/pkg/reconciler/pipelinerun/resources/pipelinerunresolution.go b/pkg/reconciler/pipelinerun/resources/pipelinerunresolution.go index 20b0609b597..1e5cb1b3287 100644 --- a/pkg/reconciler/pipelinerun/resources/pipelinerunresolution.go +++ b/pkg/reconciler/pipelinerun/resources/pipelinerunresolution.go @@ -157,6 +157,8 @@ func (state PipelineRunState) ToMap() map[string]*ResolvedPipelineRunTask { return m } +// IsDone returns true when all pipeline tasks have respective taskRun created and +// that taskRun has either succeeded or failed after all possible retry attempts func (state PipelineRunState) IsDone() (isDone bool) { isDone = true for _, t := range state { @@ -218,6 +220,74 @@ func (state PipelineRunState) SuccessfulPipelineTaskNames() []string { return done } +// SkippedPipelineTaskNames returns a list of the names of all of the PipelineTasks in state +// which have been skipped +func (state PipelineRunState) SkippedPipelineTaskNames(d *dag.Graph) []string { + skipped := []string{} + for _, t := range state { + if t.TaskRun == nil { + if isSkipped(t, state.ToMap(), d) { + skipped = append(skipped, t.PipelineTask.Name) + } + } + } + return skipped +} + +// isDAGTasksDone returns true if all DAG tasks are done executing, +// DAG tasks could have failed/succeeded/skipped +func (state PipelineRunState) isDAGTasksDone(d *dag.Graph) (isDone bool) { + isDone = true + for _, t := range state { + if _, ok := d.Nodes[t.PipelineTask.Name]; ok { + if t.TaskRun == nil { + // this task might have skipped if taskRun is nil + // continue and ignore if this task was skipped + // skipped task is considered part of done + if isSkipped(t, state.ToMap(), d) { + continue + } + return false + } + isDone = isDone && t.IsDone() + if !isDone { + return + } + } + } + return +} + +// isDAGTaskFailed return true if any single DAG task results in failure +func (state PipelineRunState) isDAGTaskFailed(d *dag.Graph) bool { + for _, t := range state { + if _, ok := d.Nodes[t.PipelineTask.Name]; ok { + if t.IsFailure() { + return true + } + } + } + return false +} + +// GetFinalTasks returns a list of final tasks without any taskRun associated with it +// GetFinalTasks returns final tasks only when all DAG tasks have finished executing successfully or skipped or +// any one DAG task resulted in failure +func (state PipelineRunState) GetFinalTasks(d *dag.Graph, dfinally *dag.Graph) []*ResolvedPipelineRunTask { + tasks := []*ResolvedPipelineRunTask{} + // check either pipeline has finished executing all DAG pipelineTasks + // or any one of the DAG pipelineTask has failed + if state.isDAGTaskFailed(d) || state.isDAGTasksDone(d) { + // return list of tasks with all final tasks + for _, t := range state { + if _, ok := dfinally.Nodes[t.PipelineTask.Name]; ok && t.TaskRun == nil { + tasks = append(tasks, t) + } + } + } + return tasks +} + // GetTaskRun is a function that will retrieve the TaskRun name. type GetTaskRun func(name string) (*v1beta1.TaskRun, error) @@ -407,10 +477,15 @@ func GetTaskRunName(taskRunsStatus map[string]*v1beta1.PipelineRunTaskRunStatus, // GetPipelineConditionStatus will return the Condition that the PipelineRun prName should be // updated with, based on the status of the TaskRuns in state. +// GetPipelineConditionStatus iterates over pipelineState to determine whether a pipeline was success or failure or hasn't finished executing +// GetPipelineConditionStatus returns failure if it finds one failed taskRun +// GetPipelineConditionStatus validates combination of the pipeline tasks with successful taskRuns and skipped pipeline tasks, +// against total number of pipeline tasks to return success +// A pipeline task is considered skipped if its condition results in failure or one of its parent has skipped func GetPipelineConditionStatus(pr *v1beta1.PipelineRun, state PipelineRunState, logger *zap.SugaredLogger, dag *dag.Graph) *apis.Condition { // We have 4 different states here: // 1. Timed out -> Failed - // 2. Any one TaskRun has failed - >Failed. This should change with #1020 and #1023 + // 2. Any one TaskRun has failed -> Failed. This should change with #1020 and #1023 // 3. All tasks are done or are skipped (i.e. condition check failed).-> Success // 4. A Task or Condition is running right now or there are things left to run -> Running if pr.IsTimedOut() { @@ -423,6 +498,8 @@ func GetPipelineConditionStatus(pr *v1beta1.PipelineRun, state PipelineRunState, } // A single failed task mean we fail the pipeline + // this failure status is still maintained with final tasks and reported as is after final tasks are finished executing + // if no failure is discovered in DAG tasks, but any one of the final tasks fail, results in overall pipeline failure for _, rprt := range state { if rprt.IsCancelled() { logger.Infof("TaskRun %s is cancelled, so PipelineRun %s is cancelled", rprt.TaskRunName, pr.Name) @@ -505,11 +582,14 @@ func isSkipped(rprt *ResolvedPipelineRunTask, stateMap map[string]*ResolvedPipel // Recursively look at parent tasks to see if they have been skipped, // if any of the parents have been skipped, skip as well - node := d.Nodes[rprt.PipelineTask.Name] - for _, p := range node.Prev { - skip := isSkipped(stateMap[p.Task.HashKey()], stateMap, d) - if skip { - return true + // we could be iterating over final tasks here which are not part of DAG graph + // check before accessing DAG node, if it exists + if node, ok := d.Nodes[rprt.PipelineTask.Name]; ok { + for _, p := range node.Prev { + skip := isSkipped(stateMap[p.Task.HashKey()], stateMap, d) + if skip { + return true + } } } return false diff --git a/pkg/reconciler/pipelinerun/resources/pipelinerunresolution_test.go b/pkg/reconciler/pipelinerun/resources/pipelinerunresolution_test.go index 294e3b285f5..879909afe13 100644 --- a/pkg/reconciler/pipelinerun/resources/pipelinerunresolution_test.go +++ b/pkg/reconciler/pipelinerun/resources/pipelinerunresolution_test.go @@ -21,6 +21,7 @@ import ( "errors" "fmt" "testing" + "time" "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" @@ -160,7 +161,7 @@ func makeFailed(tr v1beta1.TaskRun) *v1beta1.TaskRun { } func withCancelled(tr *v1beta1.TaskRun) *v1beta1.TaskRun { - tr.Status.Conditions[0].Reason = "TaskRunCancelled" + tr.Status.Conditions[0].Reason = v1beta1.TaskRunSpecStatusCancelled return tr } @@ -974,6 +975,48 @@ func TestSuccessfulPipelineTaskNames(t *testing.T) { } } +func TestPipelineRunState_SkippedPipelineTaskNames(t *testing.T) { + tcs := []struct { + name string + state PipelineRunState + expectedNames []string + }{{ + name: "conditional task not skipped as the condition execution was successful", + state: conditionCheckSuccessNoTaskStartedState, + expectedNames: []string{}, + }, { + name: "conditional task not skipped as the condition has not started executing yet", + state: conditionCheckStartedState, + expectedNames: []string{}, + }, { + name: "conditional task skipped as the condition execution resulted in failure", + state: conditionCheckFailedWithNoOtherTasksState, + expectedNames: []string{pts[5].Name}, + }, { + name: "conditional task skipped as the condition execution resulted in failure but the other pipeline task" + + "not skipped since it finished execution successfully", + state: conditionCheckFailedWithOthersPassedState, + expectedNames: []string{pts[5].Name}, + }, { + name: "conditional task skipped as the condition execution resulted in failure but the other pipeline task" + + "not skipped since it failed", + state: conditionCheckFailedWithOthersFailedState, + expectedNames: []string{pts[5].Name}, + }} + for _, tc := range tcs { + t.Run(tc.name, func(t *testing.T) { + dag, err := DagFromState(tc.state) + if err != nil { + t.Fatalf("Unexpected error while buildig DAG for state %v: %v", tc.state, err) + } + names := tc.state.SkippedPipelineTaskNames(dag) + if d := cmp.Diff(names, tc.expectedNames); d != "" { + t.Errorf("Expected to get completed names %v but got something different %s", tc.expectedNames, diff.PrintWantGot(d)) + } + }) + } +} + func TestGetPipelineConditionStatus(t *testing.T) { var taskRetriedState = PipelineRunState{{ @@ -985,6 +1028,23 @@ func TestGetPipelineConditionStatus(t *testing.T) { }, }} + var cancelledTask = PipelineRunState{{ + PipelineTask: &pts[3], // 1 retry needed + TaskRunName: "pipelinerun-mytask1", + TaskRun: &v1beta1.TaskRun{ + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{Conditions: []apis.Condition{{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + Reason: v1beta1.TaskRunSpecStatusCancelled, + }}}, + }, + }, + ResolvedTaskResources: &resources.ResolvedTaskResources{ + TaskSpec: &task.Spec, + }, + }} + tcs := []struct { name string state []*ResolvedPipelineRunTask @@ -1077,6 +1137,10 @@ func TestGetPipelineConditionStatus(t *testing.T) { name: "task with grand parents; one not run yet", state: taskWithGrandParentsOneNotRunState, expectedStatus: corev1.ConditionUnknown, + }, { + name: "cancelled task should result in cancelled pipeline", + state: cancelledTask, + expectedStatus: corev1.ConditionFalse, }} for _, tc := range tcs { t.Run(tc.name, func(t *testing.T) { @@ -1093,6 +1157,29 @@ func TestGetPipelineConditionStatus(t *testing.T) { } } +// pipeline should result in timeout if its runtime exceeds its spec.Timeout based on its status.Timeout +func TestGetPipelineConditionStatusForPipelineTimeouts(t *testing.T) { + dag, err := DagFromState(oneFinishedState) + if err != nil { + t.Fatalf("Unexpected error while buildig DAG for state %v: %v", oneFinishedState, err) + } + pr := &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "pipelinerun-no-tasks-started"}, + Spec: v1beta1.PipelineRunSpec{ + Timeout: &metav1.Duration{Duration: 1 * time.Minute}, + }, + Status: v1beta1.PipelineRunStatus{ + PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + StartTime: &metav1.Time{Time: time.Now().Add(-2 * time.Minute)}, + }, + }, + } + c := GetPipelineConditionStatus(pr, oneFinishedState, zap.NewNop().Sugar(), dag) + if c.Status != corev1.ConditionFalse && c.Reason != ReasonTimedOut { + t.Fatalf("Expected to get status %s but got %s for state %v", corev1.ConditionFalse, c.Status, oneFinishedState) + } +} + func TestGetResourcesFromBindings(t *testing.T) { pr := tb.PipelineRun("pipelinerun", tb.PipelineRunSpec("pipeline", tb.PipelineRunResourceBinding("git-resource", tb.PipelineResourceBindingRef("sweet-resource")), @@ -1971,3 +2058,131 @@ func TestIsBeforeFirstTaskRun_WithStartedTask(t *testing.T) { t.Fatalf("Expected state to be after first taskrun") } } + +func TestPipelineRunState_GetFinalTasks(t *testing.T) { + tcs := []struct { + name string + state PipelineRunState + DAGTasks []v1beta1.PipelineTask + finalTasks []v1beta1.PipelineTask + expectedFinalTasks []*ResolvedPipelineRunTask + }{{ + name: "all dag tasks started executing and finished successfully -" + + " no final tasks should be returned since pipeline didnt have any final tasks", + state: oneStartedState, + DAGTasks: []v1beta1.PipelineTask{pts[0], pts[1]}, + finalTasks: []v1beta1.PipelineTask{}, + expectedFinalTasks: []*ResolvedPipelineRunTask{}, + }, { + name: "none of the dag tasks started executing yet - no final tasks should be returned", + state: noneStartedState, + DAGTasks: []v1beta1.PipelineTask{pts[0]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{}, + }, { + name: "one dag task started executing but not finished - no final tasks should be returned", + state: oneStartedState, + DAGTasks: []v1beta1.PipelineTask{pts[0]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{}, + }, { + name: "pipeline with one dag task started executing and task finished successfully -" + + " all final tasks should be returned", + state: oneFinishedState, + DAGTasks: []v1beta1.PipelineTask{pts[0]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{oneFinishedState[1]}, + }, { + name: "one dag task started executing and resulted in failure - all final tasks should be returned", + state: oneFailedState, + DAGTasks: []v1beta1.PipelineTask{pts[0]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{oneFinishedState[1]}, + }, { + name: "pipeline with one conditional task - dag task condition has started evaluating but" + + " not finished - no final tasks should be returned", + state: append(conditionCheckStartedState, noneStartedState[0]), + DAGTasks: []v1beta1.PipelineTask{pts[5]}, + finalTasks: []v1beta1.PipelineTask{pts[0]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{}, + }, { + name: "pipeline with one conditional task - dag task condition was successful but task not" + + " started yet - no final tasks should be returned", + state: append(conditionCheckSuccessNoTaskStartedState, noneStartedState[0]), + DAGTasks: []v1beta1.PipelineTask{pts[5]}, + finalTasks: []v1beta1.PipelineTask{pts[0]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{}, + }, { + name: "pipeline with one conditional task - dag task condition failed so task was skipped -" + + " final tasks should be returned", + state: append(conditionCheckFailedWithNoOtherTasksState, noneStartedState[0]), + DAGTasks: []v1beta1.PipelineTask{pts[5]}, + finalTasks: []v1beta1.PipelineTask{pts[0]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{noneStartedState[0]}, + }, { + name: "dag task condition failed so task was skipped and rest of the dag tasks finished -" + + " final tasks should be returned", + state: append(conditionCheckFailedWithOthersPassedState, noneStartedState[1]), + DAGTasks: []v1beta1.PipelineTask{pts[5], pts[0]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{noneStartedState[1]}, + }, { + name: "dag task condition failed so task was skipped and rest of the dag task failed -" + + " final tasks should be returned", + state: append(conditionCheckFailedWithOthersFailedState, noneStartedState[1]), + DAGTasks: []v1beta1.PipelineTask{pts[5], pts[0]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{noneStartedState[1]}, + }, { + name: "dag task condition failed so task was skipped along with its dependent task -" + + " final tasks should be returned", + state: append(taskWithParentSkippedState, noneStartedState[1]), + DAGTasks: []v1beta1.PipelineTask{pts[5], pts[6]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{noneStartedState[1]}, + }, { + name: "dag task condition failed so task was skipped along with all dependent tasks -" + + " final tasks should be returned", + state: append(taskWithMultipleParentsSkippedState, noneStartedState[1]), + DAGTasks: []v1beta1.PipelineTask{pts[0], pts[5], pts[7]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{noneStartedState[1]}, + }, { + name: "dag task succeeded - next dag task condition failed so task was skipped along with all" + + " dependent tasks - final tasks should be returned", + state: append(taskWithGrandParentSkippedState, noneStartedState[1]), + DAGTasks: []v1beta1.PipelineTask{pts[0], pts[5], pts[7], pts[8]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{noneStartedState[1]}, + }, { + name: "dag task succeeded - next dag task condition succeeded but task failed -" + + " not executing rest of the dag tasks - final tasks should be returned", + state: append(taskWithGrandParentsOneFailedState, noneStartedState[1]), + DAGTasks: []v1beta1.PipelineTask{pts[0], pts[5], pts[7], pts[8]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{noneStartedState[1]}, + }, { + name: "dag task succeeded - next dag task condition succeeded but task not finished -" + + " not started executing rest of the dag tasks - no final tasks should be returned", + state: append(taskWithGrandParentsOneNotRunState, noneStartedState[1]), + DAGTasks: []v1beta1.PipelineTask{pts[0], pts[5], pts[7], pts[8]}, + finalTasks: []v1beta1.PipelineTask{pts[1]}, + expectedFinalTasks: []*ResolvedPipelineRunTask{}, + }} + for _, tc := range tcs { + dagGraph, err := dag.Build(v1beta1.PipelineTaskList(tc.DAGTasks)) + if err != nil { + t.Fatalf("Unexpected error while buildig DAG for pipelineTasks %v: %v", tc.DAGTasks, err) + } + finalGraph, err := dag.Build(v1beta1.PipelineTaskList(tc.finalTasks)) + if err != nil { + t.Fatalf("Unexpected error while buildig DAG for final pipelineTasks %v: %v", tc.finalTasks, err) + } + t.Run(tc.name, func(t *testing.T) { + next := tc.state.GetFinalTasks(dagGraph, finalGraph) + if d := cmp.Diff(next, tc.expectedFinalTasks); d != "" { + t.Errorf("Didn't get expected next Tasks %s", diff.PrintWantGot(d)) + } + }) + } +} diff --git a/test/pipelinefinally_test.go b/test/pipelinefinally_test.go new file mode 100644 index 00000000000..8ad3bceee74 --- /dev/null +++ b/test/pipelinefinally_test.go @@ -0,0 +1,350 @@ +// +build e2e + +/* +Copyright 2019 The Tekton Authors +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package test + +import ( + "strings" + "testing" + + "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1alpha1" + + "github.com/tektoncd/pipeline/pkg/reconciler/pipelinerun/resources" + + "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "knative.dev/pkg/apis" + duckv1beta1 "knative.dev/pkg/apis/duck/v1beta1" + knativetest "knative.dev/pkg/test" +) + +// Pipeline results in failure since dag task fails but final tasks does get executed +func TestPipelineLevelFinallyWhenOneDAGTaskFailed(t *testing.T) { + c, namespace := setup(t) + + taskName := "dagtask" + finalTaskName := "finaltask" + pipelineName := "pipeline-with-failed-dag-tasks" + pipelineRunName := "pipelinerun-with-failed-dag-tasks" + + knativetest.CleanupOnInterrupt(func() { tearDown(t, c, namespace) }, t.Logf) + defer tearDown(t, c, namespace) + + task := &v1beta1.Task{ + ObjectMeta: metav1.ObjectMeta{Name: taskName, Namespace: namespace}, + Spec: v1beta1.TaskSpec{ + Steps: []v1beta1.Step{{ + Container: corev1.Container{Image: "alpine"}, + Script: "exit 1", + }}, + }, + } + if _, err := c.TaskClient.Create(task); err != nil { + t.Fatalf("Failed to create non final Task: %s", err) + } + + finalTask := &v1beta1.Task{ + ObjectMeta: metav1.ObjectMeta{Name: finalTaskName, Namespace: namespace}, + Spec: v1beta1.TaskSpec{ + Steps: []v1beta1.Step{{ + Container: corev1.Container{Image: "alpine"}, + Script: "exit 0", + }}, + }, + } + if _, err := c.TaskClient.Create(finalTask); err != nil { + t.Fatalf("Failed to create final Task: %s", err) + } + + pipeline := &v1beta1.Pipeline{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineName, Namespace: namespace}, + Spec: v1beta1.PipelineSpec{ + Tasks: []v1beta1.PipelineTask{{ + Name: "dagtask1", + TaskRef: &v1beta1.TaskRef{Name: taskName}, + }}, + Finally: []v1beta1.PipelineTask{{ + Name: "finaltask1", + TaskRef: &v1beta1.TaskRef{Name: finalTaskName}, + }}, + }, + } + if _, err := c.PipelineClient.Create(pipeline); err != nil { + t.Fatalf("Failed to create Pipeline: %s", err) + } + + pipelineRun := &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineRunName, Namespace: namespace}, + Spec: v1beta1.PipelineRunSpec{ + PipelineRef: &v1beta1.PipelineRef{Name: pipelineName}, + }, + } + if _, err := c.PipelineRunClient.Create(pipelineRun); err != nil { + t.Fatalf("Failed to create Pipeline Run `%s`: %s", pipelineRunName, err) + } + + if err := WaitForPipelineRunState(c, pipelineRunName, timeout, PipelineRunFailed(pipelineRunName), "PipelineRunFailed"); err != nil { + t.Fatalf("Waiting for PipelineRun %s to fail: %v", pipelineRunName, err) + } + + taskrunList, err := c.TaskRunClient.List(metav1.ListOptions{LabelSelector: "tekton.dev/pipelineRun=" + pipelineRunName}) + if err != nil { + t.Fatalf("Error listing TaskRuns for PipelineRun %s: %s", pipelineRunName, err) + } + + // verify non final task failed and final task executed and succeeded + for _, taskrunItem := range taskrunList.Items { + switch n := taskrunItem.Name; { + case strings.HasPrefix(n, pipelineRunName+"-"+finalTaskName): + if !isSuccessful(t, n, taskrunItem.Status.Conditions) { + t.Fatalf("TaskRun %s for final task should have succedded", n) + } + case strings.HasPrefix(n, pipelineRunName+"-"+taskName): + if !isFailed(t, n, taskrunItem.Status.Conditions) { + t.Fatalf("TaskRun %s for non final task should have failed", n) + } + default: + t.Fatalf("TaskRuns were not found for both final and dag tasks") + } + } +} + +// Pipeline exits with success even if final task execution fails +func TestPipelineLevelFinallyWhenOneFinalTaskFailed(t *testing.T) { + c, namespace := setup(t) + + taskName := "dagtask" + finalTaskName := "finaltask" + pipelineName := "pipeline-with-failed-final-tasks" + pipelineRunName := "pipelinerun-with-failed-final-tasks" + + knativetest.CleanupOnInterrupt(func() { tearDown(t, c, namespace) }, t.Logf) + defer tearDown(t, c, namespace) + + task := &v1beta1.Task{ + ObjectMeta: metav1.ObjectMeta{Name: taskName, Namespace: namespace}, + Spec: v1beta1.TaskSpec{ + Steps: []v1beta1.Step{{ + Container: corev1.Container{Image: "alpine"}, + Script: "exit 0", + }}, + }, + } + if _, err := c.TaskClient.Create(task); err != nil { + t.Fatalf("Failed to create non final Task: %s", err) + } + + finalTask := &v1beta1.Task{ + ObjectMeta: metav1.ObjectMeta{Name: finalTaskName, Namespace: namespace}, + Spec: v1beta1.TaskSpec{ + Steps: []v1beta1.Step{{ + Container: corev1.Container{Image: "alpine"}, + Script: "exit 1", + }}, + }, + } + if _, err := c.TaskClient.Create(finalTask); err != nil { + t.Fatalf("Failed to create final Task: %s", err) + } + + pipeline := &v1beta1.Pipeline{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineName, Namespace: namespace}, + Spec: v1beta1.PipelineSpec{ + Tasks: []v1beta1.PipelineTask{{ + Name: "dagtask1", + TaskRef: &v1beta1.TaskRef{Name: taskName}, + }}, + Finally: []v1beta1.PipelineTask{{ + Name: "finaltask1", + TaskRef: &v1beta1.TaskRef{Name: finalTaskName}, + }}, + }, + } + if _, err := c.PipelineClient.Create(pipeline); err != nil { + t.Fatalf("Failed to create Pipeline: %s", err) + } + + pipelineRun := &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineRunName, Namespace: namespace}, + Spec: v1beta1.PipelineRunSpec{ + PipelineRef: &v1beta1.PipelineRef{Name: pipelineName}, + }, + } + if _, err := c.PipelineRunClient.Create(pipelineRun); err != nil { + t.Fatalf("Failed to create Pipeline Run `%s`: %s", pipelineRunName, err) + } + + if err := WaitForPipelineRunState(c, pipelineRunName, timeout, PipelineRunFailed(pipelineRunName), "PipelineRunFailed"); err != nil { + t.Errorf("Error waiting for PipelineRun %s to finish: %s", pipelineRunName, err) + t.Fatalf("PipelineRun execution failed") + } + + taskrunList, err := c.TaskRunClient.List(metav1.ListOptions{LabelSelector: "tekton.dev/pipelineRun=" + pipelineRunName}) + if err != nil { + t.Fatalf("Error listing TaskRuns for PipelineRun %s: %s", pipelineRunName, err) + } + + // verify non final task succeeded and final task failed + for _, taskrunItem := range taskrunList.Items { + switch n := taskrunItem.Name; { + case strings.HasPrefix(n, pipelineRunName+"-"+finalTaskName): + if !isFailed(t, n, taskrunItem.Status.Conditions) { + t.Fatalf("TaskRun %s for final task should have failed", n) + } + case strings.HasPrefix(n, pipelineRunName+"-"+taskName): + if !isSuccessful(t, n, taskrunItem.Status.Conditions) { + t.Fatalf("TaskRun %s for non final task should have succedded", n) + } + default: + t.Fatalf("TaskRuns were not found for both final and dag tasks") + } + } +} + +// The dag task is skipped due to condition failure, final tasks should still be executed +func TestPipelineLevelFinallyWhenDAGTaskSkipped(t *testing.T) { + c, namespace := setup(t) + + taskName := "dagtask" + finalTaskName := "finaltask" + pipelineName := "pipeline-with-skipped-dag-tasks" + pipelineRunName := "pipelinerun-with-skipped-dag-tasks" + condName := "failedcondition" + + knativetest.CleanupOnInterrupt(func() { tearDown(t, c, namespace) }, t.Logf) + defer tearDown(t, c, namespace) + + cond := &v1alpha1.Condition{ + ObjectMeta: metav1.ObjectMeta{Name: condName, Namespace: namespace}, + Spec: v1alpha1.ConditionSpec{ + Check: v1alpha1.Step{ + Container: corev1.Container{Image: "ubuntu"}, + Script: "exit 1", + }, + }, + } + if _, err := c.ConditionClient.Create(cond); err != nil { + t.Fatalf("Failed to create Condition `%s`: %s", cond1Name, err) + } + + task := &v1beta1.Task{ + ObjectMeta: metav1.ObjectMeta{Name: taskName, Namespace: namespace}, + Spec: v1beta1.TaskSpec{ + Steps: []v1beta1.Step{{ + Container: corev1.Container{Image: "alpine"}, + Script: "exit 0", + }}, + }, + } + if _, err := c.TaskClient.Create(task); err != nil { + t.Fatalf("Failed to create non final Task: %s", err) + } + + finalTask := &v1beta1.Task{ + ObjectMeta: metav1.ObjectMeta{Name: finalTaskName, Namespace: namespace}, + Spec: v1beta1.TaskSpec{ + Steps: []v1beta1.Step{{ + Container: corev1.Container{Image: "alpine"}, + Script: "exit 0", + }}, + }, + } + if _, err := c.TaskClient.Create(finalTask); err != nil { + t.Fatalf("Failed to create final Task: %s", err) + } + + pipeline := &v1beta1.Pipeline{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineName, Namespace: namespace}, + Spec: v1beta1.PipelineSpec{ + Tasks: []v1beta1.PipelineTask{{ + Name: "dagtask1", + TaskRef: &v1beta1.TaskRef{Name: taskName}, + Conditions: []v1beta1.PipelineTaskCondition{{ + ConditionRef: condName, + }}, + }}, + Finally: []v1beta1.PipelineTask{{ + Name: "finaltask1", + TaskRef: &v1beta1.TaskRef{Name: finalTaskName}, + }}, + }, + } + if _, err := c.PipelineClient.Create(pipeline); err != nil { + t.Fatalf("Failed to create Pipeline: %s", err) + } + + pipelineRun := &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: pipelineRunName, Namespace: namespace}, + Spec: v1beta1.PipelineRunSpec{ + PipelineRef: &v1beta1.PipelineRef{Name: pipelineName}, + }, + } + if _, err := c.PipelineRunClient.Create(pipelineRun); err != nil { + t.Fatalf("Failed to create Pipeline Run `%s`: %s", pipelineRunName, err) + } + + if err := WaitForPipelineRunState(c, pipelineRunName, timeout, PipelineRunSucceed(pipelineRunName), "PipelineRunCompleted"); err != nil { + t.Errorf("Error waiting for PipelineRun %s to finish: %s", pipelineRunName, err) + t.Fatalf("PipelineRun execution failed") + } + + taskrunList, err := c.TaskRunClient.List(metav1.ListOptions{LabelSelector: "tekton.dev/pipelineRun=" + pipelineRunName}) + if err != nil { + t.Fatalf("Error listing TaskRuns for PipelineRun %s: %s", pipelineRunName, err) + } + + // verify non final task skipped but final task executed and succeeded + for _, taskrunItem := range taskrunList.Items { + switch n := taskrunItem.Name; { + case strings.HasPrefix(n, pipelineRunName+"-"+finalTaskName): + if !isSuccessful(t, n, taskrunItem.Status.Conditions) { + t.Fatalf("TaskRun %s for final task should have succeeded", n) + } + case strings.HasPrefix(n, pipelineRunName+"-"+taskName): + if !isSkipped(t, n, taskrunItem.Status.Conditions) { + t.Fatalf("TaskRun %s for non final task should have skipped due to condition failure", n) + } + default: + t.Fatalf("TaskRuns were not found for both final and dag tasks") + } + } +} + +func isSuccessful(t *testing.T, taskRunName string, conds duckv1beta1.Conditions) bool { + for _, c := range conds { + if c.Type == apis.ConditionSucceeded { + if c.Status != corev1.ConditionTrue { + t.Errorf("TaskRun status %q is not succeeded, got %q", taskRunName, c.Status) + } + return true + } + } + t.Errorf("TaskRun status %q had no Succeeded condition", taskRunName) + return false +} + +func isSkipped(t *testing.T, taskRunName string, conds duckv1beta1.Conditions) bool { + for _, c := range conds { + if c.Type == apis.ConditionSucceeded { + if c.Status != corev1.ConditionFalse && c.Reason != resources.ReasonConditionCheckFailed { + t.Errorf("TaskRun status %q is not skipped due to condition failure, got %q", taskRunName, c.Status) + } + return true + } + } + t.Errorf("TaskRun status %q had no Succeeded condition", taskRunName) + return false +} diff --git a/third_party/github.com/hashicorp/go-multierror/go.mod b/third_party/github.com/hashicorp/go-multierror/go.mod index 2534331d5f9..0afe8e6f9d6 100644 --- a/third_party/github.com/hashicorp/go-multierror/go.mod +++ b/third_party/github.com/hashicorp/go-multierror/go.mod @@ -1,3 +1,5 @@ module github.com/hashicorp/go-multierror +go 1.14 + require github.com/hashicorp/errwrap v1.0.0