diff --git a/pkg/apis/pipeline/v1alpha1/taskrun_types.go b/pkg/apis/pipeline/v1alpha1/taskrun_types.go index dc947b32613..a2aa46f6446 100644 --- a/pkg/apis/pipeline/v1alpha1/taskrun_types.go +++ b/pkg/apis/pipeline/v1alpha1/taskrun_types.go @@ -18,7 +18,9 @@ package v1alpha1 import ( "fmt" + "time" + apisconfig "github.com/tektoncd/pipeline/pkg/apis/config" "github.com/tektoncd/pipeline/pkg/apis/pipeline" "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1" corev1 "k8s.io/api/core/v1" @@ -77,6 +79,10 @@ const ( // TaskRunSpecStatusCancelled indicates that the user wants to cancel the task, // if not already cancelled or terminated TaskRunSpecStatusCancelled = v1beta1.TaskRunSpecStatusCancelled + + // TaskRunReasonCancelled indicates that the TaskRun has been cancelled + // because it was requested so by the user + TaskRunReasonCancelled = v1beta1.TaskRunSpecStatusCancelled ) // TaskRunInputs holds the input values that this task was invoked with. @@ -223,6 +229,28 @@ func (tr *TaskRun) IsCancelled() bool { return tr.Spec.Status == TaskRunSpecStatusCancelled } +// HasTimedOut returns true if the TaskRun runtime is beyond the allowed timeout +func (tr *TaskRun) HasTimedOut() bool { + if tr.Status.StartTime.IsZero() { + return false + } + timeout := tr.GetTimeout() + // If timeout is set to 0 or defaulted to 0, there is no timeout. + if timeout == apisconfig.NoTimeoutDuration { + return false + } + runtime := time.Since(tr.Status.StartTime.Time) + return runtime > timeout +} + +func (tr *TaskRun) GetTimeout() time.Duration { + // Use the platform default is no timeout is set + if tr.Spec.Timeout == nil { + return apisconfig.DefaultTimeoutMinutes * time.Minute + } + return tr.Spec.Timeout.Duration +} + // GetRunKey return the taskrun key for timeout handler map func (tr *TaskRun) GetRunKey() string { // The address of the pointer is a threadsafe unique identifier for the taskrun diff --git a/pkg/apis/pipeline/v1alpha1/taskrun_types_test.go b/pkg/apis/pipeline/v1alpha1/taskrun_types_test.go index 65d66d0b791..f23d2a69919 100644 --- a/pkg/apis/pipeline/v1alpha1/taskrun_types_test.go +++ b/pkg/apis/pipeline/v1alpha1/taskrun_types_test.go @@ -215,3 +215,40 @@ func TestTaskRunIsOfPipelinerun(t *testing.T) { }) } } + +func TestHasTimedOut(t *testing.T) { + // IsZero reports whether t represents the zero time instant, January 1, year 1, 00:00:00 UTC + zeroTime := time.Date(1, 1, 1, 0, 0, 0, 0, time.UTC) + testCases := []struct { + name string + taskRun *v1alpha1.TaskRun + expectedStatus bool + }{{ + name: "TaskRun not started", + taskRun: tb.TaskRun("test-taskrun-not-started", "foo", tb.TaskRunSpec( + tb.TaskRunTaskRef("task-name"), + ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{}), tb.TaskRunStartTime(zeroTime))), + expectedStatus: false, + }, { + name: "TaskRun no timeout", + taskRun: tb.TaskRun("test-taskrun-no-timeout", "foo", tb.TaskRunSpec( + tb.TaskRunTaskRef("task-name"), tb.TaskRunTimeout(0), + ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{}), tb.TaskRunStartTime(time.Now().Add(-15*time.Hour)))), + expectedStatus: false, + }, { + name: "TaskRun timed out", + taskRun: tb.TaskRun("test-taskrun-timeout", "foo", tb.TaskRunSpec( + tb.TaskRunTaskRef("task-name"), tb.TaskRunTimeout(10*time.Second), + ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{}), tb.TaskRunStartTime(time.Now().Add(-15*time.Second)))), + expectedStatus: true, + }} + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + result := tc.taskRun.HasTimedOut() + if d := cmp.Diff(result, tc.expectedStatus); d != "" { + t.Fatalf("-want, +got: %v", d) + } + }) + } +} diff --git a/pkg/apis/pipeline/v1beta1/taskrun_types.go b/pkg/apis/pipeline/v1beta1/taskrun_types.go index e36391cbe12..08cefb8d195 100644 --- a/pkg/apis/pipeline/v1beta1/taskrun_types.go +++ b/pkg/apis/pipeline/v1beta1/taskrun_types.go @@ -20,6 +20,7 @@ import ( "fmt" "time" + apisconfig "github.com/tektoncd/pipeline/pkg/apis/config" "github.com/tektoncd/pipeline/pkg/apis/pipeline" corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -71,6 +72,10 @@ const ( // TaskRunSpecStatusCancelled indicates that the user wants to cancel the task, // if not already cancelled or terminated TaskRunSpecStatusCancelled = "TaskRunCancelled" + + // TaskRunReasonCancelled indicates that the TaskRun has been cancelled + // because it was requested so by the user + TaskRunReasonCancelled = "TaskRunCancelled" ) // TaskRunInputs holds the input values that this task was invoked with. @@ -120,6 +125,17 @@ func (trs *TaskRunStatus) MarkResourceNotConvertible(err *CannotConvertError) { }) } +// MarkResourceFailed sets the ConditionSucceeded condition to ConditionFalse +// based on an error that occurred and a reason +func (trs *TaskRunStatus) MarkResourceFailed(reason string, err error) { + taskRunCondSet.Manage(trs).SetCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + Reason: reason, + Message: err.Error(), + }) +} + // TaskRunStatusFields holds the fields of TaskRun's status. This is defined // separately and inlined so that other types can readily consume these fields // via duck typing. @@ -333,6 +349,28 @@ func (tr *TaskRun) IsCancelled() bool { return tr.Spec.Status == TaskRunSpecStatusCancelled } +// HasTimedOut returns true if the TaskRun runtime is beyond the allowed timeout +func (tr *TaskRun) HasTimedOut() bool { + if tr.Status.StartTime.IsZero() { + return false + } + timeout := tr.GetTimeout() + // If timeout is set to 0 or defaulted to 0, there is no timeout. + if timeout == apisconfig.NoTimeoutDuration { + return false + } + runtime := time.Since(tr.Status.StartTime.Time) + return runtime > timeout +} + +func (tr *TaskRun) GetTimeout() time.Duration { + // Use the platform default is no timeout is set + if tr.Spec.Timeout == nil { + return apisconfig.DefaultTimeoutMinutes * time.Minute + } + return tr.Spec.Timeout.Duration +} + // GetRunKey return the taskrun key for timeout handler map func (tr *TaskRun) GetRunKey() string { // The address of the pointer is a threadsafe unique identifier for the taskrun diff --git a/pkg/apis/pipeline/v1beta1/taskrun_types_test.go b/pkg/apis/pipeline/v1beta1/taskrun_types_test.go index 072ad043d98..17e0e2d6cac 100644 --- a/pkg/apis/pipeline/v1beta1/taskrun_types_test.go +++ b/pkg/apis/pipeline/v1beta1/taskrun_types_test.go @@ -262,3 +262,80 @@ func TestTaskRunIsOfPipelinerun(t *testing.T) { }) } } + +func TestHasTimedOut(t *testing.T) { + // IsZero reports whether t represents the zero time instant, January 1, year 1, 00:00:00 UTC + zeroTime := time.Date(1, 1, 1, 0, 0, 0, 0, time.UTC) + testCases := []struct { + name string + taskRun *v1beta1.TaskRun + expectedStatus bool + }{{ + name: "TaskRun not started", + taskRun: &v1beta1.TaskRun{ + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + }}, + }, + TaskRunStatusFields: v1beta1.TaskRunStatusFields{ + StartTime: &metav1.Time{Time: zeroTime}, + }, + }, + }, + expectedStatus: false, + }, { + name: "TaskRun no timeout", + taskRun: &v1beta1.TaskRun{ + Spec: v1beta1.TaskRunSpec{ + Timeout: &metav1.Duration{ + Duration: 0 * time.Minute, + }, + }, + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + }}, + }, + TaskRunStatusFields: v1beta1.TaskRunStatusFields{ + StartTime: &metav1.Time{Time: time.Now().Add(-15 * time.Hour)}, + }, + }, + }, + expectedStatus: false, + }, { + name: "TaskRun timed out", + taskRun: &v1beta1.TaskRun{ + Spec: v1beta1.TaskRunSpec{ + Timeout: &metav1.Duration{ + Duration: 10 * time.Second, + }, + }, + Status: v1beta1.TaskRunStatus{ + Status: duckv1beta1.Status{ + Conditions: []apis.Condition{{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + }}, + }, + TaskRunStatusFields: v1beta1.TaskRunStatusFields{ + StartTime: &metav1.Time{Time: time.Now().Add(-15 * time.Second)}, + }, + }, + }, + expectedStatus: true, + }} + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + result := tc.taskRun.HasTimedOut() + if d := cmp.Diff(result, tc.expectedStatus); d != "" { + t.Fatalf("-want, +got: %v", d) + } + }) + } +} diff --git a/pkg/reconciler/pipelinerun/metrics_test.go b/pkg/reconciler/pipelinerun/metrics_test.go index c0b0be6d3dd..8fdf67dded3 100644 --- a/pkg/reconciler/pipelinerun/metrics_test.go +++ b/pkg/reconciler/pipelinerun/metrics_test.go @@ -168,5 +168,4 @@ func assertErrIsNil(err error, message string, t *testing.T) { func unregisterMetrics() { metricstest.Unregister("pipelinerun_duration_seconds", "pipelinerun_count", "running_pipelineruns_count") - } diff --git a/pkg/reconciler/pipelinerun/pipelinerun_test.go b/pkg/reconciler/pipelinerun/pipelinerun_test.go index 6de3367ec40..6639395b378 100644 --- a/pkg/reconciler/pipelinerun/pipelinerun_test.go +++ b/pkg/reconciler/pipelinerun/pipelinerun_test.go @@ -66,6 +66,7 @@ func getRunName(pr *v1alpha1.PipelineRun) string { // getPipelineRunController returns an instance of the PipelineRun controller/reconciler that has been seeded with // d, where d represents the state of the system (existing resources) needed for the test. func getPipelineRunController(t *testing.T, d test.Data) (test.Assets, func()) { + //unregisterMetrics() ctx, _ := ttesting.SetupFakeContext(t) c, _ := test.SeedTestData(t, ctx, d) configMapWatcher := configmap.NewInformedWatcher(c.Kube, system.GetNamespace()) diff --git a/pkg/reconciler/taskrun/cancel.go b/pkg/reconciler/taskrun/cancel.go deleted file mode 100644 index 3249ba93a99..00000000000 --- a/pkg/reconciler/taskrun/cancel.go +++ /dev/null @@ -1,53 +0,0 @@ -/* -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 taskrun - -import ( - "fmt" - - "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1alpha1" - corev1 "k8s.io/api/core/v1" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/client-go/kubernetes" - "knative.dev/pkg/apis" -) - -type logger interface { - Warn(args ...interface{}) - Warnf(template string, args ...interface{}) -} - -// cancelTaskRun marks the TaskRun as cancelled and delete pods linked to it. -func cancelTaskRun(tr *v1alpha1.TaskRun, clientSet kubernetes.Interface, logger logger) error { - logger.Warn("task run %q has been cancelled", tr.Name) - tr.Status.SetCondition(&apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: "TaskRunCancelled", - Message: fmt.Sprintf("TaskRun %q was cancelled", tr.Name), - }) - - if tr.Status.PodName == "" { - logger.Warnf("task run %q has no pod running yet", tr.Name) - return nil - } - - if err := clientSet.CoreV1().Pods(tr.Namespace).Delete(tr.Status.PodName, &metav1.DeleteOptions{}); err != nil { - return err - } - return nil -} diff --git a/pkg/reconciler/taskrun/cancel_test.go b/pkg/reconciler/taskrun/cancel_test.go deleted file mode 100644 index 7480d0ddf64..00000000000 --- a/pkg/reconciler/taskrun/cancel_test.go +++ /dev/null @@ -1,100 +0,0 @@ -/* -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 taskrun - -import ( - "context" - "testing" - - "github.com/google/go-cmp/cmp" - "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1alpha1" - ttesting "github.com/tektoncd/pipeline/pkg/reconciler/testing" - tb "github.com/tektoncd/pipeline/test/builder" - test "github.com/tektoncd/pipeline/test/v1alpha1" - "go.uber.org/zap" - "go.uber.org/zap/zaptest/observer" - corev1 "k8s.io/api/core/v1" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - "knative.dev/pkg/apis" -) - -func TestCancelTaskRun(t *testing.T) { - testCases := []struct { - name string - taskRun *v1alpha1.TaskRun - pod *corev1.Pod - expectedStatus apis.Condition - }{{ - name: "no-pod-scheduled", - taskRun: tb.TaskRun("test-taskrun-run-cancelled", "foo", tb.TaskRunSpec( - tb.TaskRunTaskRef(simpleTask.Name), - tb.TaskRunCancelled, - ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionUnknown, - }))), - expectedStatus: apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: "TaskRunCancelled", - Message: `TaskRun "test-taskrun-run-cancelled" was cancelled`, - }, - }, { - name: "pod-scheduled", - taskRun: tb.TaskRun("test-taskrun-run-cancelled", "foo", tb.TaskRunSpec( - tb.TaskRunTaskRef(simpleTask.Name), - tb.TaskRunCancelled, - ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionUnknown, - }), tb.PodName("foo-is-bar"))), - pod: &corev1.Pod{ObjectMeta: metav1.ObjectMeta{ - Namespace: "foo", - Name: "foo-is-bar", - }}, - expectedStatus: apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: "TaskRunCancelled", - Message: `TaskRun "test-taskrun-run-cancelled" was cancelled`, - }, - }} - - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - d := test.Data{ - TaskRuns: []*v1alpha1.TaskRun{tc.taskRun}, - } - if tc.pod != nil { - d.Pods = []*corev1.Pod{tc.pod} - } - - ctx, _ := ttesting.SetupFakeContext(t) - ctx, cancel := context.WithCancel(ctx) - defer cancel() - c, _ := test.SeedTestData(t, ctx, d) - observer, _ := observer.New(zap.InfoLevel) - err := cancelTaskRun(tc.taskRun, c.Kube, zap.New(observer).Sugar()) - if err != nil { - t.Fatal(err) - } - if d := cmp.Diff(tc.taskRun.Status.GetCondition(apis.ConditionSucceeded), &tc.expectedStatus, ignoreLastTransitionTime); d != "" { - t.Fatalf("-want, +got: %v", d) - } - }) - } -} diff --git a/pkg/reconciler/taskrun/taskrun.go b/pkg/reconciler/taskrun/taskrun.go index 7f31309caf1..1f51b1e434d 100644 --- a/pkg/reconciler/taskrun/taskrun.go +++ b/pkg/reconciler/taskrun/taskrun.go @@ -18,13 +18,13 @@ package taskrun import ( "context" + "errors" "fmt" "reflect" "strings" "time" "github.com/hashicorp/go-multierror" - "github.com/tektoncd/pipeline/pkg/apis/config" "github.com/tektoncd/pipeline/pkg/apis/pipeline" "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1alpha1" "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1" @@ -42,7 +42,7 @@ import ( "go.uber.org/zap" corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/equality" - "k8s.io/apimachinery/pkg/api/errors" + k8serrors "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/client-go/tools/cache" "knative.dev/pkg/apis" @@ -92,7 +92,7 @@ func (c *Reconciler) Reconcile(ctx context.Context, key string) error { // Get the Task Run resource with this namespace/name original, err := c.taskRunLister.TaskRuns(namespace).Get(name) - if errors.IsNotFound(err) { + if k8serrors.IsNotFound(err) { // The resource no longer exists, in which case we stop processing. c.Logger.Infof("task run %q in work queue no longer exists", key) return nil @@ -113,6 +113,7 @@ func (c *Reconciler) Reconcile(ctx context.Context, key string) error { tr.Status.StartTime = &tr.CreationTimestamp } + // If the TaskRun is complete, run some post run fixtures when applicable if tr.IsDone() { c.Logger.Infof("taskrun done : %s \n", tr.Name) var merr *multierror.Error @@ -138,7 +139,7 @@ func (c *Reconciler) Reconcile(ctx context.Context, key string) error { err = updateStoppedSidecarStatus(pod, tr, c) } } - } else if errors.IsNotFound(err) { + } else if k8serrors.IsNotFound(err) { return merr.ErrorOrNil() } if err != nil { @@ -159,6 +160,28 @@ func (c *Reconciler) Reconcile(ctx context.Context, key string) error { return merr.ErrorOrNil() } + + // If the TaskRun is cancelled, kill resources and update status + if tr.IsCancelled() { + before := tr.Status.GetCondition(apis.ConditionSucceeded) + message := fmt.Sprintf("TaskRun %q was cancelled", tr.Name) + err := c.failTaskRun(tr, v1beta1.TaskRunReasonCancelled, message) + after := tr.Status.GetCondition(apis.ConditionSucceeded) + reconciler.EmitEvent(c.Recorder, before, after, tr) + return multierror.Append(err, c.updateStatusLabelsAndAnnotations(tr, original)).ErrorOrNil() + } + + // Check if the TaskRun has timed out; if it is, this will set its status + // accordingly. + if tr.HasTimedOut() { + before := tr.Status.GetCondition(apis.ConditionSucceeded) + message := fmt.Sprintf("TaskRun %q failed to finish within %q", tr.Name, tr.GetTimeout()) + err := c.failTaskRun(tr, podconvert.ReasonTimedOut, message) + after := tr.Status.GetCondition(apis.ConditionSucceeded) + reconciler.EmitEvent(c.Recorder, before, after, tr) + return multierror.Append(err, c.updateStatusLabelsAndAnnotations(tr, original)).ErrorOrNil() + } + // Reconcile this copy of the task run and then write back any status // updates regardless of whether the reconciliation errored out. if err := c.reconcile(ctx, tr); err != nil { @@ -168,69 +191,6 @@ func (c *Reconciler) Reconcile(ctx context.Context, key string) error { return multierror.Append(merr, c.updateStatusLabelsAndAnnotations(tr, original)).ErrorOrNil() } -func (c *Reconciler) updateStatusLabelsAndAnnotations(tr, original *v1alpha1.TaskRun) error { - var updated bool - - if !equality.Semantic.DeepEqual(original.Status, tr.Status) { - // If we didn't change anything then don't call updateStatus. - // This is important because the copy we loaded from the informer's - // cache may be stale and we don't want to overwrite a prior update - // to status with this stale state. - if _, err := c.updateStatus(tr); err != nil { - c.Logger.Warn("Failed to update taskRun status", zap.Error(err)) - return err - } - updated = true - } - - // When we update the status only, we use updateStatus to minimize the chances of - // racing any clients updating other parts of the Run, e.g. the spec or the labels. - // If we need to update the labels or annotations, we need to call Update with these - // changes explicitly. - if !reflect.DeepEqual(original.ObjectMeta.Labels, tr.ObjectMeta.Labels) || !reflect.DeepEqual(original.ObjectMeta.Annotations, tr.ObjectMeta.Annotations) { - if _, err := c.updateLabelsAndAnnotations(tr); err != nil { - c.Logger.Warn("Failed to update TaskRun labels/annotations", zap.Error(err)) - return err - } - updated = true - } - - if updated { - go func(metrics *Recorder) { - err := metrics.RunningTaskRuns(c.taskRunLister) - if err != nil { - c.Logger.Warnf("Failed to log the metrics : %v", err) - } - }(c.metrics) - } - - return nil -} - -func (c *Reconciler) getTaskFunc(tr *v1alpha1.TaskRun) (resources.GetTask, v1alpha1.TaskKind) { - var gtFunc resources.GetTask - kind := v1alpha1.NamespacedTaskKind - if tr.Spec.TaskRef != nil && tr.Spec.TaskRef.Kind == v1alpha1.ClusterTaskKind { - gtFunc = func(name string) (v1alpha1.TaskInterface, error) { - t, err := c.PipelineClientSet.TektonV1alpha1().ClusterTasks().Get(name, metav1.GetOptions{}) - if err != nil { - return nil, err - } - return t, nil - } - kind = v1alpha1.ClusterTaskKind - } else { - gtFunc = func(name string) (v1alpha1.TaskInterface, error) { - t, err := c.PipelineClientSet.TektonV1alpha1().Tasks(tr.Namespace).Get(name, metav1.GetOptions{}) - if err != nil { - return nil, err - } - return t, nil - } - } - return gtFunc, kind -} - func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error { // We may be reading a version of the object that was stored at an older version // and may not have had all of the assumed default specified. @@ -244,15 +204,6 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error return err } - // If the taskrun is cancelled, kill resources and update status - if tr.IsCancelled() { - before := tr.Status.GetCondition(apis.ConditionSucceeded) - err := cancelTaskRun(tr, c.KubeClientSet, c.Logger) - after := tr.Status.GetCondition(apis.ConditionSucceeded) - reconciler.EmitEvent(c.Recorder, before, after, tr) - return err - } - getTaskFunc, kind := c.getTaskFunc(tr) taskMeta, taskSpec, err := resources.GetTaskData(ctx, tr, getTaskFunc) if err != nil { @@ -261,12 +212,7 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error return nil } c.Logger.Errorf("Failed to determine Task spec to use for taskrun %s: %v", tr.Name, err) - tr.Status.SetCondition(&apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: podconvert.ReasonFailedResolution, - Message: err.Error(), - }) + tr.Status.MarkResourceFailed(podconvert.ReasonFailedResolution, err) return nil } @@ -293,18 +239,6 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error tr.ObjectMeta.Annotations[key] = value } - if tr.Spec.Timeout == nil { - tr.Spec.Timeout = &metav1.Duration{Duration: config.DefaultTimeoutMinutes * time.Minute} - } - // Check if the TaskRun has timed out; if it is, this will set its status - // accordingly. - if CheckTimeout(tr) { - if err := c.updateTaskRunStatusForTimeout(tr, c.KubeClientSet.CoreV1().Pods(tr.Namespace).Delete); err != nil { - return err - } - return nil - } - inputs := []v1beta1.TaskResourceBinding{} outputs := []v1beta1.TaskResourceBinding{} if tr.Spec.Resources != nil { @@ -314,34 +248,19 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error rtr, err := resources.ResolveTaskResources(taskSpec, taskMeta.Name, kind, inputs, outputs, c.resourceLister.PipelineResources(tr.Namespace).Get) if err != nil { c.Logger.Errorf("Failed to resolve references for taskrun %s: %v", tr.Name, err) - tr.Status.SetCondition(&apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: podconvert.ReasonFailedResolution, - Message: err.Error(), - }) + tr.Status.MarkResourceFailed(podconvert.ReasonFailedResolution, err) return nil } if err := ValidateResolvedTaskResources(tr.Spec.Params, rtr); err != nil { c.Logger.Errorf("TaskRun %q resources are invalid: %v", tr.Name, err) - tr.Status.SetCondition(&apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: podconvert.ReasonFailedValidation, - Message: err.Error(), - }) + tr.Status.MarkResourceFailed(podconvert.ReasonFailedValidation, err) return nil } if err := workspace.ValidateBindings(taskSpec.Workspaces, tr.Spec.Workspaces); err != nil { c.Logger.Errorf("TaskRun %q workspaces are invalid: %v", tr.Name, err) - tr.Status.SetCondition(&apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: podconvert.ReasonFailedValidation, - Message: err.Error(), - }) + tr.Status.MarkResourceFailed(podconvert.ReasonFailedValidation, err) return nil } @@ -360,7 +279,7 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error var pod *corev1.Pod if tr.Status.PodName != "" { pod, err = c.KubeClientSet.CoreV1().Pods(tr.Namespace).Get(tr.Status.PodName, metav1.GetOptions{}) - if errors.IsNotFound(err) { + if k8serrors.IsNotFound(err) { // Keep going, this will result in the Pod being created below. } else if err != nil { c.Logger.Errorf("Error getting pod %q: %v", tr.Status.PodName, err) @@ -386,13 +305,9 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error if tr.HasVolumeClaimTemplate() { if err = c.pvcHandler.CreatePersistentVolumeClaimsForWorkspaces(tr.Spec.Workspaces, tr.GetOwnerReference(), tr.Namespace); err != nil { c.Logger.Errorf("Failed to create PVC for TaskRun %s: %v", tr.Name, err) - tr.Status.SetCondition(&apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: volumeclaim.ReasonCouldntCreateWorkspacePVC, - Message: fmt.Sprintf("Failed to create PVC for TaskRun %s workspaces correctly: %s", - fmt.Sprintf("%s/%s", tr.Namespace, tr.Name), err), - }) + tr.Status.MarkResourceFailed(volumeclaim.ReasonCouldntCreateWorkspacePVC, + fmt.Errorf("Failed to create PVC for TaskRun %s workspaces correctly: %s", + fmt.Sprintf("%s/%s", tr.Namespace, tr.Name), err)) return nil } @@ -439,6 +354,96 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error return nil } +// Push changes (if any) to the TaskRun status, labels and annotations to +// TaskRun definition in ectd +func (c *Reconciler) updateStatusLabelsAndAnnotations(tr, original *v1alpha1.TaskRun) error { + var updated bool + + if !equality.Semantic.DeepEqual(original.Status, tr.Status) { + // If we didn't change anything then don't call updateStatus. + // This is important because the copy we loaded from the informer's + // cache may be stale and we don't want to overwrite a prior update + // to status with this stale state. + if _, err := c.updateStatus(tr); err != nil { + c.Logger.Warn("Failed to update taskRun status", zap.Error(err)) + return err + } + updated = true + } + + // When we update the status only, we use updateStatus to minimize the chances of + // racing any clients updating other parts of the Run, e.g. the spec or the labels. + // If we need to update the labels or annotations, we need to call Update with these + // changes explicitly. + if !reflect.DeepEqual(original.ObjectMeta.Labels, tr.ObjectMeta.Labels) || !reflect.DeepEqual(original.ObjectMeta.Annotations, tr.ObjectMeta.Annotations) { + if _, err := c.updateLabelsAndAnnotations(tr); err != nil { + c.Logger.Warn("Failed to update TaskRun labels/annotations", zap.Error(err)) + return err + } + updated = true + } + + if updated { + go func(metrics *Recorder) { + err := metrics.RunningTaskRuns(c.taskRunLister) + if err != nil { + c.Logger.Warnf("Failed to log the metrics : %v", err) + } + }(c.metrics) + } + + return nil +} + +func (c *Reconciler) updateStatus(taskrun *v1alpha1.TaskRun) (*v1alpha1.TaskRun, error) { + newtaskrun, err := c.taskRunLister.TaskRuns(taskrun.Namespace).Get(taskrun.Name) + if err != nil { + return nil, fmt.Errorf("error getting TaskRun %s when updating status: %w", taskrun.Name, err) + } + if !reflect.DeepEqual(taskrun.Status, newtaskrun.Status) { + newtaskrun.Status = taskrun.Status + return c.PipelineClientSet.TektonV1alpha1().TaskRuns(taskrun.Namespace).UpdateStatus(newtaskrun) + } + return newtaskrun, nil +} + +func (c *Reconciler) updateLabelsAndAnnotations(tr *v1alpha1.TaskRun) (*v1alpha1.TaskRun, error) { + newTr, err := c.taskRunLister.TaskRuns(tr.Namespace).Get(tr.Name) + if err != nil { + return nil, fmt.Errorf("error getting TaskRun %s when updating labels/annotations: %w", tr.Name, err) + } + if !reflect.DeepEqual(tr.ObjectMeta.Labels, newTr.ObjectMeta.Labels) || !reflect.DeepEqual(tr.ObjectMeta.Annotations, newTr.ObjectMeta.Annotations) { + newTr.ObjectMeta.Labels = tr.ObjectMeta.Labels + newTr.ObjectMeta.Annotations = tr.ObjectMeta.Annotations + return c.PipelineClientSet.TektonV1alpha1().TaskRuns(tr.Namespace).Update(newTr) + } + return newTr, nil +} + +func (c *Reconciler) getTaskFunc(tr *v1alpha1.TaskRun) (resources.GetTask, v1alpha1.TaskKind) { + var gtFunc resources.GetTask + kind := v1alpha1.NamespacedTaskKind + if tr.Spec.TaskRef != nil && tr.Spec.TaskRef.Kind == v1alpha1.ClusterTaskKind { + gtFunc = func(name string) (v1alpha1.TaskInterface, error) { + t, err := c.PipelineClientSet.TektonV1alpha1().ClusterTasks().Get(name, metav1.GetOptions{}) + if err != nil { + return nil, err + } + return t, nil + } + kind = v1alpha1.ClusterTaskKind + } else { + gtFunc = func(name string) (v1alpha1.TaskInterface, error) { + t, err := c.PipelineClientSet.TektonV1alpha1().Tasks(tr.Namespace).Get(name, metav1.GetOptions{}) + if err != nil { + return nil, err + } + return t, nil + } + } + return gtFunc, kind +} + func (c *Reconciler) handlePodCreationError(tr *v1alpha1.TaskRun, err error) { var reason, msg string var succeededStatus corev1.ConditionStatus @@ -469,67 +474,33 @@ func (c *Reconciler) handlePodCreationError(tr *v1alpha1.TaskRun, err error) { c.Logger.Errorf("Failed to create build pod for task %q: %v", tr.Name, err) } -func updateTaskRunResourceResult(taskRun *v1alpha1.TaskRun, podStatus corev1.PodStatus) error { - if taskRun.IsSuccessful() { - for idx, cs := range podStatus.ContainerStatuses { - if cs.State.Terminated != nil { - msg := cs.State.Terminated.Message - r, err := termination.ParseMessage(msg) - if err != nil { - return fmt.Errorf("parsing message for container status %d: %v", idx, err) - } - taskResults, pipelineResourceResults := getResults(r) - taskRun.Status.TaskRunResults = append(taskRun.Status.TaskRunResults, taskResults...) - taskRun.Status.ResourcesResult = append(taskRun.Status.ResourcesResult, pipelineResourceResults...) - } - } - } - return nil -} +// failTaskRun stops a TaskRun with the provided Reason +// If a pod is associated to the TaskRun, it stops it +// failTaskRun function may return an error in case the pod could not be deleted +// failTaskRun may update the local TaskRun status, but it won't push the updates to etcd +func (c *Reconciler) failTaskRun(tr *v1alpha1.TaskRun, reason, message string) error { -func getResults(results []v1alpha1.PipelineResourceResult) ([]v1alpha1.TaskRunResult, []v1alpha1.PipelineResourceResult) { - var taskResults []v1alpha1.TaskRunResult - var pipelineResourceResults []v1alpha1.PipelineResourceResult - for _, r := range results { - switch r.ResultType { - case v1alpha1.TaskRunResultType: - taskRunResult := v1alpha1.TaskRunResult{ - Name: r.Key, - Value: r.Value, - } - taskResults = append(taskResults, taskRunResult) - case v1alpha1.PipelineResourceResultType: - fallthrough - default: - pipelineResourceResults = append(pipelineResourceResults, r) - } - } - return taskResults, pipelineResourceResults -} + c.Logger.Warn("stopping task run %q because of %q", tr.Name, reason) + tr.Status.MarkResourceFailed(reason, errors.New(message)) -func (c *Reconciler) updateStatus(taskrun *v1alpha1.TaskRun) (*v1alpha1.TaskRun, error) { - newtaskrun, err := c.taskRunLister.TaskRuns(taskrun.Namespace).Get(taskrun.Name) - if err != nil { - return nil, fmt.Errorf("error getting TaskRun %s when updating status: %w", taskrun.Name, err) - } - if !reflect.DeepEqual(taskrun.Status, newtaskrun.Status) { - newtaskrun.Status = taskrun.Status - return c.PipelineClientSet.TektonV1alpha1().TaskRuns(taskrun.Namespace).UpdateStatus(newtaskrun) - } - return newtaskrun, nil -} + // update tr completed time + tr.Status.CompletionTime = &metav1.Time{Time: time.Now()} -func (c *Reconciler) updateLabelsAndAnnotations(tr *v1alpha1.TaskRun) (*v1alpha1.TaskRun, error) { - newTr, err := c.taskRunLister.TaskRuns(tr.Namespace).Get(tr.Name) - if err != nil { - return nil, fmt.Errorf("error getting TaskRun %s when updating labels/annotations: %w", tr.Name, err) + if tr.Status.PodName == "" { + c.Logger.Warnf("task run %q has no pod running yet", tr.Name) + return nil } - if !reflect.DeepEqual(tr.ObjectMeta.Labels, newTr.ObjectMeta.Labels) || !reflect.DeepEqual(tr.ObjectMeta.Annotations, newTr.ObjectMeta.Annotations) { - newTr.ObjectMeta.Labels = tr.ObjectMeta.Labels - newTr.ObjectMeta.Annotations = tr.ObjectMeta.Annotations - return c.PipelineClientSet.TektonV1alpha1().TaskRuns(tr.Namespace).Update(newTr) + + // tr.Status.PodName will be empty if the pod was never successfully created. This condition + // can be reached, for example, by the pod never being schedulable due to limits imposed by + // a namespace's ResourceQuota. + err := c.KubeClientSet.CoreV1().Pods(tr.Namespace).Delete(tr.Status.PodName, &metav1.DeleteOptions{}) + + if err != nil && !k8serrors.IsNotFound(err) { + c.Logger.Infof("Failed to terminate pod: %v", err) + return err } - return newTr, nil + return nil } // createPod creates a Pod based on the Task's configuration, with pvcName as a volumeMount @@ -605,33 +576,46 @@ func (c *Reconciler) createPod(tr *v1alpha1.TaskRun, rtr *resources.ResolvedTask type DeletePod func(podName string, options *metav1.DeleteOptions) error -func (c *Reconciler) updateTaskRunStatusForTimeout(tr *v1alpha1.TaskRun, dp DeletePod) error { - c.Logger.Infof("TaskRun %q has timed out, deleting pod", tr.Name) - // tr.Status.PodName will be empty if the pod was never successfully created. This condition - // can be reached, for example, by the pod never being schedulable due to limits imposed by - // a namespace's ResourceQuota. - if tr.Status.PodName != "" { - if err := dp(tr.Status.PodName, &metav1.DeleteOptions{}); err != nil && !errors.IsNotFound(err) { - c.Logger.Errorf("Failed to terminate pod: %v", err) - return err +func updateTaskRunResourceResult(taskRun *v1alpha1.TaskRun, podStatus corev1.PodStatus) error { + if taskRun.IsSuccessful() { + for idx, cs := range podStatus.ContainerStatuses { + if cs.State.Terminated != nil { + msg := cs.State.Terminated.Message + r, err := termination.ParseMessage(msg) + if err != nil { + return fmt.Errorf("parsing message for container status %d: %v", idx, err) + } + taskResults, pipelineResourceResults := getResults(r) + taskRun.Status.TaskRunResults = append(taskRun.Status.TaskRunResults, taskResults...) + taskRun.Status.ResourcesResult = append(taskRun.Status.ResourcesResult, pipelineResourceResults...) + } } } - - timeout := tr.Spec.Timeout.Duration - timeoutMsg := fmt.Sprintf("TaskRun %q failed to finish within %q", tr.Name, timeout.String()) - tr.Status.SetCondition(&apis.Condition{ - Type: apis.ConditionSucceeded, - Status: corev1.ConditionFalse, - Reason: podconvert.ReasonTimedOut, - Message: timeoutMsg, - }) - // update tr completed time - tr.Status.CompletionTime = &metav1.Time{Time: time.Now()} return nil } +func getResults(results []v1alpha1.PipelineResourceResult) ([]v1alpha1.TaskRunResult, []v1alpha1.PipelineResourceResult) { + var taskResults []v1alpha1.TaskRunResult + var pipelineResourceResults []v1alpha1.PipelineResourceResult + for _, r := range results { + switch r.ResultType { + case v1alpha1.TaskRunResultType: + taskRunResult := v1alpha1.TaskRunResult{ + Name: r.Key, + Value: r.Value, + } + taskResults = append(taskResults, taskRunResult) + case v1alpha1.PipelineResourceResultType: + fallthrough + default: + pipelineResourceResults = append(pipelineResourceResults, r) + } + } + return taskResults, pipelineResourceResults +} + func isExceededResourceQuotaError(err error) bool { - return err != nil && errors.IsForbidden(err) && strings.Contains(err.Error(), "exceeded quota") + return err != nil && k8serrors.IsForbidden(err) && strings.Contains(err.Error(), "exceeded quota") } // resourceImplBinding maps pipeline resource names to the actual resource type implementations diff --git a/pkg/reconciler/taskrun/taskrun_test.go b/pkg/reconciler/taskrun/taskrun_test.go index c08d441e028..ee808d21d0a 100644 --- a/pkg/reconciler/taskrun/taskrun_test.go +++ b/pkg/reconciler/taskrun/taskrun_test.go @@ -253,6 +253,7 @@ func getRunName(tr *v1alpha1.TaskRun) string { // getTaskRunController returns an instance of the TaskRun controller/reconciler that has been seeded with // d, where d represents the state of the system (existing resources) needed for the test. func getTaskRunController(t *testing.T, d test.Data) (test.Assets, func()) { + //unregisterMetrics() ctx, _ := ttesting.SetupFakeContext(t) ctx, cancel := context.WithCancel(ctx) cloudEventClientBehaviour := cloudevent.FakeClientBehaviour{ @@ -1791,6 +1792,7 @@ func TestUpdateTaskRunResult(t *testing.T) { }) } } + func TestUpdateTaskRunResult2(t *testing.T) { for _, c := range []struct { desc string @@ -1838,6 +1840,7 @@ func TestUpdateTaskRunResult2(t *testing.T) { }) } } + func TestUpdateTaskRunResultTwoResults(t *testing.T) { for _, c := range []struct { desc string @@ -1879,6 +1882,7 @@ func TestUpdateTaskRunResultTwoResults(t *testing.T) { }) } } + func TestUpdateTaskRunResultWhenTaskFailed(t *testing.T) { for _, c := range []struct { desc string @@ -1917,6 +1921,7 @@ func TestUpdateTaskRunResultWhenTaskFailed(t *testing.T) { }) } } + func TestUpdateTaskRunResourceResult_Errors(t *testing.T) { for _, c := range []struct { desc string @@ -2231,3 +2236,78 @@ func TestReconcileWorkspaceWithVolumeClaimTemplate(t *testing.T) { t.Fatalf("expected PVC %s to exist but instead got error when getting it: %v", expectedPVCName, err) } } + +func TestFailTaskRun(t *testing.T) { + testCases := []struct { + name string + taskRun *v1alpha1.TaskRun + pod *corev1.Pod + reason string + message string + expectedStatus apis.Condition + }{{ + name: "no-pod-scheduled", + taskRun: tb.TaskRun("test-taskrun-run-failed", "foo", tb.TaskRunSpec( + tb.TaskRunTaskRef(simpleTask.Name), + tb.TaskRunCancelled, + ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }))), + reason: "some reason", + message: "some message", + expectedStatus: apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + Reason: "some reason", + Message: "some message", + }, + }, { + name: "pod-scheduled", + taskRun: tb.TaskRun("test-taskrun-run-failed", "foo", tb.TaskRunSpec( + tb.TaskRunTaskRef(simpleTask.Name), + tb.TaskRunCancelled, + ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + }), tb.PodName("foo-is-bar"))), + pod: &corev1.Pod{ObjectMeta: metav1.ObjectMeta{ + Namespace: "foo", + Name: "foo-is-bar", + }}, + reason: "some reason", + message: "some message", + expectedStatus: apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionFalse, + Reason: "some reason", + Message: "some message", + }, + }} + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + d := test.Data{ + TaskRuns: []*v1alpha1.TaskRun{tc.taskRun}, + } + if tc.pod != nil { + d.Pods = []*corev1.Pod{tc.pod} + } + + testAssets, cancel := getTaskRunController(t, d) + defer cancel() + c, ok := testAssets.Controller.Reconciler.(*Reconciler) + if !ok { + t.Errorf("failed to construct instance of taskrun reconciler") + return + } + err := c.failTaskRun(tc.taskRun, tc.reason, tc.message) + if err != nil { + t.Fatal(err) + } + if d := cmp.Diff(tc.taskRun.Status.GetCondition(apis.ConditionSucceeded), &tc.expectedStatus, ignoreLastTransitionTime); d != "" { + t.Fatalf("-want, +got: %v", d) + } + }) + } +} diff --git a/pkg/reconciler/taskrun/timeout_check.go b/pkg/reconciler/taskrun/timeout_check.go deleted file mode 100644 index 4dc69cfe2f7..00000000000 --- a/pkg/reconciler/taskrun/timeout_check.go +++ /dev/null @@ -1,39 +0,0 @@ -/* -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 taskrun - -import ( - "time" - - apisconfig "github.com/tektoncd/pipeline/pkg/apis/config" - "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1alpha1" -) - -func CheckTimeout(tr *v1alpha1.TaskRun) bool { - // If tr has not started, startTime should be zero. - if tr.Status.StartTime.IsZero() { - return false - } - - timeout := tr.Spec.Timeout.Duration - // If timeout is set to 0 or defaulted to 0, there is no timeout. - if timeout == apisconfig.NoTimeoutDuration { - return false - } - runtime := time.Since(tr.Status.StartTime.Time) - return runtime > timeout -} diff --git a/pkg/reconciler/taskrun/timeout_check_test.go b/pkg/reconciler/taskrun/timeout_check_test.go deleted file mode 100644 index 73b77ef5145..00000000000 --- a/pkg/reconciler/taskrun/timeout_check_test.go +++ /dev/null @@ -1,64 +0,0 @@ -/* -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 taskrun - -import ( - "testing" - "time" - - "github.com/google/go-cmp/cmp" - "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1alpha1" - tb "github.com/tektoncd/pipeline/test/builder" - "knative.dev/pkg/apis" -) - -func TestCheckTimeout(t *testing.T) { - // IsZero reports whether t represents the zero time instant, January 1, year 1, 00:00:00 UTC - zeroTime := time.Date(1, 1, 1, 0, 0, 0, 0, time.UTC) - testCases := []struct { - name string - taskRun *v1alpha1.TaskRun - expectedStatus bool - }{{ - name: "TaskRun not started", - taskRun: tb.TaskRun("test-taskrun-not-started", "foo", tb.TaskRunSpec( - tb.TaskRunTaskRef(simpleTask.Name), - ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{}), tb.TaskRunStartTime(zeroTime))), - expectedStatus: false, - }, { - name: "TaskRun no timeout", - taskRun: tb.TaskRun("test-taskrun-no-timeout", "foo", tb.TaskRunSpec( - tb.TaskRunTaskRef(simpleTask.Name), tb.TaskRunTimeout(0), - ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{}), tb.TaskRunStartTime(time.Now().Add(-15*time.Hour)))), - expectedStatus: false, - }, { - name: "TaskRun timed out", - taskRun: tb.TaskRun("test-taskrun-timeout", "foo", tb.TaskRunSpec( - tb.TaskRunTaskRef(simpleTask.Name), tb.TaskRunTimeout(10*time.Second), - ), tb.TaskRunStatus(tb.StatusCondition(apis.Condition{}), tb.TaskRunStartTime(time.Now().Add(-15*time.Second)))), - expectedStatus: true, - }} - - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - result := CheckTimeout(tc.taskRun) - if d := cmp.Diff(result, tc.expectedStatus); d != "" { - t.Fatalf("-want, +got: %v", d) - } - }) - } -}