Skip to content

Commit

Permalink
[TEP-0089] - Implement Non-falsifiable provenance support
Browse files Browse the repository at this point in the history
Signed-off-by: pxp928 <parth.psu@gmail.com>
  • Loading branch information
pxp928 committed May 3, 2022
1 parent 025e1fd commit b482840
Show file tree
Hide file tree
Showing 18 changed files with 679 additions and 0 deletions.
28 changes: 28 additions & 0 deletions pkg/reconciler/taskrun/taskrun.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,20 @@ func (c *Reconciler) ReconcileKind(ctx context.Context, tr *v1beta1.TaskRun) pkg
// on the event to perform user facing initialisations, such has reset a CI check status
afterCondition := tr.Status.GetCondition(apis.ConditionSucceeded)
events.Emit(ctx, nil, afterCondition, tr)
} else if config.FromContextOrDefaults(ctx).FeatureFlags.EnableSpire {
var verified = false
if c.SpireClient != nil {
if err := c.SpireClient.VerifyStatusInternalAnnotation(ctx, tr, logger); err == nil {
verified = true
}
if !verified {
if tr.Status.Annotations == nil {
tr.Status.Annotations = map[string]string{}
}
tr.Status.Annotations[spire.NotVerifiedAnnotation] = "yes"
}
logger.Infof("taskrun verification status: %t with hash %v \n", verified, tr.Status.Annotations[spire.TaskRunStatusHashAnnotation])
}
}

// If the TaskRun is complete, run some post run fixtures when applicable
Expand Down Expand Up @@ -260,6 +274,20 @@ func (c *Reconciler) finishReconcileUpdateEmitEvents(ctx context.Context, tr *v1
events.Emit(ctx, beforeCondition, afterCondition, tr)

var err error
// Add status internal annotations hash only if it was verified
if config.FromContextOrDefaults(ctx).FeatureFlags.EnableSpire &&
c.SpireClient != nil && c.SpireClient.CheckSpireVerifiedFlag(tr) {
if err := spire.CheckStatusInternalAnnotation(tr); err != nil {
err = c.SpireClient.AppendStatusInternalAnnotation(ctx, tr)
if err != nil {
logger.Warn("Failed to sign TaskRun internal status hash", zap.Error(err))
events.EmitError(controller.GetEventRecorder(ctx), err, tr)
} else {
logger.Infof("Successfully signed TaskRun internal status with hash: %v",
tr.Status.Annotations[spire.TaskRunStatusHashAnnotation])
}
}
}

merr := multierror.Append(previousError, err).ErrorOrNil()

Expand Down
120 changes: 120 additions & 0 deletions pkg/reconciler/taskrun/taskrun_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import (
"github.com/tektoncd/pipeline/pkg/reconciler/taskrun/resources"
ttesting "github.com/tektoncd/pipeline/pkg/reconciler/testing"
"github.com/tektoncd/pipeline/pkg/reconciler/volumeclaim"
"github.com/tektoncd/pipeline/pkg/spire"
spireconfig "github.com/tektoncd/pipeline/pkg/spire/config"
"github.com/tektoncd/pipeline/test"
"github.com/tektoncd/pipeline/test/diff"
Expand All @@ -63,6 +64,7 @@ import (
ktesting "k8s.io/client-go/testing"
"k8s.io/client-go/tools/record"
"knative.dev/pkg/apis"
duckv1beta1 "knative.dev/pkg/apis/duck/v1beta1"
"knative.dev/pkg/changeset"
cminformer "knative.dev/pkg/configmap/informer"
"knative.dev/pkg/controller"
Expand Down Expand Up @@ -3800,6 +3802,124 @@ status:
}
}

func TestReconcileOnTaskRunSign(t *testing.T) {
sc := &spire.MockClient{}

taskSt := &apis.Condition{
Type: apis.ConditionSucceeded,
Status: corev1.ConditionTrue,
Reason: "Build succeeded",
Message: "Build succeeded",
}
taskRunStartedUnsigned := &v1beta1.TaskRun{
ObjectMeta: objectMeta("taskrun-started-unsigned", "foo"),
Spec: v1beta1.TaskRunSpec{
TaskRef: &v1beta1.TaskRef{
Name: simpleTask.Name,
},
},
Status: v1beta1.TaskRunStatus{
Status: duckv1beta1.Status{
Conditions: duckv1beta1.Conditions{
*taskSt,
},
},
TaskRunStatusFields: v1beta1.TaskRunStatusFields{
StartTime: &metav1.Time{Time: now.Add(-15 * time.Second)},
},
},
}
taskRunUnstarted := &v1beta1.TaskRun{
ObjectMeta: objectMeta("taskrun-unstarted", "foo"),
Spec: v1beta1.TaskRunSpec{
TaskRef: &v1beta1.TaskRef{
Name: simpleTask.Name,
},
},
Status: v1beta1.TaskRunStatus{
Status: duckv1beta1.Status{
Conditions: duckv1beta1.Conditions{
*taskSt,
},
},
},
}
taskRunStartedSigned := &v1beta1.TaskRun{
ObjectMeta: objectMeta("taskrun-started-signed", "foo"),
Spec: v1beta1.TaskRunSpec{
TaskRef: &v1beta1.TaskRef{
Name: simpleTask.Name,
},
},
Status: v1beta1.TaskRunStatus{
Status: duckv1beta1.Status{
Conditions: duckv1beta1.Conditions{
*taskSt,
},
},
},
}
if err := sc.AppendStatusInternalAnnotation(context.Background(), taskRunStartedSigned); err != nil {
t.Fatal("failed to sign test taskrun")
}

d := test.Data{
ConfigMaps: []*corev1.ConfigMap{{
ObjectMeta: metav1.ObjectMeta{Name: config.GetFeatureFlagsConfigName(), Namespace: system.Namespace()},
Data: map[string]string{
"enable-spire": "true",
},
}},

TaskRuns: []*v1beta1.TaskRun{
taskRunStartedUnsigned, taskRunUnstarted, taskRunStartedSigned,
},
Tasks: []*v1beta1.Task{simpleTask},
}
testAssets, cancel := getTaskRunController(t, d)
defer cancel()
c := testAssets.Controller
clients := testAssets.Clients

testCases := []struct {
name string
tr *v1beta1.TaskRun
verifiable bool
}{
{
name: "sign/verify unstarted taskrun",
tr: taskRunUnstarted,
verifiable: true,
},
{
name: "sign/verify signed started taskrun",
tr: taskRunStartedSigned,
verifiable: true,
},
{
name: "sign/verify unsigned started taskrun should fail",
tr: taskRunStartedUnsigned,
verifiable: false,
},
}

for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
if err := c.Reconciler.Reconcile(testAssets.Ctx, getRunName(tc.tr)); err != nil {
t.Fatalf("Unexpected error when reconciling completed TaskRun : %v", err)
}
newTr, err := clients.Pipeline.TektonV1beta1().TaskRuns(tc.tr.Namespace).Get(testAssets.Ctx, tc.tr.Name, metav1.GetOptions{})
if err != nil {
t.Fatalf("Expected completed TaskRun %s to exist but instead got error when getting it: %v", tc.tr.Name, err)
}
verified := sc.CheckSpireVerifiedFlag(newTr)
if verified != tc.verifiable {
t.Fatalf("expected verifiable: %v, got %v", tc.verifiable, verified)
}
})
}
}

func Test_validateTaskSpecRequestResources_ValidResources(t *testing.T) {
ctx := context.Background()

Expand Down
47 changes: 47 additions & 0 deletions pkg/spire/sign.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"crypto/sha256"
"encoding/base64"
"encoding/pem"
"fmt"
"strings"

"github.com/spiffe/go-spiffe/v2/svid/x509svid"
Expand Down Expand Up @@ -101,3 +102,49 @@ func getManifest(results []v1beta1.PipelineResourceResult) string {
}
return strings.Join(keys, ",")
}

// AppendStatusInternalAnnotation creates the status annotations which are used by the controller to verify the status hash
func (sc *spireControllerAPIClient) AppendStatusInternalAnnotation(ctx context.Context, tr *v1beta1.TaskRun) error {
err := sc.checkClient(ctx)
if err != nil {
return err
}

// Add status hash
currentHash, err := hashTaskrunStatusInternal(tr)
if err != nil {
return err
}

// Sign with controller private key
xsvid, err := sc.fetchSVID(ctx)
if err != nil {
return err
}

sig, err := signWithKey(xsvid, currentHash)
if err != nil {
return err
}

// Store Controller SVID
p := pem.EncodeToMemory(&pem.Block{
Bytes: xsvid.Certificates[0].Raw,
Type: "CERTIFICATE",
})
if tr.Status.Annotations == nil {
tr.Status.Annotations = map[string]string{}
}
tr.Status.Annotations[controllerSvidAnnotation] = string(p)
tr.Status.Annotations[TaskRunStatusHashAnnotation] = currentHash
tr.Status.Annotations[taskRunStatusHashSigAnnotation] = base64.StdEncoding.EncodeToString(sig)
return nil
}

func (sc *spireControllerAPIClient) fetchSVID(ctx context.Context) (*x509svid.SVID, error) {
xsvid, err := sc.workloadAPI.FetchX509SVID(ctx)
if err != nil {
return nil, fmt.Errorf("failed to fetch controller SVID: %w", err)
}
return xsvid, nil
}
12 changes: 12 additions & 0 deletions pkg/spire/spire.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,19 @@ import (
"context"

"github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1"
"go.uber.org/zap"
corev1 "k8s.io/api/core/v1"
)

const (
// TaskRunStatusHashAnnotation TaskRun status annotation Hash Key
TaskRunStatusHashAnnotation = "tekton.dev/status-hash"
// taskRunStatusHashSigAnnotation TaskRun status annotation hash signature Key
taskRunStatusHashSigAnnotation = "tekton.dev/status-hash-sig"
// controllerSvidAnnotation TaskRun status annotation controller SVID Key
controllerSvidAnnotation = "tekton.dev/controller-svid"
// NotVerifiedAnnotation TaskRun status annotation not verified by spire key that get set when status match fails
NotVerifiedAnnotation = "tekton.dev/not-verified"
// KeySVID key used by TaskRun SVID
KeySVID = "SVID"
// KeySignatureSuffix is the suffix of the keys that contain signatures
Expand All @@ -34,9 +43,12 @@ const (

// ControllerAPIClient interface maps to the spire controller API to interact with spire
type ControllerAPIClient interface {
AppendStatusInternalAnnotation(ctx context.Context, tr *v1beta1.TaskRun) error
CheckSpireVerifiedFlag(tr *v1beta1.TaskRun) bool
Close()
CreateEntries(ctx context.Context, tr *v1beta1.TaskRun, pod *corev1.Pod, ttl int) error
DeleteEntry(ctx context.Context, tr *v1beta1.TaskRun, pod *corev1.Pod) error
VerifyStatusInternalAnnotation(ctx context.Context, tr *v1beta1.TaskRun, logger *zap.SugaredLogger) error
VerifyTaskRunResults(ctx context.Context, prs []v1beta1.PipelineResourceResult, tr *v1beta1.TaskRun) error
}

Expand Down
88 changes: 88 additions & 0 deletions pkg/spire/spire_mock.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (

"github.com/pkg/errors"
"github.com/tektoncd/pipeline/pkg/apis/pipeline/v1beta1"
"go.uber.org/zap"
corev1 "k8s.io/api/core/v1"
)

Expand All @@ -43,16 +44,30 @@ type MockClient struct {

// VerifyAlwaysReturns defines whether to always verify successfully or to always fail verification if non-nil.
// This only take effect on Verify functions:
// - VerifyStatusInternalAnnotationOverride
// - VerifyTaskRunResultsOverride
VerifyAlwaysReturns *bool

// VerifyStatusInternalAnnotationOverride contains the function to overwrite a call to VerifyStatusInternalAnnotation
VerifyStatusInternalAnnotationOverride func(ctx context.Context, tr *v1beta1.TaskRun, logger *zap.SugaredLogger) error

// VerifyTaskRunResultsOverride contains the function to overwrite a call to VerifyTaskRunResults
VerifyTaskRunResultsOverride func(ctx context.Context, prs []v1beta1.PipelineResourceResult, tr *v1beta1.TaskRun) error

// AppendStatusInternalAnnotationOverride contains the function to overwrite a call to AppendStatusInternalAnnotation
AppendStatusInternalAnnotationOverride func(ctx context.Context, tr *v1beta1.TaskRun) error

// CheckSpireVerifiedFlagOverride contains the function to overwrite a call to CheckSpireVerifiedFlag
CheckSpireVerifiedFlagOverride func(tr *v1beta1.TaskRun) bool

// SignOverride contains the function to overwrite a call to Sign
SignOverride func(ctx context.Context, results []v1beta1.PipelineResourceResult) ([]v1beta1.PipelineResourceResult, error)
}

const (
controllerSvid = "CONTROLLER_SVID_DATA"
)

func (*MockClient) mockSign(content, signedBy string) string {
return fmt.Sprintf("signed-by-%s:%x", signedBy, sha256.Sum256([]byte(content)))
}
Expand All @@ -66,6 +81,38 @@ func (*MockClient) GetIdentity(tr *v1beta1.TaskRun) string {
return fmt.Sprintf("/ns/%v/taskrun/%v", tr.Namespace, tr.Name)
}

// AppendStatusInternalAnnotation creates the status annotations which are used by the controller to verify the status hash
func (sc *MockClient) AppendStatusInternalAnnotation(ctx context.Context, tr *v1beta1.TaskRun) error {
if sc.AppendStatusInternalAnnotationOverride != nil {
return sc.AppendStatusInternalAnnotationOverride(ctx, tr)
}
// Add status hash
currentHash, err := hashTaskrunStatusInternal(tr)
if err != nil {
return err
}

if tr.Status.Annotations == nil {
tr.Status.Annotations = map[string]string{}
}
tr.Status.Annotations[controllerSvidAnnotation] = controllerSvid
tr.Status.Annotations[TaskRunStatusHashAnnotation] = currentHash
tr.Status.Annotations[taskRunStatusHashSigAnnotation] = sc.mockSign(currentHash, "controller")
return nil
}

// CheckSpireVerifiedFlag checks if the not-verified status annotation is set which would result in spire verification failed
func (sc *MockClient) CheckSpireVerifiedFlag(tr *v1beta1.TaskRun) bool {
if sc.CheckSpireVerifiedFlagOverride != nil {
return sc.CheckSpireVerifiedFlagOverride(tr)
}

if _, notVerified := tr.Status.Annotations[NotVerifiedAnnotation]; !notVerified {
return true
}
return false
}

// CreateEntries adds entries to the dictionary of entries that mock the SPIRE server datastore
func (sc *MockClient) CreateEntries(ctx context.Context, tr *v1beta1.TaskRun, pod *corev1.Pod, ttl int) error {
id := fmt.Sprintf("/ns/%v/taskrun/%v", tr.Namespace, tr.Name)
Expand All @@ -85,6 +132,47 @@ func (sc *MockClient) DeleteEntry(ctx context.Context, tr *v1beta1.TaskRun, pod
return nil
}

// VerifyStatusInternalAnnotation checks that the internal status annotations are valid by the mocked spire client
func (sc *MockClient) VerifyStatusInternalAnnotation(ctx context.Context, tr *v1beta1.TaskRun, logger *zap.SugaredLogger) error {
if sc.VerifyStatusInternalAnnotationOverride != nil {
return sc.VerifyStatusInternalAnnotationOverride(ctx, tr, logger)
}

if sc.VerifyAlwaysReturns != nil {
if *sc.VerifyAlwaysReturns {
return nil
}
return errors.New("failed to verify from mock VerifyAlwaysReturns")
}

if !sc.CheckSpireVerifiedFlag(tr) {
return errors.New("annotation tekton.dev/not-verified = yes failed spire verification")
}

annotations := tr.Status.Annotations

// Verify annotations are there
if annotations[controllerSvidAnnotation] != controllerSvid {
return errors.New("svid annotation missing")
}

// Check signature
currentHash, err := hashTaskrunStatusInternal(tr)
if err != nil {
return err
}
if !sc.mockVerify(currentHash, annotations[taskRunStatusHashSigAnnotation], "controller") {
return errors.New("signature was not able to be verified")
}

// check current status hash vs annotation status hash by controller
if err := CheckStatusInternalAnnotation(tr); err != nil {
return err
}

return nil
}

// VerifyTaskRunResults checks that all the TaskRun results are valid by the mocked spire client
func (sc *MockClient) VerifyTaskRunResults(ctx context.Context, prs []v1beta1.PipelineResourceResult, tr *v1beta1.TaskRun) error {
if sc.VerifyTaskRunResultsOverride != nil {
Expand Down
Loading

0 comments on commit b482840

Please sign in to comment.