diff --git a/pkg/apis/pipeline/v1beta1/taskrun_types.go b/pkg/apis/pipeline/v1beta1/taskrun_types.go index 44a0f02b816..2a96b39d46a 100644 --- a/pkg/apis/pipeline/v1beta1/taskrun_types.go +++ b/pkg/apis/pipeline/v1beta1/taskrun_types.go @@ -88,6 +88,9 @@ const ( // TaskRunSpecStatusCancelled indicates that the user wants to cancel the task, // if not already cancelled or terminated TaskRunSpecStatusCancelled = "TaskRunCancelled" + // TaskRunSpecStatusTimedOut indicates that the PipelineRun owning this task wants to mark it as timed out, + // if not already cancelled or terminated + TaskRunSpecStatusTimedOut = "TaskRunTimedOut" ) // TaskRunDebug defines the breakpoint config for a particular TaskRun @@ -424,6 +427,11 @@ func (tr *TaskRun) IsCancelled() bool { return tr.Spec.Status == TaskRunSpecStatusCancelled } +// IsSpecTimedOut returns true if the TaskRun's spec status is set to TimedOut state +func (tr *TaskRun) IsSpecTimedOut() bool { + return tr.Spec.Status == TaskRunSpecStatusTimedOut +} + // HasTimedOut returns true if the TaskRun runtime is beyond the allowed timeout func (tr *TaskRun) HasTimedOut(ctx context.Context, c clock.PassiveClock) bool { if tr.Status.StartTime.IsZero() { diff --git a/pkg/apis/pipeline/v1beta1/taskrun_validation.go b/pkg/apis/pipeline/v1beta1/taskrun_validation.go index 2eed5c9e609..20838c4025f 100644 --- a/pkg/apis/pipeline/v1beta1/taskrun_validation.go +++ b/pkg/apis/pipeline/v1beta1/taskrun_validation.go @@ -79,8 +79,8 @@ func (ts *TaskRunSpec) Validate(ctx context.Context) (errs *apis.FieldError) { } if ts.Status != "" { - if ts.Status != TaskRunSpecStatusCancelled { - errs = errs.Also(apis.ErrInvalidValue(fmt.Sprintf("%s should be %s", ts.Status, TaskRunSpecStatusCancelled), "status")) + if ts.Status != TaskRunSpecStatusCancelled && ts.Status != TaskRunSpecStatusTimedOut { + errs = errs.Also(apis.ErrInvalidValue(fmt.Sprintf("%s should be %s or %s", ts.Status, TaskRunSpecStatusCancelled, TaskRunSpecStatusTimedOut), "status")) } } if ts.Timeout != nil { diff --git a/pkg/apis/pipeline/v1beta1/taskrun_validation_test.go b/pkg/apis/pipeline/v1beta1/taskrun_validation_test.go index 94c2392e367..6c860625f65 100644 --- a/pkg/apis/pipeline/v1beta1/taskrun_validation_test.go +++ b/pkg/apis/pipeline/v1beta1/taskrun_validation_test.go @@ -188,7 +188,7 @@ func TestTaskRunSpec_Invalidate(t *testing.T) { }, Status: "TaskRunCancell", }, - wantErr: apis.ErrInvalidValue("TaskRunCancell should be TaskRunCancelled", "status"), + wantErr: apis.ErrInvalidValue("TaskRunCancell should be TaskRunCancelled or TaskRunTimedOut", "status"), }, { name: "invalid taskspec", spec: v1beta1.TaskRunSpec{ diff --git a/pkg/reconciler/pipelinerun/pipelinerun.go b/pkg/reconciler/pipelinerun/pipelinerun.go index 19038cd6cb7..a05883f9a2a 100644 --- a/pkg/reconciler/pipelinerun/pipelinerun.go +++ b/pkg/reconciler/pipelinerun/pipelinerun.go @@ -110,6 +110,9 @@ const ( // ReasonCouldntCancel indicates that a PipelineRun was cancelled but attempting to update // all of the running TaskRuns as cancelled failed. ReasonCouldntCancel = "PipelineRunCouldntCancel" + // ReasonCouldntTimeOut indicates that a PipelineRun was timed out but attempting to update + // all of the running TaskRuns as timed out failed. + ReasonCouldntTimeOut = "PipelineRunCouldntTimeOut" // ReasonInvalidTaskResultReference indicates a task result was declared // but was not initialized by that task ReasonInvalidTaskResultReference = "InvalidTaskResultReference" @@ -579,6 +582,13 @@ func (c *Reconciler) reconcile(ctx context.Context, pr *v1beta1.PipelineRun, get // Reset the skipped status to trigger recalculation pipelineRunFacts.ResetSkippedCache() + // If the pipelinerun has timed out, mark tasks as timed out and update status + if pr.HasTimedOut(ctx, c.Clock) { + if err := timeoutPipelineRun(ctx, logger, pr, c.PipelineClientSet); err != nil { + return err + } + } + after := pipelineRunFacts.GetPipelineConditionStatus(ctx, pr, logger, c.Clock) switch after.Status { case corev1.ConditionTrue: @@ -621,7 +631,7 @@ func (c *Reconciler) processRunTimeouts(ctx context.Context, pr *v1beta1.Pipelin } for _, rpt := range pipelineState { if rpt.IsCustomTask() { - if rpt.Run != nil && !rpt.Run.IsCancelled() && (pr.HasTimedOut(ctx, c.Clock) || (rpt.Run.HasTimedOut(c.Clock) && !rpt.Run.IsDone())) { + if rpt.Run != nil && !rpt.Run.IsCancelled() && rpt.Run.HasTimedOut(c.Clock) && !rpt.Run.IsDone() { logger.Infof("Cancelling run task: %s due to timeout.", rpt.RunName) err := cancelRun(ctx, rpt.RunName, pr.Namespace, c.PipelineClientSet) if err != nil { @@ -1141,12 +1151,10 @@ func calculateTaskRunTimeout(timeout time.Duration, pr *v1beta1.PipelineRun, rpt if pElapsedTime > timeout { return &metav1.Duration{Duration: 1 * time.Second} } - timeRemaining := (timeout - pElapsedTime) - // Return the smaller of timeRemaining and rpt.pipelineTask.timeout - if rpt.PipelineTask.Timeout != nil && rpt.PipelineTask.Timeout.Duration < timeRemaining { + if rpt.PipelineTask.Timeout != nil && rpt.PipelineTask.Timeout.Duration < timeout { return &metav1.Duration{Duration: rpt.PipelineTask.Timeout.Duration} } - return &metav1.Duration{Duration: timeRemaining} + return &metav1.Duration{Duration: timeout} } if timeout == apisconfig.NoTimeoutDuration && rpt.PipelineTask.Timeout != nil { diff --git a/pkg/reconciler/pipelinerun/pipelinerun_test.go b/pkg/reconciler/pipelinerun/pipelinerun_test.go index c3f8779db2f..7863777d3fe 100644 --- a/pkg/reconciler/pipelinerun/pipelinerun_test.go +++ b/pkg/reconciler/pipelinerun/pipelinerun_test.go @@ -1736,16 +1736,44 @@ spec: name: test-pipeline serviceAccountName: test-sa status: + conditions: + - message: running... + reason: Running + status: Unknown + type: Succeeded startTime: "2021-12-31T00:00:00Z" + runs: + test-pipeline-run-custom-task-hello-world-1: + pipelineTaskName: hello-world-1 + status: + conditions: + - status: Unknown + type: Succeeded `)} prs[0].Spec.Timeout = tc.timeout prs[0].Spec.Timeouts = tc.timeouts + runs := []*v1alpha1.Run{mustParseRunWithObjectMeta(t, + taskRunObjectMeta("test-pipeline-run-custom-task-hello-world-1", "test", "test-pipeline-run-custom-task", + "test-pipeline", "hello-world-1", true), + ` +spec: + ref: + apiVersion: example.dev/v0 + kind: Example +status: + conditions: + - status: Unknown + type: Succeeded + startTime: "2021-12-31T11:58:59Z" +`)} + cms := []*corev1.ConfigMap{withCustomTasks(newFeatureFlagsConfigMap())} d := test.Data{ PipelineRuns: prs, Pipelines: ps, ConfigMaps: cms, + Runs: runs, } prt := newPipelineRunTest(d, t) defer prt.Cancel() @@ -1763,9 +1791,9 @@ status: } gotTimeoutValue := postReconcileRun.GetTimeout() - expectedTimeoutValue := time.Second + expectedTimeoutValue := time.Hour - if d := cmp.Diff(gotTimeoutValue, expectedTimeoutValue); d != "" { + if d := cmp.Diff(expectedTimeoutValue, gotTimeoutValue); d != "" { t.Fatalf("Expected timeout for created Run, but got a mismatch %s", diff.PrintWantGot(d)) } @@ -2755,6 +2783,120 @@ spec: } } +func TestReconcileFailsTaskRunTimeOut(t *testing.T) { + prName := "test-pipeline-fails-to-timeout" + + // TestReconcileFailsTaskRunTimeOut runs "Reconcile" on a PipelineRun with a single TaskRun. + // The TaskRun cannot be timed out. Check that the pipelinerun timeout fails, that reconcile fails and + // an event is generated + names.TestingSeed() + prs := []*v1beta1.PipelineRun{parse.MustParsePipelineRun(t, ` +metadata: + name: test-pipeline-fails-to-timeout + namespace: foo +spec: + pipelineRef: + name: test-pipeline + timeout: 1h0m0s +status: + conditions: + - message: running... + reason: Running + status: Unknown + type: Succeeded + startTime: "2021-12-31T22:59:00Z" + taskRuns: + test-pipeline-fails-to-timeouthello-world-1: + pipelineTaskName: hello-world-1 +`)} + ps := []*v1beta1.Pipeline{parse.MustParsePipeline(t, ` +metadata: + name: test-pipeline + namespace: foo +spec: + tasks: + - name: hello-world-1 + taskRef: + name: hello-world + - name: hello-world-2 + taskRef: + name: hello-world +`)} + tasks := []*v1beta1.Task{simpleHelloWorldTask} + taskRuns := []*v1beta1.TaskRun{ + getTaskRun( + t, + "test-pipeline-fails-to-timeouthello-world-1", + prName, + "test-pipeline", + "hello-world", + corev1.ConditionUnknown, + ), + } + + cms := []*corev1.ConfigMap{withEnabledAlphaAPIFields(newFeatureFlagsConfigMap())} + + d := test.Data{ + PipelineRuns: prs, + Pipelines: ps, + Tasks: tasks, + TaskRuns: taskRuns, + ConfigMaps: cms, + } + + testAssets, cancel := getPipelineRunController(t, d) + defer cancel() + c := testAssets.Controller + clients := testAssets.Clients + failingReactorActivated := true + + // Make the patch call fail, i.e. make it so that the controller fails to cancel the TaskRun + clients.Pipeline.PrependReactor("patch", "taskruns", func(action ktesting.Action) (bool, runtime.Object, error) { + return failingReactorActivated, nil, fmt.Errorf("i'm sorry Dave, i'm afraid i can't do that") + }) + + err := c.Reconciler.Reconcile(testAssets.Ctx, "foo/test-pipeline-fails-to-timeout") + if err == nil { + t.Errorf("Expected to see error returned from reconcile after failing to timeout TaskRun but saw none!") + } + + // Check that the PipelineRun is still running with correct error message + reconciledRun, err := clients.Pipeline.TektonV1beta1().PipelineRuns("foo").Get(testAssets.Ctx, "test-pipeline-fails-to-timeout", metav1.GetOptions{}) + if err != nil { + t.Fatalf("Somehow had error getting reconciled run out of fake client: %s", err) + } + + if val, ok := reconciledRun.GetLabels()[pipeline.PipelineLabelKey]; !ok { + t.Fatalf("expected pipeline label") + } else if d := cmp.Diff("test-pipeline", val); d != "" { + t.Errorf("expected to see pipeline label. Diff %s", diff.PrintWantGot(d)) + } + + // The PipelineRun should not be timed out b/c we couldn't timeout the TaskRun + checkPipelineRunConditionStatusAndReason(t, reconciledRun, corev1.ConditionUnknown, ReasonCouldntTimeOut) + // The event here is "Normal" because in case we fail to timeout we leave the condition to unknown + // Further reconcile might converge then the status of the pipeline. + // See https://github.com/tektoncd/pipeline/issues/2647 for further details. + wantEvents := []string{ + "Normal PipelineRunCouldntTimeOut PipelineRun \"test-pipeline-fails-to-timeout\" was timed out but had errors trying to time out TaskRuns and/or Runs", + "Warning InternalError 1 error occurred", + } + err = eventstest.CheckEventsOrdered(t, testAssets.Recorder.Events, prName, wantEvents) + if err != nil { + t.Errorf(err.Error()) + } + + // Turn off failing reactor and retry reconciliation + failingReactorActivated = false + + err = c.Reconciler.Reconcile(testAssets.Ctx, "foo/test-pipeline-fails-to-timeout") + if err == nil { + // No error is ok + } else if ok, _ := controller.IsRequeueKey(err); !ok { // Requeue is also fine. + t.Errorf("Expected to timeout TaskRun successfully!") + } +} + func TestReconcilePropagateLabelsAndAnnotations(t *testing.T) { names.TestingSeed() @@ -3094,7 +3236,7 @@ status: reconciledRun, _ := prt.reconcileRun("foo", "test-pipeline-retry-run-with-timeout", []string{}, false) if len(reconciledRun.Status.TaskRuns["hello-world-1"].Status.RetriesStatus) != tc.retries { - t.Fatalf(" %d retry expected but %d ", tc.retries, len(reconciledRun.Status.TaskRuns["hello-world-1"].Status.RetriesStatus)) + t.Fatalf(" %d retries expected but got %d ", tc.retries, len(reconciledRun.Status.TaskRuns["hello-world-1"].Status.RetriesStatus)) } if status := reconciledRun.Status.TaskRuns["hello-world-1"].Status.GetCondition(apis.ConditionSucceeded).Status; status != tc.conditionSucceeded { @@ -3240,7 +3382,7 @@ func TestGetTaskRunTimeout(t *testing.T) { }, }, }, - expected: &metav1.Duration{Duration: 10 * time.Minute}, + expected: &metav1.Duration{Duration: 20 * time.Minute}, }, { name: "taskrun with elapsed time; task.timeout applies", timeoutFields: &v1beta1.TimeoutFields{ @@ -3259,7 +3401,7 @@ func TestGetTaskRunTimeout(t *testing.T) { }, }, }, - expected: &metav1.Duration{Duration: 10 * time.Minute}, + expected: &metav1.Duration{Duration: 15 * time.Minute}, }, { name: "taskrun with elapsed time; timeouts.pipeline applies", timeoutFields: &v1beta1.TimeoutFields{ @@ -3278,7 +3420,7 @@ func TestGetTaskRunTimeout(t *testing.T) { }, }, }, - expected: &metav1.Duration{Duration: 10 * time.Minute}, + expected: &metav1.Duration{Duration: 15 * time.Minute}, }} for _, tc := range tcs { @@ -3296,7 +3438,7 @@ func TestGetTaskRunTimeout(t *testing.T) { }, }, } - if d := cmp.Diff(getTaskRunTimeout(context.TODO(), pr, tc.rpt, testClock), tc.expected); d != "" { + if d := cmp.Diff(tc.expected, getTaskRunTimeout(context.TODO(), pr, tc.rpt, testClock)); d != "" { t.Errorf("Unexpected task run timeout. Diff %s", diff.PrintWantGot(d)) } }) @@ -3449,7 +3591,7 @@ func TestGetFinallyTaskRunTimeout(t *testing.T) { }, }, }, - expected: &metav1.Duration{Duration: 11 * time.Minute}, + expected: &metav1.Duration{Duration: 15 * time.Minute}, }} for _, tc := range tcs { diff --git a/pkg/reconciler/pipelinerun/timeout.go b/pkg/reconciler/pipelinerun/timeout.go new file mode 100644 index 00000000000..8862c016922 --- /dev/null +++ b/pkg/reconciler/pipelinerun/timeout.go @@ -0,0 +1,109 @@ +/* +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 pipelinerun + +import ( + "context" + "encoding/json" + "fmt" + "log" + "strings" + "time" + + "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1" + clientset "github.com/tektoncd/pipeline/pkg/client/clientset/versioned" + "go.uber.org/zap" + jsonpatch "gomodules.xyz/jsonpatch/v2" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" + "knative.dev/pkg/apis" +) + +var timeoutTaskRunPatchBytes []byte + +func init() { + var err error + timeoutTaskRunPatchBytes, err = json.Marshal([]jsonpatch.JsonPatchOperation{{ + Operation: "add", + Path: "/spec/status", + Value: v1beta1.TaskRunSpecStatusTimedOut, + }}) + if err != nil { + log.Fatalf("failed to marshal TaskRun timed out patch bytes: %v", err) + } +} + +// timeoutPipelineRun marks the PipelineRun as timed out and any resolved TaskRun(s) too. +func timeoutPipelineRun(ctx context.Context, logger *zap.SugaredLogger, pr *v1beta1.PipelineRun, clientSet clientset.Interface) error { + errs := timeoutPipelineTaskRuns(ctx, logger, pr, clientSet) + + // If we successfully timed out all the TaskRuns and Runs, we can consider the PipelineRun timed out. + if len(errs) == 0 { + reason := v1beta1.PipelineRunReasonTimedOut.String() + + pr.Status.SetCondition(&apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + Reason: reason, + Message: fmt.Sprintf("PipelineRun %q failed to finish within %q", pr.Name, pr.PipelineTimeout(ctx).String()), + }) + // update pr completed time + pr.Status.CompletionTime = &metav1.Time{Time: time.Now()} + } else { + e := strings.Join(errs, "\n") + // Indicate that we failed to time out the PipelineRun + pr.Status.SetCondition(&apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + Reason: ReasonCouldntTimeOut, + Message: fmt.Sprintf("PipelineRun %q was timed out but had errors trying to time out TaskRuns and/or Runs: %s", pr.Name, e), + }) + return fmt.Errorf("error(s) from timing out TaskRun(s) from PipelineRun %s: %s", pr.Name, e) + } + return nil +} + +// timeoutPipelineTaskRuns patches `TaskRun` and `Run` with timed out status +func timeoutPipelineTaskRuns(ctx context.Context, logger *zap.SugaredLogger, pr *v1beta1.PipelineRun, clientSet clientset.Interface) []string { + errs := []string{} + + trNames, runNames, err := getChildObjectsFromPRStatus(ctx, pr.Status) + if err != nil { + errs = append(errs, err.Error()) + } + + for _, taskRunName := range trNames { + logger.Infof("timing out TaskRun %s", taskRunName) + + if _, err := clientSet.TektonV1beta1().TaskRuns(pr.Namespace).Patch(ctx, taskRunName, types.JSONPatchType, timeoutTaskRunPatchBytes, metav1.PatchOptions{}, ""); err != nil { + errs = append(errs, fmt.Errorf("failed to patch TaskRun `%s` with timeout: %s", taskRunName, err).Error()) + continue + } + } + + for _, runName := range runNames { + logger.Infof("cancelling Run %s", runName) + + if err := cancelRun(ctx, runName, pr.Namespace, clientSet); err != nil { + errs = append(errs, fmt.Errorf("Failed to patch Run `%s` with cancellation: %s", runName, err).Error()) + continue + } + } + + return errs +} diff --git a/pkg/reconciler/pipelinerun/timeout_test.go b/pkg/reconciler/pipelinerun/timeout_test.go new file mode 100644 index 00000000000..f0d5b0ef0e8 --- /dev/null +++ b/pkg/reconciler/pipelinerun/timeout_test.go @@ -0,0 +1,249 @@ +/* +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 pipelinerun + +import ( + "context" + "testing" + + "github.com/tektoncd/pipeline/pkg/apis/config" + "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1alpha1" + "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1" + _ "github.com/tektoncd/pipeline/pkg/pipelinerunmetrics/fake" // Make sure the pipelinerunmetrics are setup + ttesting "github.com/tektoncd/pipeline/pkg/reconciler/testing" + "github.com/tektoncd/pipeline/test" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" + "knative.dev/pkg/apis" + logtesting "knative.dev/pkg/logging/testing" +) + +func TestTimeoutPipelineRun(t *testing.T) { + testCases := []struct { + name string + embeddedStatus string + pipelineRun *v1beta1.PipelineRun + taskRuns []*v1beta1.TaskRun + runs []*v1alpha1.Run + wantErr bool + }{{ + name: "no-resolved-taskrun", + embeddedStatus: config.DefaultEmbeddedStatus, + pipelineRun: &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "test-pipeline-run-timedout"}, + Spec: v1beta1.PipelineRunSpec{}, + }, + }, { + name: "one-taskrun", + embeddedStatus: config.DefaultEmbeddedStatus, + pipelineRun: &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "test-pipeline-run-timedout"}, + Spec: v1beta1.PipelineRunSpec{}, + Status: v1beta1.PipelineRunStatus{PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + TaskRuns: map[string]*v1beta1.PipelineRunTaskRunStatus{ + "t1": {PipelineTaskName: "task-1"}, + }, + }}, + }, + taskRuns: []*v1beta1.TaskRun{ + {ObjectMeta: metav1.ObjectMeta{Name: "t1"}}, + }, + }, { + name: "multiple-taskruns", + embeddedStatus: config.DefaultEmbeddedStatus, + pipelineRun: &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "test-pipeline-run-timedout"}, + Spec: v1beta1.PipelineRunSpec{}, + Status: v1beta1.PipelineRunStatus{PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + TaskRuns: map[string]*v1beta1.PipelineRunTaskRunStatus{ + "t1": {PipelineTaskName: "task-1"}, + "t2": {PipelineTaskName: "task-2"}, + }, + }}, + }, + taskRuns: []*v1beta1.TaskRun{ + {ObjectMeta: metav1.ObjectMeta{Name: "t1"}}, + {ObjectMeta: metav1.ObjectMeta{Name: "t2"}}, + }, + }, { + name: "multiple-runs", + embeddedStatus: config.DefaultEmbeddedStatus, + pipelineRun: &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "test-pipeline-run-timedout"}, + Spec: v1beta1.PipelineRunSpec{}, + Status: v1beta1.PipelineRunStatus{PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + Runs: map[string]*v1beta1.PipelineRunRunStatus{ + "t1": {PipelineTaskName: "task-1"}, + "t2": {PipelineTaskName: "task-2"}, + }, + }}, + }, + runs: []*v1alpha1.Run{ + {ObjectMeta: metav1.ObjectMeta{Name: "t1"}}, + {ObjectMeta: metav1.ObjectMeta{Name: "t2"}}, + }, + }, { + name: "child-references-with-both", + embeddedStatus: config.BothEmbeddedStatus, + pipelineRun: &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "test-pipeline-run-timedout"}, + Spec: v1beta1.PipelineRunSpec{}, + Status: v1beta1.PipelineRunStatus{PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + ChildReferences: []v1beta1.ChildStatusReference{ + { + TypeMeta: runtime.TypeMeta{Kind: "TaskRun"}, + Name: "t1", + PipelineTaskName: "task-1", + }, + { + TypeMeta: runtime.TypeMeta{Kind: "TaskRun"}, + Name: "t2", + PipelineTaskName: "task-2", + }, + { + TypeMeta: runtime.TypeMeta{Kind: "Run"}, + Name: "r1", + PipelineTaskName: "run-1", + }, + { + TypeMeta: runtime.TypeMeta{Kind: "Run"}, + Name: "r2", + PipelineTaskName: "run-2", + }, + }, + }}, + }, + taskRuns: []*v1beta1.TaskRun{ + {ObjectMeta: metav1.ObjectMeta{Name: "t1"}}, + {ObjectMeta: metav1.ObjectMeta{Name: "t2"}}, + }, + runs: []*v1alpha1.Run{ + {ObjectMeta: metav1.ObjectMeta{Name: "r1"}}, + {ObjectMeta: metav1.ObjectMeta{Name: "r2"}}, + }, + }, { + name: "child-references-with-minimal", + embeddedStatus: config.MinimalEmbeddedStatus, + pipelineRun: &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "test-pipeline-run-timedout"}, + Spec: v1beta1.PipelineRunSpec{}, + Status: v1beta1.PipelineRunStatus{PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + ChildReferences: []v1beta1.ChildStatusReference{ + { + TypeMeta: runtime.TypeMeta{Kind: "TaskRun"}, + Name: "t1", + PipelineTaskName: "task-1", + }, + { + TypeMeta: runtime.TypeMeta{Kind: "TaskRun"}, + Name: "t2", + PipelineTaskName: "task-2", + }, + { + TypeMeta: runtime.TypeMeta{Kind: "Run"}, + Name: "r1", + PipelineTaskName: "run-1", + }, + { + TypeMeta: runtime.TypeMeta{Kind: "Run"}, + Name: "r2", + PipelineTaskName: "run-2", + }, + }, + }}, + }, + taskRuns: []*v1beta1.TaskRun{ + {ObjectMeta: metav1.ObjectMeta{Name: "t1"}}, + {ObjectMeta: metav1.ObjectMeta{Name: "t2"}}, + }, + runs: []*v1alpha1.Run{ + {ObjectMeta: metav1.ObjectMeta{Name: "r1"}}, + {ObjectMeta: metav1.ObjectMeta{Name: "r2"}}, + }, + }, { + name: "unknown-kind-on-child-references", + embeddedStatus: config.MinimalEmbeddedStatus, + pipelineRun: &v1beta1.PipelineRun{ + ObjectMeta: metav1.ObjectMeta{Name: "test-pipeline-run-timedout"}, + Spec: v1beta1.PipelineRunSpec{}, + Status: v1beta1.PipelineRunStatus{PipelineRunStatusFields: v1beta1.PipelineRunStatusFields{ + ChildReferences: []v1beta1.ChildStatusReference{{ + TypeMeta: runtime.TypeMeta{Kind: "InvalidKind"}, + Name: "t1", + PipelineTaskName: "task-1", + }}, + }}, + }, + wantErr: true, + }} + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + + d := test.Data{ + PipelineRuns: []*v1beta1.PipelineRun{tc.pipelineRun}, + TaskRuns: tc.taskRuns, + Runs: tc.runs, + } + ctx, _ := ttesting.SetupFakeContext(t) + cfg := config.NewStore(logtesting.TestLogger(t)) + cfg.OnConfigChanged(withCustomTasks(withEmbeddedStatus(newFeatureFlagsConfigMap(), tc.embeddedStatus))) + ctx = cfg.ToContext(ctx) + ctx, cancel := context.WithCancel(ctx) + defer cancel() + c, _ := test.SeedTestData(t, ctx, d) + + err := timeoutPipelineRun(ctx, logtesting.TestLogger(t), tc.pipelineRun, c.Pipeline) + if tc.wantErr { + if err == nil { + t.Error("expected an error, but did not get one") + } + } else { + if err != nil { + t.Fatal(err) + } + // This PipelineRun should still be complete and false, and the status should reflect that + cond := tc.pipelineRun.Status.GetCondition(apis.ConditionSucceeded) + if cond.IsTrue() { + t.Errorf("Expected PipelineRun status to be complete and false, but was %v", cond) + } + if tc.taskRuns != nil { + for _, expectedTR := range tc.taskRuns { + tr, err := c.Pipeline.TektonV1beta1().TaskRuns("").Get(ctx, expectedTR.Name, metav1.GetOptions{}) + if err != nil { + t.Fatalf("couldn't get expected TaskRun %s, got error %s", expectedTR.Name, err) + } + if tr.Spec.Status != v1beta1.TaskRunSpecStatusTimedOut { + t.Errorf("expected task %q to be marked as timed out, was %q", tr.Name, tr.Spec.Status) + } + } + } + if tc.runs != nil { + for _, expectedRun := range tc.runs { + r, err := c.Pipeline.TektonV1alpha1().Runs("").Get(ctx, expectedRun.Name, metav1.GetOptions{}) + if err != nil { + t.Fatalf("couldn't get expected Run %s, got error %s", expectedRun.Name, err) + } + if r.Spec.Status != v1alpha1.RunSpecStatusCancelled { + t.Errorf("expected task %q to be marked as cancelled, was %q", r.Name, r.Spec.Status) + } + } + } + } + }) + } +} diff --git a/pkg/reconciler/taskrun/taskrun.go b/pkg/reconciler/taskrun/taskrun.go index 90f7beb56e6..9775d4d809e 100644 --- a/pkg/reconciler/taskrun/taskrun.go +++ b/pkg/reconciler/taskrun/taskrun.go @@ -149,6 +149,13 @@ func (c *Reconciler) ReconcileKind(ctx context.Context, tr *v1beta1.TaskRun) pkg return c.finishReconcileUpdateEmitEvents(ctx, tr, before, err) } + // If the TaskRun is timed out via the spec status, kill resources and update status + if tr.IsSpecTimedOut() { + message := fmt.Sprintf("TaskRun %q failed to finish before the PipelineRun it belongs to timed out", tr.Name) + err := c.failTaskRun(ctx, tr, v1beta1.TaskRunReasonTimedOut, message) + return c.finishReconcileUpdateEmitEvents(ctx, tr, before, err) + } + // Check if the TaskRun has timed out; if it is, this will set its status // accordingly. if tr.HasTimedOut(ctx, c.Clock) { diff --git a/pkg/reconciler/taskrun/taskrun_test.go b/pkg/reconciler/taskrun/taskrun_test.go index 878e9a7bf34..5da95c275ed 100644 --- a/pkg/reconciler/taskrun/taskrun_test.go +++ b/pkg/reconciler/taskrun/taskrun_test.go @@ -1975,6 +1975,78 @@ status: } } +func TestReconcileOnTimedOutTaskRun(t *testing.T) { + taskRun := parse.MustParseTaskRun(t, ` +metadata: + name: test-taskrun-run-timedout + namespace: foo +spec: + status: TaskRunTimedOut + taskRef: + name: test-task +status: + conditions: + - status: Unknown + type: Succeeded + podName: test-taskrun-run-timedout-pod +`) + pod, err := makePod(taskRun, simpleTask) + if err != nil { + t.Fatalf("MakePod: %v", err) + } + d := test.Data{ + TaskRuns: []*v1beta1.TaskRun{taskRun}, + Tasks: []*v1beta1.Task{simpleTask}, + Pods: []*corev1.Pod{pod}, + } + + testAssets, cancel := getTaskRunController(t, d) + defer cancel() + c := testAssets.Controller + clients := testAssets.Clients + + if err := c.Reconciler.Reconcile(testAssets.Ctx, getRunName(taskRun)); err != nil { + t.Fatalf("Unexpected error when reconciling completed TaskRun : %v", err) + } + newTr, err := clients.Pipeline.TektonV1beta1().TaskRuns(taskRun.Namespace).Get(testAssets.Ctx, taskRun.Name, metav1.GetOptions{}) + if err != nil { + t.Fatalf("Expected completed TaskRun %s to exist but instead got error when getting it: %v", taskRun.Name, err) + } + + expectedStatus := &apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + Reason: v1beta1.TaskRunReasonTimedOut.String(), + Message: `TaskRun "test-taskrun-run-timedout" failed to finish before the PipelineRun it belongs to timed out`, + } + if d := cmp.Diff(expectedStatus, newTr.Status.GetCondition(apis.ConditionSucceeded), ignoreLastTransitionTime); d != "" { + t.Fatalf("Did not get expected condition %s", diff.PrintWantGot(d)) + } + + wantEvents := []string{ + "Normal Started", + "Warning Failed TaskRun \"test-taskrun-run-timedout\" failed to finish before the PipelineRun it belongs to timed out", + } + err = eventstest.CheckEventsOrdered(t, testAssets.Recorder.Events, "test-reconcile-on-timedout-taskrun", wantEvents) + if !(err == nil) { + t.Errorf(err.Error()) + } + + // reconcile the completed TaskRun again without the pod as that was deleted + d = test.Data{ + TaskRuns: []*v1beta1.TaskRun{newTr}, + Tasks: []*v1beta1.Task{simpleTask}, + } + + testAssets, cancel = getTaskRunController(t, d) + defer cancel() + c = testAssets.Controller + + if err := c.Reconciler.Reconcile(testAssets.Ctx, getRunName(newTr)); err != nil { + t.Fatalf("Unexpected error when reconciling completed TaskRun : %v", err) + } +} + func TestReconcilePodFailuresStepImagePullFailed(t *testing.T) { taskRun := parse.MustParseTaskRun(t, ` metadata: