From a963e63ba37b53fe533f81f0407c492f1c2d6ebb Mon Sep 17 00:00:00 2001 From: Andrea Frittoli Date: Thu, 16 Apr 2020 18:45:53 +0100 Subject: [PATCH] Split reconcile into prepare and reconcile The current "reconcile" function does a lot of things, which means it's hard to test properly, it's hard to follow the logic and it's hard to handle all possible errors and events properly. This splits reconcile into two parts, the first one that deals preparing and validating the taskrun and all the resources it depends on, the second one that actually reconciles the TaskRun by creating the pod if required and updating the TaskRun from the pod. This adds events that were missing for error situations that happen during preparation and validation. The new events go past the burst limit of the event broadcaster, which delays some events a lot for a pipeline with a few tasks. Tweaking the default burst rate and QPS is required to pass CI. We might want to add those configs to a dedicated config map in future. --- pkg/pod/status.go | 4 + pkg/reconciler/event.go | 18 +++- pkg/reconciler/pipelinerun/pipelinerun.go | 2 +- pkg/reconciler/reconciler.go | 8 +- pkg/reconciler/taskrun/taskrun.go | 107 ++++++++++++++-------- pkg/reconciler/taskrun/taskrun_test.go | 76 ++++++++++----- test/cancel_test.go | 2 +- test/init_test.go | 2 +- test/pipelinerun_test.go | 9 +- test/v1alpha1/pipelinerun_test.go | 9 +- 10 files changed, 165 insertions(+), 72 deletions(-) diff --git a/pkg/pod/status.go b/pkg/pod/status.go index e7ed666030d..36e56205da8 100644 --- a/pkg/pod/status.go +++ b/pkg/pod/status.go @@ -64,6 +64,10 @@ const ( // config error of container ReasonCreateContainerConfigError = "CreateContainerConfigError" + // ReasonPodCreationFailed indicates that the reason for the current condition + // is that the creation of the pod backing the TaskRun failed + ReasonPodCreationFailed = "PodCreationFailed" + // ReasonSucceeded indicates that the reason for the finished status is that all of the steps // completed successfully ReasonSucceeded = "Succeeded" diff --git a/pkg/reconciler/event.go b/pkg/reconciler/event.go index 5341ad73c87..e35560b88ef 100644 --- a/pkg/reconciler/event.go +++ b/pkg/reconciler/event.go @@ -22,15 +22,25 @@ import ( "knative.dev/pkg/apis" ) +const ( + // EventReasonSucceded is the reason set for events about successful completion of TaskRuns / PipelineRuns + EventReasonSucceded = "Succeeded" + // EventReasonFailed is the reason set for events about unsuccessful completion of TaskRuns / PipelineRuns + EventReasonFailed = "Failed" +) + // EmitEvent emits success or failed event for object // if afterCondition is different from beforeCondition func EmitEvent(c record.EventRecorder, beforeCondition *apis.Condition, afterCondition *apis.Condition, object runtime.Object) { if beforeCondition != afterCondition && afterCondition != nil { // Create events when the obj result is in. - if afterCondition.Status == corev1.ConditionTrue { - c.Event(object, corev1.EventTypeNormal, "Succeeded", afterCondition.Message) - } else if afterCondition.Status == corev1.ConditionFalse { - c.Event(object, corev1.EventTypeWarning, "Failed", afterCondition.Message) + switch afterCondition.Status { + case corev1.ConditionTrue: + c.Event(object, corev1.EventTypeNormal, EventReasonSucceded, afterCondition.Message) + case corev1.ConditionUnknown: + c.Event(object, corev1.EventTypeNormal, afterCondition.Reason, afterCondition.Message) + case corev1.ConditionFalse: + c.Event(object, corev1.EventTypeWarning, EventReasonFailed, afterCondition.Message) } } } diff --git a/pkg/reconciler/pipelinerun/pipelinerun.go b/pkg/reconciler/pipelinerun/pipelinerun.go index eaae85c631a..34cbf8cfac2 100644 --- a/pkg/reconciler/pipelinerun/pipelinerun.go +++ b/pkg/reconciler/pipelinerun/pipelinerun.go @@ -119,7 +119,7 @@ var ( // converge the two. It then updates the Status block of the Pipeline Run // resource with the current status of the resource. func (c *Reconciler) Reconcile(ctx context.Context, key string) error { - c.Logger.Infof("Reconciling %v", time.Now()) + c.Logger.Infof("Reconciling key %s at %v", key, time.Now()) // Convert the namespace/name string into a distinct namespace and name namespace, name, err := cache.SplitMetaNamespaceKey(key) diff --git a/pkg/reconciler/reconciler.go b/pkg/reconciler/reconciler.go index a997dd420c3..9c0d4339e0e 100644 --- a/pkg/reconciler/reconciler.go +++ b/pkg/reconciler/reconciler.go @@ -97,7 +97,13 @@ func NewBase(opt Options, controllerAgentName string, images pipeline.Images) *B if recorder == nil { // Create event broadcaster logger.Debug("Creating event broadcaster") - eventBroadcaster := record.NewBroadcaster() + + correlatorOptions := record.CorrelatorOptions{ + // The default burst size is 25 + BurstSize: 50, + QPS: 1, + } + eventBroadcaster := record.NewBroadcasterWithCorrelatorOptions(correlatorOptions) eventBroadcaster.StartLogging(logger.Named("event-broadcaster").Infof) eventBroadcaster.StartRecordingToSink(&typedcorev1.EventSinkImpl{Interface: opt.KubeClientSet.CoreV1().Events("")}) diff --git a/pkg/reconciler/taskrun/taskrun.go b/pkg/reconciler/taskrun/taskrun.go index 1f51b1e434d..47c412857c2 100644 --- a/pkg/reconciler/taskrun/taskrun.go +++ b/pkg/reconciler/taskrun/taskrun.go @@ -79,9 +79,6 @@ var _ controller.Reconciler = (*Reconciler)(nil) // converge the two. It then updates the Status block of the Task Run // resource with the current status of the resource. func (c *Reconciler) Reconcile(ctx context.Context, key string) error { - // In case of reconcile errors, we store the error in a multierror, attempt - // to update, and return the original error combined with any update error - var merr error // Convert the namespace/name string into a distinct namespace and name namespace, name, err := cache.SplitMetaNamespaceKey(key) @@ -182,16 +179,46 @@ func (c *Reconciler) Reconcile(ctx context.Context, key string) error { return multierror.Append(err, c.updateStatusLabelsAndAnnotations(tr, original)).ErrorOrNil() } + // prepare fetches all required resources, validates them together with the + // taskrun, runs API convertions. Errors that come out of prepare are + // permanent one, so in case of error we update, emit events and return + taskSpec, rtr, err := c.prepare(ctx, tr) + if err != nil { + c.Logger.Errorf("TaskRun prepare error: %v", err.Error()) + after := tr.Status.GetCondition(apis.ConditionSucceeded) + reconciler.EmitEvent(c.Recorder, nil, after, tr) + // We only return an error if update failed, otherwise we don't want to + // reconcile an invalid TaskRun anymore + return c.updateStatusLabelsAndAnnotations(tr, original) + } + + // Store the condition before reconcile + before := tr.Status.GetCondition(apis.ConditionSucceeded) + // 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 { + if err = c.reconcile(ctx, tr, taskSpec, rtr); err != nil { c.Logger.Errorf("Reconcile error: %v", err.Error()) - merr = multierror.Append(merr, err) } - return multierror.Append(merr, c.updateStatusLabelsAndAnnotations(tr, original)).ErrorOrNil() + + // Emit events (only when ConditionSucceeded was changed) + after := tr.Status.GetCondition(apis.ConditionSucceeded) + reconciler.EmitEvent(c.Recorder, before, after, tr) + + return multierror.Append(err, c.updateStatusLabelsAndAnnotations(tr, original)).ErrorOrNil() } -func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error { +// `prepare` fetches resources the taskrun depends on, runs validation and convertion +// It may report errors back to Reconcile, it updates the taskrun status in case of +// error but it does not sync updates back to etcd. It does not emit events. +// All errors returned by `prepare` are always handled by `Reconcile`, so they don't cause +// the key to be re-queued directly. Once we start using `PermanentErrors` code in +// `prepare` will be able to control which error is handled by `Reconcile` and which is not +// See https://github.com/tektoncd/pipeline/issues/2474 for details. +// `prepare` returns spec and resources. In future we might store +// them in the TaskRun.Status so we don't need to re-run `prepare` at every +// reconcile (see https://github.com/tektoncd/pipeline/issues/2473). +func (c *Reconciler) prepare(ctx context.Context, tr *v1alpha1.TaskRun) (*v1alpha1.TaskSpec, *resources.ResolvedTaskResources, 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. tr.SetDefaults(contexts.WithUpgradeViaDefaulting(ctx)) @@ -199,9 +226,9 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error if err := tr.ConvertTo(ctx, &v1beta1.TaskRun{}); err != nil { if ce, ok := err.(*v1beta1.CannotConvertError); ok { tr.Status.MarkResourceNotConvertible(ce) - return nil + return nil, nil, nil } - return err + return nil, nil, err } getTaskFunc, kind := c.getTaskFunc(tr) @@ -209,11 +236,10 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error if err != nil { if ce, ok := err.(*v1beta1.CannotConvertError); ok { tr.Status.MarkResourceNotConvertible(ce) - return nil } c.Logger.Errorf("Failed to determine Task spec to use for taskrun %s: %v", tr.Name, err) tr.Status.MarkResourceFailed(podconvert.ReasonFailedResolution, err) - return nil + return nil, nil, err } // Propagate labels from Task to TaskRun. @@ -249,19 +275,19 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error if err != nil { c.Logger.Errorf("Failed to resolve references for taskrun %s: %v", tr.Name, err) tr.Status.MarkResourceFailed(podconvert.ReasonFailedResolution, err) - return nil + return nil, nil, err } if err := ValidateResolvedTaskResources(tr.Spec.Params, rtr); err != nil { c.Logger.Errorf("TaskRun %q resources are invalid: %v", tr.Name, err) tr.Status.MarkResourceFailed(podconvert.ReasonFailedValidation, err) - return nil + return nil, nil, err } 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.MarkResourceFailed(podconvert.ReasonFailedValidation, err) - return nil + return nil, nil, err } // Initialize the cloud events if at least a CloudEventResource is defined @@ -275,8 +301,20 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error } cloudevent.InitializeCloudEvents(tr, prs) + return taskSpec, rtr, nil +} + +// `reconcile` creates the Pod associated to the TaskRun, and it pulls back status +// updates from the Pod to the TaskRun. +// It reports errors back to Reconcile, it updates the taskrun status in case of +// error but it does not sync updates back to etcd. It does not emit events. +// `reconcile` consumes spec and resources returned by `prepare` +func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun, + taskSpec *v1alpha1.TaskSpec, rtr *resources.ResolvedTaskResources) error { // Get the TaskRun's Pod if it should have one. Otherwise, create the Pod. var pod *corev1.Pod + var err error + if tr.Status.PodName != "" { pod, err = c.KubeClientSet.CoreV1().Pods(tr.Namespace).Get(tr.Status.PodName, metav1.GetOptions{}) if k8serrors.IsNotFound(err) { @@ -303,7 +341,7 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error if pod == nil { if tr.HasVolumeClaimTemplate() { - if err = c.pvcHandler.CreatePersistentVolumeClaimsForWorkspaces(tr.Spec.Workspaces, tr.GetOwnerReference(), tr.Namespace); err != nil { + 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.MarkResourceFailed(volumeclaim.ReasonCouldntCreateWorkspacePVC, fmt.Errorf("Failed to create PVC for TaskRun %s workspaces correctly: %s", @@ -312,6 +350,7 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error } taskRunWorkspaces := applyVolumeClaimTemplates(tr.Spec.Workspaces, tr.GetOwnerReference()) + // This is used by createPod below. Changes to the Spec are not updated. tr.Spec.Workspaces = taskRunWorkspaces } @@ -337,8 +376,6 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error } } - before := tr.Status.GetCondition(apis.ConditionSucceeded) - // Convert the Pod's status to the equivalent TaskRun Status. tr.Status = podconvert.MakeTaskRunStatus(c.Logger, *tr, pod, *taskSpec) @@ -346,10 +383,7 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error return err } - after := tr.Status.GetCondition(apis.ConditionSucceeded) - - reconciler.EmitEvent(c.Recorder, before, after, tr) - c.Logger.Infof("Successfully reconciled taskrun %s/%s with status: %#v", tr.Name, tr.Namespace, after) + c.Logger.Infof("Successfully reconciled taskrun %s/%s with status: %#v", tr.Name, tr.Namespace, tr.Status.GetCondition(apis.ConditionSucceeded)) return nil } @@ -445,33 +479,32 @@ func (c *Reconciler) getTaskFunc(tr *v1alpha1.TaskRun) (resources.GetTask, v1alp } func (c *Reconciler) handlePodCreationError(tr *v1alpha1.TaskRun, err error) { - var reason, msg string - var succeededStatus corev1.ConditionStatus + var msg string if isExceededResourceQuotaError(err) { - succeededStatus = corev1.ConditionUnknown - reason = podconvert.ReasonExceededResourceQuota backoff, currentlyBackingOff := c.timeoutHandler.GetBackoff(tr) if !currentlyBackingOff { go c.timeoutHandler.SetTaskRunTimer(tr, time.Until(backoff.NextAttempt)) } msg = fmt.Sprintf("TaskRun Pod exceeded available resources, reattempted %d times", backoff.NumAttempts) + tr.Status.SetCondition(&apis.Condition{ + Type: apis.ConditionSucceeded, + Status: corev1.ConditionUnknown, + Reason: podconvert.ReasonExceededResourceQuota, + Message: fmt.Sprintf("%s: %v", msg, err), + }) } else { - succeededStatus = corev1.ConditionFalse - reason = podconvert.ReasonCouldntGetTask + // The pod creation failed, not because of quota issues. The most likely + // reason is that something is wrong with the spec of the Task, that we could + // not check with validation before - i.e. pod template fields + msg = fmt.Sprintf("failed to create task run pod %q: %v. Maybe ", tr.Name, err) if tr.Spec.TaskRef != nil { - msg = fmt.Sprintf("Missing or invalid Task %s/%s", tr.Namespace, tr.Spec.TaskRef.Name) + msg += fmt.Sprintf("missing or invalid Task %s/%s", tr.Namespace, tr.Spec.TaskRef.Name) } else { - msg = "Invalid TaskSpec" + msg += "invalid TaskSpec" } + tr.Status.MarkResourceFailed(podconvert.ReasonCouldntGetTask, errors.New(msg)) } - tr.Status.SetCondition(&apis.Condition{ - Type: apis.ConditionSucceeded, - Status: succeededStatus, - Reason: reason, - Message: fmt.Sprintf("%s: %v", msg, err), - }) - c.Recorder.Eventf(tr, corev1.EventTypeWarning, "BuildCreationFailed", "Failed to create build pod %q: %v", tr.Name, err) - c.Logger.Errorf("Failed to create build pod for task %q: %v", tr.Name, err) + c.Logger.Error("Failed to create task run pod for task %q: %v", tr.Name, err) } // failTaskRun stops a TaskRun with the provided Reason diff --git a/pkg/reconciler/taskrun/taskrun_test.go b/pkg/reconciler/taskrun/taskrun_test.go index bc53db52762..e4fb64f7980 100644 --- a/pkg/reconciler/taskrun/taskrun_test.go +++ b/pkg/reconciler/taskrun/taskrun_test.go @@ -568,12 +568,14 @@ func TestReconcile(t *testing.T) { PipelineResources: []*v1alpha1.PipelineResource{gitResource, anotherGitResource, imageResource}, } for _, tc := range []struct { - name string - taskRun *v1alpha1.TaskRun - wantPod *corev1.Pod + name string + taskRun *v1alpha1.TaskRun + wantPod *corev1.Pod + wantEvents int }{{ - name: "success", - taskRun: taskRunSuccess, + name: "success", + taskRun: taskRunSuccess, + wantEvents: 1, wantPod: tb.Pod("test-taskrun-run-success-pod-abcde", tb.PodNamespace("foo"), tb.PodAnnotation(podconvert.ReleaseAnnotation, podconvert.ReleaseAnnotationValue), @@ -611,8 +613,9 @@ func TestReconcile(t *testing.T) { ), ), }, { - name: "serviceaccount", - taskRun: taskRunWithSaSuccess, + name: "serviceaccount", + taskRun: taskRunWithSaSuccess, + wantEvents: 1, wantPod: tb.Pod("test-taskrun-with-sa-run-success-pod-abcde", tb.PodNamespace("foo"), tb.PodAnnotation(podconvert.ReleaseAnnotation, podconvert.ReleaseAnnotationValue), @@ -651,8 +654,9 @@ func TestReconcile(t *testing.T) { ), ), }, { - name: "params", - taskRun: taskRunSubstitution, + name: "params", + taskRun: taskRunSubstitution, + wantEvents: 1, wantPod: tb.Pod("test-taskrun-substitution-pod-abcde", tb.PodNamespace("foo"), tb.PodAnnotation(podconvert.ReleaseAnnotation, podconvert.ReleaseAnnotationValue), @@ -733,8 +737,9 @@ func TestReconcile(t *testing.T) { ), ), }, { - name: "taskrun-with-taskspec", - taskRun: taskRunWithTaskSpec, + name: "taskrun-with-taskspec", + taskRun: taskRunWithTaskSpec, + wantEvents: 1, wantPod: tb.Pod("test-taskrun-with-taskspec-pod-abcde", tb.PodNamespace("foo"), tb.PodAnnotation(podconvert.ReleaseAnnotation, podconvert.ReleaseAnnotationValue), @@ -790,8 +795,9 @@ func TestReconcile(t *testing.T) { ), ), }, { - name: "success-with-cluster-task", - taskRun: taskRunWithClusterTask, + name: "success-with-cluster-task", + taskRun: taskRunWithClusterTask, + wantEvents: 1, wantPod: tb.Pod("test-taskrun-with-cluster-task-pod-abcde", tb.PodNamespace("foo"), tb.PodAnnotation(podconvert.ReleaseAnnotation, podconvert.ReleaseAnnotationValue), @@ -829,8 +835,9 @@ func TestReconcile(t *testing.T) { ), ), }, { - name: "taskrun-with-resource-spec-task-spec", - taskRun: taskRunWithResourceSpecAndTaskSpec, + name: "taskrun-with-resource-spec-task-spec", + taskRun: taskRunWithResourceSpecAndTaskSpec, + wantEvents: 1, wantPod: tb.Pod("test-taskrun-with-resource-spec-pod-abcde", tb.PodNamespace("foo"), tb.PodAnnotation(podconvert.ReleaseAnnotation, podconvert.ReleaseAnnotationValue), @@ -885,8 +892,9 @@ func TestReconcile(t *testing.T) { ), ), }, { - name: "taskrun-with-pod", - taskRun: taskRunWithPod, + name: "taskrun-with-pod", + taskRun: taskRunWithPod, + wantEvents: 1, wantPod: tb.Pod("test-taskrun-with-pod-pod-abcde", tb.PodNamespace("foo"), tb.PodAnnotation(podconvert.ReleaseAnnotation, podconvert.ReleaseAnnotationValue), @@ -923,8 +931,9 @@ func TestReconcile(t *testing.T) { ), ), }, { - name: "taskrun-with-credentials-variable-default-tekton-home", - taskRun: taskRunWithCredentialsVariable, + name: "taskrun-with-credentials-variable-default-tekton-home", + taskRun: taskRunWithCredentialsVariable, + wantEvents: 1, wantPod: tb.Pod("test-taskrun-with-credentials-variable-pod-9l9zj", tb.PodNamespace("foo"), tb.PodAnnotation(podconvert.ReleaseAnnotation, podconvert.ReleaseAnnotationValue), @@ -1024,6 +1033,17 @@ func TestReconcile(t *testing.T) { if len(clients.Kube.Actions()) == 0 { t.Fatalf("Expected actions to be logged in the kubeclient, got none") } + + actions := clients.Kube.Actions() + var eventCount = 0 + for _, action := range actions { + if action.GetVerb() == "create" && action.GetResource().Resource == "events" { + eventCount++ + } + } + if d := cmp.Diff(tc.wantEvents, eventCount); d != "" { + t.Errorf("Event count does not match (-want, +got): %s. ", d) + } }) } } @@ -1178,7 +1198,7 @@ func TestReconcileInvalidTaskRuns(t *testing.T) { taskRun: noTaskRun, reason: podconvert.ReasonFailedResolution, }, { - name: "task run with no task", + name: "task run with wrong ref", taskRun: withWrongRef, reason: podconvert.ReasonFailedResolution, }} @@ -1190,16 +1210,22 @@ func TestReconcileInvalidTaskRuns(t *testing.T) { c := testAssets.Controller clients := testAssets.Clients err := c.Reconciler.Reconcile(context.Background(), getRunName(tc.taskRun)) + // Events are sent in a goroutine, let's sleep a bit to make sure they're + // captured by the fake client-go action list + time.Sleep(100 * time.Millisecond) // When a TaskRun is invalid and can't run, we don't want to return an error because // an error will tell the Reconciler to keep trying to reconcile; instead we want to stop // and forget about the Run. if err != nil { t.Errorf("Did not expect to see error when reconciling invalid TaskRun but saw %q", err) } - if len(clients.Kube.Actions()) != 1 || - clients.Kube.Actions()[0].GetVerb() != "list" || - clients.Kube.Actions()[0].GetResource().Resource != "namespaces" { - t.Errorf("expected only one action (list namespaces) created by the reconciler, got %+v", clients.Kube.Actions()) + actions := clients.Kube.Actions() + if len(actions) != 2 || + actions[0].GetVerb() != "list" || + actions[0].GetResource().Resource != "namespaces" || + actions[1].GetVerb() != "create" || + actions[1].GetResource().Resource != "events" { + t.Errorf("expected two actions (list namespaces + event) created by the reconciler, got %+v", actions) } // Since the TaskRun is invalid, the status should say it has failed condition := tc.taskRun.Status.GetCondition(apis.ConditionSucceeded) @@ -1235,7 +1261,7 @@ func TestReconcilePodFetchError(t *testing.T) { }) if err := c.Reconciler.Reconcile(context.Background(), getRunName(taskRun)); err == nil { - t.Fatal("expected error when reconciling a Task for which we couldn't get the corresponding Build Pod but got nil") + t.Fatal("expected error when reconciling a Task for which we couldn't get the corresponding Pod but got nil") } } diff --git a/test/cancel_test.go b/test/cancel_test.go index 6840d1a27f1..31cad260211 100644 --- a/test/cancel_test.go +++ b/test/cancel_test.go @@ -178,7 +178,7 @@ func TestTaskRunPipelineRunCancel(t *testing.T) { t.Fatalf("Failed to collect matching events: %q", err) } if len(events) != expectedNumberOfEvents { - t.Fatalf("Expected %d number of successful events from pipelinerun and taskrun but got %d; list of receieved events : %#v", expectedNumberOfEvents, len(events), events) + t.Fatalf("Expected %d number of successful events from pipelinerun and taskrun but got %d; list of received events : %#v", expectedNumberOfEvents, len(events), events) } }) } diff --git a/test/init_test.go b/test/init_test.go index 4420c1626df..9b5da5516da 100644 --- a/test/init_test.go +++ b/test/init_test.go @@ -102,7 +102,7 @@ func tearDown(t *testing.T, cs *clients, namespace string) { } } - if os.Getenv("TEST_KEEP_NAMESPACES") == "" { + if os.Getenv("TEST_KEEP_NAMESPACES") == "" && !t.Failed() { t.Logf("Deleting namespace %s", namespace) if err := cs.KubeClient.Kube.CoreV1().Namespaces().Delete(namespace, &metav1.DeleteOptions{}); err != nil { t.Errorf("Failed to delete namespace %s: %s", namespace, err) diff --git a/test/pipelinerun_test.go b/test/pipelinerun_test.go index b871a18cbed..a1e4c289625 100644 --- a/test/pipelinerun_test.go +++ b/test/pipelinerun_test.go @@ -265,7 +265,14 @@ func TestPipelineRun(t *testing.T) { t.Fatalf("Failed to collect matching events: %q", err) } if len(events) != td.expectedNumberOfEvents { - t.Fatalf("Expected %d number of successful events from pipelinerun and taskrun but got %d; list of receieved events : %#v", td.expectedNumberOfEvents, len(events), events) + collectedEvents := "" + for i, event := range events { + collectedEvents += fmt.Sprintf("%#v", event) + if i < (len(events) - 1) { + collectedEvents += ", " + } + } + t.Fatalf("Expected %d number of successful events from pipelinerun and taskrun but got %d; list of receieved events : %#v", td.expectedNumberOfEvents, len(events), collectedEvents) } // Wait for up to 10 minutes and restart every second to check if diff --git a/test/v1alpha1/pipelinerun_test.go b/test/v1alpha1/pipelinerun_test.go index 26edeef547d..06641200e3b 100644 --- a/test/v1alpha1/pipelinerun_test.go +++ b/test/v1alpha1/pipelinerun_test.go @@ -204,7 +204,14 @@ func TestPipelineRun(t *testing.T) { t.Fatalf("Failed to collect matching events: %q", err) } if len(events) != td.expectedNumberOfEvents { - t.Fatalf("Expected %d number of successful events from pipelinerun and taskrun but got %d; list of receieved events : %#v", td.expectedNumberOfEvents, len(events), events) + collectedEvents := "" + for i, event := range events { + collectedEvents += fmt.Sprintf("%#v", event) + if i < (len(events) - 1) { + collectedEvents += ", " + } + } + t.Fatalf("Expected %d number of successful events from pipelinerun and taskrun but got %d; list of receieved events : %#v", td.expectedNumberOfEvents, len(events), collectedEvents) } // Wait for up to 10 minutes and restart every second to check if