Skip to content

Commit

Permalink
Perform webhook validation for remote pipelines
Browse files Browse the repository at this point in the history
Prior to this commit, remote pipelines were only validated
by calling `pipelineSpec.Validate` in the PipelineRun reconciler.
This omits some validation that is only done when validating Pipelines,
rather than Pipeline specs, such as validation for propagated params
and workspaces. In addition, if a cluster operator or vendor defines
any validating admission webhooks for Pipelines, this validation would
apply only to local Pipelines but not remote Pipelines.

This commit issues a dry-run create request for remote Pipelines
and fails the PipelineRun if the apiserver rejects the request.
This allows us to do webhook-based validation of remote Pipelines
without ever having to create them on the cluster, ensuring validation
of remote Pipelines matches validation of local Pipelines.

Similar validation will be added for remote Tasks in a separate commit.
  • Loading branch information
lbernick committed Jun 28, 2023
1 parent 7b23f93 commit a5de06d
Show file tree
Hide file tree
Showing 4 changed files with 122 additions and 12 deletions.
3 changes: 3 additions & 0 deletions pkg/reconciler/pipelinerun/pipelinerun.go
Original file line number Diff line number Diff line change
Expand Up @@ -406,6 +406,9 @@ func (c *Reconciler) reconcile(ctx context.Context, pr *v1.PipelineRun, getPipel
message := fmt.Sprintf("PipelineRun %s/%s awaiting remote resource", pr.Namespace, pr.Name)
pr.Status.MarkRunning(ReasonResolvingPipelineRef, message)
return nil
case errors.Is(err, resources.ErrRemotePipelineValidationFailed):
pr.Status.MarkFailed(ReasonCouldntGetPipeline, err.Error())
return controller.NewPermanentError(err)
case err != nil:
logger.Errorf("Failed to determine Pipeline spec to use for pipelinerun %s: %v", pr.Name, err)
pr.Status.MarkFailed(ReasonCouldntGetPipeline,
Expand Down
77 changes: 75 additions & 2 deletions pkg/reconciler/pipelinerun/pipelinerun_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,6 @@ import (
"testing"
"time"

"sigs.k8s.io/yaml"

"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"github.com/sigstore/sigstore/pkg/signature"
Expand Down Expand Up @@ -58,10 +56,13 @@ import (
"gomodules.xyz/jsonpatch/v2"
appsv1 "k8s.io/api/apps/v1"
corev1 "k8s.io/api/core/v1"
apierrors "k8s.io/apimachinery/pkg/api/errors"
"k8s.io/apimachinery/pkg/api/resource"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/runtime/schema"
"k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/validation/field"
fakek8s "k8s.io/client-go/kubernetes/fake"
ktesting "k8s.io/client-go/testing"
"k8s.io/client-go/tools/record"
Expand All @@ -76,6 +77,7 @@ import (
"knative.dev/pkg/reconciler"
"knative.dev/pkg/system"
_ "knative.dev/pkg/system/testing" // Setup system.Namespace()
"sigs.k8s.io/yaml"
)

var (
Expand Down Expand Up @@ -6705,6 +6707,77 @@ spec:
verifyTaskRunStatusesNames(t, reconciledRun.Status, "test-pipeline-run-success-unit-test-1")
}

func TestReconcile_InvalidRemotePipeline(t *testing.T) {
namespace := "foo"
prName := "test-pipeline-run-success"
prs := []*v1.PipelineRun{parse.MustParseV1PipelineRun(t, `
metadata:
name: test-pipeline-run-success
namespace: foo
spec:
pipelineRef:
resolver: bar
`)}
ps := parse.MustParseV1Pipeline(t, `
metadata:
name: test-pipeline
namespace: foo
spec:
tasks:
- name: unit-test-1
taskSpec:
steps:
- image: busybox
script: echo hello
`)

pipelineBytes, err := yaml.Marshal(ps)
if err != nil {
t.Fatal("fail to marshal task", err)
}
pipelineReq := getResolvedResolutionRequest(t, "bar", pipelineBytes, "foo", prName)

// Unlike the tests above, we do *not* locally define our pipeline or unit-test task.
d := test.Data{
PipelineRuns: prs,
ConfigMaps: []*corev1.ConfigMap{
{
ObjectMeta: metav1.ObjectMeta{Name: config.GetFeatureFlagsConfigName(), Namespace: system.Namespace()},
Data: map[string]string{
"enable-api-fields": "beta",
},
},
},
ResolutionRequests: []*resolutionv1beta1.ResolutionRequest{&pipelineReq},
}
testAssets, cancel := getPipelineRunController(t, d)
defer cancel()
c := testAssets.Controller
clients := testAssets.Clients
pipelineGK := schema.GroupKind{Group: "tekton.dev/v1", Kind: "pipeline"}
// Create an error when the Pipeline client attempts to create Pipelines
clients.Pipeline.PrependReactor("create", "pipelines", func(action ktesting.Action) (bool, runtime.Object, error) {
return true, nil, apierrors.NewInvalid(pipelineGK, "invalid", field.ErrorList{})
})
err = c.Reconciler.Reconcile(testAssets.Ctx, fmt.Sprintf("%s/%s", namespace, prName))
if !controller.IsPermanentError(err) {
t.Errorf("expected permanent error but got %s", err)
}
reconciledRun, err := clients.Pipeline.TektonV1().PipelineRuns(namespace).Get(testAssets.Ctx, prName, metav1.GetOptions{})
if err != nil {
t.Fatalf("Somehow had error getting reconciled run out of fake client: %s", err)
}

if reconciledRun.Status.CompletionTime == nil {
t.Errorf("Expected a CompletionTime on invalid PipelineRun but was nil")
}

// The PipelineRun should be create run failed.
if reconciledRun.Status.GetCondition(apis.ConditionSucceeded).Reason != ReasonCouldntGetPipeline {
t.Errorf("Expected PipelineRun to have reason CouldntGetPipeline, but condition reason is %s", reconciledRun.Status.GetCondition(apis.ConditionSucceeded))
}
}

// TestReconcile_OptionalWorkspacesOmitted checks that an optional workspace declared by
// a Task and a Pipeline can be omitted by a PipelineRun and the run will still start
// successfully without an error.
Expand Down
42 changes: 33 additions & 9 deletions pkg/reconciler/pipelinerun/resources/pipelineref.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,11 +30,15 @@ import (
"github.com/tektoncd/pipeline/pkg/remote/resolution"
remoteresource "github.com/tektoncd/pipeline/pkg/resolution/resource"
"github.com/tektoncd/pipeline/pkg/trustedresources"
apierrors "k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/client-go/kubernetes"
"knative.dev/pkg/logging"
)

var ErrRemotePipelineValidationFailed = errors.New("validation failed for remote Pipeline")

// GetPipelineFunc is a factory function that will use the given PipelineRef to return a valid GetPipeline function that
// looks up the pipeline. It uses as context a k8s client, tekton client, namespace, and service account name to return
// the pipeline. It knows whether it needs to look in the cluster or in a remote location to fetch the reference.
Expand Down Expand Up @@ -70,7 +74,7 @@ func GetPipelineFunc(ctx context.Context, k8s kubernetes.Interface, tekton clien
replacedParams := pr.Params.ReplaceVariables(stringReplacements, arrayReplacements, objectReplacements)

resolver := resolution.NewResolver(requester, pipelineRun, string(pr.Resolver), "", "", replacedParams)
return resolvePipeline(ctx, resolver, name, k8s, verificationPolicies)
return resolvePipeline(ctx, resolver, name, namespace, k8s, tekton, verificationPolicies)
}
default:
// Even if there is no pipeline ref, we should try to return a local resolver.
Expand Down Expand Up @@ -111,12 +115,12 @@ func (l *LocalPipelineRefResolver) GetPipeline(ctx context.Context, name string)
// An error is returned if the remoteresource doesn't work
// A VerificationResult is returned if trusted resources is enabled, VerificationResult contains the result type and err.
// or the returned data isn't a valid *v1.Pipeline.
func resolvePipeline(ctx context.Context, resolver remote.Resolver, name string, k8s kubernetes.Interface, verificationPolicies []*v1alpha1.VerificationPolicy) (*v1.Pipeline, *v1.RefSource, *trustedresources.VerificationResult, error) {
func resolvePipeline(ctx context.Context, resolver remote.Resolver, name string, namespace string, k8s kubernetes.Interface, tekton clientset.Interface, verificationPolicies []*v1alpha1.VerificationPolicy) (*v1.Pipeline, *v1.RefSource, *trustedresources.VerificationResult, error) {
obj, refSource, err := resolver.Get(ctx, "pipeline", name)
if err != nil {
return nil, nil, nil, err
}
pipelineObj, vr, err := readRuntimeObjectAsPipeline(ctx, obj, k8s, refSource, verificationPolicies)
pipelineObj, vr, err := readRuntimeObjectAsPipeline(ctx, namespace, obj, k8s, tekton, refSource, verificationPolicies)
if err != nil {
return nil, nil, nil, err
}
Expand All @@ -132,11 +136,27 @@ func resolvePipeline(ctx context.Context, resolver remote.Resolver, name string,
// PipelineObject or if there is an error validating or upgrading an
// older PipelineObject into its v1beta1 equivalent.
// TODO(#5541): convert v1beta1 obj to v1 once we use v1 as the stored version
func readRuntimeObjectAsPipeline(ctx context.Context, obj runtime.Object, k8s kubernetes.Interface, refSource *v1.RefSource, verificationPolicies []*v1alpha1.VerificationPolicy) (*v1.Pipeline, *trustedresources.VerificationResult, error) {
func readRuntimeObjectAsPipeline(ctx context.Context, namespace string, obj runtime.Object, k8s kubernetes.Interface, tekton clientset.Interface, refSource *v1.RefSource, verificationPolicies []*v1alpha1.VerificationPolicy) (*v1.Pipeline, *trustedresources.VerificationResult, error) {
logger := logging.FromContext(ctx)
switch obj := obj.(type) {
case *v1beta1.Pipeline:
// Verify the Pipeline once we fetch from the remote resolution, mutating, validation and conversion of the pipeline should happen after the verification, since signatures are based on the remote pipeline contents
vr := trustedresources.VerifyResource(ctx, obj, k8s, refSource, verificationPolicies)
// Issue a dry-run request to create the remote Pipeline, so that it can undergo validation from validating admission webhooks
// without actually creating the Pipeline on the cluster.
// Validation must happen before the v1beta1 Pipeline is converted into the storage version of the API,
// since validation of beta features differs between v1 and v1beta1
// TODO(#6592): Decouple API versioning from feature versioning
if _, err := tekton.TektonV1beta1().Pipelines(namespace).Create(ctx, obj, metav1.CreateOptions{DryRun: []string{metav1.DryRunAll}}); err != nil {
switch {
case apierrors.IsInvalid(err): // Pipeline rejected by validating webhook
return nil, nil, fmt.Errorf("%w %s: %v", ErrRemotePipelineValidationFailed, obj.Name, err)
}
return nil, nil, fmt.Errorf("%w %s: %v", ErrRemotePipelineValidationFailed, obj.Name, err)
} else {

logger.Infof("no error on dry run")
}
p := &v1.Pipeline{
TypeMeta: metav1.TypeMeta{
Kind: "Pipeline",
Expand All @@ -149,13 +169,17 @@ func readRuntimeObjectAsPipeline(ctx context.Context, obj runtime.Object, k8s ku
return p, &vr, nil
case *v1.Pipeline:
vr := trustedresources.VerifyResource(ctx, obj, k8s, refSource, verificationPolicies)
// Validation of beta fields must happen before the V1 Pipeline is converted into the storage version of the API.
// TODO(#6592): Decouple API versioning from feature versioning
if err := obj.Spec.ValidateBetaFields(ctx); err != nil {
return nil, nil, fmt.Errorf("invalid Pipeline %s: %w", obj.GetName(), err)
// Issue a dry-run request to create the remote Pipeline, so that it can undergo validation from validating admission webhooks
// without actually creating the Pipeline on the cluster
if _, err := tekton.TektonV1().Pipelines(namespace).Create(ctx, obj, metav1.CreateOptions{DryRun: []string{metav1.DryRunAll}}); err != nil {
switch {
case apierrors.IsBadRequest(err): // Pipeline rejected by validating webhook
return nil, nil, fmt.Errorf("%w %s: %v", ErrRemotePipelineValidationFailed, obj.Name, err)
}
} else {
logger.Infof("no error on dry run")
}
return obj, &vr, nil
}

return nil, nil, errors.New("resource is not a pipeline")
}
12 changes: 11 additions & 1 deletion pkg/reconciler/pipelinerun/resources/pipelineref_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,9 @@ import (
"github.com/tektoncd/pipeline/pkg/apis/pipeline/v1alpha1"
"github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1"
"github.com/tektoncd/pipeline/pkg/client/clientset/versioned/fake"
clientset "github.com/tektoncd/pipeline/pkg/client/clientset/versioned/fake"
"github.com/tektoncd/pipeline/pkg/reconciler/pipelinerun/resources"
ttesting "github.com/tektoncd/pipeline/pkg/reconciler/testing"
"github.com/tektoncd/pipeline/pkg/trustedresources"
"github.com/tektoncd/pipeline/test"
"github.com/tektoncd/pipeline/test/diff"
Expand Down Expand Up @@ -89,6 +91,13 @@ var (
})
)

func getFakePipelineClient(t *testing.T) (context.Context, *clientset.Clientset) {
t.Helper()
ctx, _ := ttesting.SetupFakeContext(t)
clients, _ := test.SeedTestData(t, ctx, test.Data{})
return ctx, clients.Pipeline
}

func TestLocalPipelineRef(t *testing.T) {
testcases := []struct {
name string
Expand Down Expand Up @@ -267,7 +276,8 @@ func TestGetPipelineFuncSpecAlreadyFetched(t *testing.T) {
}

func TestGetPipelineFunc_RemoteResolution(t *testing.T) {
ctx := config.EnableStableAPIFields(context.Background())
ctx, _ := getFakePipelineClient(t)
ctx = config.EnableStableAPIFields(ctx)
cfg := config.FromContextOrDefaults(ctx)
ctx = config.ToContext(ctx, cfg)
pipelineRef := &v1.PipelineRef{ResolverRef: v1.ResolverRef{Resolver: "git"}}
Expand Down

0 comments on commit a5de06d

Please sign in to comment.