Skip to content

Commit

Permalink
allow for retry on typically transient k8s errors in both core contro…
Browse files Browse the repository at this point in the history
…ller and resolver for remote resolution

During both sides of remote resolution (core controller and resolver) typically transient kubernetes errors were being treated as permanent knative errors and no attempts at trying to reconcile again were made, leading to failures which could be avoided.

Then, while diagnosing this, discovered the TaskNotFoundError was missing the Task name when identification comes from params.  That is also addressed.
  • Loading branch information
gabemontero authored and tekton-robot committed May 16, 2024
1 parent f95c4b7 commit 13f45bf
Show file tree
Hide file tree
Showing 16 changed files with 242 additions and 49 deletions.
3 changes: 2 additions & 1 deletion pkg/reconciler/pipelinerun/pipelinerun.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ import (
"github.com/tektoncd/pipeline/pkg/reconciler/volumeclaim"
"github.com/tektoncd/pipeline/pkg/remote"
resolution "github.com/tektoncd/pipeline/pkg/remoteresolution/resource"
resolutioncommon "github.com/tektoncd/pipeline/pkg/resolution/common"
"github.com/tektoncd/pipeline/pkg/substitution"
"github.com/tektoncd/pipeline/pkg/trustedresources"
"github.com/tektoncd/pipeline/pkg/workspace"
Expand Down Expand Up @@ -373,7 +374,7 @@ func (c *Reconciler) resolvePipelineState(
pst,
)
if err != nil {
if tresources.IsErrTransient(err) {
if resolutioncommon.IsErrTransient(err) {
return nil, err
}
if errors.Is(err, remote.ErrRequestInProgress) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1"
"github.com/tektoncd/pipeline/pkg/reconciler/taskrun/resources"
"github.com/tektoncd/pipeline/pkg/remote"
"github.com/tektoncd/pipeline/pkg/resolution/resource"
"github.com/tektoncd/pipeline/pkg/substitution"
kerrors "k8s.io/apimachinery/pkg/api/errors"
"knative.dev/pkg/apis"
Expand Down Expand Up @@ -645,8 +646,12 @@ func resolveTask(
case errors.Is(err, remote.ErrRequestInProgress):
return rt, err
case err != nil:
name := pipelineTask.TaskRef.Name
if len(strings.TrimSpace(name)) == 0 {
name = resource.GenerateErrorLogString(string(pipelineTask.TaskRef.Resolver), pipelineTask.TaskRef.Params)
}
return rt, &TaskNotFoundError{
Name: pipelineTask.TaskRef.Name,
Name: name,
Msg: err.Error(),
}
default:
Expand Down
15 changes: 15 additions & 0 deletions pkg/reconciler/pipelinerun/resources/pipelinerunresolution_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2490,6 +2490,18 @@ func TestResolvePipelineRun_TaskDoesntExist(t *testing.T) {
Value: *v1.NewStructuredValues("b", "a", "r"),
}},
},
}, {
Name: "mytask3",
TaskRef: &v1.TaskRef{ResolverRef: v1.ResolverRef{Params: v1.Params{{Name: "name", Value: v1.ParamValue{Type: v1.ParamTypeString, StringVal: "foo"}}}}},
Matrix: &v1.Matrix{
Params: v1.Params{{
Name: "foo",
Value: *v1.NewStructuredValues("f", "o", "o"),
}, {
Name: "bar",
Value: *v1.NewStructuredValues("b", "a", "r"),
}},
},
}}

// Return an error when the Task is retrieved, as if it didn't exist
Expand All @@ -2512,6 +2524,9 @@ func TestResolvePipelineRun_TaskDoesntExist(t *testing.T) {
t.Fatalf("Pipeline %s: want error, got nil", p.Name)
case errors.As(err, &tnf):
// expected error
if len(tnf.Name) == 0 {
t.Fatalf("Pipeline %s: TaskNotFoundError did not have name set: %s", p.Name, tnf.Error())
}
default:
t.Fatalf("Pipeline %s: Want %T, got %s of type %T", p.Name, tnf, err, err)
}
Expand Down
11 changes: 0 additions & 11 deletions pkg/reconciler/taskrun/resources/taskref.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,12 +39,6 @@ import (
"knative.dev/pkg/kmeta"
)

// This error is defined in etcd at
// https://github.com/etcd-io/etcd/blob/5b226e0abf4100253c94bb71f47d6815877ed5a2/server/etcdserver/errors.go#L30
// TODO: If/when https://github.com/kubernetes/kubernetes/issues/106491 is addressed,
// we should stop relying on a hardcoded string.
var errEtcdLeaderChange = "etcdserver: leader changed"

// GetTaskKind returns the referenced Task kind (Task, ClusterTask, ...) if the TaskRun is using TaskRef.
func GetTaskKind(taskrun *v1.TaskRun) v1.TaskKind {
kind := v1.NamespacedTaskKind
Expand Down Expand Up @@ -366,11 +360,6 @@ func (l *LocalStepActionRefResolver) GetStepAction(ctx context.Context, name str
return stepAction, nil, nil
}

// IsErrTransient returns true if an error returned by GetTask/GetStepAction is retryable.
func IsErrTransient(err error) bool {
return strings.Contains(err.Error(), errEtcdLeaderChange)
}

// convertClusterTaskToTask converts deprecated v1beta1 ClusterTasks to Tasks for
// the rest of reconciling process since GetTask func and its upstream callers only
// fetches the task spec and stores it in the taskrun status while the kind info
Expand Down
5 changes: 3 additions & 2 deletions pkg/reconciler/taskrun/taskrun.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ import (
"github.com/tektoncd/pipeline/pkg/reconciler/volumeclaim"
"github.com/tektoncd/pipeline/pkg/remote"
resolution "github.com/tektoncd/pipeline/pkg/remoteresolution/resource"
resolutioncommon "github.com/tektoncd/pipeline/pkg/resolution/common"
"github.com/tektoncd/pipeline/pkg/spire"
"github.com/tektoncd/pipeline/pkg/taskrunmetrics"
_ "github.com/tektoncd/pipeline/pkg/taskrunmetrics/fake" // Make sure the taskrunmetrics are setup
Expand Down Expand Up @@ -409,7 +410,7 @@ func (c *Reconciler) prepare(ctx context.Context, tr *v1.TaskRun) (*v1.TaskSpec,
return nil, nil, err
case err != nil:
logger.Errorf("Failed to determine Task spec to use for taskrun %s: %v", tr.Name, err)
if resources.IsErrTransient(err) {
if resolutioncommon.IsErrTransient(err) {
return nil, nil, err
}
tr.Status.MarkResourceFailed(v1.TaskRunReasonFailedResolution, err)
Expand All @@ -434,7 +435,7 @@ func (c *Reconciler) prepare(ctx context.Context, tr *v1.TaskRun) (*v1.TaskSpec,
return nil, nil, err
case err != nil:
logger.Errorf("Failed to determine StepAction to use for TaskRun %s: %v", tr.Name, err)
if resources.IsErrTransient(err) {
if resolutioncommon.IsErrTransient(err) {
return nil, nil, err
}
tr.Status.MarkResourceFailed(v1.TaskRunReasonFailedResolution, err)
Expand Down
66 changes: 37 additions & 29 deletions pkg/reconciler/taskrun/taskrun_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1965,38 +1965,46 @@ spec:
Tasks: []*v1.Task{simpleTask},
ClusterTasks: []*v1beta1.ClusterTask{},
}
testAssets, cancel := getTaskRunController(t, d)
defer cancel()
c := testAssets.Controller
clients := testAssets.Clients
createServiceAccount(t, testAssets, "default", tr.Namespace)
for _, v := range []error{
errors.New("etcdserver: leader changed"),
context.DeadlineExceeded,
apierrors.NewConflict(pipeline.TaskRunResource, "", nil),
apierrors.NewServerTimeout(pipeline.TaskRunResource, "", 0),
apierrors.NewTimeoutError("", 0),
} {
testAssets, cancel := getTaskRunController(t, d)
defer cancel()
c := testAssets.Controller
clients := testAssets.Clients
createServiceAccount(t, testAssets, "default", tr.Namespace)

failingReactorActivated := true
clients.Pipeline.PrependReactor("*", "tasks", func(action ktesting.Action) (handled bool, ret runtime.Object, err error) {
return failingReactorActivated, &v1.Task{}, errors.New("etcdserver: leader changed")
})
err := c.Reconciler.Reconcile(testAssets.Ctx, getRunName(tr))
if err == nil {
t.Error("Wanted a wrapped error, but got nil.")
}
if controller.IsPermanentError(err) {
t.Errorf("Unexpected permanent error %v", err)
}
failingReactorActivated := true
clients.Pipeline.PrependReactor("*", "tasks", func(action ktesting.Action) (handled bool, ret runtime.Object, err error) {
return failingReactorActivated, &v1.Task{}, v
})
err := c.Reconciler.Reconcile(testAssets.Ctx, getRunName(tr))
if err == nil {
t.Error("Wanted a wrapped error, but got nil.")
}
if controller.IsPermanentError(err) {
t.Errorf("Unexpected permanent error %v", err)
}

failingReactorActivated = false
err = c.Reconciler.Reconcile(testAssets.Ctx, getRunName(tr))
if err != nil {
if ok, _ := controller.IsRequeueKey(err); !ok {
t.Errorf("unexpected error in TaskRun reconciliation: %v", err)
failingReactorActivated = false
err = c.Reconciler.Reconcile(testAssets.Ctx, getRunName(tr))
if err != nil {
if ok, _ := controller.IsRequeueKey(err); !ok {
t.Errorf("unexpected error in TaskRun reconciliation: %v", err)
}
}
reconciledRun, err := clients.Pipeline.TektonV1().TaskRuns("foo").Get(testAssets.Ctx, tr.Name, metav1.GetOptions{})
if err != nil {
t.Fatalf("Somehow had error getting reconciled run out of fake client: %s", err)
}
condition := reconciledRun.Status.GetCondition(apis.ConditionSucceeded)
if !condition.IsUnknown() {
t.Errorf("Expected TaskRun to still be running but succeeded condition is %v", condition.Status)
}
}
reconciledRun, err := clients.Pipeline.TektonV1().TaskRuns("foo").Get(testAssets.Ctx, tr.Name, metav1.GetOptions{})
if err != nil {
t.Fatalf("Somehow had error getting reconciled run out of fake client: %s", err)
}
condition := reconciledRun.Status.GetCondition(apis.ConditionSucceeded)
if !condition.IsUnknown() {
t.Errorf("Expected TaskRun to still be running but succeeded condition is %v", condition.Status)
}
}

Expand Down
5 changes: 5 additions & 0 deletions pkg/remoteresolution/resolver/framework/reconciler.go
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,9 @@ func (r *Reconciler) resolve(ctx context.Context, key string, rr *v1beta1.Resolu
// OnError is used to handle any situation where a ResolutionRequest has
// reached a terminal situation that cannot be recovered from.
func (r *Reconciler) OnError(ctx context.Context, rr *v1beta1.ResolutionRequest, err error) error {
if resolutioncommon.IsErrTransient(err) {
return err
}
if rr == nil {
return controller.NewPermanentError(err)
}
Expand Down Expand Up @@ -213,13 +216,15 @@ func (r *Reconciler) writeResolvedData(ctx context.Context, rr *v1beta1.Resoluti
},
})
if err != nil {
logging.FromContext(ctx).Warnf("writeResolvedData error serializing resource request patch for resolution request %s:%s: %s", rr.Namespace, rr.Name, err.Error())
return r.OnError(ctx, rr, &resolutioncommon.UpdatingRequestError{
ResolutionRequestKey: fmt.Sprintf("%s/%s", rr.Namespace, rr.Name),
Original: fmt.Errorf("error serializing resource request patch: %w", err),
})
}
_, err = r.resolutionRequestClientSet.ResolutionV1beta1().ResolutionRequests(rr.Namespace).Patch(ctx, rr.Name, types.MergePatchType, patchBytes, metav1.PatchOptions{}, "status")
if err != nil {
logging.FromContext(ctx).Warnf("writeResolvedData error patching resolution request %s:%s: %s", rr.Namespace, rr.Name, err.Error())
return r.OnError(ctx, rr, &resolutioncommon.UpdatingRequestError{
ResolutionRequestKey: fmt.Sprintf("%s/%s", rr.Namespace, rr.Name),
Original: err,
Expand Down
5 changes: 5 additions & 0 deletions pkg/remoteresolution/resolver/framework/reconciler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ func TestReconcile(t *testing.T) {
reconcilerTimeout time.Duration
expectedStatus *v1beta1.ResolutionRequestStatus
expectedErr error
transient bool
}{
{
name: "unknown value",
Expand Down Expand Up @@ -343,6 +344,7 @@ func TestReconcile(t *testing.T) {
},
reconcilerTimeout: 1 * time.Second,
expectedErr: errors.New("context deadline exceeded"),
transient: true,
},
}

Expand All @@ -369,6 +371,9 @@ func TestReconcile(t *testing.T) {
if tc.expectedErr.Error() != err.Error() {
t.Fatalf("expected to get error %v, but got %v", tc.expectedErr, err)
}
if tc.transient && controller.IsPermanentError(err) {
t.Fatalf("exepected error to not be wrapped as permanent %v", err)
}
} else {
if err != nil {
if ok, _ := controller.IsRequeueKey(err); !ok {
Expand Down
27 changes: 27 additions & 0 deletions pkg/resolution/common/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,21 @@ limitations under the License.
package common

import (
"context"
"errors"
"fmt"
"slices"
"strings"

apierrors "k8s.io/apimachinery/pkg/api/errors"
)

// This error is defined in etcd at
// https://github.com/etcd-io/etcd/blob/5b226e0abf4100253c94bb71f47d6815877ed5a2/server/etcdserver/errors.go#L30
// TODO: If/when https://github.com/kubernetes/kubernetes/issues/106491 is addressed,
// we should stop relying on a hardcoded string.
var errEtcdLeaderChange = "etcdserver: leader changed"

// Error embeds both a short machine-readable string reason for resolution
// problems alongside the original error generated during the resolution flow.
type Error struct {
Expand Down Expand Up @@ -165,3 +176,19 @@ func ReasonError(err error) (string, error) {

return reason, resolutionError
}

// IsErrTransient returns true if an error returned by GetTask/GetStepAction is retryable.
func IsErrTransient(err error) bool {
switch {
case apierrors.IsConflict(err):
return true
case apierrors.IsServerTimeout(err):
return true
case apierrors.IsTimeout(err):
return true
default:
return slices.ContainsFunc([]string{errEtcdLeaderChange, context.DeadlineExceeded.Error()}, func(s string) bool {
return strings.Contains(err.Error(), s)
})
}
}
3 changes: 2 additions & 1 deletion pkg/resolution/resolver/bundle/params.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/google/go-containerregistry/pkg/name"
pipelinev1 "github.com/tektoncd/pipeline/pkg/apis/pipeline/v1"
"github.com/tektoncd/pipeline/pkg/resolution/resolver/framework"
"github.com/tektoncd/pipeline/pkg/resolution/resource"
)

// ParamImagePullSecret is the parameter defining what secret
Expand All @@ -32,7 +33,7 @@ const ParamBundle = "bundle"

// ParamName is the parameter defining what the layer name in the bundle
// image is.
const ParamName = "name"
const ParamName = resource.ParamName

// ParamKind is the parameter defining what the layer kind in the bundle
// image is.
Expand Down
2 changes: 1 addition & 1 deletion pkg/resolution/resolver/framework/reconciler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
"github.com/tektoncd/pipeline/pkg/apis/resolution/v1beta1"
ttesting "github.com/tektoncd/pipeline/pkg/reconciler/testing"
resolutioncommon "github.com/tektoncd/pipeline/pkg/resolution/common"
framework "github.com/tektoncd/pipeline/pkg/resolution/resolver/framework"
"github.com/tektoncd/pipeline/pkg/resolution/resolver/framework"
"github.com/tektoncd/pipeline/test"
"github.com/tektoncd/pipeline/test/diff"
"github.com/tektoncd/pipeline/test/names"
Expand Down
4 changes: 3 additions & 1 deletion pkg/resolution/resolver/git/params.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,11 @@ limitations under the License.

package git

import "github.com/tektoncd/pipeline/pkg/resolution/resource"

const (
// UrlParam is the git repo Url when using the anonymous/full clone approach
UrlParam string = "url"
UrlParam string = resource.ParamURL
// OrgParam is the organization to find the repository in when using the SCM API approach
OrgParam = "org"
// RepoParam is the repository to use when using the SCM API approach
Expand Down
4 changes: 3 additions & 1 deletion pkg/resolution/resolver/http/params.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,11 @@ limitations under the License.

package http

import "github.com/tektoncd/pipeline/pkg/resolution/resource"

const (
// UrlParam is the URL to fetch the task from
UrlParam string = "url"
UrlParam string = resource.ParamURL

// HttpBasicAuthUsername is the user name to use for basic auth
HttpBasicAuthUsername string = "http-username"
Expand Down
4 changes: 3 additions & 1 deletion pkg/resolution/resolver/hub/params.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@ limitations under the License.

package hub

import "github.com/tektoncd/pipeline/pkg/resolution/resource"

// DefaultArtifactHubURL is the default url for the Artifact hub api
const DefaultArtifactHubURL = "https://artifacthub.io"

Expand All @@ -30,7 +32,7 @@ const ArtifactHubListTasksEndpoint = "api/v1/packages/tekton-%s/%s/%s"

// ParamName is the parameter defining what the layer name in the bundle
// image is.
const ParamName = "name"
const ParamName = resource.ParamName

// ParamKind is the parameter defining what the layer kind in the bundle
// image is.
Expand Down
Loading

0 comments on commit 13f45bf

Please sign in to comment.