From 279957f3aad16b7ffbf0ee057ac1a873ee8bb4b7 Mon Sep 17 00:00:00 2001 From: Andrea Frittoli Date: Thu, 9 Apr 2020 22:03:36 +0100 Subject: [PATCH] Consolidate cancel and timeout logic Cancel and timeout do very similar things when they happen: they update the status of the taskrun, set the completion time and try and delete the pod. Today this is done for the two cases in different places, the code structured differently and the behaviour slightly different: - log levels of the messages are different - cancel does not set the completion time - cancel does not check if the error on pod deletion is a NotFound This commit introduces "HasTimedOut" to tasktun_types, which matches what "IsCancelled" does. It introduces a "killTaskRun" function that can be used by both cancel and timeout, with the only different being the "Reason" and termination message. The timeout_check module is not necessary anymore. The check for IsCancelled and HasTimedOut are move out of "reconcile" into "Reconcile", so that now "Reconcile" checks: - HasStarted - isDone - IsCancelled - HasTimedOut and finally, if applicable, it invokes "reconcile". --- pkg/apis/pipeline/v1alpha1/taskrun_types.go | 28 ++ .../pipeline/v1alpha1/taskrun_types_test.go | 37 ++ pkg/apis/pipeline/v1beta1/taskrun_types.go | 38 ++ .../pipeline/v1beta1/taskrun_types_test.go | 77 ++++ pkg/reconciler/pipelinerun/metrics_test.go | 1 - .../pipelinerun/pipelinerun_test.go | 1 + pkg/reconciler/taskrun/cancel.go | 53 --- pkg/reconciler/taskrun/cancel_test.go | 100 ----- pkg/reconciler/taskrun/taskrun.go | 378 +++++++++--------- pkg/reconciler/taskrun/taskrun_test.go | 80 ++++ pkg/reconciler/taskrun/timeout_check.go | 39 -- pkg/reconciler/taskrun/timeout_check_test.go | 64 --- 12 files changed, 442 insertions(+), 454 deletions(-) delete mode 100644 pkg/reconciler/taskrun/cancel.go delete mode 100644 pkg/reconciler/taskrun/cancel_test.go delete mode 100644 pkg/reconciler/taskrun/timeout_check.go delete mode 100644 pkg/reconciler/taskrun/timeout_check_test.go 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 256fbe536ca..1ac247771ef 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 } @@ -289,18 +235,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 { @@ -310,34 +244,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 } @@ -356,7 +275,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) @@ -382,13 +301,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 } @@ -435,6 +350,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 @@ -465,67 +470,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 @@ -601,33 +572,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 b2b69c68471..58a67fbc012 100644 --- a/pkg/reconciler/taskrun/taskrun_test.go +++ b/pkg/reconciler/taskrun/taskrun_test.go @@ -252,6 +252,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{ @@ -1790,6 +1791,7 @@ func TestUpdateTaskRunResult(t *testing.T) { }) } } + func TestUpdateTaskRunResult2(t *testing.T) { for _, c := range []struct { desc string @@ -1837,6 +1839,7 @@ func TestUpdateTaskRunResult2(t *testing.T) { }) } } + func TestUpdateTaskRunResultTwoResults(t *testing.T) { for _, c := range []struct { desc string @@ -1878,6 +1881,7 @@ func TestUpdateTaskRunResultTwoResults(t *testing.T) { }) } } + func TestUpdateTaskRunResultWhenTaskFailed(t *testing.T) { for _, c := range []struct { desc string @@ -1916,6 +1920,7 @@ func TestUpdateTaskRunResultWhenTaskFailed(t *testing.T) { }) } } + func TestUpdateTaskRunResourceResult_Errors(t *testing.T) { for _, c := range []struct { desc string @@ -2230,3 +2235,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) - } - }) - } -}