Merge pull request #1001 from fluxcd/artifact-digest

RFC-0005: introduction of Digest and change of Revision format
This commit is contained in:
Hidde Beydals 2023-02-14 14:42:21 +01:00 committed by GitHub
commit e24ce868d1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
39 changed files with 2487 additions and 1248 deletions

View File

@ -18,6 +18,7 @@ package v1beta2
import (
"path"
"regexp"
"strings"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
@ -43,8 +44,14 @@ type Artifact struct {
Revision string `json:"revision"`
// Checksum is the SHA256 checksum of the Artifact file.
// Deprecated: use Artifact.Digest instead.
// +optional
Checksum string `json:"checksum"`
Checksum string `json:"checksum,omitempty"`
// Digest is the digest of the file in the form of '<algorithm>:<checksum>'.
// +optional
// +kubebuilder:validation:Pattern="^[a-z0-9]+(?:[.+_-][a-z0-9]+)*:[a-zA-Z0-9=_-]+$"
Digest string `json:"digest"`
// LastUpdateTime is the timestamp corresponding to the last update of the
// Artifact.
@ -66,7 +73,7 @@ func (in *Artifact) HasRevision(revision string) bool {
if in == nil {
return false
}
return in.Revision == revision
return TransformLegacyRevision(in.Revision) == TransformLegacyRevision(revision)
}
// HasChecksum returns if the given checksum matches the current Checksum of
@ -90,3 +97,60 @@ func ArtifactDir(kind, namespace, name string) string {
func ArtifactPath(kind, namespace, name, filename string) string {
return path.Join(ArtifactDir(kind, namespace, name), filename)
}
// TransformLegacyRevision transforms a "legacy" revision string into a "new"
// revision string. It accepts the following formats:
//
// - main/5394cb7f48332b2de7c17dd8b8384bbc84b7e738
// - feature/branch/5394cb7f48332b2de7c17dd8b8384bbc84b7e738
// - HEAD/5394cb7f48332b2de7c17dd8b8384bbc84b7e738
// - tag/55609ff9d959589ed917ce32e6bc0f0a36809565f308602c15c3668965979edc
// - d52bde83c5b2bd0fa7910264e0afc3ac9cfe9b6636ca29c05c09742f01d5a4bd
//
// Which are transformed into the following formats respectively:
//
// - main@sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738
// - feature/branch@sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738
// - sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738
// - tag@sha256:55609ff9d959589ed917ce32e6bc0f0a36809565f308602c15c3668965979edc
// - sha256:d52bde83c5b2bd0fa7910264e0afc3ac9cfe9b6636ca29c05c09742f01d5a4bd
//
// Deprecated, this function exists for backwards compatibility with existing
// resources, and to provide a transition period. Will be removed in a future
// release.
func TransformLegacyRevision(rev string) string {
if rev != "" && strings.LastIndex(rev, ":") == -1 {
if i := strings.LastIndex(rev, "/"); i >= 0 {
sha := rev[i+1:]
if algo := determineSHAType(sha); algo != "" {
if name := rev[:i]; name != "HEAD" {
return name + "@" + algo + ":" + sha
}
return algo + ":" + sha
}
}
if algo := determineSHAType(rev); algo != "" {
return algo + ":" + rev
}
}
return rev
}
// isAlphaNumHex returns true if the given string only contains 0-9 and a-f
// characters.
var isAlphaNumHex = regexp.MustCompile(`^[0-9a-f]+$`).MatchString
// determineSHAType returns the SHA algorithm used to compute the provided hex.
// The determination is heuristic and based on the length of the hex string. If
// the size is not recognized, an empty string is returned.
func determineSHAType(hex string) string {
if isAlphaNumHex(hex) {
switch len(hex) {
case 40:
return "sha1"
case 64:
return "sha256"
}
}
return ""
}

View File

@ -0,0 +1,78 @@
/*
Copyright 2023 The Flux 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 v1beta2
import "testing"
func TestTransformLegacyRevision(t *testing.T) {
tests := []struct {
rev string
want string
}{
{
rev: "HEAD/5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
want: "sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
},
{
rev: "main/5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
want: "main@sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
},
{
rev: "main@sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
want: "main@sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
},
{
rev: "feature/branch/5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
want: "feature/branch@sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
},
{
rev: "feature/branch@sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
want: "feature/branch@sha1:5394cb7f48332b2de7c17dd8b8384bbc84b7e738",
},
{
rev: "5ac85ca617f3774baff4ae0a420b810b2546dbc9af9f346b1d55c5ed9873c55c",
want: "sha256:5ac85ca617f3774baff4ae0a420b810b2546dbc9af9f346b1d55c5ed9873c55c",
},
{
rev: "v1.0.0",
want: "v1.0.0",
},
{
rev: "v1.0.0-rc1",
want: "v1.0.0-rc1",
},
{
rev: "v1.0.0-rc1+metadata",
want: "v1.0.0-rc1+metadata",
},
{
rev: "arbitrary/revision",
want: "arbitrary/revision",
},
{
rev: "5394cb7f48332b2de7c17dd8b8384bbc84b7xxxx",
want: "5394cb7f48332b2de7c17dd8b8384bbc84b7xxxx",
},
}
for _, tt := range tests {
t.Run(tt.rev, func(t *testing.T) {
if got := TransformLegacyRevision(tt.rev); got != tt.want {
t.Errorf("TransformLegacyRevision() = %v, want %v", got, tt.want)
}
})
}
}

View File

@ -377,7 +377,12 @@ spec:
description: Artifact represents the last successful Bucket reconciliation.
properties:
checksum:
description: Checksum is the SHA256 checksum of the Artifact file.
description: 'Checksum is the SHA256 checksum of the Artifact
file. Deprecated: use Artifact.Digest instead.'
type: string
digest:
description: Digest is the digest of the file in the form of '<algorithm>:<checksum>'.
pattern: ^[a-z0-9]+(?:[.+_-][a-z0-9]+)*:[a-zA-Z0-9=_-]+$
type: string
lastUpdateTime:
description: LastUpdateTime is the timestamp corresponding to

View File

@ -554,7 +554,12 @@ spec:
reconciliation.
properties:
checksum:
description: Checksum is the SHA256 checksum of the Artifact file.
description: 'Checksum is the SHA256 checksum of the Artifact
file. Deprecated: use Artifact.Digest instead.'
type: string
digest:
description: Digest is the digest of the file in the form of '<algorithm>:<checksum>'.
pattern: ^[a-z0-9]+(?:[.+_-][a-z0-9]+)*:[a-zA-Z0-9=_-]+$
type: string
lastUpdateTime:
description: LastUpdateTime is the timestamp corresponding to
@ -676,8 +681,13 @@ spec:
description: Artifact represents the output of a Source reconciliation.
properties:
checksum:
description: Checksum is the SHA256 checksum of the Artifact
file.
description: 'Checksum is the SHA256 checksum of the Artifact
file. Deprecated: use Artifact.Digest instead.'
type: string
digest:
description: Digest is the digest of the file in the form of
'<algorithm>:<checksum>'.
pattern: ^[a-z0-9]+(?:[.+_-][a-z0-9]+)*:[a-zA-Z0-9=_-]+$
type: string
lastUpdateTime:
description: LastUpdateTime is the timestamp corresponding to

View File

@ -452,7 +452,12 @@ spec:
reconciliation.
properties:
checksum:
description: Checksum is the SHA256 checksum of the Artifact file.
description: 'Checksum is the SHA256 checksum of the Artifact
file. Deprecated: use Artifact.Digest instead.'
type: string
digest:
description: Digest is the digest of the file in the form of '<algorithm>:<checksum>'.
pattern: ^[a-z0-9]+(?:[.+_-][a-z0-9]+)*:[a-zA-Z0-9=_-]+$
type: string
lastUpdateTime:
description: LastUpdateTime is the timestamp corresponding to

View File

@ -369,7 +369,12 @@ spec:
reconciliation.
properties:
checksum:
description: Checksum is the SHA256 checksum of the Artifact file.
description: 'Checksum is the SHA256 checksum of the Artifact
file. Deprecated: use Artifact.Digest instead.'
type: string
digest:
description: Digest is the digest of the file in the form of '<algorithm>:<checksum>'.
pattern: ^[a-z0-9]+(?:[.+_-][a-z0-9]+)*:[a-zA-Z0-9=_-]+$
type: string
lastUpdateTime:
description: LastUpdateTime is the timestamp corresponding to

View File

@ -195,7 +195,12 @@ spec:
OCI Repository sync.
properties:
checksum:
description: Checksum is the SHA256 checksum of the Artifact file.
description: 'Checksum is the SHA256 checksum of the Artifact
file. Deprecated: use Artifact.Digest instead.'
type: string
digest:
description: Digest is the digest of the file in the form of '<algorithm>:<checksum>'.
pattern: ^[a-z0-9]+(?:[.+_-][a-z0-9]+)*:[a-zA-Z0-9=_-]+$
type: string
lastUpdateTime:
description: LastUpdateTime is the timestamp corresponding to

View File

@ -37,25 +37,3 @@ outer:
}
return false
}
// hasArtifactUpdated returns true if any of the revisions in the current artifacts
// does not match any of the artifacts in the updated artifacts
// NOTE: artifactSet is a replacement for this. Remove this once it's not used
// anywhere.
func hasArtifactUpdated(current []*sourcev1.Artifact, updated []*sourcev1.Artifact) bool {
if len(current) != len(updated) {
return true
}
OUTER:
for _, c := range current {
for _, u := range updated {
if u.HasRevision(c.Revision) {
continue OUTER
}
}
return true
}
return false
}

View File

@ -18,17 +18,14 @@ package controllers
import (
"context"
"crypto/sha256"
"errors"
"fmt"
"os"
"path/filepath"
"sort"
"strings"
"sync"
"time"
"github.com/fluxcd/source-controller/pkg/azure"
"github.com/opencontainers/go-digest"
"golang.org/x/sync/errgroup"
"golang.org/x/sync/semaphore"
corev1 "k8s.io/api/core/v1"
@ -51,10 +48,14 @@ import (
eventv1 "github.com/fluxcd/pkg/apis/event/v1beta1"
"github.com/fluxcd/pkg/sourceignore"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
intdigest "github.com/fluxcd/source-controller/internal/digest"
serror "github.com/fluxcd/source-controller/internal/error"
"github.com/fluxcd/source-controller/internal/index"
sreconcile "github.com/fluxcd/source-controller/internal/reconcile"
"github.com/fluxcd/source-controller/internal/reconcile/summarize"
"github.com/fluxcd/source-controller/pkg/azure"
"github.com/fluxcd/source-controller/pkg/gcp"
"github.com/fluxcd/source-controller/pkg/minio"
)
@ -154,83 +155,7 @@ type BucketProvider interface {
// bucketReconcileFunc is the function type for all the v1beta2.Bucket
// (sub)reconcile functions. The type implementations are grouped and
// executed serially to perform the complete reconcile of the object.
type bucketReconcileFunc func(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.Bucket, index *etagIndex, dir string) (sreconcile.Result, error)
// etagIndex is an index of storage object keys and their Etag values.
type etagIndex struct {
sync.RWMutex
index map[string]string
}
// newEtagIndex returns a new etagIndex with an empty initialized index.
func newEtagIndex() *etagIndex {
return &etagIndex{
index: make(map[string]string),
}
}
func (i *etagIndex) Add(key, etag string) {
i.Lock()
defer i.Unlock()
i.index[key] = etag
}
func (i *etagIndex) Delete(key string) {
i.Lock()
defer i.Unlock()
delete(i.index, key)
}
func (i *etagIndex) Get(key string) string {
i.RLock()
defer i.RUnlock()
return i.index[key]
}
func (i *etagIndex) Has(key string) bool {
i.RLock()
defer i.RUnlock()
_, ok := i.index[key]
return ok
}
func (i *etagIndex) Index() map[string]string {
i.RLock()
defer i.RUnlock()
index := make(map[string]string)
for k, v := range i.index {
index[k] = v
}
return index
}
func (i *etagIndex) Len() int {
i.RLock()
defer i.RUnlock()
return len(i.index)
}
// Revision calculates the SHA256 checksum of the index.
// The keys are stable sorted, and the SHA256 sum is then calculated for the
// string representation of the key/value pairs, each pair written on a newline
// with a space between them. The sum result is returned as a string.
func (i *etagIndex) Revision() (string, error) {
i.RLock()
defer i.RUnlock()
keyIndex := make([]string, 0, len(i.index))
for k := range i.index {
keyIndex = append(keyIndex, k)
}
sort.Strings(keyIndex)
sum := sha256.New()
for _, k := range keyIndex {
if _, err := sum.Write([]byte(fmt.Sprintf("%s %s\n", k, i.index[k]))); err != nil {
return "", err
}
}
return fmt.Sprintf("%x", sum.Sum(nil)), nil
}
type bucketReconcileFunc func(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.Bucket, index *index.Digester, dir string) (sreconcile.Result, error)
func (r *BucketReconciler) SetupWithManager(mgr ctrl.Manager) error {
return r.SetupWithManagerAndOptions(mgr, BucketReconcilerOptions{})
@ -371,7 +296,7 @@ func (r *BucketReconciler) reconcile(ctx context.Context, sp *patch.SerialPatche
var (
res sreconcile.Result
resErr error
index = newEtagIndex()
index = index.NewDigester()
)
for _, rec := range reconcilers {
@ -397,7 +322,7 @@ func (r *BucketReconciler) reconcile(ctx context.Context, sp *patch.SerialPatche
}
// notify emits notification related to the reconciliation.
func (r *BucketReconciler) notify(ctx context.Context, oldObj, newObj *sourcev1.Bucket, index *etagIndex, res sreconcile.Result, resErr error) {
func (r *BucketReconciler) notify(ctx context.Context, oldObj, newObj *sourcev1.Bucket, index *index.Digester, res sreconcile.Result, resErr error) {
// Notify successful reconciliation for new artifact and recovery from any
// failure.
if resErr == nil && res == sreconcile.ResultSuccess && newObj.Status.Artifact != nil {
@ -405,6 +330,9 @@ func (r *BucketReconciler) notify(ctx context.Context, oldObj, newObj *sourcev1.
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaRevisionKey): newObj.Status.Artifact.Revision,
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaChecksumKey): newObj.Status.Artifact.Checksum,
}
if newObj.Status.Artifact.Digest != "" {
annotations[sourcev1.GroupVersion.Group+"/"+eventv1.MetaDigestKey] = newObj.Status.Artifact.Digest
}
var oldChecksum string
if oldObj.GetArtifact() != nil {
@ -440,7 +368,7 @@ func (r *BucketReconciler) notify(ctx context.Context, oldObj, newObj *sourcev1.
// condition is added.
// The hostname of any URL in the Status of the object are updated, to ensure
// they match the Storage server hostname of current runtime.
func (r *BucketReconciler) reconcileStorage(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.Bucket, _ *etagIndex, _ string) (sreconcile.Result, error) {
func (r *BucketReconciler) reconcileStorage(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.Bucket, _ *index.Digester, _ string) (sreconcile.Result, error) {
// Garbage collect previous advertised artifact(s) from storage
_ = r.garbageCollect(ctx, obj)
@ -481,7 +409,7 @@ func (r *BucketReconciler) reconcileStorage(ctx context.Context, sp *patch.Seria
// When a SecretRef is defined, it attempts to fetch the Secret before calling
// the provider. If this fails, it records v1beta2.FetchFailedCondition=True on
// the object and returns early.
func (r *BucketReconciler) reconcileSource(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.Bucket, index *etagIndex, dir string) (sreconcile.Result, error) {
func (r *BucketReconciler) reconcileSource(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.Bucket, index *index.Digester, dir string) (sreconcile.Result, error) {
secret, err := r.getBucketSecret(ctx, obj)
if err != nil {
e := &serror.Event{Err: err, Reason: sourcev1.AuthenticationFailedReason}
@ -535,26 +463,21 @@ func (r *BucketReconciler) reconcileSource(ctx context.Context, sp *patch.Serial
return sreconcile.ResultEmpty, e
}
// Calculate revision
revision, err := index.Revision()
if err != nil {
return sreconcile.ResultEmpty, &serror.Event{
Err: fmt.Errorf("failed to calculate revision: %w", err),
Reason: meta.FailedReason,
}
// Check if index has changed compared to current Artifact revision.
var changed bool
if artifact := obj.Status.Artifact; artifact != nil && artifact.Revision != "" {
curRev := digest.Digest(sourcev1.TransformLegacyRevision(artifact.Revision))
changed = curRev.Validate() != nil || curRev != index.Digest(curRev.Algorithm())
}
// Mark observations about the revision on the object
defer func() {
// As fetchIndexFiles can make last-minute modifications to the etag
// index, we need to re-calculate the revision at the end
revision, err := index.Revision()
if err != nil {
ctrl.LoggerFrom(ctx).Error(err, "failed to calculate revision after fetching etag index")
return
}
// Fetch the bucket objects if required to.
if artifact := obj.GetArtifact(); artifact == nil || changed {
// Mark observations about the revision on the object
defer func() {
// As fetchIndexFiles can make last-minute modifications to the etag
// index, we need to re-calculate the revision at the end
revision := index.Digest(intdigest.Canonical)
if !obj.GetArtifact().HasRevision(revision) {
message := fmt.Sprintf("new upstream revision '%s'", revision)
if obj.GetArtifact() != nil {
conditions.MarkTrue(obj, sourcev1.ArtifactOutdatedCondition, "NewRevision", message)
@ -564,10 +487,8 @@ func (r *BucketReconciler) reconcileSource(ctx context.Context, sp *patch.Serial
ctrl.LoggerFrom(ctx).Error(err, "failed to patch")
return
}
}
}()
}()
if !obj.GetArtifact().HasRevision(revision) {
if err = fetchIndexFiles(ctx, provider, obj, index, dir); err != nil {
e := &serror.Event{Err: err, Reason: sourcev1.BucketOperationFailedReason}
conditions.MarkTrue(obj, sourcev1.FetchFailedCondition, e.Reason, e.Error())
@ -588,32 +509,32 @@ func (r *BucketReconciler) reconcileSource(ctx context.Context, sp *patch.Serial
// early.
// On a successful archive, the Artifact in the Status of the object is set,
// and the symlink in the Storage is updated to its path.
func (r *BucketReconciler) reconcileArtifact(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.Bucket, index *etagIndex, dir string) (sreconcile.Result, error) {
func (r *BucketReconciler) reconcileArtifact(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.Bucket, index *index.Digester, dir string) (sreconcile.Result, error) {
// Calculate revision
revision, err := index.Revision()
if err != nil {
return sreconcile.ResultEmpty, &serror.Event{
Err: fmt.Errorf("failed to calculate revision of new artifact: %w", err),
Reason: meta.FailedReason,
}
}
revision := index.Digest(intdigest.Canonical)
// Create artifact
artifact := r.Storage.NewArtifactFor(obj.Kind, obj, revision, fmt.Sprintf("%s.tar.gz", revision))
artifact := r.Storage.NewArtifactFor(obj.Kind, obj, revision.String(), fmt.Sprintf("%s.tar.gz", revision.Encoded()))
// Set the ArtifactInStorageCondition if there's no drift.
defer func() {
if obj.GetArtifact().HasRevision(artifact.Revision) {
conditions.Delete(obj, sourcev1.ArtifactOutdatedCondition)
conditions.MarkTrue(obj, sourcev1.ArtifactInStorageCondition, meta.SucceededReason,
"stored artifact: revision '%s'", artifact.Revision)
if curArtifact := obj.GetArtifact(); curArtifact != nil && curArtifact.Revision != "" {
curRev := digest.Digest(sourcev1.TransformLegacyRevision(curArtifact.Revision))
if curRev.Validate() == nil && index.Digest(curRev.Algorithm()) == curRev {
conditions.Delete(obj, sourcev1.ArtifactOutdatedCondition)
conditions.MarkTrue(obj, sourcev1.ArtifactInStorageCondition, meta.SucceededReason,
"stored artifact: revision '%s'", artifact.Revision)
}
}
}()
// The artifact is up-to-date
if obj.GetArtifact().HasRevision(artifact.Revision) {
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.ArtifactUpToDateReason, "artifact up-to-date with remote revision: '%s'", artifact.Revision)
return sreconcile.ResultSuccess, nil
if curArtifact := obj.GetArtifact(); curArtifact != nil && curArtifact.Revision != "" {
curRev := digest.Digest(sourcev1.TransformLegacyRevision(curArtifact.Revision))
if curRev.Validate() == nil && index.Digest(curRev.Algorithm()) == curRev {
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.ArtifactUpToDateReason, "artifact up-to-date with remote revision: '%s'", artifact.Revision)
return sreconcile.ResultSuccess, nil
}
}
// Ensure target path exists and is a directory
@ -778,7 +699,7 @@ func (r *BucketReconciler) annotatedEventLogf(ctx context.Context,
// bucket using the given provider, while filtering them using .sourceignore
// rules. After fetching an object, the etag value in the index is updated to
// the current value to ensure accuracy.
func fetchEtagIndex(ctx context.Context, provider BucketProvider, obj *sourcev1.Bucket, index *etagIndex, tempDir string) error {
func fetchEtagIndex(ctx context.Context, provider BucketProvider, obj *sourcev1.Bucket, index *index.Digester, tempDir string) error {
ctxTimeout, cancel := context.WithTimeout(ctx, obj.Spec.Timeout.Duration)
defer cancel()
@ -832,7 +753,7 @@ func fetchEtagIndex(ctx context.Context, provider BucketProvider, obj *sourcev1.
// using the given provider, and stores them into tempDir. It downloads in
// parallel, but limited to the maxConcurrentBucketFetches.
// Given an index is provided, the bucket is assumed to exist.
func fetchIndexFiles(ctx context.Context, provider BucketProvider, obj *sourcev1.Bucket, index *etagIndex, tempDir string) error {
func fetchIndexFiles(ctx context.Context, provider BucketProvider, obj *sourcev1.Bucket, index *index.Digester, tempDir string) error {
ctxTimeout, cancel := context.WithTimeout(ctx, obj.Spec.Timeout.Duration)
defer cancel()

View File

@ -28,6 +28,7 @@ import (
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
"github.com/fluxcd/source-controller/internal/index"
)
type mockBucketObject struct {
@ -88,8 +89,8 @@ func (m *mockBucketClient) addObject(key string, object mockBucketObject) {
m.objects[key] = object
}
func (m *mockBucketClient) objectsToEtagIndex() *etagIndex {
i := newEtagIndex()
func (m *mockBucketClient) objectsToDigestIndex() *index.Digester {
i := index.NewDigester()
for k, v := range m.objects {
i.Add(k, v.etag)
}
@ -114,7 +115,7 @@ func Test_fetchEtagIndex(t *testing.T) {
client.addObject("bar.yaml", mockBucketObject{data: "bar.yaml", etag: "etag2"})
client.addObject("baz.yaml", mockBucketObject{data: "baz.yaml", etag: "etag3"})
index := newEtagIndex()
index := index.NewDigester()
err := fetchEtagIndex(context.TODO(), client, bucket.DeepCopy(), index, tmp)
if err != nil {
t.Fatal(err)
@ -128,7 +129,7 @@ func Test_fetchEtagIndex(t *testing.T) {
client := mockBucketClient{bucketName: "other-bucket-name"}
index := newEtagIndex()
index := index.NewDigester()
err := fetchEtagIndex(context.TODO(), client, bucket.DeepCopy(), index, tmp)
assert.ErrorContains(t, err, "not found")
})
@ -141,7 +142,7 @@ func Test_fetchEtagIndex(t *testing.T) {
client.addObject("foo.yaml", mockBucketObject{etag: "etag1", data: "foo.yaml"})
client.addObject("foo.txt", mockBucketObject{etag: "etag2", data: "foo.txt"})
index := newEtagIndex()
index := index.NewDigester()
err := fetchEtagIndex(context.TODO(), client, bucket.DeepCopy(), index, tmp)
if err != nil {
t.Fatal(err)
@ -168,7 +169,7 @@ func Test_fetchEtagIndex(t *testing.T) {
bucket := bucket.DeepCopy()
bucket.Spec.Ignore = &ignore
index := newEtagIndex()
index := index.NewDigester()
err := fetchEtagIndex(context.TODO(), client, bucket.DeepCopy(), index, tmp)
if err != nil {
t.Fatal(err)
@ -203,7 +204,7 @@ func Test_fetchFiles(t *testing.T) {
client.addObject("bar.yaml", mockBucketObject{data: "bar.yaml", etag: "etag2"})
client.addObject("baz.yaml", mockBucketObject{data: "baz.yaml", etag: "etag3"})
index := client.objectsToEtagIndex()
index := client.objectsToDigestIndex()
err := fetchIndexFiles(context.TODO(), client, bucket.DeepCopy(), index, tmp)
if err != nil {
@ -225,7 +226,7 @@ func Test_fetchFiles(t *testing.T) {
client := mockBucketClient{bucketName: bucketName, objects: map[string]mockBucketObject{}}
client.objects["error"] = mockBucketObject{}
err := fetchIndexFiles(context.TODO(), client, bucket.DeepCopy(), client.objectsToEtagIndex(), tmp)
err := fetchIndexFiles(context.TODO(), client, bucket.DeepCopy(), client.objectsToDigestIndex(), tmp)
if err == nil {
t.Fatal("expected error but got nil")
}
@ -237,7 +238,7 @@ func Test_fetchFiles(t *testing.T) {
client := mockBucketClient{bucketName: bucketName}
client.addObject("foo.yaml", mockBucketObject{data: "foo.yaml", etag: "etag2"})
index := newEtagIndex()
index := index.NewDigester()
index.Add("foo.yaml", "etag1")
err := fetchIndexFiles(context.TODO(), client, bucket.DeepCopy(), index, tmp)
if err != nil {
@ -253,7 +254,7 @@ func Test_fetchFiles(t *testing.T) {
client := mockBucketClient{bucketName: bucketName}
client.addObject("foo.yaml", mockBucketObject{data: "foo.yaml", etag: "etag1"})
index := newEtagIndex()
index := index.NewDigester()
index.Add("foo.yaml", "etag1")
// Does not exist on server
index.Add("bar.yaml", "etag2")
@ -276,7 +277,7 @@ func Test_fetchFiles(t *testing.T) {
f := fmt.Sprintf("file-%d", i)
client.addObject(f, mockBucketObject{etag: f, data: f})
}
index := client.objectsToEtagIndex()
index := client.objectsToDigestIndex()
err := fetchIndexFiles(context.TODO(), client, bucket.DeepCopy(), index, tmp)
if err != nil {

View File

@ -43,6 +43,8 @@ import (
"github.com/fluxcd/pkg/runtime/patch"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
intdigest "github.com/fluxcd/source-controller/internal/digest"
"github.com/fluxcd/source-controller/internal/index"
gcsmock "github.com/fluxcd/source-controller/internal/mock/gcs"
s3mock "github.com/fluxcd/source-controller/internal/mock/s3"
sreconcile "github.com/fluxcd/source-controller/internal/reconcile"
@ -297,7 +299,7 @@ func TestBucketReconciler_reconcileStorage(t *testing.T) {
g.Expect(r.Client.Delete(context.TODO(), obj)).ToNot(HaveOccurred())
}()
index := newEtagIndex()
index := index.NewDigester()
sp := patch.NewSerialPatcher(obj, r.Client)
got, err := r.reconcileStorage(context.TODO(), sp, obj, index, "")
@ -336,7 +338,7 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
beforeFunc func(obj *sourcev1.Bucket)
want sreconcile.Result
wantErr bool
assertIndex *etagIndex
assertIndex *index.Digester
assertConditions []metav1.Condition
}{
{
@ -351,14 +353,12 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
},
},
// TODO(hidde): middleware for mock server
@ -377,7 +377,7 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
},
wantErr: true,
assertIndex: newEtagIndex(),
assertIndex: index.NewDigester(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.FetchFailedCondition, sourcev1.AuthenticationFailedReason, "failed to get secret '/dummy': secrets \"dummy\" not found"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
@ -400,7 +400,7 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
},
wantErr: true,
assertIndex: newEtagIndex(),
assertIndex: index.NewDigester(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.FetchFailedCondition, sourcev1.AuthenticationFailedReason, "invalid 'dummy' secret data: required fields 'accesskey' and 'secretkey'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
@ -416,7 +416,7 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
},
wantErr: true,
assertIndex: newEtagIndex(),
assertIndex: index.NewDigester(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.FetchFailedCondition, sourcev1.BucketOperationFailedReason, "bucket 'invalid' not found"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
@ -432,7 +432,7 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
},
wantErr: true,
assertIndex: newEtagIndex(),
assertIndex: index.NewDigester(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.FetchFailedCondition, sourcev1.BucketOperationFailedReason, "failed to confirm existence of 'unavailable' bucket"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
@ -463,14 +463,12 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"included/file.txt": "5a4bc7048b3301f677fe15b8678be2f8",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"included/file.txt": "5a4bc7048b3301f677fe15b8678be2f8",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision '9fc2ddfc4a6f44e6c3efee40af36578b9e76d4d930eaf384b8435a0aa0bf7a0f'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision '9fc2ddfc4a6f44e6c3efee40af36578b9e76d4d930eaf384b8435a0aa0bf7a0f'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:9fc2ddfc4a6f44e6c3efee40af36578b9e76d4d930eaf384b8435a0aa0bf7a0f'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:9fc2ddfc4a6f44e6c3efee40af36578b9e76d4d930eaf384b8435a0aa0bf7a0f'"),
},
},
{
@ -501,15 +499,13 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"ignored/file.txt": "f08907038338288420ae7dc2d30c0497",
"included/file.txt": "5a4bc7048b3301f677fe15b8678be2f8",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"ignored/file.txt": "f08907038338288420ae7dc2d30c0497",
"included/file.txt": "5a4bc7048b3301f677fe15b8678be2f8",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision '117f586dc64cfc559329e21d286edcbb94cb6b1581517eaddc0ab5292b470cd5'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision '117f586dc64cfc559329e21d286edcbb94cb6b1581517eaddc0ab5292b470cd5'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:117f586dc64cfc559329e21d286edcbb94cb6b1581517eaddc0ab5292b470cd5'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:117f586dc64cfc559329e21d286edcbb94cb6b1581517eaddc0ab5292b470cd5'"),
},
},
{
@ -531,11 +527,9 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
@ -556,14 +550,12 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
},
},
{
@ -584,15 +576,13 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
}
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactOutdatedCondition, "NewRevision", "new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(sourcev1.ArtifactOutdatedCondition, "NewRevision", "new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
},
},
}
@ -650,7 +640,7 @@ func TestBucketReconciler_reconcileSource_generic(t *testing.T) {
g.Expect(r.Client.Delete(context.TODO(), obj)).ToNot(HaveOccurred())
}()
index := newEtagIndex()
index := index.NewDigester()
sp := patch.NewSerialPatcher(obj, r.Client)
got, err := r.reconcileSource(context.TODO(), sp, obj, index, tmpDir)
@ -676,7 +666,7 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
beforeFunc func(obj *sourcev1.Bucket)
want sreconcile.Result
wantErr bool
assertIndex *etagIndex
assertIndex *index.Digester
assertConditions []metav1.Condition
}{
{
@ -706,14 +696,12 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
}
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
},
},
{
@ -728,7 +716,7 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
},
want: sreconcile.ResultEmpty,
wantErr: true,
assertIndex: newEtagIndex(),
assertIndex: index.NewDigester(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.FetchFailedCondition, sourcev1.AuthenticationFailedReason, "failed to get secret '/dummy': secrets \"dummy\" not found"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
@ -752,7 +740,7 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
},
want: sreconcile.ResultEmpty,
wantErr: true,
assertIndex: newEtagIndex(),
assertIndex: index.NewDigester(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.FetchFailedCondition, sourcev1.AuthenticationFailedReason, "invalid 'dummy' secret data: required fields"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
@ -769,7 +757,7 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
},
want: sreconcile.ResultEmpty,
wantErr: true,
assertIndex: newEtagIndex(),
assertIndex: index.NewDigester(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.FetchFailedCondition, sourcev1.BucketOperationFailedReason, "bucket 'invalid' not found"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
@ -786,7 +774,7 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
},
want: sreconcile.ResultEmpty,
wantErr: true,
assertIndex: newEtagIndex(),
assertIndex: index.NewDigester(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.FetchFailedCondition, sourcev1.BucketOperationFailedReason, "failed to confirm existence of 'unavailable' bucket"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
@ -817,14 +805,12 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"included/file.txt": "5a4bc7048b3301f677fe15b8678be2f8",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"included/file.txt": "5a4bc7048b3301f677fe15b8678be2f8",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision '9fc2ddfc4a6f44e6c3efee40af36578b9e76d4d930eaf384b8435a0aa0bf7a0f'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision '9fc2ddfc4a6f44e6c3efee40af36578b9e76d4d930eaf384b8435a0aa0bf7a0f'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:9fc2ddfc4a6f44e6c3efee40af36578b9e76d4d930eaf384b8435a0aa0bf7a0f'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:9fc2ddfc4a6f44e6c3efee40af36578b9e76d4d930eaf384b8435a0aa0bf7a0f'"),
},
},
{
@ -855,15 +841,13 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"ignored/file.txt": "f08907038338288420ae7dc2d30c0497",
"included/file.txt": "5a4bc7048b3301f677fe15b8678be2f8",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"ignored/file.txt": "f08907038338288420ae7dc2d30c0497",
"included/file.txt": "5a4bc7048b3301f677fe15b8678be2f8",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision '117f586dc64cfc559329e21d286edcbb94cb6b1581517eaddc0ab5292b470cd5'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision '117f586dc64cfc559329e21d286edcbb94cb6b1581517eaddc0ab5292b470cd5'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:117f586dc64cfc559329e21d286edcbb94cb6b1581517eaddc0ab5292b470cd5'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:117f586dc64cfc559329e21d286edcbb94cb6b1581517eaddc0ab5292b470cd5'"),
},
},
{
@ -885,11 +869,9 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
@ -910,14 +892,12 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
},
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
},
},
{
@ -938,15 +918,13 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
}
},
want: sreconcile.ResultSuccess,
assertIndex: &etagIndex{
index: map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
},
},
assertIndex: index.NewDigester(index.WithIndex(map[string]string{
"test.txt": "098f6bcd4621d373cade4e832627b4f6",
})),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactOutdatedCondition, "NewRevision", "new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(sourcev1.ArtifactOutdatedCondition, "NewRevision", "new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'sha256:b4c2a60ce44b67f5b659a95ce4e4cc9e2a86baf13afb72bd397c5384cbc0e479'"),
},
},
// TODO: Middleware for mock server to test authentication using secret.
@ -1009,11 +987,10 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
g.Expect(r.Client.Delete(context.TODO(), obj)).ToNot(HaveOccurred())
}()
index := newEtagIndex()
index := index.NewDigester()
sp := patch.NewSerialPatcher(obj, r.Client)
got, err := r.reconcileSource(context.TODO(), sp, obj, index, tmpDir)
t.Log(err)
g.Expect(err != nil).To(Equal(tt.wantErr))
g.Expect(got).To(Equal(tt.want))
@ -1030,7 +1007,7 @@ func TestBucketReconciler_reconcileSource_gcs(t *testing.T) {
func TestBucketReconciler_reconcileArtifact(t *testing.T) {
tests := []struct {
name string
beforeFunc func(t *WithT, obj *sourcev1.Bucket, index *etagIndex, dir string)
beforeFunc func(t *WithT, obj *sourcev1.Bucket, index *index.Digester, dir string)
afterFunc func(t *WithT, obj *sourcev1.Bucket, dir string)
want sreconcile.Result
wantErr bool
@ -1038,25 +1015,25 @@ func TestBucketReconciler_reconcileArtifact(t *testing.T) {
}{
{
name: "Archiving artifact to storage makes ArtifactInStorage=True",
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *etagIndex, dir string) {
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *index.Digester, dir string) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
},
{
name: "Up-to-date artifact should not persist and update status",
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *etagIndex, dir string) {
revision, _ := index.Revision()
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *index.Digester, dir string) {
revision := index.Digest(intdigest.Canonical)
obj.Spec.Interval = metav1.Duration{Duration: interval}
// Incomplete artifact
obj.Status.Artifact = &sourcev1.Artifact{Revision: revision}
obj.Status.Artifact = &sourcev1.Artifact{Revision: revision.String()}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
},
@ -1066,14 +1043,14 @@ func TestBucketReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
},
{
name: "Removes ArtifactOutdatedCondition after creating a new artifact",
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *etagIndex, dir string) {
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *index.Digester, dir string) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
conditions.MarkTrue(obj, sourcev1.ArtifactOutdatedCondition, "Foo", "")
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
@ -1081,14 +1058,14 @@ func TestBucketReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
},
{
name: "Creates latest symlink to the created artifact",
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *etagIndex, dir string) {
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *index.Digester, dir string) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
@ -1102,14 +1079,14 @@ func TestBucketReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
},
{
name: "Dir path deleted",
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *etagIndex, dir string) {
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *index.Digester, dir string) {
t.Expect(os.RemoveAll(dir)).ToNot(HaveOccurred())
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
@ -1124,7 +1101,7 @@ func TestBucketReconciler_reconcileArtifact(t *testing.T) {
},
{
name: "Dir path is not a directory",
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *etagIndex, dir string) {
beforeFunc: func(t *WithT, obj *sourcev1.Bucket, index *index.Digester, dir string) {
// Remove the given directory and create a file for the same
// path.
t.Expect(os.RemoveAll(dir)).ToNot(HaveOccurred())
@ -1174,7 +1151,7 @@ func TestBucketReconciler_reconcileArtifact(t *testing.T) {
},
}
index := newEtagIndex()
index := index.NewDigester()
if tt.beforeFunc != nil {
tt.beforeFunc(g, obj, index, tmpDir)
@ -1206,57 +1183,6 @@ func TestBucketReconciler_reconcileArtifact(t *testing.T) {
}
}
func Test_etagIndex_Revision(t *testing.T) {
tests := []struct {
name string
list map[string]string
want string
wantErr bool
}{
{
name: "index with items",
list: map[string]string{
"one": "one",
"two": "two",
"three": "three",
},
want: "c0837b3f32bb67c5275858fdb96595f87801cf3c2f622c049918a051d29b2c7f",
},
{
name: "index with items in different order",
list: map[string]string{
"three": "three",
"one": "one",
"two": "two",
},
want: "c0837b3f32bb67c5275858fdb96595f87801cf3c2f622c049918a051d29b2c7f",
},
{
name: "empty index",
list: map[string]string{},
want: "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855",
},
{
name: "nil index",
list: nil,
want: "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855",
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
index := &etagIndex{index: tt.list}
got, err := index.Revision()
if (err != nil) != tt.wantErr {
t.Errorf("revision() error = %v, wantErr %v", err, tt.wantErr)
return
}
if got != tt.want {
t.Errorf("revision() got = %v, want %v", got, tt.want)
}
})
}
}
func TestBucketReconciler_statusConditions(t *testing.T) {
tests := []struct {
name string
@ -1439,12 +1365,10 @@ func TestBucketReconciler_notify(t *testing.T) {
EventRecorder: recorder,
patchOptions: getPatchOptions(bucketReadyCondition.Owned, "sc"),
}
index := &etagIndex{
index: map[string]string{
"zzz": "qqq",
"bbb": "ddd",
},
}
index := index.NewDigester(index.WithIndex(map[string]string{
"zzz": "qqq",
"bbb": "ddd",
}))
reconciler.notify(ctx, oldObj, newObj, index, tt.res, tt.resErr)
select {

View File

@ -327,6 +327,9 @@ func (r *GitRepositoryReconciler) notify(ctx context.Context, oldObj, newObj *so
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaRevisionKey): newObj.Status.Artifact.Revision,
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaChecksumKey): newObj.Status.Artifact.Checksum,
}
if newObj.Status.Artifact.Digest != "" {
annotations[sourcev1.GroupVersion.Group+"/"+eventv1.MetaDigestKey] = newObj.Status.Artifact.Digest
}
var oldChecksum string
if oldObj.GetArtifact() != nil {
@ -623,20 +626,20 @@ func (r *GitRepositoryReconciler) reconcileArtifact(ctx context.Context, sp *pat
// Set the ArtifactInStorageCondition if there's no drift.
defer func() {
if obj.GetArtifact().HasRevision(artifact.Revision) &&
if curArtifact := obj.GetArtifact(); curArtifact.HasRevision(artifact.Revision) &&
!includes.Diff(obj.Status.IncludedArtifacts) &&
!gitContentConfigChanged(obj, includes) {
conditions.Delete(obj, sourcev1.ArtifactOutdatedCondition)
conditions.MarkTrue(obj, sourcev1.ArtifactInStorageCondition, meta.SucceededReason,
"stored artifact for revision '%s'", artifact.Revision)
"stored artifact for revision '%s'", curArtifact.Revision)
}
}()
// The artifact is up-to-date
if obj.GetArtifact().HasRevision(artifact.Revision) &&
if curArtifact := obj.GetArtifact(); curArtifact.HasRevision(artifact.Revision) &&
!includes.Diff(obj.Status.IncludedArtifacts) &&
!gitContentConfigChanged(obj, includes) {
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.ArtifactUpToDateReason, "artifact up-to-date with remote revision: '%s'", artifact.Revision)
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.ArtifactUpToDateReason, "artifact up-to-date with remote revision: '%s'", curArtifact.Revision)
return sreconcile.ResultSuccess, nil
}
@ -1021,7 +1024,7 @@ func gitContentConfigChanged(obj *sourcev1.GitRepository, includes *artifactSet)
}
// Check if the included repositories are still the same.
if observedInclArtifact.Revision != currentIncl.Revision {
if !observedInclArtifact.HasRevision(currentIncl.Revision) {
return true
}
if observedInclArtifact.Checksum != currentIncl.Checksum {

View File

@ -294,8 +294,8 @@ func TestGitRepositoryReconciler_reconcileSource_authStrategy(t *testing.T) {
protocol: "http",
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
},
},
{
@ -319,8 +319,8 @@ func TestGitRepositoryReconciler_reconcileSource_authStrategy(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
},
},
{
@ -344,8 +344,8 @@ func TestGitRepositoryReconciler_reconcileSource_authStrategy(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
},
},
{
@ -404,8 +404,8 @@ func TestGitRepositoryReconciler_reconcileSource_authStrategy(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>"),
},
},
{
@ -429,8 +429,8 @@ func TestGitRepositoryReconciler_reconcileSource_authStrategy(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
},
},
{
@ -478,9 +478,9 @@ func TestGitRepositoryReconciler_reconcileSource_authStrategy(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactOutdatedCondition, "NewRevision", "new upstream revision 'master/<commit>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master/<commit>'"),
*conditions.TrueCondition(sourcev1.ArtifactOutdatedCondition, "NewRevision", "new upstream revision 'master@sha1:<commit>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new upstream revision 'master@sha1:<commit>'"),
},
},
}
@ -614,7 +614,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
{
name: "Nil reference (default branch)",
want: sreconcile.ResultSuccess,
wantRevision: "master/<commit>",
wantRevision: "master@sha1:<commit>",
wantReconciling: true,
},
{
@ -623,7 +623,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
Branch: "staging",
},
want: sreconcile.ResultSuccess,
wantRevision: "staging/<commit>",
wantRevision: "staging@sha1:<commit>",
wantReconciling: true,
},
{
@ -632,7 +632,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
Tag: "v0.1.0",
},
want: sreconcile.ResultSuccess,
wantRevision: "v0.1.0/<commit>",
wantRevision: "v0.1.0@sha1:<commit>",
wantReconciling: true,
},
{
@ -642,7 +642,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
Commit: "<commit>",
},
want: sreconcile.ResultSuccess,
wantRevision: "staging/<commit>",
wantRevision: "staging@sha1:<commit>",
wantReconciling: true,
},
{
@ -651,7 +651,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
SemVer: "*",
},
want: sreconcile.ResultSuccess,
wantRevision: "v2.0.0/<commit>",
wantRevision: "v2.0.0@sha1:<commit>",
wantReconciling: true,
},
{
@ -660,7 +660,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
SemVer: "<v0.2.1",
},
want: sreconcile.ResultSuccess,
wantRevision: "0.2.0/<commit>",
wantRevision: "0.2.0@sha1:<commit>",
wantReconciling: true,
},
{
@ -668,7 +668,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
reference: &sourcev1.GitRepositoryRef{
SemVer: ">=1.0.0-0 <1.1.0-0",
},
wantRevision: "v1.0.0-alpha/<commit>",
wantRevision: "v1.0.0-alpha@sha1:<commit>",
want: sreconcile.ResultSuccess,
wantReconciling: true,
},
@ -688,7 +688,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
conditions.MarkTrue(obj, meta.ReadyCondition, meta.SucceededReason, "foo")
},
want: sreconcile.ResultSuccess,
wantRevision: "staging/<commit>",
wantRevision: "staging@sha1:<commit>",
wantArtifactOutdated: true,
wantReconciling: true,
},
@ -697,6 +697,27 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
reference: &sourcev1.GitRepositoryRef{
Branch: "staging",
},
beforeFunc: func(obj *sourcev1.GitRepository, latestRev string) {
// Add existing artifact on the object and storage.
obj.Status = sourcev1.GitRepositoryStatus{
Artifact: &sourcev1.Artifact{
Revision: "staging@sha1:" + latestRev,
Path: randStringRunes(10),
},
}
conditions.MarkTrue(obj, sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "foo")
conditions.MarkTrue(obj, meta.ReadyCondition, meta.SucceededReason, "foo")
},
want: sreconcile.ResultEmpty,
wantErr: true,
wantRevision: "staging@sha1:<commit>",
wantReconciling: false,
},
{
name: "Optimized clone (legacy revision format)",
reference: &sourcev1.GitRepositoryRef{
Branch: "staging",
},
beforeFunc: func(obj *sourcev1.GitRepository, latestRev string) {
// Add existing artifact on the object and storage.
obj.Status = sourcev1.GitRepositoryStatus{
@ -710,7 +731,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
},
want: sreconcile.ResultEmpty,
wantErr: true,
wantRevision: "staging/<commit>",
wantRevision: "staging@sha1:<commit>",
wantReconciling: false,
},
{
@ -718,6 +739,28 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
reference: &sourcev1.GitRepositoryRef{
Branch: "staging",
},
beforeFunc: func(obj *sourcev1.GitRepository, latestRev string) {
// Set new ignore value.
obj.Spec.Ignore = pointer.StringPtr("foo")
// Add existing artifact on the object and storage.
obj.Status = sourcev1.GitRepositoryStatus{
Artifact: &sourcev1.Artifact{
Revision: "staging@sha1:" + latestRev,
Path: randStringRunes(10),
},
}
conditions.MarkTrue(obj, sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "foo")
conditions.MarkTrue(obj, meta.ReadyCondition, meta.SucceededReason, "foo")
},
want: sreconcile.ResultSuccess,
wantRevision: "staging@sha1:<commit>",
wantReconciling: false,
},
{
name: "Optimized clone different ignore (legacy revision format)",
reference: &sourcev1.GitRepositoryRef{
Branch: "staging",
},
beforeFunc: func(obj *sourcev1.GitRepository, latestRev string) {
// Set new ignore value.
obj.Spec.Ignore = pointer.StringPtr("foo")
@ -732,7 +775,7 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
conditions.MarkTrue(obj, meta.ReadyCondition, meta.SucceededReason, "foo")
},
want: sreconcile.ResultSuccess,
wantRevision: "staging/<commit>",
wantRevision: "staging@sha1:<commit>",
wantReconciling: false,
},
}
@ -770,6 +813,8 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
g := NewWithT(t)
obj := &sourcev1.GitRepository{
ObjectMeta: metav1.ObjectMeta{
GenerateName: "checkout-strategy-",
@ -802,8 +847,8 @@ func TestGitRepositoryReconciler_reconcileSource_checkoutStrategy(t *testing.T)
var includes artifactSet
sp := patch.NewSerialPatcher(obj, r.Client)
got, err := r.reconcileSource(ctx, sp, obj, &commit, &includes, tmpDir)
if err != nil {
println(err.Error())
if err != nil && !tt.wantErr {
t.Log(err)
}
g.Expect(err != nil).To(Equal(tt.wantErr))
g.Expect(got).To(Equal(tt.want))
@ -843,13 +888,13 @@ func TestGitRepositoryReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main/revision'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'"),
},
},
{
name: "Archiving artifact to storage with includes makes ArtifactInStorage=True",
dir: "testdata/git/repository",
includes: artifactSet{&sourcev1.Artifact{Revision: "main/revision"}},
includes: artifactSet{&sourcev1.Artifact{Revision: "main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91"}},
beforeFunc: func(obj *sourcev1.GitRepository) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
obj.Spec.Include = []sourcev1.GitRepositoryInclude{
@ -864,20 +909,20 @@ func TestGitRepositoryReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main/revision'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'"),
},
},
{
name: "Up-to-date artifact should not update status",
dir: "testdata/git/repository",
includes: artifactSet{&sourcev1.Artifact{Revision: "main/revision", Checksum: "some-checksum"}},
includes: artifactSet{&sourcev1.Artifact{Revision: "main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91", Checksum: "some-checksum"}},
beforeFunc: func(obj *sourcev1.GitRepository) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
obj.Spec.Include = []sourcev1.GitRepositoryInclude{
{GitRepositoryRef: meta.LocalObjectReference{Name: "foo"}},
}
obj.Status.Artifact = &sourcev1.Artifact{Revision: "main/revision"}
obj.Status.IncludedArtifacts = []*sourcev1.Artifact{{Revision: "main/revision", Checksum: "some-checksum"}}
obj.Status.Artifact = &sourcev1.Artifact{Revision: "main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91"}
obj.Status.IncludedArtifacts = []*sourcev1.Artifact{{Revision: "main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91", Checksum: "some-checksum"}}
obj.Status.ObservedInclude = obj.Spec.Include
},
afterFunc: func(t *WithT, obj *sourcev1.GitRepository) {
@ -885,7 +930,29 @@ func TestGitRepositoryReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main/revision'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'"),
},
},
{
name: "Up-to-date artifact with legacy revision format should not update status",
dir: "testdata/git/repository",
includes: artifactSet{&sourcev1.Artifact{Revision: "main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91", Checksum: "some-checksum"}},
beforeFunc: func(obj *sourcev1.GitRepository) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
obj.Spec.Include = []sourcev1.GitRepositoryInclude{
{GitRepositoryRef: meta.LocalObjectReference{Name: "foo"}},
}
obj.Status.Artifact = &sourcev1.Artifact{Revision: "main/b9b3feadba509cb9b22e968a5d27e96c2bc2ff91"}
obj.Status.IncludedArtifacts = []*sourcev1.Artifact{{Revision: "main/b9b3feadba509cb9b22e968a5d27e96c2bc2ff91", Checksum: "some-checksum"}}
obj.Status.ObservedInclude = obj.Spec.Include
},
afterFunc: func(t *WithT, obj *sourcev1.GitRepository) {
t.Expect(obj.Status.URL).To(BeEmpty())
t.Expect(obj.Status.Artifact.Revision).To(Equal("main/b9b3feadba509cb9b22e968a5d27e96c2bc2ff91"))
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main/b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'"),
},
},
{
@ -901,11 +968,11 @@ func TestGitRepositoryReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main/revision'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'"),
},
},
{
name: "source ignore for subdir ignore patterns",
name: "Source ignore for subdir ignore patterns",
dir: "testdata/git/repowithsubdirs",
beforeFunc: func(obj *sourcev1.GitRepository) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
@ -916,7 +983,7 @@ func TestGitRepositoryReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main/revision'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'"),
},
},
{
@ -933,7 +1000,7 @@ func TestGitRepositoryReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main/revision'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'"),
},
},
{
@ -953,7 +1020,7 @@ func TestGitRepositoryReconciler_reconcileArtifact(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main/revision'"),
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact for revision 'main@sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'"),
},
},
{
@ -1010,7 +1077,7 @@ func TestGitRepositoryReconciler_reconcileArtifact(t *testing.T) {
}
commit := git.Commit{
Hash: []byte("revision"),
Hash: []byte("b9b3feadba509cb9b22e968a5d27e96c2bc2ff91"),
Reference: "refs/heads/main",
}
sp := patch.NewSerialPatcher(obj, r.Client)
@ -1918,12 +1985,12 @@ func TestGitRepositoryReconciler_statusConditions(t *testing.T) {
func TestGitRepositoryReconciler_notify(t *testing.T) {
concreteCommit := git.Commit{
Hash: git.Hash("some-hash"),
Hash: git.Hash("b9b3feadba509cb9b22e968a5d27e96c2bc2ff91"),
Message: "test commit",
Encoded: []byte("content"),
}
partialCommit := git.Commit{
Hash: git.Hash("some-hash"),
Hash: git.Hash("b9b3feadba509cb9b22e968a5d27e96c2bc2ff91"),
}
noopErr := serror.NewGeneric(fmt.Errorf("some no-op error"), "NoOpReason")
@ -2012,7 +2079,7 @@ func TestGitRepositoryReconciler_notify(t *testing.T) {
conditions.MarkTrue(obj, meta.ReadyCondition, meta.SucceededReason, "ready")
},
commit: partialCommit, // no-op will always result in partial commit.
wantEvent: "Normal Succeeded stored artifact for commit 'HEAD/some-hash'",
wantEvent: "Normal Succeeded stored artifact for commit 'sha1:b9b3feadba509cb9b22e968a5d27e96c2bc2ff91'",
},
}

View File

@ -28,12 +28,12 @@ import (
"strings"
"time"
eventv1 "github.com/fluxcd/pkg/apis/event/v1beta1"
soci "github.com/fluxcd/source-controller/internal/oci"
"github.com/google/go-containerregistry/pkg/authn"
"github.com/google/go-containerregistry/pkg/v1/remote"
"github.com/opencontainers/go-digest"
helmgetter "helm.sh/helm/v3/pkg/getter"
helmreg "helm.sh/helm/v3/pkg/registry"
helmrepo "helm.sh/helm/v3/pkg/repo"
corev1 "k8s.io/api/core/v1"
apierrs "k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
@ -52,7 +52,9 @@ import (
"sigs.k8s.io/controller-runtime/pkg/reconcile"
"sigs.k8s.io/controller-runtime/pkg/source"
eventv1 "github.com/fluxcd/pkg/apis/event/v1beta1"
"github.com/fluxcd/pkg/apis/meta"
"github.com/fluxcd/pkg/git"
"github.com/fluxcd/pkg/oci"
"github.com/fluxcd/pkg/runtime/conditions"
helper "github.com/fluxcd/pkg/runtime/controller"
@ -68,6 +70,7 @@ import (
"github.com/fluxcd/source-controller/internal/helm/getter"
"github.com/fluxcd/source-controller/internal/helm/registry"
"github.com/fluxcd/source-controller/internal/helm/repository"
soci "github.com/fluxcd/source-controller/internal/oci"
sreconcile "github.com/fluxcd/source-controller/internal/reconcile"
"github.com/fluxcd/source-controller/internal/reconcile/summarize"
"github.com/fluxcd/source-controller/internal/util"
@ -330,6 +333,9 @@ func (r *HelmChartReconciler) notify(ctx context.Context, oldObj, newObj *source
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaRevisionKey): newObj.Status.Artifact.Revision,
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaChecksumKey): newObj.Status.Artifact.Checksum,
}
if newObj.Status.Artifact.Digest != "" {
annotations[sourcev1.GroupVersion.Group+"/"+eventv1.MetaDigestKey] = newObj.Status.Artifact.Digest
}
var oldChecksum string
if oldObj.GetArtifact() != nil {
@ -522,7 +528,7 @@ func (r *HelmChartReconciler) buildFromHelmRepository(ctx context.Context, obj *
}
// Build client options from secret
opts, tls, err := r.clientOptionsFromSecret(secret, normalizedURL)
opts, tlsCfg, err := r.clientOptionsFromSecret(secret, normalizedURL)
if err != nil {
e := &serror.Event{
Err: err,
@ -533,7 +539,7 @@ func (r *HelmChartReconciler) buildFromHelmRepository(ctx context.Context, obj *
return sreconcile.ResultEmpty, e
}
clientOpts = append(clientOpts, opts...)
tlsConfig = tls
tlsConfig = tlsCfg
// Build registryClient options from secret
keychain, err = registry.LoginOptionFromSecret(normalizedURL, *secret)
@ -646,35 +652,38 @@ func (r *HelmChartReconciler) buildFromHelmRepository(ctx context.Context, obj *
}
}
default:
httpChartRepo, err := repository.NewChartRepository(normalizedURL, r.Storage.LocalPath(*repo.GetArtifact()), r.Getters, tlsConfig, clientOpts,
repository.WithMemoryCache(r.Storage.LocalPath(*repo.GetArtifact()), r.Cache, r.TTL, func(event string) {
r.IncCacheEvents(event, obj.Name, obj.Namespace)
}))
httpChartRepo, err := repository.NewChartRepository(normalizedURL, r.Storage.LocalPath(*repo.GetArtifact()), r.Getters, tlsConfig, clientOpts...)
if err != nil {
return chartRepoConfigErrorReturn(err, obj)
}
chartRepo = httpChartRepo
defer func() {
if httpChartRepo == nil {
return
}
// Cache the index if it was successfully retrieved
// and the chart was successfully built
if r.Cache != nil && httpChartRepo.Index != nil {
// The cache key have to be safe in multi-tenancy environments,
// as otherwise it could be used as a vector to bypass the helm repository's authentication.
// Using r.Storage.LocalPath(*repo.GetArtifact() is safe as the path is in the format /<helm-repository-name>/<chart-name>/<filename>.
err := httpChartRepo.CacheIndexInMemory()
if err != nil {
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.CacheOperationFailedReason, "failed to cache index: %s", err)
}
}
// Delete the index reference
if httpChartRepo.Index != nil {
httpChartRepo.Unload()
// NB: this needs to be deferred first, as otherwise the Index will disappear
// before we had a chance to cache it.
defer func() {
if err := httpChartRepo.Clear(); err != nil {
ctrl.LoggerFrom(ctx).Error(err, "failed to clear Helm repository index")
}
}()
// Attempt to load the index from the cache.
if r.Cache != nil {
if index, ok := r.Cache.Get(repo.GetArtifact().Path); ok {
r.IncCacheEvents(cache.CacheEventTypeHit, repo.Name, repo.Namespace)
r.Cache.SetExpiration(repo.GetArtifact().Path, r.TTL)
httpChartRepo.Index = index.(*helmrepo.IndexFile)
} else {
r.IncCacheEvents(cache.CacheEventTypeMiss, repo.Name, repo.Namespace)
defer func() {
// If we succeed in loading the index, cache it.
if httpChartRepo.Index != nil {
if err = r.Cache.Set(repo.GetArtifact().Path, httpChartRepo.Index, r.TTL); err != nil {
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.CacheOperationFailedReason, "failed to cache index: %s", err)
}
}
}()
}
}
chartRepo = httpChartRepo
}
// Construct the chart builder with scoped configuration
@ -786,10 +795,12 @@ func (r *HelmChartReconciler) buildFromTarballArtifact(ctx context.Context, obj
if obj.Spec.ReconcileStrategy == sourcev1.ReconcileStrategyRevision {
rev := source.Revision
if obj.Spec.SourceRef.Kind == sourcev1.GitRepositoryKind {
// Split the reference by the `/` delimiter which may be present,
// and take the last entry which contains the SHA.
split := strings.Split(source.Revision, "/")
rev = split[len(split)-1]
rev = git.ExtractHashFromRevision(rev).String()
}
if obj.Spec.SourceRef.Kind == sourcev1.BucketKind {
if dig := digest.Digest(sourcev1.TransformLegacyRevision(rev)); dig.Validate() == nil {
rev = dig.Hex()
}
}
if kind := obj.Spec.SourceRef.Kind; kind == sourcev1.GitRepositoryKind || kind == sourcev1.BucketKind {
// The SemVer from the metadata is at times used in e.g. the label metadata for a resource
@ -838,7 +849,7 @@ func (r *HelmChartReconciler) buildFromTarballArtifact(ctx context.Context, obj
// early.
// On a successful archive, the Artifact in the Status of the object is set,
// and the symlink in the Storage is updated to its path.
func (r *HelmChartReconciler) reconcileArtifact(ctx context.Context, sp *patch.SerialPatcher, obj *sourcev1.HelmChart, b *chart.Build) (sreconcile.Result, error) {
func (r *HelmChartReconciler) reconcileArtifact(ctx context.Context, _ *patch.SerialPatcher, obj *sourcev1.HelmChart, b *chart.Build) (sreconcile.Result, error) {
// Without a complete chart build, there is little to reconcile
if !b.Complete() {
return sreconcile.ResultRequeue, nil
@ -1009,14 +1020,15 @@ func (r *HelmChartReconciler) namespacedChartRepositoryCallback(ctx context.Cont
authenticator authn.Authenticator
keychain authn.Keychain
)
normalizedURL := repository.NormalizeURL(url)
repo, err := r.resolveDependencyRepository(ctx, url, namespace)
obj, err := r.resolveDependencyRepository(ctx, url, namespace)
if err != nil {
// Return Kubernetes client errors, but ignore others
if apierrs.ReasonForError(err) != metav1.StatusReasonUnknown {
return nil, err
}
repo = &sourcev1.HelmRepository{
obj = &sourcev1.HelmRepository{
Spec: sourcev1.HelmRepositorySpec{
URL: url,
Timeout: &metav1.Duration{Duration: 60 * time.Second},
@ -1025,37 +1037,37 @@ func (r *HelmChartReconciler) namespacedChartRepositoryCallback(ctx context.Cont
}
// Used to login with the repository declared provider
ctxTimeout, cancel := context.WithTimeout(ctx, repo.Spec.Timeout.Duration)
ctxTimeout, cancel := context.WithTimeout(ctx, obj.Spec.Timeout.Duration)
defer cancel()
clientOpts := []helmgetter.Option{
helmgetter.WithURL(normalizedURL),
helmgetter.WithTimeout(repo.Spec.Timeout.Duration),
helmgetter.WithPassCredentialsAll(repo.Spec.PassCredentials),
helmgetter.WithTimeout(obj.Spec.Timeout.Duration),
helmgetter.WithPassCredentialsAll(obj.Spec.PassCredentials),
}
if secret, err := r.getHelmRepositorySecret(ctx, repo); secret != nil || err != nil {
if secret, err := r.getHelmRepositorySecret(ctx, obj); secret != nil || err != nil {
if err != nil {
return nil, err
}
// Build client options from secret
opts, tls, err := r.clientOptionsFromSecret(secret, normalizedURL)
opts, tlsCfg, err := r.clientOptionsFromSecret(secret, normalizedURL)
if err != nil {
return nil, err
}
clientOpts = append(clientOpts, opts...)
tlsConfig = tls
tlsConfig = tlsCfg
// Build registryClient options from secret
keychain, err = registry.LoginOptionFromSecret(normalizedURL, *secret)
if err != nil {
return nil, fmt.Errorf("failed to create login options for HelmRepository '%s': %w", repo.Name, err)
return nil, fmt.Errorf("failed to create login options for HelmRepository '%s': %w", obj.Name, err)
}
} else if repo.Spec.Provider != sourcev1.GenericOCIProvider && repo.Spec.Type == sourcev1.HelmRepositoryTypeOCI {
auth, authErr := oidcAuth(ctxTimeout, repo.Spec.URL, repo.Spec.Provider)
} else if obj.Spec.Provider != sourcev1.GenericOCIProvider && obj.Spec.Type == sourcev1.HelmRepositoryTypeOCI {
auth, authErr := oidcAuth(ctxTimeout, obj.Spec.URL, obj.Spec.Provider)
if authErr != nil && !errors.Is(authErr, oci.ErrUnconfiguredProvider) {
return nil, fmt.Errorf("failed to get credential from %s: %w", repo.Spec.Provider, authErr)
return nil, fmt.Errorf("failed to get credential from %s: %w", obj.Spec.Provider, authErr)
}
if auth != nil {
authenticator = auth
@ -1071,7 +1083,7 @@ func (r *HelmChartReconciler) namespacedChartRepositoryCallback(ctx context.Cont
if helmreg.IsOCI(normalizedURL) {
registryClient, credentialsFile, err := r.RegistryClientGenerator(loginOpt != nil)
if err != nil {
return nil, fmt.Errorf("failed to create registry client for HelmRepository '%s': %w", repo.Name, err)
return nil, fmt.Errorf("failed to create registry client for HelmRepository '%s': %w", obj.Name, err)
}
var errs []error
@ -1082,7 +1094,7 @@ func (r *HelmChartReconciler) namespacedChartRepositoryCallback(ctx context.Cont
repository.WithOCIRegistryClient(registryClient),
repository.WithCredentialsFile(credentialsFile))
if err != nil {
errs = append(errs, fmt.Errorf("failed to create OCI chart repository for HelmRepository '%s': %w", repo.Name, err))
errs = append(errs, fmt.Errorf("failed to create OCI chart repository for HelmRepository '%s': %w", obj.Name, err))
// clean up the credentialsFile
if credentialsFile != "" {
if err := os.Remove(credentialsFile); err != nil {
@ -1097,7 +1109,7 @@ func (r *HelmChartReconciler) namespacedChartRepositoryCallback(ctx context.Cont
if loginOpt != nil {
err = ociChartRepo.Login(loginOpt)
if err != nil {
errs = append(errs, fmt.Errorf("failed to login to OCI chart repository for HelmRepository '%s': %w", repo.Name, err))
errs = append(errs, fmt.Errorf("failed to login to OCI chart repository for HelmRepository '%s': %w", obj.Name, err))
// clean up the credentialsFile
errs = append(errs, ociChartRepo.Clear())
return nil, kerrors.NewAggregate(errs)
@ -1106,19 +1118,28 @@ func (r *HelmChartReconciler) namespacedChartRepositoryCallback(ctx context.Cont
chartRepo = ociChartRepo
} else {
httpChartRepo, err := repository.NewChartRepository(normalizedURL, "", r.Getters, tlsConfig, clientOpts)
httpChartRepo, err := repository.NewChartRepository(normalizedURL, "", r.Getters, tlsConfig, clientOpts...)
if err != nil {
return nil, err
}
// Ensure that the cache key is the same as the artifact path
// otherwise don't enable caching. We don't want to cache indexes
// for repositories that are not reconciled by the source controller.
if repo.Status.Artifact != nil {
httpChartRepo.CachePath = r.Storage.LocalPath(*repo.GetArtifact())
httpChartRepo.SetMemCache(r.Storage.LocalPath(*repo.GetArtifact()), r.Cache, r.TTL, func(event string) {
r.IncCacheEvents(event, name, namespace)
})
if artifact := obj.GetArtifact(); artifact != nil {
httpChartRepo.Path = r.Storage.LocalPath(*artifact)
// Attempt to load the index from the cache.
if r.Cache != nil {
if index, ok := r.Cache.Get(artifact.Path); ok {
r.IncCacheEvents(cache.CacheEventTypeHit, name, namespace)
r.Cache.SetExpiration(artifact.Path, r.TTL)
httpChartRepo.Index = index.(*helmrepo.IndexFile)
} else {
r.IncCacheEvents(cache.CacheEventTypeMiss, name, namespace)
if err := httpChartRepo.LoadFromPath(); err != nil {
return nil, err
}
r.Cache.Set(artifact.Path, httpChartRepo.Index, r.TTL)
}
}
}
chartRepo = httpChartRepo
@ -1242,7 +1263,7 @@ func (r *HelmChartReconciler) requestsForGitRepositoryChange(o client.Object) []
var reqs []reconcile.Request
for _, i := range list.Items {
if i.Status.ObservedSourceArtifactRevision != repo.GetArtifact().Revision {
if !repo.GetArtifact().HasRevision(i.Status.ObservedSourceArtifactRevision) {
reqs = append(reqs, reconcile.Request{NamespacedName: client.ObjectKeyFromObject(&i)})
}
}
@ -1269,7 +1290,7 @@ func (r *HelmChartReconciler) requestsForBucketChange(o client.Object) []reconci
var reqs []reconcile.Request
for _, i := range list.Items {
if i.Status.ObservedSourceArtifactRevision != bucket.GetArtifact().Revision {
if !bucket.GetArtifact().HasRevision(i.Status.ObservedSourceArtifactRevision) {
reqs = append(reqs, reconcile.Request{NamespacedName: client.ObjectKeyFromObject(&i)})
}
}

View File

@ -137,8 +137,7 @@ func TestHelmChartReconciler_Reconcile(t *testing.T) {
repoKey := client.ObjectKey{Name: repository.Name, Namespace: repository.Namespace}
err = testEnv.Get(ctx, repoKey, repository)
g.Expect(err).ToNot(HaveOccurred())
localPath := testStorage.LocalPath(*repository.GetArtifact())
_, found := testCache.Get(localPath)
_, found := testCache.Get(repository.GetArtifact().Path)
g.Expect(found).To(BeTrue())
g.Expect(testEnv.Delete(ctx, obj)).To(Succeed())
@ -1394,6 +1393,9 @@ func TestHelmChartReconciler_buildFromTarballArtifact(t *testing.T) {
}
got, err := r.buildFromTarballArtifact(context.TODO(), obj, tt.source, &b)
if err != nil {
t.Log(err)
}
g.Expect(err != nil).To(Equal(tt.wantErr != nil))
if tt.wantErr != nil {
g.Expect(reflect.TypeOf(err).String()).To(Equal(reflect.TypeOf(tt.wantErr).String()))

View File

@ -25,6 +25,7 @@ import (
"time"
"github.com/docker/go-units"
"github.com/opencontainers/go-digest"
helmgetter "helm.sh/helm/v3/pkg/getter"
corev1 "k8s.io/api/core/v1"
"k8s.io/apimachinery/pkg/runtime"
@ -47,6 +48,7 @@ import (
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
"github.com/fluxcd/source-controller/internal/cache"
intdigest "github.com/fluxcd/source-controller/internal/digest"
serror "github.com/fluxcd/source-controller/internal/error"
"github.com/fluxcd/source-controller/internal/helm/getter"
"github.com/fluxcd/source-controller/internal/helm/repository"
@ -277,13 +279,13 @@ func (r *HelmRepositoryReconciler) reconcile(ctx context.Context, sp *patch.Seri
res = sreconcile.LowestRequeuingResult(res, recResult)
}
r.notify(ctx, oldObj, obj, chartRepo, res, resErr)
r.notify(ctx, oldObj, obj, &chartRepo, res, resErr)
return res, resErr
}
// notify emits notification related to the reconciliation.
func (r *HelmRepositoryReconciler) notify(ctx context.Context, oldObj, newObj *sourcev1.HelmRepository, chartRepo repository.ChartRepository, res sreconcile.Result, resErr error) {
func (r *HelmRepositoryReconciler) notify(ctx context.Context, oldObj, newObj *sourcev1.HelmRepository, chartRepo *repository.ChartRepository, res sreconcile.Result, resErr error) {
// Notify successful reconciliation for new artifact and recovery from any
// failure.
if resErr == nil && res == sreconcile.ResultSuccess && newObj.Status.Artifact != nil {
@ -291,6 +293,9 @@ func (r *HelmRepositoryReconciler) notify(ctx context.Context, oldObj, newObj *s
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaRevisionKey): newObj.Status.Artifact.Revision,
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaChecksumKey): newObj.Status.Artifact.Checksum,
}
if newObj.Status.Artifact.Digest != "" {
annotations[sourcev1.GroupVersion.Group+"/"+eventv1.MetaDigestKey] = newObj.Status.Artifact.Digest
}
humanReadableSize := "unknown size"
if size := newObj.Status.Artifact.Size; size != nil {
@ -430,7 +435,7 @@ func (r *HelmRepositoryReconciler) reconcileSource(ctx context.Context, sp *patc
}
// Construct Helm chart repository with options and download index
newChartRepo, err := repository.NewChartRepository(obj.Spec.URL, "", r.Getters, tlsConfig, clientOpts)
newChartRepo, err := repository.NewChartRepository(obj.Spec.URL, "", r.Getters, tlsConfig, clientOpts...)
if err != nil {
switch err.(type) {
case *url.Error:
@ -451,8 +456,7 @@ func (r *HelmRepositoryReconciler) reconcileSource(ctx context.Context, sp *patc
}
// Fetch the repository index from remote.
checksum, err := newChartRepo.CacheIndex()
if err != nil {
if err := newChartRepo.CacheIndex(); err != nil {
e := &serror.Event{
Err: fmt.Errorf("failed to fetch Helm repository index: %w", err),
Reason: meta.FailedReason,
@ -463,20 +467,48 @@ func (r *HelmRepositoryReconciler) reconcileSource(ctx context.Context, sp *patc
}
*chartRepo = *newChartRepo
// Short-circuit based on the fetched index being an exact match to the
// stored Artifact. This prevents having to unmarshal the YAML to calculate
// the (stable) revision, which is a memory expensive operation.
if obj.GetArtifact().HasChecksum(checksum) {
*artifact = *obj.GetArtifact()
conditions.Delete(obj, sourcev1.FetchFailedCondition)
return sreconcile.ResultSuccess, nil
// Early comparison to current Artifact.
if curArtifact := obj.GetArtifact(); curArtifact != nil {
curDig := digest.Digest(curArtifact.Digest)
if curDig == "" {
curDig = digest.Digest(sourcev1.TransformLegacyRevision(curArtifact.Checksum))
}
if curDig.Validate() == nil {
// Short-circuit based on the fetched index being an exact match to the
// stored Artifact. This prevents having to unmarshal the YAML to calculate
// the (stable) revision, which is a memory expensive operation.
if newDig := chartRepo.Digest(curDig.Algorithm()); newDig.Validate() == nil && (newDig == curDig) {
*artifact = *curArtifact
conditions.Delete(obj, sourcev1.FetchFailedCondition)
return sreconcile.ResultSuccess, nil
}
}
}
// Load the cached repository index to ensure it passes validation. This
// also populates chartRepo.Checksum.
if err := chartRepo.LoadFromCache(); err != nil {
// Load the cached repository index to ensure it passes validation.
if err := chartRepo.LoadFromPath(); err != nil {
e := &serror.Event{
Err: fmt.Errorf("failed to load Helm repository from cache: %w", err),
Err: fmt.Errorf("failed to load Helm repository from index YAML: %w", err),
Reason: sourcev1.IndexationFailedReason,
}
conditions.MarkTrue(obj, sourcev1.FetchFailedCondition, e.Reason, e.Err.Error())
return sreconcile.ResultEmpty, e
}
// Delete any stale failure observation
conditions.Delete(obj, sourcev1.FetchFailedCondition)
// Check if index has changed compared to current Artifact revision.
var changed bool
if artifact := obj.Status.Artifact; artifact != nil {
curRev := digest.Digest(sourcev1.TransformLegacyRevision(artifact.Revision))
changed = curRev.Validate() != nil || curRev != chartRepo.Revision(curRev.Algorithm())
}
// Calculate revision.
revision := chartRepo.Revision(intdigest.Canonical)
if revision.Validate() != nil {
e := &serror.Event{
Err: fmt.Errorf("failed to calculate revision: %w", err),
Reason: sourcev1.IndexationFailedReason,
}
conditions.MarkTrue(obj, sourcev1.FetchFailedCondition, e.Reason, e.Err.Error())
@ -484,8 +516,8 @@ func (r *HelmRepositoryReconciler) reconcileSource(ctx context.Context, sp *patc
}
// Mark observations about the revision on the object.
if !obj.GetArtifact().HasRevision(chartRepo.Checksum) {
message := fmt.Sprintf("new index revision '%s'", checksum)
if obj.Status.Artifact == nil || changed {
message := fmt.Sprintf("new index revision '%s'", revision)
if obj.GetArtifact() != nil {
conditions.MarkTrue(obj, sourcev1.ArtifactOutdatedCondition, "NewRevision", message)
}
@ -497,15 +529,11 @@ func (r *HelmRepositoryReconciler) reconcileSource(ctx context.Context, sp *patc
}
// Create potential new artifact.
// Note: Since this is a potential artifact, artifact.Checksum is empty at
// this stage. It's populated when the artifact is written in storage.
*artifact = r.Storage.NewArtifactFor(obj.Kind,
obj.ObjectMeta.GetObjectMeta(),
chartRepo.Checksum,
fmt.Sprintf("index-%s.yaml", checksum))
// Delete any stale failure observation
conditions.Delete(obj, sourcev1.FetchFailedCondition)
revision.String(),
fmt.Sprintf("index-%s.yaml", revision.Hex()),
)
return sreconcile.ResultSuccess, nil
}
@ -527,15 +555,17 @@ func (r *HelmRepositoryReconciler) reconcileArtifact(ctx context.Context, sp *pa
conditions.MarkTrue(obj, sourcev1.ArtifactInStorageCondition, meta.SucceededReason,
"stored artifact: revision '%s'", artifact.Revision)
}
chartRepo.Unload()
if err := chartRepo.RemoveCache(); err != nil {
if err := chartRepo.Clear(); err != nil {
ctrl.LoggerFrom(ctx).Error(err, "failed to remove temporary cached index file")
}
}()
if obj.GetArtifact().HasRevision(artifact.Revision) && obj.GetArtifact().HasChecksum(artifact.Checksum) {
// Extend TTL of the Index in the cache (if present).
if r.Cache != nil {
r.Cache.SetExpiration(artifact.Path, r.TTL)
}
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.ArtifactUpToDateReason, "artifact up-to-date with remote revision: '%s'", artifact.Revision)
return sreconcile.ResultSuccess, nil
}
@ -561,7 +591,7 @@ func (r *HelmRepositoryReconciler) reconcileArtifact(ctx context.Context, sp *pa
defer unlock()
// Save artifact to storage.
if err = r.Storage.CopyFromPath(artifact, chartRepo.CachePath); err != nil {
if err = r.Storage.CopyFromPath(artifact, chartRepo.Path); err != nil {
e := &serror.Event{
Err: fmt.Errorf("unable to save artifact to storage: %w", err),
Reason: sourcev1.ArchiveOperationFailedReason,
@ -573,6 +603,17 @@ func (r *HelmRepositoryReconciler) reconcileArtifact(ctx context.Context, sp *pa
// Record it on the object.
obj.Status.Artifact = artifact.DeepCopy()
// Cache the index if it was successfully retrieved.
if r.Cache != nil && chartRepo.Index != nil {
// The cache keys have to be safe in multi-tenancy environments, as
// otherwise it could be used as a vector to bypass the repository's
// authentication. Using the Artifact.Path is safe as the path is in
// the format of: /<repository-name>/<chart-name>/<filename>.
if err := r.Cache.Set(artifact.Path, chartRepo.Index, r.TTL); err != nil {
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.CacheOperationFailedReason, "failed to cache index: %s", err)
}
}
// Update index symlink.
indexURL, err := r.Storage.Symlink(*artifact, "index.yaml")
if err != nil {
@ -583,26 +624,6 @@ func (r *HelmRepositoryReconciler) reconcileArtifact(ctx context.Context, sp *pa
obj.Status.URL = indexURL
}
conditions.Delete(obj, sourcev1.StorageOperationFailedCondition)
// enable cache if applicable
if r.Cache != nil && chartRepo.IndexCache == nil {
chartRepo.SetMemCache(r.Storage.LocalPath(*artifact), r.Cache, r.TTL, func(event string) {
r.IncCacheEvents(event, obj.GetName(), obj.GetNamespace())
})
}
// Cache the index if it was successfully retrieved
// and the chart was successfully built
if r.Cache != nil && chartRepo.Index != nil {
// The cache key have to be safe in multi-tenancy environments,
// as otherwise it could be used as a vector to bypass the helm repository's authentication.
// Using r.Storage.LocalPath(*repo.GetArtifact() is safe as the path is in the format /<helm-repository-name>/<chart-name>/<filename>.
err := chartRepo.CacheIndexInMemory()
if err != nil {
r.eventLogf(ctx, obj, eventv1.EventTypeTrace, sourcev1.CacheOperationFailedReason, "failed to cache index: %s", err)
}
}
return sreconcile.ResultSuccess, nil
}

View File

@ -29,7 +29,9 @@ import (
"time"
. "github.com/onsi/gomega"
"github.com/opencontainers/go-digest"
helmgetter "helm.sh/helm/v3/pkg/getter"
"helm.sh/helm/v3/pkg/repo"
corev1 "k8s.io/api/core/v1"
apierrors "k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
@ -46,6 +48,8 @@ import (
"github.com/fluxcd/pkg/runtime/patch"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
"github.com/fluxcd/source-controller/internal/cache"
intdigest "github.com/fluxcd/source-controller/internal/digest"
"github.com/fluxcd/source-controller/internal/helm/getter"
"github.com/fluxcd/source-controller/internal/helm/repository"
sreconcile "github.com/fluxcd/source-controller/internal/reconcile"
@ -312,8 +316,8 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
server options
url string
secret *corev1.Secret
beforeFunc func(t *WithT, obj *sourcev1.HelmRepository, checksum string)
afterFunc func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository)
beforeFunc func(t *WithT, obj *sourcev1.HelmRepository, revision, digest digest.Digest)
afterFunc func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository)
want sreconcile.Result
wantErr bool
assertConditions []metav1.Condition
@ -344,9 +348,9 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new index revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new index revision"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
t.Expect(chartRepo.Checksum).ToNot(BeEmpty())
t.Expect(chartRepo.CachePath).ToNot(BeEmpty())
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
t.Expect(chartRepo.Path).ToNot(BeEmpty())
t.Expect(chartRepo.Index).ToNot(BeNil())
t.Expect(artifact.Checksum).To(BeEmpty())
t.Expect(artifact.Revision).ToNot(BeEmpty())
},
@ -367,7 +371,7 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
"password": []byte("1234"),
},
},
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Spec.SecretRef = &meta.LocalObjectReference{Name: "basic-auth"}
},
want: sreconcile.ResultSuccess,
@ -375,9 +379,9 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new index revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new index revision"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
t.Expect(chartRepo.Checksum).ToNot(BeEmpty())
t.Expect(chartRepo.CachePath).ToNot(BeEmpty())
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
t.Expect(chartRepo.Path).ToNot(BeEmpty())
t.Expect(chartRepo.Index).ToNot(BeNil())
t.Expect(artifact.Checksum).To(BeEmpty())
t.Expect(artifact.Revision).ToNot(BeEmpty())
},
@ -398,7 +402,7 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
"caFile": tlsCA,
},
},
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Spec.SecretRef = &meta.LocalObjectReference{Name: "ca-file"}
},
want: sreconcile.ResultSuccess,
@ -406,9 +410,9 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new index revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new index revision"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
t.Expect(chartRepo.Checksum).ToNot(BeEmpty())
t.Expect(chartRepo.CachePath).ToNot(BeEmpty())
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
t.Expect(chartRepo.Path).ToNot(BeEmpty())
t.Expect(chartRepo.Index).ToNot(BeNil())
t.Expect(artifact.Checksum).To(BeEmpty())
t.Expect(artifact.Revision).ToNot(BeEmpty())
},
@ -429,7 +433,7 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
"caFile": []byte("invalid"),
},
},
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Spec.SecretRef = &meta.LocalObjectReference{Name: "invalid-ca"}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
@ -440,10 +444,10 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
// No repo index due to fetch fail.
t.Expect(chartRepo.Checksum).To(BeEmpty())
t.Expect(chartRepo.CachePath).To(BeEmpty())
t.Expect(chartRepo.Path).To(BeEmpty())
t.Expect(chartRepo.Index).To(BeNil())
t.Expect(artifact.Checksum).To(BeEmpty())
t.Expect(artifact.Revision).To(BeEmpty())
},
@ -451,7 +455,7 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
{
name: "Invalid URL makes FetchFailed=True and returns stalling error",
protocol: "http",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Spec.URL = strings.ReplaceAll(obj.Spec.URL, "http://", "")
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
@ -463,10 +467,10 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
// No repo index due to fetch fail.
t.Expect(chartRepo.Checksum).To(BeEmpty())
t.Expect(chartRepo.CachePath).To(BeEmpty())
t.Expect(chartRepo.Path).To(BeEmpty())
t.Expect(chartRepo.Index).To(BeNil())
t.Expect(artifact.Checksum).To(BeEmpty())
t.Expect(artifact.Revision).To(BeEmpty())
},
@ -474,7 +478,7 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
{
name: "Unsupported scheme makes FetchFailed=True and returns stalling error",
protocol: "http",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Spec.URL = strings.ReplaceAll(obj.Spec.URL, "http://", "ftp://")
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
@ -486,10 +490,10 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
// No repo index due to fetch fail.
t.Expect(chartRepo.Checksum).To(BeEmpty())
t.Expect(chartRepo.CachePath).To(BeEmpty())
t.Expect(chartRepo.Path).To(BeEmpty())
t.Expect(chartRepo.Index).To(BeNil())
t.Expect(artifact.Checksum).To(BeEmpty())
t.Expect(artifact.Revision).To(BeEmpty())
},
@ -497,7 +501,7 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
{
name: "Missing secret returns FetchFailed=True and returns error",
protocol: "http",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Spec.SecretRef = &meta.LocalObjectReference{Name: "non-existing"}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
@ -508,10 +512,10 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
// No repo index due to fetch fail.
t.Expect(chartRepo.Checksum).To(BeEmpty())
t.Expect(chartRepo.CachePath).To(BeEmpty())
t.Expect(chartRepo.Path).To(BeEmpty())
t.Expect(chartRepo.Index).To(BeNil())
t.Expect(artifact.Checksum).To(BeEmpty())
t.Expect(artifact.Revision).To(BeEmpty())
},
@ -527,7 +531,7 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
"username": []byte("git"),
},
},
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Spec.SecretRef = &meta.LocalObjectReference{Name: "malformed-basic-auth"}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
@ -538,66 +542,125 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
// No repo index due to fetch fail.
t.Expect(chartRepo.Checksum).To(BeEmpty())
t.Expect(chartRepo.CachePath).To(BeEmpty())
t.Expect(chartRepo.Path).To(BeEmpty())
t.Expect(chartRepo.Index).To(BeNil())
t.Expect(artifact.Checksum).To(BeEmpty())
t.Expect(artifact.Revision).To(BeEmpty())
},
},
{
name: "cached index with same checksum",
name: "Stored index with same digest and revision",
protocol: "http",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, digest digest.Digest) {
obj.Status.Artifact = &sourcev1.Artifact{
Revision: checksum,
Checksum: checksum,
Revision: revision.String(),
Digest: digest.String(),
Checksum: digest.Hex(),
}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
conditions.MarkTrue(obj, sourcev1.FetchFailedCondition, "foo", "bar")
},
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
// chartRepo.Checksum isn't populated, artifact.Checksum is
// populated from the cached repo index data.
t.Expect(chartRepo.Checksum).To(BeEmpty())
t.Expect(chartRepo.CachePath).ToNot(BeEmpty())
t.Expect(artifact.Checksum).To(Equal(obj.Status.Artifact.Checksum))
t.Expect(artifact.Revision).To(Equal(obj.Status.Artifact.Revision))
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
t.Expect(chartRepo.Path).ToNot(BeEmpty())
t.Expect(chartRepo.Index).To(BeNil())
t.Expect(&artifact).To(BeEquivalentTo(obj.Status.Artifact))
},
want: sreconcile.ResultSuccess,
},
{
name: "cached index with different checksum",
name: "Stored index with same checksum and (legacy) revision",
protocol: "http",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, digest digest.Digest) {
obj.Status.Artifact = &sourcev1.Artifact{
Revision: checksum,
Checksum: "foo",
Revision: revision.Hex(),
Checksum: digest.Hex(),
}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
conditions.MarkTrue(obj, sourcev1.FetchFailedCondition, "foo", "bar")
},
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo repository.ChartRepository) {
t.Expect(chartRepo.Checksum).ToNot(BeEmpty())
t.Expect(chartRepo.CachePath).ToNot(BeEmpty())
t.Expect(artifact.Checksum).To(BeEmpty())
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
t.Expect(chartRepo.Path).ToNot(BeEmpty())
t.Expect(chartRepo.Index).To(BeNil())
t.Expect(&artifact).To(BeEquivalentTo(obj.Status.Artifact))
},
want: sreconcile.ResultSuccess,
},
{
name: "Stored index with different digest and same revision",
protocol: "http",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, digest digest.Digest) {
obj.Status.Artifact = &sourcev1.Artifact{
Revision: revision.String(),
Digest: "sha256:80bb3dd67c63095d985850459834ea727603727a370079de90d221191d375a86",
Checksum: "80bb3dd67c63095d985850459834ea727603727a370079de90d221191d375a86",
}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
conditions.MarkTrue(obj, sourcev1.FetchFailedCondition, "foo", "bar")
},
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "foo"),
*conditions.UnknownCondition(meta.ReadyCondition, "foo", "bar"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
t.Expect(chartRepo.Path).ToNot(BeEmpty())
t.Expect(chartRepo.Index).ToNot(BeNil())
t.Expect(artifact.Revision).To(Equal(obj.Status.Artifact.Revision))
t.Expect(artifact.Digest).ToNot(Equal(obj.Status.Artifact.Digest))
t.Expect(artifact.Checksum).ToNot(Equal(obj.Status.Artifact.Checksum))
},
want: sreconcile.ResultSuccess,
},
{
name: "Stored index with different revision and digest",
protocol: "http",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Status.Artifact = &sourcev1.Artifact{
Revision: "80bb3dd67c63095d985850459834ea727603727a370079de90d221191d375a86",
Checksum: "80bb3dd67c63095d985850459834ea727603727a370079de90d221191d375a86",
Digest: "sha256:80bb3dd67c63095d985850459834ea727603727a370079de90d221191d375a86",
}
conditions.MarkReconciling(obj, meta.ProgressingReason, "foo")
conditions.MarkUnknown(obj, meta.ReadyCondition, "foo", "bar")
},
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactOutdatedCondition, "NewRevision", "new index revision"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new index revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new index revision"),
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, chartRepo *repository.ChartRepository) {
t.Expect(chartRepo.Path).ToNot(BeEmpty())
t.Expect(chartRepo.Index).ToNot(BeNil())
t.Expect(artifact.Path).To(Not(BeEmpty()))
t.Expect(artifact.Revision).ToNot(Equal(obj.Status.Artifact.Revision))
t.Expect(artifact.Digest).ToNot(Equal(obj.Status.Artifact.Digest))
t.Expect(artifact.Checksum).ToNot(Equal(obj.Status.Artifact.Checksum))
},
want: sreconcile.ResultSuccess,
},
{
name: "Existing artifact makes ArtifactOutdated=True",
protocol: "http",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, checksum string) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, revision, checksum digest.Digest) {
obj.Status.Artifact = &sourcev1.Artifact{
Path: "some-path",
Revision: "some-rev",
@ -698,22 +761,24 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
if serr != nil {
validSecret = false
}
newChartRepo, err = repository.NewChartRepository(obj.Spec.URL, "", testGetters, tOpts, clientOpts)
newChartRepo, err = repository.NewChartRepository(obj.Spec.URL, "", testGetters, tOpts, clientOpts...)
} else {
newChartRepo, err = repository.NewChartRepository(obj.Spec.URL, "", testGetters, nil, nil)
newChartRepo, err = repository.NewChartRepository(obj.Spec.URL, "", testGetters, nil)
}
g.Expect(err).ToNot(HaveOccurred())
// NOTE: checksum will be empty in beforeFunc for invalid repo
// configurations as the client can't get the repo.
var indexChecksum string
var revision, checksum digest.Digest
if validSecret {
indexChecksum, err = newChartRepo.CacheIndex()
g.Expect(err).ToNot(HaveOccurred())
}
g.Expect(newChartRepo.CacheIndex()).To(Succeed())
checksum = newChartRepo.Digest(intdigest.Canonical)
g.Expect(newChartRepo.LoadFromPath()).To(Succeed())
revision = newChartRepo.Revision(intdigest.Canonical)
}
if tt.beforeFunc != nil {
tt.beforeFunc(g, obj, indexChecksum)
tt.beforeFunc(g, obj, revision, checksum)
}
r := &HelmRepositoryReconciler{
@ -734,14 +799,14 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
sp := patch.NewSerialPatcher(obj, r.Client)
got, err := r.reconcileSource(context.TODO(), sp, obj, &artifact, &chartRepo)
defer os.Remove(chartRepo.CachePath)
defer os.Remove(chartRepo.Path)
g.Expect(obj.Status.Conditions).To(conditions.MatchConditions(tt.assertConditions))
g.Expect(err != nil).To(Equal(tt.wantErr))
g.Expect(got).To(Equal(tt.want))
if tt.afterFunc != nil {
tt.afterFunc(g, obj, artifact, chartRepo)
tt.afterFunc(g, obj, artifact, &chartRepo)
}
// In-progress status condition validity.
@ -754,8 +819,9 @@ func TestHelmRepositoryReconciler_reconcileSource(t *testing.T) {
func TestHelmRepositoryReconciler_reconcileArtifact(t *testing.T) {
tests := []struct {
name string
cache *cache.Cache
beforeFunc func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, index *repository.ChartRepository)
afterFunc func(t *WithT, obj *sourcev1.HelmRepository)
afterFunc func(t *WithT, obj *sourcev1.HelmRepository, cache *cache.Cache)
want sreconcile.Result
wantErr bool
assertConditions []metav1.Condition
@ -770,13 +836,33 @@ func TestHelmRepositoryReconciler_reconcileArtifact(t *testing.T) {
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'existing'"),
},
},
{
name: "Archiving (loaded) artifact to storage adds to cache",
cache: cache.New(10, time.Minute),
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, index *repository.ChartRepository) {
index.Index = &repo.IndexFile{
APIVersion: "v1",
Generated: time.Now(),
}
obj.Spec.Interval = metav1.Duration{Duration: interval}
},
want: sreconcile.ResultSuccess,
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, cache *cache.Cache) {
i, ok := cache.Get(obj.GetArtifact().Path)
t.Expect(ok).To(BeTrue())
t.Expect(i).To(BeAssignableToTypeOf(&repo.IndexFile{}))
},
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.ArtifactInStorageCondition, meta.SucceededReason, "stored artifact: revision 'existing'"),
},
},
{
name: "Up-to-date artifact should not update status",
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, index *repository.ChartRepository) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
obj.Status.Artifact = artifact.DeepCopy()
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository) {
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, _ *cache.Cache) {
t.Expect(obj.Status.URL).To(BeEmpty())
},
want: sreconcile.ResultSuccess,
@ -800,7 +886,7 @@ func TestHelmRepositoryReconciler_reconcileArtifact(t *testing.T) {
beforeFunc: func(t *WithT, obj *sourcev1.HelmRepository, artifact sourcev1.Artifact, index *repository.ChartRepository) {
obj.Spec.Interval = metav1.Duration{Duration: interval}
},
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository) {
afterFunc: func(t *WithT, obj *sourcev1.HelmRepository, _ *cache.Cache) {
localPath := testStorage.LocalPath(*obj.GetArtifact())
symlinkPath := filepath.Join(filepath.Dir(localPath), "index.yaml")
targetFile, err := os.Readlink(symlinkPath)
@ -822,6 +908,8 @@ func TestHelmRepositoryReconciler_reconcileArtifact(t *testing.T) {
Client: fakeclient.NewClientBuilder().WithScheme(testEnv.GetScheme()).Build(),
EventRecorder: record.NewFakeRecorder(32),
Storage: testStorage,
Cache: tt.cache,
TTL: 1 * time.Minute,
patchOptions: getPatchOptions(helmRepositoryReadyCondition.Owned, "sc"),
}
@ -848,9 +936,9 @@ func TestHelmRepositoryReconciler_reconcileArtifact(t *testing.T) {
g.Expect(err).ToNot(HaveOccurred())
g.Expect(cacheFile.Close()).ToNot(HaveOccurred())
chartRepo, err := repository.NewChartRepository(obj.Spec.URL, "", testGetters, nil, nil)
chartRepo, err := repository.NewChartRepository(obj.Spec.URL, "", testGetters, nil)
g.Expect(err).ToNot(HaveOccurred())
chartRepo.CachePath = cachePath
chartRepo.Path = cachePath
artifact := testStorage.NewArtifactFor(obj.Kind, obj, "existing", "foo.tar.gz")
// Checksum of the index file calculated by the ChartRepository.
@ -873,7 +961,7 @@ func TestHelmRepositoryReconciler_reconcileArtifact(t *testing.T) {
g.Expect(obj.Status.Conditions).To(conditions.MatchConditions(tt.assertConditions))
if tt.afterFunc != nil {
tt.afterFunc(g, obj)
tt.afterFunc(g, obj, tt.cache)
}
})
}
@ -1209,7 +1297,7 @@ func TestHelmRepositoryReconciler_notify(t *testing.T) {
chartRepo := repository.ChartRepository{
URL: "some-address",
}
reconciler.notify(ctx, oldObj, newObj, chartRepo, tt.res, tt.resErr)
reconciler.notify(ctx, oldObj, newObj, &chartRepo, tt.res, tt.resErr)
select {
case x, ok := <-recorder.Events:
@ -1493,7 +1581,6 @@ func TestHelmRepositoryReconciler_InMemoryCaching(t *testing.T) {
err = testEnv.Get(ctx, key, helmRepo)
g.Expect(err).ToNot(HaveOccurred())
localPath := testStorage.LocalPath(*helmRepo.GetArtifact())
_, cacheHit := testCache.Get(localPath)
_, cacheHit := testCache.Get(helmRepo.GetArtifact().Path)
g.Expect(cacheHit).To(BeTrue())
}

View File

@ -582,7 +582,8 @@ func (r *OCIRepositoryReconciler) selectLayer(obj *sourcev1.OCIRepository, image
return blob, nil
}
// getRevision fetches the upstream digest and returns the revision in the format `<tag>/<digest>`
// getRevision fetches the upstream digest, returning the revision in the
// format '<tag>@<digest>'.
func (r *OCIRepositoryReconciler) getRevision(url string, options []crane.Option) (string, error) {
ref, err := name.ParseReference(url)
if err != nil {
@ -609,21 +610,22 @@ func (r *OCIRepositoryReconciler) getRevision(url string, options []crane.Option
return "", err
}
revision := digestHash.Hex
revision := digestHash.String()
if repoTag != "" {
revision = fmt.Sprintf("%s/%s", repoTag, digestHash.Hex)
revision = fmt.Sprintf("%s@%s", repoTag, revision)
}
return revision, nil
}
// digestFromRevision extract the digest from the revision string
// digestFromRevision extracts the digest from the revision string.
func (r *OCIRepositoryReconciler) digestFromRevision(revision string) string {
parts := strings.Split(revision, "/")
parts := strings.Split(revision, "@")
return parts[len(parts)-1]
}
// verifySignature verifies the authenticity of the given image reference url. First, it tries using a key
// if a secret with a valid public key is provided. If not, it falls back to a keyless approach for verification.
// verifySignature verifies the authenticity of the given image reference URL.
// First, it tries to use a key if a Secret with a valid public key is provided.
// If not, it falls back to a keyless approach for verification.
func (r *OCIRepositoryReconciler) verifySignature(ctx context.Context, obj *sourcev1.OCIRepository, url string, opt ...remote.Option) error {
ctxTimeout, cancel := context.WithTimeout(ctx, obj.Spec.Timeout.Duration)
defer cancel()
@ -722,7 +724,7 @@ func (r *OCIRepositoryReconciler) parseRepositoryURL(obj *sourcev1.OCIRepository
return ref.Context().Name(), nil
}
// getArtifactURL determines which tag or digest should be used and returns the OCI artifact FQN.
// getArtifactURL determines which tag or revision should be used and returns the OCI artifact FQN.
func (r *OCIRepositoryReconciler) getArtifactURL(obj *sourcev1.OCIRepository, options []crane.Option) (string, error) {
url, err := r.parseRepositoryURL(obj)
if err != nil {
@ -951,11 +953,9 @@ func (r *OCIRepositoryReconciler) reconcileStorage(ctx context.Context, sp *patc
// and the symlink in the Storage is updated to its path.
func (r *OCIRepositoryReconciler) reconcileArtifact(ctx context.Context, sp *patch.SerialPatcher,
obj *sourcev1.OCIRepository, metadata *sourcev1.Artifact, dir string) (sreconcile.Result, error) {
revision := metadata.Revision
// Create artifact
artifact := r.Storage.NewArtifactFor(obj.Kind, obj, revision,
fmt.Sprintf("%s.tar.gz", r.digestFromRevision(revision)))
artifact := r.Storage.NewArtifactFor(obj.Kind, obj, metadata.Revision,
fmt.Sprintf("%s.tar.gz", r.digestFromRevision(metadata.Revision)))
// Set the ArtifactInStorageCondition if there's no drift.
defer func() {
@ -1140,6 +1140,9 @@ func (r *OCIRepositoryReconciler) notify(ctx context.Context, oldObj, newObj *so
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaRevisionKey): newObj.Status.Artifact.Revision,
fmt.Sprintf("%s/%s", sourcev1.GroupVersion.Group, eventv1.MetaChecksumKey): newObj.Status.Artifact.Checksum,
}
if newObj.Status.Artifact.Digest != "" {
annotations[sourcev1.GroupVersion.Group+"/"+eventv1.MetaDigestKey] = newObj.Status.Artifact.Digest
}
var oldChecksum string
if oldObj.GetArtifact() != nil {

View File

@ -86,7 +86,7 @@ func TestOCIRepository_Reconcile(t *testing.T) {
url string
tag string
semver string
digest string
revision string
mediaType string
operation string
assertArtifact []artifactFixture
@ -95,7 +95,7 @@ func TestOCIRepository_Reconcile(t *testing.T) {
name: "public tag",
url: podinfoVersions["6.1.6"].url,
tag: podinfoVersions["6.1.6"].tag,
digest: fmt.Sprintf("%s/%s", podinfoVersions["6.1.6"].tag, podinfoVersions["6.1.6"].digest.Hex),
revision: fmt.Sprintf("%s@%s", podinfoVersions["6.1.6"].tag, podinfoVersions["6.1.6"].digest.String()),
mediaType: "application/vnd.docker.image.rootfs.diff.tar.gzip",
operation: sourcev1.OCILayerCopy,
assertArtifact: []artifactFixture{
@ -110,10 +110,10 @@ func TestOCIRepository_Reconcile(t *testing.T) {
},
},
{
name: "public semver",
url: podinfoVersions["6.1.5"].url,
semver: ">= 6.1 <= 6.1.5",
digest: fmt.Sprintf("%s/%s", podinfoVersions["6.1.5"].tag, podinfoVersions["6.1.5"].digest.Hex),
name: "public semver",
url: podinfoVersions["6.1.5"].url,
semver: ">= 6.1 <= 6.1.5",
revision: fmt.Sprintf("%s@%s", podinfoVersions["6.1.5"].tag, podinfoVersions["6.1.5"].digest.String()),
assertArtifact: []artifactFixture{
{
expectedPath: "kustomize/deployment.yaml",
@ -177,8 +177,8 @@ func TestOCIRepository_Reconcile(t *testing.T) {
// Wait for the object to be Ready
waitForSourceReadyWithArtifact(ctx, g, obj)
// Check if the revision matches the expected digest
g.Expect(obj.Status.Artifact.Revision).To(Equal(tt.digest))
// Check if the revision matches the expected revision
g.Expect(obj.Status.Artifact.Revision).To(Equal(tt.revision))
// Check if the metadata matches the expected annotations
g.Expect(obj.Status.Artifact.Metadata[oci.SourceAnnotation]).To(ContainSubstring("podinfo"))
@ -293,7 +293,6 @@ func TestOCIRepository_Reconcile_MediaType(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
g := NewWithT(t)
ns, err := testEnv.CreateNamespace(ctx, "ocirepository-mediatype-test")
@ -383,8 +382,8 @@ func TestOCIRepository_reconcileSource_authStrategy(t *testing.T) {
name: "HTTP without basic auth",
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
},
},
{
@ -404,8 +403,8 @@ func TestOCIRepository_reconcileSource_authStrategy(t *testing.T) {
includeSecret: true,
},
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
},
},
{
@ -425,8 +424,8 @@ func TestOCIRepository_reconcileSource_authStrategy(t *testing.T) {
includeSA: true,
},
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
},
},
{
@ -508,8 +507,8 @@ func TestOCIRepository_reconcileSource_authStrategy(t *testing.T) {
},
},
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
},
},
{
@ -580,8 +579,8 @@ func TestOCIRepository_reconcileSource_authStrategy(t *testing.T) {
},
provider: "azure",
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
},
},
}
@ -678,7 +677,7 @@ func TestOCIRepository_reconcileSource_authStrategy(t *testing.T) {
assertConditions := tt.assertConditions
for k := range assertConditions {
assertConditions[k].Message = strings.ReplaceAll(assertConditions[k].Message, "<digest>", fmt.Sprintf("%s/%s", img.tag, img.digest.Hex))
assertConditions[k].Message = strings.ReplaceAll(assertConditions[k].Message, "<revision>", fmt.Sprintf("%s@%s", img.tag, img.digest.String()))
assertConditions[k].Message = strings.ReplaceAll(assertConditions[k].Message, "<url>", repoURL)
}
@ -750,7 +749,7 @@ func TestOCIRepository_CertSecret(t *testing.T) {
digest: pi.digest,
certSecret: &tlsSecretClientCert,
expectreadyconition: true,
expectedstatusmessage: fmt.Sprintf("stored artifact for digest '%s'", pi.digest.Hex),
expectedstatusmessage: fmt.Sprintf("stored artifact for digest '%s'", pi.digest.String()),
},
{
name: "test connection with no secret",
@ -874,7 +873,7 @@ func TestOCIRepository_reconcileSource_remoteReference(t *testing.T) {
{
name: "no reference (latest tag)",
want: sreconcile.ResultSuccess,
wantRevision: fmt.Sprintf("latest/%s", img6.digest.Hex),
wantRevision: fmt.Sprintf("latest@%s", img6.digest.String()),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision"),
@ -886,7 +885,7 @@ func TestOCIRepository_reconcileSource_remoteReference(t *testing.T) {
Tag: "6.1.6",
},
want: sreconcile.ResultSuccess,
wantRevision: fmt.Sprintf("%s/%s", img6.tag, img6.digest.Hex),
wantRevision: fmt.Sprintf("%s@%s", img6.tag, img6.digest.String()),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision"),
@ -898,7 +897,7 @@ func TestOCIRepository_reconcileSource_remoteReference(t *testing.T) {
SemVer: ">= 6.1.5",
},
want: sreconcile.ResultSuccess,
wantRevision: fmt.Sprintf("%s/%s", img6.tag, img6.digest.Hex),
wantRevision: fmt.Sprintf("%s@%s", img6.tag, img6.digest.String()),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision"),
@ -909,7 +908,7 @@ func TestOCIRepository_reconcileSource_remoteReference(t *testing.T) {
reference: &sourcev1.OCIRepositoryRef{
Digest: img6.digest.String(),
},
wantRevision: img6.digest.Hex,
wantRevision: img6.digest.String(),
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision"),
@ -956,7 +955,7 @@ func TestOCIRepository_reconcileSource_remoteReference(t *testing.T) {
Tag: "6.1.5",
},
want: sreconcile.ResultSuccess,
wantRevision: fmt.Sprintf("%s/%s", img6.tag, img6.digest.Hex),
wantRevision: fmt.Sprintf("%s@%s", img6.tag, img6.digest.String()),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision"),
@ -970,7 +969,7 @@ func TestOCIRepository_reconcileSource_remoteReference(t *testing.T) {
Digest: img5.digest.String(),
},
want: sreconcile.ResultSuccess,
wantRevision: img5.digest.Hex,
wantRevision: img5.digest.String(),
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision"),
@ -1058,13 +1057,13 @@ func TestOCIRepository_reconcileSource_verifyOCISourceSignature(t *testing.T) {
reference: &sourcev1.OCIRepositoryRef{
Tag: "6.1.4",
},
digest: img4.digest.Hex,
digest: img4.digest.String(),
shouldSign: true,
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(sourcev1.SourceVerifiedCondition, meta.SucceededReason, "verified signature of revision <digest>"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.TrueCondition(sourcev1.SourceVerifiedCondition, meta.SucceededReason, "verified signature of revision <revision>"),
},
},
{
@ -1072,13 +1071,13 @@ func TestOCIRepository_reconcileSource_verifyOCISourceSignature(t *testing.T) {
reference: &sourcev1.OCIRepositoryRef{
Tag: "6.1.5",
},
digest: img5.digest.Hex,
digest: img5.digest.String(),
wantErr: true,
wantErrMsg: "failed to verify the signature using provider 'cosign': no matching signatures were found for '<url>'",
want: sreconcile.ResultEmpty,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.FalseCondition(sourcev1.SourceVerifiedCondition, sourcev1.VerificationError, "failed to verify the signature using provider '<provider>': no matching signatures were found for '<url>'"),
},
},
@ -1087,34 +1086,34 @@ func TestOCIRepository_reconcileSource_verifyOCISourceSignature(t *testing.T) {
reference: &sourcev1.OCIRepositoryRef{
Tag: "6.1.5",
},
digest: img5.digest.Hex,
digest: img5.digest.String(),
wantErr: true,
want: sreconcile.ResultEmpty,
keyless: true,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.FalseCondition(sourcev1.SourceVerifiedCondition, sourcev1.VerificationError, "failed to verify the signature using provider '<provider> keyless': no matching signatures"),
},
},
{
name: "verify failed before, removed from spec, remove condition",
reference: &sourcev1.OCIRepositoryRef{Tag: "6.1.4"},
digest: img4.digest.Hex,
digest: img4.digest.String(),
beforeFunc: func(obj *sourcev1.OCIRepository) {
conditions.MarkFalse(obj, sourcev1.SourceVerifiedCondition, "VerifyFailed", "fail msg")
obj.Spec.Verify = nil
obj.Status.Artifact = &sourcev1.Artifact{Revision: fmt.Sprintf("%s/%s", img4.tag, img4.digest.Hex)}
obj.Status.Artifact = &sourcev1.Artifact{Revision: fmt.Sprintf("%s@%s", img4.tag, img4.digest.String())}
},
want: sreconcile.ResultSuccess,
},
{
name: "same artifact, verified before, change in obj gen verify again",
reference: &sourcev1.OCIRepositoryRef{Tag: "6.1.4"},
digest: img4.digest.Hex,
digest: img4.digest.String(),
shouldSign: true,
beforeFunc: func(obj *sourcev1.OCIRepository) {
obj.Status.Artifact = &sourcev1.Artifact{Revision: fmt.Sprintf("%s/%s", img4.tag, img4.digest.Hex)}
obj.Status.Artifact = &sourcev1.Artifact{Revision: fmt.Sprintf("%s@%s", img4.tag, img4.digest.String())}
// Set Verified with old observed generation and different reason/message.
conditions.MarkTrue(obj, sourcev1.SourceVerifiedCondition, "Verified", "verified")
// Set new object generation.
@ -1122,17 +1121,17 @@ func TestOCIRepository_reconcileSource_verifyOCISourceSignature(t *testing.T) {
},
want: sreconcile.ResultSuccess,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(sourcev1.SourceVerifiedCondition, meta.SucceededReason, "verified signature of revision <digest>"),
*conditions.TrueCondition(sourcev1.SourceVerifiedCondition, meta.SucceededReason, "verified signature of revision <revision>"),
},
},
{
name: "no verify for already verified, verified condition remains the same",
reference: &sourcev1.OCIRepositoryRef{Tag: "6.1.4"},
digest: img4.digest.Hex,
digest: img4.digest.String(),
shouldSign: true,
beforeFunc: func(obj *sourcev1.OCIRepository) {
// Artifact present and custom verified condition reason/message.
obj.Status.Artifact = &sourcev1.Artifact{Revision: fmt.Sprintf("%s/%s", img4.tag, img4.digest.Hex)}
obj.Status.Artifact = &sourcev1.Artifact{Revision: fmt.Sprintf("%s@%s", img4.tag, img4.digest.String())}
conditions.MarkTrue(obj, sourcev1.SourceVerifiedCondition, "Verified", "verified")
},
want: sreconcile.ResultSuccess,
@ -1145,14 +1144,14 @@ func TestOCIRepository_reconcileSource_verifyOCISourceSignature(t *testing.T) {
reference: &sourcev1.OCIRepositoryRef{
Tag: "6.1.4",
},
digest: img4.digest.Hex,
digest: img4.digest.String(),
shouldSign: true,
insecure: true,
wantErr: true,
want: sreconcile.ResultEmpty,
assertConditions: []metav1.Condition{
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<digest>' for '<url>'"),
*conditions.TrueCondition(meta.ReconcilingCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.UnknownCondition(meta.ReadyCondition, meta.ProgressingReason, "building artifact: new revision '<revision>' for '<url>'"),
*conditions.FalseCondition(sourcev1.SourceVerifiedCondition, sourcev1.VerificationError, "cosign does not support insecure registries"),
},
},
@ -1248,7 +1247,7 @@ func TestOCIRepository_reconcileSource_verifyOCISourceSignature(t *testing.T) {
assertConditions := tt.assertConditions
for k := range assertConditions {
assertConditions[k].Message = strings.ReplaceAll(assertConditions[k].Message, "<digest>", fmt.Sprintf("%s/%s", tt.reference.Tag, tt.digest))
assertConditions[k].Message = strings.ReplaceAll(assertConditions[k].Message, "<revision>", fmt.Sprintf("%s@%s", tt.reference.Tag, tt.digest))
assertConditions[k].Message = strings.ReplaceAll(assertConditions[k].Message, "<url>", artifactURL)
assertConditions[k].Message = strings.ReplaceAll(assertConditions[k].Message, "<provider>", "cosign")
}
@ -1282,7 +1281,7 @@ func TestOCIRepository_reconcileSource_verifyOCISourceSignature(t *testing.T) {
func TestOCIRepository_reconcileSource_noop(t *testing.T) {
g := NewWithT(t)
testRevision := "6.1.5/d1fc4595915714af2492dc4b66097de1e10f80150c8899907d8f8e61c6d6f67d"
testRevision := "6.1.5@sha256:8e4057c22d531d40e12b065443cb0d80394b7257c4dc557cb1fbd4dce892b86d"
tmpDir := t.TempDir()
server, err := setupRegistryServer(ctx, tmpDir, registryOptions{})
@ -1316,6 +1315,17 @@ func TestOCIRepository_reconcileSource_noop(t *testing.T) {
g.Expect(artifact.Metadata).To(BeEmpty())
},
},
{
name: "noop - artifact revisions match (legacy)",
beforeFunc: func(obj *sourcev1.OCIRepository) {
obj.Status.Artifact = &sourcev1.Artifact{
Revision: "6.1.5/8e4057c22d531d40e12b065443cb0d80394b7257c4dc557cb1fbd4dce892b86d",
}
},
afterFunc: func(g *WithT, artifact *sourcev1.Artifact) {
g.Expect(artifact.Metadata).To(BeEmpty())
},
},
{
name: "full reconcile - same rev, unobserved ignore",
beforeFunc: func(obj *sourcev1.OCIRepository) {
@ -1723,9 +1733,9 @@ func TestOCIRepository_getArtifactURL(t *testing.T) {
name: "valid url with digest reference",
url: "oci://ghcr.io/stefanprodan/charts",
reference: &sourcev1.OCIRepositoryRef{
Digest: imgs["6.1.6"].digest.Hex,
Digest: imgs["6.1.6"].digest.String(),
},
want: "ghcr.io/stefanprodan/charts@" + imgs["6.1.6"].digest.Hex,
want: "ghcr.io/stefanprodan/charts@" + imgs["6.1.6"].digest.String(),
},
{
name: "valid url with semver reference",
@ -2236,7 +2246,7 @@ func pushMultiplePodinfoImages(serverURL string, versions ...string) (map[string
func setPodinfoImageAnnotations(img gcrv1.Image, tag string) gcrv1.Image {
metadata := map[string]string{
oci.SourceAnnotation: "https://github.com/stefanprodan/podinfo",
oci.RevisionAnnotation: fmt.Sprintf("%s/SHA", tag),
oci.RevisionAnnotation: fmt.Sprintf("%s@sha1:b3b00fe35424a45d373bf4c7214178bc36fd7872", tag),
}
return mutate.Annotations(img, metadata).(gcrv1.Image)
}

View File

@ -33,15 +33,17 @@ import (
"time"
securejoin "github.com/cyphar/filepath-securejoin"
"github.com/fluxcd/go-git/v5/plumbing/format/gitignore"
"github.com/fluxcd/pkg/lockedfile"
"github.com/fluxcd/pkg/untar"
"github.com/opencontainers/go-digest"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
kerrors "k8s.io/apimachinery/pkg/util/errors"
"github.com/fluxcd/pkg/lockedfile"
"github.com/fluxcd/pkg/sourceignore"
"github.com/fluxcd/pkg/untar"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
intdigest "github.com/fluxcd/source-controller/internal/digest"
sourcefs "github.com/fluxcd/source-controller/internal/fs"
)
@ -358,9 +360,12 @@ func (s *Storage) Archive(artifact *sourcev1.Artifact, dir string, filter Archiv
}
}()
h := newHash()
md, err := intdigest.NewMultiDigester(intdigest.Canonical, digest.SHA256)
if err != nil {
return fmt.Errorf("failed to create digester: %w", err)
}
sz := &writeCounter{}
mw := io.MultiWriter(h, tf, sz)
mw := io.MultiWriter(md, tf, sz)
gw := gzip.NewWriter(mw)
tw := tar.NewWriter(gw)
@ -450,7 +455,8 @@ func (s *Storage) Archive(artifact *sourcev1.Artifact, dir string, filter Archiv
return err
}
artifact.Checksum = fmt.Sprintf("%x", h.Sum(nil))
artifact.Digest = md.Digest(intdigest.Canonical).String()
artifact.Checksum = md.Digest(digest.SHA256).Encoded()
artifact.LastUpdateTime = metav1.Now()
artifact.Size = &sz.written
@ -472,9 +478,12 @@ func (s *Storage) AtomicWriteFile(artifact *sourcev1.Artifact, reader io.Reader,
}
}()
h := newHash()
md, err := intdigest.NewMultiDigester(intdigest.Canonical, digest.SHA256)
if err != nil {
return fmt.Errorf("failed to create digester: %w", err)
}
sz := &writeCounter{}
mw := io.MultiWriter(h, tf, sz)
mw := io.MultiWriter(md, tf, sz)
if _, err := io.Copy(mw, reader); err != nil {
tf.Close()
@ -492,7 +501,8 @@ func (s *Storage) AtomicWriteFile(artifact *sourcev1.Artifact, reader io.Reader,
return err
}
artifact.Checksum = fmt.Sprintf("%x", h.Sum(nil))
artifact.Digest = md.Digest(intdigest.Canonical).String()
artifact.Checksum = md.Digest(digest.SHA256).Encoded()
artifact.LastUpdateTime = metav1.Now()
artifact.Size = &sz.written
@ -514,9 +524,12 @@ func (s *Storage) Copy(artifact *sourcev1.Artifact, reader io.Reader) (err error
}
}()
h := newHash()
md, err := intdigest.NewMultiDigester(intdigest.Canonical, digest.SHA256)
if err != nil {
return fmt.Errorf("failed to create digester: %w", err)
}
sz := &writeCounter{}
mw := io.MultiWriter(h, tf, sz)
mw := io.MultiWriter(md, tf, sz)
if _, err := io.Copy(mw, reader); err != nil {
tf.Close()
@ -530,7 +543,8 @@ func (s *Storage) Copy(artifact *sourcev1.Artifact, reader io.Reader) (err error
return err
}
artifact.Checksum = fmt.Sprintf("%x", h.Sum(nil))
artifact.Digest = md.Digest(intdigest.Canonical).String()
artifact.Checksum = md.Digest(digest.SHA256).Encoded()
artifact.LastUpdateTime = metav1.Now()
artifact.Size = &sz.written

View File

@ -1253,7 +1253,20 @@ string
</td>
<td>
<em>(Optional)</em>
<p>Checksum is the SHA256 checksum of the Artifact file.</p>
<p>Checksum is the SHA256 checksum of the Artifact file.
Deprecated: use Artifact.Digest instead.</p>
</td>
</tr>
<tr>
<td>
<code>digest</code><br>
<em>
string
</em>
</td>
<td>
<em>(Optional)</em>
<p>Digest is the digest of the file in the form of &lsquo;<algorithm>:<checksum>&rsquo;.</p>
</td>
</tr>
<tr>

View File

@ -48,8 +48,8 @@ In the above example:
- A list of object keys and their [etags](https://en.wikipedia.org/wiki/HTTP_ETag)
in the `.spec.bucketName` bucket is compiled, while filtering the keys using
[default ignore rules](#default-exclusions).
- The SHA256 sum of the list is used as Artifact revision, reported
in-cluster in the `.status.artifact.revision` field.
- The digest (algorithm defaults to SHA256) of the list is used as Artifact
revision, reported in-cluster in the `.status.artifact.revision` field.
- When the current Bucket revision differs from the latest calculated revision,
all objects are fetched and archived.
- The new Artifact is reported in the `.status.artifact` field.
@ -71,7 +71,7 @@ control over.
```console
NAME ENDPOINT AGE READY STATUS
minio-bucket minio.example.com 34s True stored artifact for revision 'e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'
minio-bucket minio.example.com 34s True stored artifact for revision 'sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'
```
3. Run `kubectl describe bucket minio-bucket` to see the [Artifact](#artifact)
@ -82,19 +82,21 @@ control over.
Status:
Artifact:
Checksum: 72aa638abb455ca5f9ef4825b949fd2de4d4be0a74895bf7ed2338622cd12686
Digest: sha256:72aa638abb455ca5f9ef4825b949fd2de4d4be0a74895bf7ed2338622cd12686
Last Update Time: 2022-02-01T23:43:38Z
Path: bucket/default/minio-bucket/e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855.tar.gz
Revision: e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855
Revision: sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855
Size: 38099
URL: http://source-controller.source-system.svc.cluster.local./bucket/default/minio-bucket/e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855.tar.gz
Conditions:
Last Transition Time: 2022-02-01T23:43:38Z
Message: stored artifact for revision 'e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'
Message: stored artifact for revision 'sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'
Observed Generation: 1
Reason: Succeeded
Status: True
Type: Ready
Last Transition Time: 2022-02-01T23:43:38Z
Message: stored artifact for revision 'e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'
Message: stored artifact for revision 'sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'
Observed Generation: 1
Reason: Succeeded
Status: True
@ -104,7 +106,7 @@ control over.
Events:
Type Reason Age From Message
---- ------ ---- ---- -------
Normal NewArtifact 82s source-controller fetched 16 files from 'example'
Normal NewArtifact 82s source-controller stored artifact with 16 fetched files from 'example' bucket
```
## Writing a Bucket spec
@ -906,7 +908,7 @@ lists
```console
LAST SEEN TYPE REASON OBJECT MESSAGE
2m30s Normal NewArtifact bucket/<bucket-name> fetched 16 files with revision from 'my-new-bucket'
36s Normal ArtifactUpToDate bucket/<bucket-name> artifact up-to-date with remote revision: 'e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'
36s Normal ArtifactUpToDate bucket/<bucket-name> artifact up-to-date with remote revision: 'sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855'
18s Warning BucketOperationFailed bucket/<bucket-name> bucket 'my-new-bucket' does not exist
```
@ -936,9 +938,11 @@ metadata:
status:
artifact:
checksum: cbec34947cc2f36dee8adcdd12ee62ca6a8a36699fc6e56f6220385ad5bd421a
digest: sha256:cbec34947cc2f36dee8adcdd12ee62ca6a8a36699fc6e56f6220385ad5bd421a
lastUpdateTime: "2022-01-28T10:30:30Z"
path: bucket/<namespace>/<bucket-name>/c3ab8ff13720e8ad9047dd39466b3c8974e592c2fa383d4a3960714caef0c4f2.tar.gz
revision: c3ab8ff13720e8ad9047dd39466b3c8974e592c2fa383d4a3960714caef0c4f2
revision: sha256:c3ab8ff13720e8ad9047dd39466b3c8974e592c2fa383d4a3960714caef0c4f2
size: 38099
url: http://source-controller.<namespace>.svc.cluster.local./bucket/<namespace>/<bucket-name>/c3ab8ff13720e8ad9047dd39466b3c8974e592c2fa383d4a3960714caef0c4f2.tar.gz
```

View File

@ -49,7 +49,7 @@ You can run this example by saving the manifest into `gitrepository.yaml`.
```console
NAME URL AGE READY STATUS
podinfo https://github.com/stefanprodan/podinfo 5s True stored artifact for revision 'master/132f4e719209eb10b9485302f8593fc0e680f4fc'
podinfo https://github.com/stefanprodan/podinfo 5s True stored artifact for revision 'master@sha1:132f4e719209eb10b9485302f8593fc0e680f4fc'
```
3. Run `kubectl describe gitrepository podinfo` to see the [Artifact](#artifact)
@ -60,19 +60,21 @@ You can run this example by saving the manifest into `gitrepository.yaml`.
Status:
Artifact:
Checksum: 95e386f421272710c4cedbbd8607dbbaa019d500e7a5a0b6720bc7bebefc7bf2
Digest: sha256:95e386f421272710c4cedbbd8607dbbaa019d500e7a5a0b6720bc7bebefc7bf2
Last Update Time: 2022-02-14T11:23:36Z
Path: gitrepository/default/podinfo/132f4e719209eb10b9485302f8593fc0e680f4fc.tar.gz
Revision: master/132f4e719209eb10b9485302f8593fc0e680f4fc
Revision: master@sha1:132f4e719209eb10b9485302f8593fc0e680f4fc
Size: 91318
URL: http://source-controller.source-system.svc.cluster.local./gitrepository/default/podinfo/132f4e719209eb10b9485302f8593fc0e680f4fc.tar.gz
Conditions:
Last Transition Time: 2022-02-14T11:23:36Z
Message: stored artifact for revision 'master/132f4e719209eb10b9485302f8593fc0e680f4fc'
Message: stored artifact for revision 'master@sha1:132f4e719209eb10b9485302f8593fc0e680f4fc'
Observed Generation: 1
Reason: Succeeded
Status: True
Type: Ready
Last Transition Time: 2022-02-14T11:23:36Z
Message: stored artifact for revision 'master/132f4e719209eb10b9485302f8593fc0e680f4fc'
Message: stored artifact for revision 'master@sha1:132f4e719209eb10b9485302f8593fc0e680f4fc'
Observed Generation: 1
Reason: Succeeded
Status: True
@ -670,7 +672,7 @@ lists
```console
LAST SEEN TYPE REASON OBJECT MESSAGE
2m14s Normal NewArtifact gitrepository/<repository-name> stored artifact for commit 'Merge pull request #160 from stefanprodan/release-6.0.3'
36s Normal ArtifactUpToDate gitrepository/<repository-name> artifact up-to-date with remote revision: 'master/132f4e719209eb10b9485302f8593fc0e680f4fc'
36s Normal ArtifactUpToDate gitrepository/<repository-name> artifact up-to-date with remote revision: 'master@sha1:132f4e719209eb10b9485302f8593fc0e680f4fc'
94s Warning GitOperationFailed gitrepository/<repository-name> failed to checkout and determine revision: unable to clone 'https://github.com/stefanprodan/podinfo': couldn't find remote ref "refs/heads/invalid"
```
@ -700,9 +702,11 @@ metadata:
status:
artifact:
checksum: e750c7a46724acaef8f8aa926259af30bbd9face2ae065ae8896ba5ee5ab832b
digest: sha256:e750c7a46724acaef8f8aa926259af30bbd9face2ae065ae8896ba5ee5ab832b
lastUpdateTime: "2022-01-29T06:59:23Z"
path: gitrepository/<namespace>/<repository-name>/c3ab8ff13720e8ad9047dd39466b3c8974e592c2fa383d4a3960714caef0c4f2.tar.gz
revision: master/363a6a8fe6a7f13e05d34c163b0ef02a777da20a
revision: master@sha1:363a6a8fe6a7f13e05d34c163b0ef02a777da20a
size: 91318
url: http://source-controller.<namespace>.svc.cluster.local./gitrepository/<namespace>/<repository-name>/363a6a8fe6a7f13e05d34c163b0ef02a777da20a.tar.gz
```

View File

@ -65,12 +65,14 @@ helm-controller.
```console
Status:
Observed Source Artifact Revision: 83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
Observed Source Artifact Revision: sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
Artifact:
Checksum: 6c3cc3b955bce1686036ae6822ee2ca0ef6ecb994e3f2d19eaf3ec03dcba84b3
Digest: sha256:6c3cc3b955bce1686036ae6822ee2ca0ef6ecb994e3f2d19eaf3ec03dcba84b3
Last Update Time: 2022-02-13T11:24:10Z
Path: helmchart/default/podinfo/podinfo-5.2.1.tgz
Revision: 5.2.1
Size: 14166
URL: http://source-controller.flux-system.svc.cluster.local./helmchart/default/podinfo/podinfo-5.2.1.tgz
Conditions:
Last Transition Time: 2022-02-13T11:24:10Z
@ -555,9 +557,11 @@ metadata:
status:
artifact:
checksum: e30b95a08787de69ffdad3c232d65cfb131b5b50c6fd44295f48a078fceaa44e
digest: sha256:e30b95a08787de69ffdad3c232d65cfb131b5b50c6fd44295f48a078fceaa44e
lastUpdateTime: "2022-02-10T18:53:47Z"
path: helmchart/<source-namespace>/<chart-name>/<chart-name>-<chart-version>.tgz
revision: 6.0.3
size: 14166
url: http://source-controller.flux-system.svc.cluster.local./helmchart/<source-namespace>/<chart-name>/<chart-name>-<chart-version>.tgz
```
@ -576,9 +580,11 @@ metadata:
status:
artifact:
checksum: ee68224ded207ebb18a8e9730cf3313fa6bc1f31e6d8d3943ab541113559bb52
digest: sha256:ee68224ded207ebb18a8e9730cf3313fa6bc1f31e6d8d3943ab541113559bb52
lastUpdateTime: "2022-02-28T08:07:12Z"
path: helmchart/<source-namespace>/<chart-name>/<chart-name>-6.0.3+1.tgz
revision: 6.0.3+1
size: 14166
url: http://source-controller.flux-system.svc.cluster.local./helmchart/<source-namespace>/<chart-name>/<chart-name>-6.0.3+1.tgz
observedGeneration: 1
...
@ -600,9 +606,11 @@ metadata:
status:
artifact:
checksum: 8d1f0ac3f4b0e8759a32180086f17ac87ca04e5d46c356e67f97e97616ef4718
digest: sha256:8d1f0ac3f4b0e8759a32180086f17ac87ca04e5d46c356e67f97e97616ef4718
lastUpdateTime: "2022-02-28T08:07:12Z"
path: helmchart/<source-namespace>/<chart-name>/<chart-name>-6.0.3+4e5cbb7b97d0.tgz
revision: 6.0.3+4e5cbb7b97d0
size: 14166
url: http://source-controller.flux-system.svc.cluster.local./helmchart/<source-namespace>/<chart-name>/<chart-name>-6.0.3+4e5cbb7b97d0.tgz
```

View File

@ -34,9 +34,9 @@ In the above example:
- The source-controller fetches the Helm repository index YAML every five
minutes from `https://stefanprodan.github.io/podinfo`, indicated by the
`.spec.interval` and `.spec.url` fields.
- The SHA256 sum of the Helm repository index after stable sorting the entries
is used as Artifact revision, reported in-cluster in the
`.status.artifact.revision` field.
- The digest (algorithm defaults to SHA256) of the Helm repository index after
stable sorting the entries is used as Artifact revision, reported in-cluster
in the `.status.artifact.revision` field.
- When the current HelmRepository revision differs from the latest fetched
revision, it is stored as a new Artifact.
- The new Artifact is reported in the `.status.artifact` field.
@ -53,7 +53,7 @@ You can run this example by saving the manifest into `helmrepository.yaml`.
```console
NAME URL AGE READY STATUS
podinfo https://stefanprodan.github.io/podinfo 4s True stored artifact for revision '83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111'
podinfo https://stefanprodan.github.io/podinfo 4s True stored artifact for revision 'sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111'
```
3. Run `kubectl describe helmrepository podinfo` to see the [Artifact](#artifact)
@ -64,19 +64,21 @@ You can run this example by saving the manifest into `helmrepository.yaml`.
Status:
Artifact:
Checksum: 83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
Digest: sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
Last Update Time: 2022-02-04T09:55:58Z
Path: helmrepository/default/podinfo/index-83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111.yaml
Revision: 83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
Revision: sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
Size: 40898
URL: http://source-controller.flux-system.svc.cluster.local./helmrepository/default/podinfo/index-83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111.yaml
Conditions:
Last Transition Time: 2022-02-04T09:55:58Z
Message: stored artifact for revision '83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111'
Message: stored artifact for revision 'sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111'
Observed Generation: 1
Reason: Succeeded
Status: True
Type: Ready
Last Transition Time: 2022-02-04T09:55:58Z
Message: stored artifact for revision '83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111'
Message: stored artifact for revision 'sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111'
Observed Generation: 1
Reason: Succeeded
Status: True
@ -609,7 +611,7 @@ lists
LAST SEEN TYPE REASON OBJECT MESSAGE
107s Warning Failed helmrepository/<repository-name> failed to construct Helm client: scheme "invalid" not supported
7s Normal NewArtifact helmrepository/<repository-name> fetched index of size 30.88kB from 'https://stefanprodan.github.io/podinfo'
3s Normal ArtifactUpToDate helmrepository/<repository-name> artifact up-to-date with remote revision: '83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111'
3s Normal ArtifactUpToDate helmrepository/<repository-name> artifact up-to-date with remote revision: 'sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111'
```
Besides being reported in Events, the reconciliation errors are also logged by
@ -640,9 +642,11 @@ metadata:
status:
artifact:
checksum: 83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
digest: sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
lastUpdateTime: "2022-02-04T09:55:58Z"
path: helmrepository/<namespace>/<repository-name>/index-83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111.yaml
revision: 83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
revision: sha256:83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111
size: 40898
url: http://source-controller.flux-system.svc.cluster.local./helmrepository/<namespace>/<repository-name>/index-83a3c595163a6ff0333e0154c790383b5be441b9db632cb36da11db1c4ece111.yaml
```

View File

@ -49,7 +49,7 @@ You can run this example by saving the manifest into `ocirepository.yaml`.
```console
NAME URL AGE READY STATUS
podinfo oci://ghcr.io/stefanprodan/manifests/podinfo 5s True stored artifact with revision 'latest/3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
podinfo oci://ghcr.io/stefanprodan/manifests/podinfo 5s True stored artifact with revision 'latest@sha256:3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
```
3. Run `kubectl describe ocirepository podinfo` to see the [Artifact](#artifact)
@ -60,19 +60,21 @@ You can run this example by saving the manifest into `ocirepository.yaml`.
Status:
Artifact:
Checksum: d7e924b4882e55b97627355c7b3d2e711e9b54303afa2f50c25377f4df66a83b
Digest: sha256:d7e924b4882e55b97627355c7b3d2e711e9b54303afa2f50c25377f4df66a83b
Last Update Time: 2022-06-14T11:23:36Z
Path: ocirepository/default/podinfo/3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de.tar.gz
Revision: latest/3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de
Revision: latest@sha256:3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de
Size: 1105
URL: http://source-controller.flux-system.svc.cluster.local./ocirepository/oci/podinfo/3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de.tar.gz
Conditions:
Last Transition Time: 2022-06-14T11:23:36Z
Message: stored artifact for revision 'latest/3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
Message: stored artifact for revision 'latest@sha256:3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
Observed Generation: 1
Reason: Succeeded
Status: True
Type: Ready
Last Transition Time: 2022-06-14T11:23:36Z
Message: stored artifact for revision 'latest/3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
Message: stored artifact for revision 'latest@sha256:3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
Observed Generation: 1
Reason: Succeeded
Status: True
@ -691,8 +693,8 @@ lists
```console
LAST SEEN TYPE REASON OBJECT MESSAGE
2m14s Normal NewArtifact ocirepository/<repository-name> stored artifact for revision 'latest/3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
36s Normal ArtifactUpToDate ocirepository/<repository-name> artifact up-to-date with remote revision: 'latest/3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
2m14s Normal NewArtifact ocirepository/<repository-name> stored artifact for revision 'latest@sha256:3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
36s Normal ArtifactUpToDate ocirepository/<repository-name> artifact up-to-date with remote revision: 'latest@sha256:3b6cdcc7adcc9a84d3214ee1c029543789d90b5ae69debe9efa3f66e982875de'
94s Warning OCIOperationFailed ocirepository/<repository-name> failed to pull artifact from 'oci://ghcr.io/stefanprodan/manifests/podinfo': couldn't find tag "0.0.1"
```
@ -731,13 +733,15 @@ metadata:
status:
artifact:
checksum: 9f3bc0f341d4ecf2bab460cc59320a2a9ea292f01d7b96e32740a9abfd341088
digest: sha256:9f3bc0f341d4ecf2bab460cc59320a2a9ea292f01d7b96e32740a9abfd341088
lastUpdateTime: "2022-08-08T09:35:45Z"
metadata:
org.opencontainers.image.created: "2022-08-08T12:31:41+03:00"
org.opencontainers.image.revision: 6.1.8/b3b00fe35424a45d373bf4c7214178bc36fd7872
org.opencontainers.image.source: https://github.com/stefanprodan/podinfo.git
path: ocirepository/<namespace>/<repository-name>/<digest>.tar.gz
revision: <tag>/<digest>
revision: <tag>@<digest>
size: 1105
url: http://source-controller.<namespace>.svc.cluster.local./ocirepository/<namespace>/<repository-name>/<digest>.tar.gz
```

37
go.mod
View File

@ -7,8 +7,9 @@ replace github.com/fluxcd/source-controller/api => ./api
// Fix CVE-2022-1996 (for v2, Go Modules incompatible)
replace github.com/emicklei/go-restful => github.com/emicklei/go-restful v2.16.0+incompatible
// The util.Walk func was never release as a tag.
replace github.com/go-git/go-billy/v5 => github.com/go-git/go-billy/v5 v5.0.0-20210804024030-7ab80d7c013d
// Replace digest lib to master to gather access to BLAKE3.
// xref: https://github.com/opencontainers/go-digest/pull/66
replace github.com/opencontainers/go-digest => github.com/opencontainers/go-digest v1.0.1-0.20220411205349-bde1400a84be
require (
cloud.google.com/go/storage v1.29.0
@ -22,10 +23,10 @@ require (
github.com/docker/cli v20.10.23+incompatible
github.com/docker/go-units v0.5.0
github.com/fluxcd/go-git/v5 v5.0.0-20221219190809-2e5c9d01cfc4
github.com/fluxcd/pkg/apis/event v0.3.0
github.com/fluxcd/pkg/apis/event v0.4.0
github.com/fluxcd/pkg/apis/meta v0.19.0
github.com/fluxcd/pkg/git v0.8.0
github.com/fluxcd/pkg/git/gogit v0.5.0
github.com/fluxcd/pkg/git v0.9.0
github.com/fluxcd/pkg/git/gogit v0.6.0
github.com/fluxcd/pkg/gittestserver v0.8.0
github.com/fluxcd/pkg/helmtestserver v0.11.0
github.com/fluxcd/pkg/lockedfile v0.1.0
@ -38,13 +39,15 @@ require (
github.com/fluxcd/pkg/untar v0.2.0
github.com/fluxcd/pkg/version v0.2.0
github.com/fluxcd/source-controller/api v0.34.0
github.com/go-git/go-billy/v5 v5.4.0
github.com/go-git/go-billy/v5 v5.4.1
github.com/go-logr/logr v1.2.3
github.com/google/go-containerregistry v0.13.0
github.com/google/go-containerregistry/pkg/authn/k8schain v0.0.0-20230126080250-11843ba2d084
github.com/google/uuid v1.3.0
github.com/minio/minio-go/v7 v7.0.47
github.com/onsi/gomega v1.26.0
github.com/opencontainers/go-digest v1.0.0
github.com/opencontainers/go-digest/blake3 v0.0.0-20220411205349-bde1400a84be
github.com/ory/dockertest/v3 v3.9.1
github.com/otiai10/copy v1.9.0
github.com/phayes/freeport v0.0.0-20220201140144-74d24b5ae9f5
@ -53,7 +56,7 @@ require (
github.com/sigstore/sigstore v1.5.1
github.com/sirupsen/logrus v1.9.0
github.com/spf13/pflag v1.0.5
golang.org/x/crypto v0.5.0
golang.org/x/crypto v0.6.0
golang.org/x/sync v0.1.0
google.golang.org/api v0.108.0
gotest.tools v2.2.0+incompatible
@ -93,7 +96,7 @@ require (
github.com/Masterminds/squirrel v1.5.3 // indirect
github.com/Microsoft/go-winio v0.6.0 // indirect
github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5 // indirect
github.com/ProtonMail/go-crypto v0.0.0-20221026131551-cf6655e29de4 // indirect
github.com/ProtonMail/go-crypto v0.0.0-20230201104953-d1d05f4e2bfb // indirect
github.com/Shopify/logrus-bugsnag v0.0.0-20171204204709-577dee27f20d // indirect
github.com/ThalesIgnite/crypto11 v1.2.5 // indirect
github.com/acomagu/bufpipe v1.0.3 // indirect
@ -137,7 +140,7 @@ require (
github.com/chai2010/gettext-go v1.0.2 // indirect
github.com/chrismellard/docker-credential-acr-env v0.0.0-20221002210726-e883f69e0206 // indirect
github.com/clbanning/mxj/v2 v2.5.6 // indirect
github.com/cloudflare/circl v1.3.0 // indirect
github.com/cloudflare/circl v1.3.2 // indirect
github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4 // indirect
github.com/cncf/xds/go v0.0.0-20211130200136-a8f946100490 // indirect
github.com/common-nighthawk/go-figure v0.0.0-20210622060536-734e95fb86be // indirect
@ -277,7 +280,6 @@ require (
github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect
github.com/oklog/ulid v1.3.1 // indirect
github.com/olekukonko/tablewriter v0.0.5 // indirect
github.com/opencontainers/go-digest v1.0.0 // indirect
github.com/opencontainers/image-spec v1.1.0-rc2 // indirect
github.com/opencontainers/runc v1.1.2 // indirect
github.com/opentracing/opentracing-go v1.2.0 // indirect
@ -298,7 +300,7 @@ require (
github.com/sassoftware/relic v0.0.0-20210427151427-dfb082b79b74 // indirect
github.com/secure-systems-lab/go-securesystemslib v0.4.0 // indirect
github.com/segmentio/ksuid v1.0.4 // indirect
github.com/sergi/go-diff v1.2.0 // indirect
github.com/sergi/go-diff v1.3.1 // indirect
github.com/shibumi/go-pathspec v1.3.0 // indirect
github.com/shopspring/decimal v1.3.1 // indirect
github.com/sigstore/fulcio v0.6.0 // indirect
@ -334,6 +336,7 @@ require (
github.com/yvasiyarov/go-metrics v0.0.0-20150112132944-c25f46c4b940 // indirect
github.com/yvasiyarov/gorelic v0.0.7 // indirect
github.com/yvasiyarov/newrelic_platform_go v0.0.0-20160601141957-9c099fbc30e9 // indirect
github.com/zeebo/blake3 v0.1.1 // indirect
github.com/zeebo/errs v1.2.2 // indirect
go.etcd.io/bbolt v1.3.6 // indirect
go.etcd.io/etcd/api/v3 v3.6.0-alpha.0 // indirect
@ -362,14 +365,14 @@ require (
go.uber.org/multierr v1.8.0 // indirect
go.uber.org/zap v1.24.0 // indirect
golang.org/x/exp v0.0.0-20220823124025-807a23277127 // indirect
golang.org/x/mod v0.7.0 // indirect
golang.org/x/net v0.5.0 // indirect
golang.org/x/mod v0.8.0 // indirect
golang.org/x/net v0.6.0 // indirect
golang.org/x/oauth2 v0.4.0 // indirect
golang.org/x/sys v0.4.0 // indirect
golang.org/x/term v0.4.0 // indirect
golang.org/x/text v0.6.0 // indirect
golang.org/x/sys v0.5.0 // indirect
golang.org/x/term v0.5.0 // indirect
golang.org/x/text v0.7.0 // indirect
golang.org/x/time v0.3.0 // indirect
golang.org/x/tools v0.3.0 // indirect
golang.org/x/tools v0.6.0 // indirect
golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
gomodules.xyz/jsonpatch/v2 v2.2.0 // indirect
google.golang.org/appengine v1.6.7 // indirect

68
go.sum
View File

@ -171,8 +171,9 @@ github.com/Microsoft/hcsshim v0.9.6 h1:VwnDOgLeoi2du6dAznfmspNqTiwczvjv4K7NxuY9j
github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5 h1:TngWCqHvy9oXAN6lEVMRuU21PR1EtLVZJmdB18Gu3Rw=
github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5/go.mod h1:lmUJ/7eu/Q8D7ML55dXQrVaamCz2vxCfdQBasLZfHKk=
github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU=
github.com/ProtonMail/go-crypto v0.0.0-20221026131551-cf6655e29de4 h1:ra2OtmuW0AE5csawV4YXMNGNQQXvLRps3z2Z59OPO+I=
github.com/ProtonMail/go-crypto v0.0.0-20221026131551-cf6655e29de4/go.mod h1:UBYPn8k0D56RtnR8RFQMjmh4KrZzWJ5o7Z9SYjossQ8=
github.com/ProtonMail/go-crypto v0.0.0-20230201104953-d1d05f4e2bfb h1:Vx1Bw/nGULx+FuY7Sw+8ZDpOx9XOdA+mOfo678SqkbU=
github.com/ProtonMail/go-crypto v0.0.0-20230201104953-d1d05f4e2bfb/go.mod h1:I0gYDMZ6Z5GRU7l58bNFSkPTFN6Yl12dsUlAZ8xy98g=
github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0=
github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE=
github.com/Shopify/logrus-bugsnag v0.0.0-20171204204709-577dee27f20d h1:UrqY+r/OJnIp5u0s1SbQ8dVfLCZJsnvazdBP5hS4iRs=
@ -370,8 +371,8 @@ github.com/clbanning/mxj/v2 v2.5.6/go.mod h1:hNiWqW14h+kc+MdF9C6/YoRfjEJoR3ou6tn
github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE=
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
github.com/cloudflare/circl v1.1.0/go.mod h1:prBCrKB9DV4poKZY1l9zBXg2QJY7mvgRvtMxxK7fi4I=
github.com/cloudflare/circl v1.3.0 h1:Anq00jxDtoyX3+aCaYUZ0vXC5r4k4epberfWGDXV1zE=
github.com/cloudflare/circl v1.3.0/go.mod h1:+CauBF6R70Jqcyl8N2hC8pAXYbWkGIezuSbuGLtRhnw=
github.com/cloudflare/circl v1.3.2 h1:VWp8dY3yH69fdM7lM6A1+NhhVoDu9vqK0jOgmkQHFWk=
github.com/cloudflare/circl v1.3.2/go.mod h1:+CauBF6R70Jqcyl8N2hC8pAXYbWkGIezuSbuGLtRhnw=
github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc=
github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk=
github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk=
@ -523,14 +524,14 @@ github.com/fluxcd/go-git/v5 v5.0.0-20221219190809-2e5c9d01cfc4 h1:Gm5sGGk+/Wq6Rh
github.com/fluxcd/go-git/v5 v5.0.0-20221219190809-2e5c9d01cfc4/go.mod h1:raWgfUV7lDQVXp4QXUaeNNJkRVKz97UQuF+0kdY7Vmo=
github.com/fluxcd/pkg/apis/acl v0.1.0 h1:EoAl377hDQYL3WqanWCdifauXqXbMyFuK82NnX6pH4Q=
github.com/fluxcd/pkg/apis/acl v0.1.0/go.mod h1:zfEZzz169Oap034EsDhmCAGgnWlcWmIObZjYMusoXS8=
github.com/fluxcd/pkg/apis/event v0.3.0 h1:B+IXmfSniUGfoczheNAH0YULgS+ejxMl58RyWlvLa1c=
github.com/fluxcd/pkg/apis/event v0.3.0/go.mod h1:xYOOlf+9gCBSYcs93N2XAbJvSVwuVBDBUzqhR+cAo7M=
github.com/fluxcd/pkg/apis/event v0.4.0 h1:UPCC269KjgKgkmtiCiBq/DNue/EpXy8Tq1zFx7oRXZM=
github.com/fluxcd/pkg/apis/event v0.4.0/go.mod h1:xYOOlf+9gCBSYcs93N2XAbJvSVwuVBDBUzqhR+cAo7M=
github.com/fluxcd/pkg/apis/meta v0.19.0 h1:CX75e/eaRWZDTzNdMSWomY1InlssLKcS8GQDSg/aopI=
github.com/fluxcd/pkg/apis/meta v0.19.0/go.mod h1:7b6prDPsViyAzoY7eRfSPS0/MbXpGGsOMvRq2QrTKa4=
github.com/fluxcd/pkg/git v0.8.0 h1:7mIbdqSf+qXwY17+A+Kge2yWIJCMJ1p1DiBDGnKRohg=
github.com/fluxcd/pkg/git v0.8.0/go.mod h1:3deiLPws4DSQ3hqwtQd7Dt66GXTN/4RcT/yHAljXaHo=
github.com/fluxcd/pkg/git/gogit v0.5.0 h1:3Fzx2W16K/37ZHT6WmLFuRYgs+CGvzka+dwY7ktoxJE=
github.com/fluxcd/pkg/git/gogit v0.5.0/go.mod h1:cqoJhKXCmWuN2ezD/2ECUYwR8gR7svMRJoHRr9VyTQc=
github.com/fluxcd/pkg/git v0.9.0 h1:e/RBMBe9rGUEi+B4DQpVPmDmAyHGj/fztqxTUeUxnsM=
github.com/fluxcd/pkg/git v0.9.0/go.mod h1:3deiLPws4DSQ3hqwtQd7Dt66GXTN/4RcT/yHAljXaHo=
github.com/fluxcd/pkg/git/gogit v0.6.0 h1:3RWWmviQzcsAkZcLMVtvPVZvAmx77m5cCdL7B5SzuKg=
github.com/fluxcd/pkg/git/gogit v0.6.0/go.mod h1:3PgGDssi637wrQTf3EKg1HdodvsGxWe9ZnSzDdi3qXw=
github.com/fluxcd/pkg/gittestserver v0.8.0 h1:YrYe63KScKlLxx0GAiQthx2XqHDx0vKitIIx4JnDtIo=
github.com/fluxcd/pkg/gittestserver v0.8.0/go.mod h1:/LI/xKMrnQbIsTDnTyABQ71iaYhFIZ8fb4cvY7WAlBU=
github.com/fluxcd/pkg/helmtestserver v0.11.0 h1:eVKE6DtwkPej5YByskpgMWhnINzuK3SmeJvOeYBYoKU=
@ -585,8 +586,9 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI
github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og=
github.com/go-git/gcfg v1.5.0 h1:Q5ViNfGF8zFgyJWPqYwA7qGFoMTEiBmdlkcfRmpIMa4=
github.com/go-git/gcfg v1.5.0/go.mod h1:5m20vg6GwYabIxaOonVkTdrILxQMpEShl1xiMF4ua+E=
github.com/go-git/go-billy/v5 v5.0.0-20210804024030-7ab80d7c013d h1:O796bLOF253EfLnGYMjJth4mLrxcJBxbyem4mhyJFow=
github.com/go-git/go-billy/v5 v5.0.0-20210804024030-7ab80d7c013d/go.mod h1:pmpqyWchKfYfrkb/UVH4otLvyi/5gJlGI4Hb3ZqZ3W0=
github.com/go-git/go-billy/v5 v5.3.1/go.mod h1:pmpqyWchKfYfrkb/UVH4otLvyi/5gJlGI4Hb3ZqZ3W0=
github.com/go-git/go-billy/v5 v5.4.1 h1:Uwp5tDRkPr+l/TnbHOQzp+tmJfLceOlbVucgpTz8ix4=
github.com/go-git/go-billy/v5 v5.4.1/go.mod h1:vjbugF6Fz7JIflbVpl1hJsGjSHNltrSw45YK/ukIvQg=
github.com/go-git/go-git-fixtures/v4 v4.3.1 h1:y5z6dd3qi8Hl+stezc8p3JxDkoTRqMAlKnXHuzrfjTQ=
github.com/go-git/go-git-fixtures/v4 v4.3.1/go.mod h1:8LHG1a3SRW71ettAD/jW13h8c6AqjVSeL11RAdgaqpo=
github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
@ -1253,8 +1255,10 @@ github.com/onsi/gomega v1.19.0/go.mod h1:LY+I3pBVzYsTBU1AnDwOSxaYi9WoWiqgwooUqq9
github.com/onsi/gomega v1.26.0 h1:03cDLK28U6hWvCAns6NeydX3zIm4SF3ci69ulidS32Q=
github.com/onsi/gomega v1.26.0/go.mod h1:r+zV744Re+DiYCIPRlYOTxn0YkOLcAnW8k1xXdMPGhM=
github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk=
github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U=
github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM=
github.com/opencontainers/go-digest v1.0.1-0.20220411205349-bde1400a84be h1:f2PlhC9pm5sqpBZFvnAoKj+KzXRzbjFMA+TqXfJdgho=
github.com/opencontainers/go-digest v1.0.1-0.20220411205349-bde1400a84be/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM=
github.com/opencontainers/go-digest/blake3 v0.0.0-20220411205349-bde1400a84be h1:yJISmqboKE7zWqC2Nlg3pBkelqCblzZBoMHv2nbrUjQ=
github.com/opencontainers/go-digest/blake3 v0.0.0-20220411205349-bde1400a84be/go.mod h1:amaK2C3q0MwQTE9OgeDacYr8Qac7uKwICGry1fn3UrI=
github.com/opencontainers/image-spec v1.1.0-rc2 h1:2zx/Stx4Wc5pIPDvIxHXvXtQFW/7XWJGmnM7r3wg034=
github.com/opencontainers/image-spec v1.1.0-rc2/go.mod h1:3OVijpioIKYWTqjiG0zfF6wvoJ4fAXGbjdZuI2NgsRQ=
github.com/opencontainers/runc v1.1.2 h1:2VSZwLx5k/BfsBxMMipG/LYUnmqOD/BPkIVgQUcTlLw=
@ -1412,8 +1416,9 @@ github.com/segmentio/ksuid v1.0.4 h1:sBo2BdShXjmcugAMwjugoGUdUV0pcxY5mW4xKRn3v4c
github.com/segmentio/ksuid v1.0.4/go.mod h1:/XUiZBD3kVx5SmUOl55voK5yeAbBNNIed+2O73XgrPE=
github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo=
github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM=
github.com/sergi/go-diff v1.2.0 h1:XU+rvMAioB0UC3q1MFrIQy4Vo5/4VsRDQQXHsEya6xQ=
github.com/sergi/go-diff v1.2.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM=
github.com/sergi/go-diff v1.3.1 h1:xkr+Oxo4BOQKmkn/B9eMK0g5Kg/983T9DqqPHwYqD+8=
github.com/sergi/go-diff v1.3.1/go.mod h1:aMJSSKb2lpPvRNec0+w3fl7LP9IOFzdc9Pa4NFbPK1I=
github.com/shibumi/go-pathspec v1.3.0 h1:QUyMZhFo0Md5B8zV8x2tesohbb5kfbpTi9rBnKh5dkI=
github.com/shibumi/go-pathspec v1.3.0/go.mod h1:Xutfslp817l2I1cZvgcfeMQJG5QnU2lh5tVaaMCl3jE=
github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o=
@ -1600,8 +1605,14 @@ github.com/yvasiyarov/gorelic v0.0.7/go.mod h1:NUSPSUX/bi6SeDMUh6brw0nXpxHnc96Tg
github.com/yvasiyarov/newrelic_platform_go v0.0.0-20160601141957-9c099fbc30e9 h1:AsFN8kXcCVkUFHyuzp1FtYbzp1nCO/H6+1uPSGEyPzM=
github.com/yvasiyarov/newrelic_platform_go v0.0.0-20160601141957-9c099fbc30e9/go.mod h1:GlGEuHIJweS1mbCqG+7vt2nvWLzLLnRHbXz5JKd/Qbg=
github.com/zalando/go-keyring v0.1.0/go.mod h1:RaxNwUITJaHVdQ0VC7pELPZ3tOWn13nr0gZMZEhpVU0=
github.com/zeebo/assert v1.1.0 h1:hU1L1vLTHsnO8x8c9KAR5GmM5QscxHg5RNU5z5qbUWY=
github.com/zeebo/assert v1.1.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0=
github.com/zeebo/blake3 v0.1.1 h1:Nbsts7DdKThRHHd+YNlqiGlRqGEF2bE2eXN+xQ1hsEs=
github.com/zeebo/blake3 v0.1.1/go.mod h1:G9pM4qQwjRzF1/v7+vabMj/c5mWpGZ2Wzo3Eb4z0pb4=
github.com/zeebo/errs v1.2.2 h1:5NFypMTuSdoySVTqlNs1dEoU21QVamMQJxW/Fii5O7g=
github.com/zeebo/errs v1.2.2/go.mod h1:sgbWHsvVuTPHcqJJGQ1WhI5KbWlHYz+2+2C/LSEtCw4=
github.com/zeebo/pcg v1.0.0 h1:dt+dx+HvX8g7Un32rY9XWoYnd0NmKmrIzpHF7qiTDj0=
github.com/zeebo/pcg v1.0.0/go.mod h1:09F0S9iiKrwn9rlI5yjLkmrug154/YRW6KnnXVDM/l4=
github.com/ziutek/mymysql v1.5.4/go.mod h1:LMSpPZ6DbqWFxNCHW77HeMg9I646SAhApZ/wKdgO/C0=
go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
@ -1768,8 +1779,8 @@ golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0
golang.org/x/crypto v0.0.0-20220826181053-bd7e27e6170d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4=
golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw=
golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4=
golang.org/x/crypto v0.5.0 h1:U/0M97KRkSFvyD/3FSmdP5W5swImpNgle/EHFhOsQPE=
golang.org/x/crypto v0.5.0/go.mod h1:NK/OQwhpMQP3MwtdjgLlYHnH9ebylxKWv3e0fK+mkQU=
golang.org/x/crypto v0.6.0 h1:qfktjS5LUO+fFKeJXZ+ikTRijMmljikvG68fpMMruSc=
golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58=
golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8=
@ -1811,8 +1822,8 @@ golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.5.0/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro=
golang.org/x/mod v0.6.0-dev.0.20220106191415-9b9b3d81d5e3/go.mod h1:3p9vT2HGsQu2K1YbXdKPJLVgG5VJdoTa1poYQBtP1AY=
golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4=
golang.org/x/mod v0.7.0 h1:LapD9S96VoQRhi/GrNTqeBJFrUjs5UHCAtTlgwA5oZA=
golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs=
golang.org/x/mod v0.8.0 h1:LUYupSeNrTNCGzR/hVBk2NHZO4hXcVaW1k4Qx7rjPx8=
golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs=
golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
@ -1888,8 +1899,8 @@ golang.org/x/net v0.1.0/go.mod h1:Cx3nUiGt4eDBEyega/BKRp+/AlGL8hYe7U9odMt2Cco=
golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY=
golang.org/x/net v0.3.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE=
golang.org/x/net v0.4.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE=
golang.org/x/net v0.5.0 h1:GyT4nK/YDHSqa1c4753ouYCDajOYKTja9Xb/OHtgvSw=
golang.org/x/net v0.5.0/go.mod h1:DivGGAXEgPSlEBzxGzZI+ZLohi+xUj054jfeKui00ws=
golang.org/x/net v0.6.0 h1:L4ZwwTvKW9gr0ZMS1yrHD9GZhIuVjOBBnaKH+SPQK0Q=
golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs=
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
golang.org/x/oauth2 v0.0.0-20181106182150-f42d05182288/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
@ -1994,6 +2005,7 @@ golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201009025420-dfb3f7c4e634/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201014080544-cc95f250f6bc/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@ -2054,8 +2066,8 @@ golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.4.0 h1:Zr2JFtRQNX3BCZ8YtxRE9hNJYC8J6I1MVbMg6owUp18=
golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU=
golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw=
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
@ -2064,8 +2076,8 @@ golang.org/x/term v0.0.0-20220722155259-a9ba230a4035/go.mod h1:jbD1KX2456YbFQfuX
golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc=
golang.org/x/term v0.3.0/go.mod h1:q750SLmJuPmVoN1blW3UFBPREJfb1KmY3vwxfr+nFDA=
golang.org/x/term v0.4.0 h1:O7UWfv5+A2qiuulQk30kVinPoMtoIPeVaKLEgLpVkvg=
golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ=
golang.org/x/term v0.5.0 h1:n2a8QNdAb0sZNpU9R1ALUXBbY+w51fCQDN+7EdxNBsY=
golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k=
golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
@ -2077,8 +2089,8 @@ golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ=
golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
golang.org/x/text v0.6.0 h1:3XmdazWV+ubf7QgHSTWeykHOci5oeekaGJBLkrkaw4k=
golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo=
golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
@ -2168,8 +2180,8 @@ golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/tools v0.1.7/go.mod h1:LGqMHiF4EqQNHR1JncWGqT5BVaXmza+X+BDGol+dOxo=
golang.org/x/tools v0.1.10/go.mod h1:Uh6Zz+xoGYZom868N8YTex3t7RhtHDBrE8Gzo9bV56E=
golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc=
golang.org/x/tools v0.3.0 h1:SrNbZl6ECOS1qFzgTdQfWXZM9XBkiA6tkFrH9YSTPHM=
golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k=
golang.org/x/tools v0.6.0 h1:BOw41kyTf3PuCW1pVQf8+Cyg8pMlkYB1oo9iJ6D/lKM=
golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU=
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=

52
internal/digest/digest.go Normal file
View File

@ -0,0 +1,52 @@
/*
Copyright 2022 The Flux 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 digest
import (
"crypto"
_ "crypto/sha1"
_ "crypto/sha256"
_ "crypto/sha512"
"fmt"
"github.com/opencontainers/go-digest"
_ "github.com/opencontainers/go-digest/blake3"
)
const (
SHA1 digest.Algorithm = "sha1"
)
var (
// Canonical is the primary digest algorithm used to calculate checksums.
Canonical = digest.SHA256
)
func init() {
// Register SHA-1 algorithm for support of e.g. Git commit SHAs.
digest.RegisterAlgorithm(SHA1, crypto.SHA1)
}
// AlgorithmForName returns the digest algorithm for the given name, or an
// error of type digest.ErrDigestUnsupported if the algorithm is unavailable.
func AlgorithmForName(name string) (digest.Algorithm, error) {
a := digest.Algorithm(name)
if !a.Available() {
return "", fmt.Errorf("%w: %s", digest.ErrDigestUnsupported, name)
}
return a, nil
}

View File

@ -0,0 +1,71 @@
/*
Copyright 2022 The Flux 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 digest
import (
"errors"
"testing"
. "github.com/onsi/gomega"
"github.com/opencontainers/go-digest"
)
func TestAlgorithmForName(t *testing.T) {
tests := []struct {
name string
want digest.Algorithm
wantErr error
}{
{
name: "sha256",
want: digest.SHA256,
},
{
name: "sha384",
want: digest.SHA384,
},
{
name: "sha512",
want: digest.SHA512,
},
{
name: "blake3",
want: digest.BLAKE3,
},
{
name: "sha1",
want: SHA1,
},
{
name: "not-available",
wantErr: digest.ErrDigestUnsupported,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
g := NewWithT(t)
got, err := AlgorithmForName(tt.name)
if tt.wantErr != nil {
g.Expect(err).To(HaveOccurred())
g.Expect(errors.Is(err, tt.wantErr)).To(BeTrue())
return
}
g.Expect(err).ToNot(HaveOccurred())
g.Expect(got).To(Equal(tt.want))
})
}
}

71
internal/digest/writer.go Normal file
View File

@ -0,0 +1,71 @@
/*
Copyright 2022 The Flux 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 digest
import (
"fmt"
"io"
"github.com/opencontainers/go-digest"
)
// MultiDigester is a digester that writes to multiple digesters to calculate
// the checksum of different algorithms.
type MultiDigester struct {
d map[digest.Algorithm]digest.Digester
}
// NewMultiDigester returns a new MultiDigester that writes to newly
// initialized digesters for the given algorithms. If a provided algorithm is
// not available, it returns a digest.ErrDigestUnsupported error.
func NewMultiDigester(algos ...digest.Algorithm) (*MultiDigester, error) {
d := make(map[digest.Algorithm]digest.Digester, len(algos))
for _, a := range algos {
if _, ok := d[a]; ok {
continue
}
if !a.Available() {
return nil, fmt.Errorf("%w: %s", digest.ErrDigestUnsupported, a)
}
d[a] = a.Digester()
}
return &MultiDigester{d: d}, nil
}
// Write writes p to all underlying digesters.
func (w *MultiDigester) Write(p []byte) (n int, err error) {
for _, d := range w.d {
n, err = d.Hash().Write(p)
if err != nil {
return
}
if n != len(p) {
err = io.ErrShortWrite
return
}
}
return len(p), nil
}
// Digest returns the digest of the data written to the digester of the given
// algorithm, or an empty digest if the algorithm is not available.
func (w *MultiDigester) Digest(algo digest.Algorithm) digest.Digest {
if d, ok := w.d[algo]; ok {
return d.Digest()
}
return ""
}

View File

@ -0,0 +1,128 @@
/*
Copyright 2022 The Flux 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 digest
import (
"crypto/rand"
"testing"
. "github.com/onsi/gomega"
"github.com/opencontainers/go-digest"
)
func TestNewMultiDigester(t *testing.T) {
t.Run("constructs a MultiDigester", func(t *testing.T) {
g := NewWithT(t)
d, err := NewMultiDigester(Canonical, digest.SHA512)
g.Expect(err).ToNot(HaveOccurred())
g.Expect(d.d).To(HaveLen(2))
})
t.Run("returns an error if an algorithm is not available", func(t *testing.T) {
g := NewWithT(t)
_, err := NewMultiDigester(digest.Algorithm("not-available"))
g.Expect(err).To(HaveOccurred())
})
}
func TestMultiDigester_Write(t *testing.T) {
t.Run("writes to all digesters", func(t *testing.T) {
g := NewWithT(t)
d, err := NewMultiDigester(Canonical, digest.SHA512)
g.Expect(err).ToNot(HaveOccurred())
n, err := d.Write([]byte("hello"))
g.Expect(err).ToNot(HaveOccurred())
g.Expect(n).To(Equal(5))
n, err = d.Write([]byte(" world"))
g.Expect(err).ToNot(HaveOccurred())
g.Expect(n).To(Equal(6))
g.Expect(d.Digest(Canonical)).To(BeEquivalentTo("sha256:b94d27b9934d3e08a52e52d7da7dabfac484efe37a5380ee9088f7ace2efcde9"))
g.Expect(d.Digest(digest.SHA512)).To(BeEquivalentTo("sha512:309ecc489c12d6eb4cc40f50c902f2b4d0ed77ee511a7c7a9bcd3ca86d4cd86f989dd35bc5ff499670da34255b45b0cfd830e81f605dcf7dc5542e93ae9cd76f"))
})
}
func TestMultiDigester_Digest(t *testing.T) {
t.Run("returns the digest for the given algorithm", func(t *testing.T) {
g := NewWithT(t)
d, err := NewMultiDigester(Canonical, digest.SHA512)
g.Expect(err).ToNot(HaveOccurred())
g.Expect(d.Digest(Canonical)).To(BeEquivalentTo("sha256:e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855"))
g.Expect(d.Digest(digest.SHA512)).To(BeEquivalentTo("sha512:cf83e1357eefb8bdf1542850d66d8007d620e4050b5715dc83f4a921d36ce9ce47d0d13c5d85f2b0ff8318d2877eec2f63b931bd47417a81a538327af927da3e"))
})
t.Run("returns an empty digest if the algorithm is not supported", func(t *testing.T) {
g := NewWithT(t)
d, err := NewMultiDigester(Canonical, digest.SHA512)
g.Expect(err).ToNot(HaveOccurred())
g.Expect(d.Digest(digest.Algorithm("not-available"))).To(BeEmpty())
})
}
func benchmarkMultiDigesterWrite(b *testing.B, algos []digest.Algorithm, pSize int64) {
md, err := NewMultiDigester(algos...)
if err != nil {
b.Fatal(err)
}
p := make([]byte, pSize)
if _, err = rand.Read(p); err != nil {
b.Fatal(err)
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
md.Write(p)
}
}
func BenchmarkMultiDigester_Write(b *testing.B) {
const pSize = 1024 * 2
b.Run("sha1", func(b *testing.B) {
benchmarkMultiDigesterWrite(b, []digest.Algorithm{SHA1}, pSize)
})
b.Run("sha256", func(b *testing.B) {
benchmarkMultiDigesterWrite(b, []digest.Algorithm{digest.SHA256}, pSize)
})
b.Run("blake3", func(b *testing.B) {
benchmarkMultiDigesterWrite(b, []digest.Algorithm{digest.BLAKE3}, pSize)
})
b.Run("sha256+sha384", func(b *testing.B) {
benchmarkMultiDigesterWrite(b, []digest.Algorithm{digest.SHA256, digest.SHA384}, pSize)
})
b.Run("sha256+sha512", func(b *testing.B) {
benchmarkMultiDigesterWrite(b, []digest.Algorithm{digest.SHA256, digest.SHA512}, pSize)
})
b.Run("sha256+blake3", func(b *testing.B) {
benchmarkMultiDigesterWrite(b, []digest.Algorithm{digest.SHA256, digest.BLAKE3}, pSize)
})
}

View File

@ -193,10 +193,9 @@ entries:
targetPath := filepath.Join(tmpDir, "chart.tgz")
if tt.repository != nil {
_, err := tt.repository.CacheIndex()
g.Expect(err).ToNot(HaveOccurred())
g.Expect(tt.repository.CacheIndex()).ToNot(HaveOccurred())
// Cleanup the cache index path.
defer os.Remove(tt.repository.CachePath)
defer os.Remove(tt.repository.Path)
}
b := NewRemoteBuilder(tt.repository)
@ -411,10 +410,10 @@ entries:
reference := RemoteReference{Name: "helmchart"}
repository := mockRepo()
_, err = repository.CacheIndex()
err = repository.CacheIndex()
g.Expect(err).ToNot(HaveOccurred())
// Cleanup the cache index path.
defer os.Remove(repository.CachePath)
defer os.Remove(repository.Path)
b := NewRemoteBuilder(repository)

View File

@ -86,11 +86,6 @@ func TestDependencyManager_Clear(t *testing.T) {
Index: repo.NewIndexFile(),
RWMutex: &sync.RWMutex{},
},
"cached cache path": &repository.ChartRepository{
CachePath: "/invalid/path/resets",
Cached: true,
RWMutex: &sync.RWMutex{},
},
"with credentials": ociRepoWithCreds,
"without credentials": &repository.OCIChartRepository{},
"nil downloader": nil,
@ -103,8 +98,6 @@ func TestDependencyManager_Clear(t *testing.T) {
switch v := v.(type) {
case *repository.ChartRepository:
g.Expect(v.Index).To(BeNil())
g.Expect(v.CachePath).To(BeEmpty())
g.Expect(v.Cached).To(BeFalse())
case *repository.OCIChartRepository:
g.Expect(v.HasCredentials()).To(BeFalse())
}
@ -441,14 +434,14 @@ func TestDependencyManager_addRemoteDependency(t *testing.T) {
name: "strategic load error",
downloaders: map[string]repository.Downloader{
"https://example.com/": &repository.ChartRepository{
CachePath: "/invalid/cache/path/foo",
RWMutex: &sync.RWMutex{},
Client: &mockGetter{},
RWMutex: &sync.RWMutex{},
},
},
dep: &helmchart.Dependency{
Repository: "https://example.com",
},
wantErr: "failed to strategically load index",
wantErr: "failed to load index",
},
{
name: "repository get error",

View File

@ -19,9 +19,7 @@ package repository
import (
"bytes"
"context"
"crypto/sha256"
"crypto/tls"
"encoding/hex"
"errors"
"fmt"
"io"
@ -31,22 +29,79 @@ import (
"sort"
"strings"
"sync"
"time"
"github.com/Masterminds/semver/v3"
"github.com/opencontainers/go-digest"
"helm.sh/helm/v3/pkg/chart"
"helm.sh/helm/v3/pkg/getter"
"helm.sh/helm/v3/pkg/repo"
kerrors "k8s.io/apimachinery/pkg/util/errors"
"sigs.k8s.io/yaml"
"github.com/fluxcd/pkg/version"
"github.com/fluxcd/source-controller/internal/cache"
"github.com/fluxcd/source-controller/internal/helm"
"github.com/fluxcd/source-controller/internal/transport"
)
var ErrNoChartIndex = errors.New("no chart index")
var (
ErrNoChartIndex = errors.New("no chart index")
)
// IndexFromFile loads a repo.IndexFile from the given path. It returns an
// error if the file does not exist, is not a regular file, exceeds the
// maximum index file size, or if the file cannot be parsed.
func IndexFromFile(path string) (*repo.IndexFile, error) {
st, err := os.Lstat(path)
if err != nil {
return nil, err
}
if !st.Mode().IsRegular() {
return nil, fmt.Errorf("%s is not a regular file", path)
}
if st.Size() > helm.MaxIndexSize {
return nil, fmt.Errorf("%s exceeds the maximum index file size of %d bytes", path, helm.MaxIndexSize)
}
b, err := os.ReadFile(path)
if err != nil {
return nil, err
}
return IndexFromBytes(b)
}
// IndexFromBytes loads a repo.IndexFile from the given bytes. It returns an
// error if the bytes cannot be parsed, or if the API version is not set.
// The entries are sorted before the index is returned.
func IndexFromBytes(b []byte) (*repo.IndexFile, error) {
if len(b) == 0 {
return nil, repo.ErrEmptyIndexYaml
}
i := &repo.IndexFile{}
if err := yaml.UnmarshalStrict(b, i); err != nil {
return nil, err
}
if i.APIVersion == "" {
return nil, repo.ErrNoAPIVersion
}
for _, cvs := range i.Entries {
for idx := len(cvs) - 1; idx >= 0; idx-- {
if cvs[idx] == nil {
continue
}
if cvs[idx].APIVersion == "" {
cvs[idx].APIVersion = chart.APIVersionV1
}
if err := cvs[idx].Validate(); err != nil {
cvs = append(cvs[:idx], cvs[idx+1:]...)
}
}
}
i.SortEntries()
return i, nil
}
// ChartRepository represents a Helm chart repository, and the configuration
// required to download the chart index and charts from the repository.
@ -55,73 +110,32 @@ type ChartRepository struct {
// URL the ChartRepository's index.yaml can be found at,
// without the index.yaml suffix.
URL string
// Path is the absolute path to the Index file.
Path string
// Index of the ChartRepository.
Index *repo.IndexFile
// Client to use while downloading the Index or a chart from the URL.
Client getter.Getter
// Options to configure the Client with while downloading the Index
// or a chart from the URL.
Options []getter.Option
// CachePath is the path of a cached index.yaml for read-only operations.
CachePath string
// Cached indicates if the ChartRepository index.yaml has been cached
// to CachePath.
Cached bool
// Index contains a loaded chart repository index if not nil.
Index *repo.IndexFile
// Checksum contains the SHA256 checksum of the loaded chart repository
// index bytes. This is different from the checksum of the CachePath, which
// may contain unordered entries.
Checksum string
tlsConfig *tls.Config
cached bool
revisions map[digest.Algorithm]digest.Digest
digests map[digest.Algorithm]digest.Digest
*sync.RWMutex
cacheInfo
}
type cacheInfo struct {
// In memory cache of the index.yaml file.
IndexCache *cache.Cache
// IndexKey is the cache key for the index.yaml file.
IndexKey string
// IndexTTL is the cache TTL for the index.yaml file.
IndexTTL time.Duration
// RecordIndexCacheMetric records the cache hit/miss metrics for the index.yaml file.
RecordIndexCacheMetric RecordMetricsFunc
}
// ChartRepositoryOption is a function that can be passed to NewChartRepository
// to configure a ChartRepository.
type ChartRepositoryOption func(*ChartRepository) error
// RecordMetricsFunc is a function that records metrics.
type RecordMetricsFunc func(event string)
// WithMemoryCache returns a ChartRepositoryOptions that will enable the
// ChartRepository to cache the index.yaml file in memory.
// The cache key have to be safe in multi-tenancy environments,
// as otherwise it could be used as a vector to bypass the helm repository's authentication.
func WithMemoryCache(key string, c *cache.Cache, ttl time.Duration, rec RecordMetricsFunc) ChartRepositoryOption {
return func(r *ChartRepository) error {
if c != nil {
if key == "" {
return errors.New("cache key cannot be empty")
}
}
r.IndexCache = c
r.IndexKey = key
r.IndexTTL = ttl
r.RecordIndexCacheMetric = rec
return nil
}
}
// NewChartRepository constructs and returns a new ChartRepository with
// the ChartRepository.Client configured to the getter.Getter for the
// repository URL scheme. It returns an error on URL parsing failures,
// or if there is no getter available for the scheme.
func NewChartRepository(repositoryURL, cachePath string, providers getter.Providers, tlsConfig *tls.Config, getterOpts []getter.Option, chartRepoOpts ...ChartRepositoryOption) (*ChartRepository, error) {
u, err := url.Parse(repositoryURL)
func NewChartRepository(URL, path string, providers getter.Providers, tlsConfig *tls.Config, getterOpts ...getter.Option) (*ChartRepository, error) {
u, err := url.Parse(URL)
if err != nil {
return nil, err
}
@ -131,24 +145,20 @@ func NewChartRepository(repositoryURL, cachePath string, providers getter.Provid
}
r := newChartRepository()
r.URL = repositoryURL
r.CachePath = cachePath
r.URL = URL
r.Path = path
r.Client = c
r.Options = getterOpts
r.tlsConfig = tlsConfig
for _, opt := range chartRepoOpts {
if err := opt(r); err != nil {
return nil, err
}
}
return r, nil
}
func newChartRepository() *ChartRepository {
return &ChartRepository{
RWMutex: &sync.RWMutex{},
revisions: make(map[digest.Algorithm]digest.Digest, 0),
digests: make(map[digest.Algorithm]digest.Digest, 0),
RWMutex: &sync.RWMutex{},
}
}
@ -205,10 +215,10 @@ func (r *ChartRepository) getChartVersion(name, ver string) (*repo.ChartVersion,
}
}
// Filter out chart versions that doesn't satisfy constraints if any,
// Filter out chart versions that don't satisfy constraints if any,
// parse semver and build a lookup table
var matchedVersions semver.Collection
lookup := make(map[*semver.Version]*repo.ChartVersion)
lookup := make(map[*semver.Version]*repo.ChartVersion, 0)
for _, cv := range cvs {
v, err := version.ParseVersion(cv.Version)
if err != nil {
@ -288,155 +298,86 @@ func (r *ChartRepository) DownloadChart(chart *repo.ChartVersion) (*bytes.Buffer
return r.Client.Get(u.String(), clientOpts...)
}
// LoadIndexFromBytes loads Index from the given bytes.
// It returns a repo.ErrNoAPIVersion error if the API version is not set
func (r *ChartRepository) LoadIndexFromBytes(b []byte) error {
i := &repo.IndexFile{}
if err := yaml.UnmarshalStrict(b, i); err != nil {
return err
}
if i.APIVersion == "" {
return repo.ErrNoAPIVersion
}
i.SortEntries()
r.Lock()
r.Index = i
r.Checksum = fmt.Sprintf("%x", sha256.Sum256(b))
r.Unlock()
return nil
}
// LoadFromFile reads the file at the given path and loads it into Index.
func (r *ChartRepository) LoadFromFile(path string) error {
stat, err := os.Stat(path)
if err != nil || stat.IsDir() {
if err == nil {
err = fmt.Errorf("'%s' is a directory", path)
}
return err
}
if stat.Size() > helm.MaxIndexSize {
return fmt.Errorf("size of index '%s' exceeds '%d' bytes limit", stat.Name(), helm.MaxIndexSize)
}
b, err := os.ReadFile(path)
if err != nil {
return err
}
return r.LoadIndexFromBytes(b)
}
// CacheIndex attempts to write the index from the remote into a new temporary file
// using DownloadIndex, and sets CachePath and Cached.
// using DownloadIndex, and sets Path and cached.
// It returns the SHA256 checksum of the downloaded index bytes, or an error.
// The caller is expected to handle the garbage collection of CachePath, and to
// load the Index separately using LoadFromCache if required.
func (r *ChartRepository) CacheIndex() (string, error) {
// The caller is expected to handle the garbage collection of Path, and to
// load the Index separately using LoadFromPath if required.
func (r *ChartRepository) CacheIndex() error {
f, err := os.CreateTemp("", "chart-index-*.yaml")
if err != nil {
return "", fmt.Errorf("failed to create temp file to cache index to: %w", err)
return fmt.Errorf("failed to create temp file to cache index to: %w", err)
}
h := sha256.New()
mw := io.MultiWriter(f, h)
if err = r.DownloadIndex(mw); err != nil {
if err = r.DownloadIndex(f); err != nil {
f.Close()
os.RemoveAll(f.Name())
return "", fmt.Errorf("failed to cache index to temporary file: %w", err)
os.Remove(f.Name())
return fmt.Errorf("failed to cache index to temporary file: %w", err)
}
if err = f.Close(); err != nil {
os.RemoveAll(f.Name())
return "", fmt.Errorf("failed to close cached index file '%s': %w", f.Name(), err)
os.Remove(f.Name())
return fmt.Errorf("failed to close cached index file '%s': %w", f.Name(), err)
}
r.Lock()
r.CachePath = f.Name()
r.Cached = true
r.Path = f.Name()
r.Index = nil
r.cached = true
r.invalidate()
r.Unlock()
return hex.EncodeToString(h.Sum(nil)), nil
}
// CacheIndexInMemory attempts to cache the index in memory.
// It returns an error if it fails.
// The cache key have to be safe in multi-tenancy environments,
// as otherwise it could be used as a vector to bypass the helm repository's authentication.
func (r *ChartRepository) CacheIndexInMemory() error {
// Cache the index if it was successfully retrieved
// and the chart was successfully built
if r.IndexCache != nil && r.Index != nil {
err := r.IndexCache.Set(r.IndexKey, r.Index, r.IndexTTL)
if err != nil {
return err
}
}
return nil
}
// StrategicallyLoadIndex lazy-loads the Index
// first from Indexcache,
// then from CachePath using oadFromCache if it does not HasIndex.
// If not HasCacheFile, a cache attempt is made using CacheIndex
// before continuing to load.
// StrategicallyLoadIndex lazy-loads the Index if required, first
// attempting to load it from Path if the file exists, before falling
// back to caching it.
func (r *ChartRepository) StrategicallyLoadIndex() (err error) {
if r.HasIndex() {
return
}
if r.IndexCache != nil {
if found := r.LoadFromMemCache(); found {
if !r.HasFile() {
if err = r.CacheIndex(); err != nil {
err = fmt.Errorf("failed to cache index: %w", err)
return
}
}
if !r.HasCacheFile() {
if _, err = r.CacheIndex(); err != nil {
err = fmt.Errorf("failed to strategically load index: %w", err)
return
}
}
if err = r.LoadFromCache(); err != nil {
err = fmt.Errorf("failed to strategically load index: %w", err)
if err = r.LoadFromPath(); err != nil {
err = fmt.Errorf("failed to load index: %w", err)
return
}
return
}
// LoadFromMemCache attempts to load the Index from the provided cache.
// It returns true if the Index was found in the cache, and false otherwise.
func (r *ChartRepository) LoadFromMemCache() bool {
if index, found := r.IndexCache.Get(r.IndexKey); found {
r.Lock()
r.Index = index.(*repo.IndexFile)
r.Unlock()
// LoadFromPath attempts to load the Index from the configured Path.
// It returns an error if no Path is set, or if the load failed.
func (r *ChartRepository) LoadFromPath() error {
r.Lock()
defer r.Unlock()
// record the cache hit
if r.RecordIndexCacheMetric != nil {
r.RecordIndexCacheMetric(cache.CacheEventTypeHit)
}
return true
if len(r.Path) == 0 {
return fmt.Errorf("no cache path")
}
// record the cache miss
if r.RecordIndexCacheMetric != nil {
r.RecordIndexCacheMetric(cache.CacheEventTypeMiss)
i, err := IndexFromFile(r.Path)
if err != nil {
return fmt.Errorf("failed to load index: %w", err)
}
return false
}
// LoadFromCache attempts to load the Index from the configured CachePath.
// It returns an error if no CachePath is set, or if the load failed.
func (r *ChartRepository) LoadFromCache() error {
if cachePath := r.CachePath; cachePath != "" {
return r.LoadFromFile(cachePath)
}
return fmt.Errorf("no cache path set")
r.Index = i
r.revisions = make(map[digest.Algorithm]digest.Digest, 0)
return nil
}
// DownloadIndex attempts to download the chart repository index using
// the Client and set Options, and writes the index to the given io.Writer.
// It returns an url.Error if the URL failed to parse.
func (r *ChartRepository) DownloadIndex(w io.Writer) (err error) {
r.RLock()
defer r.RUnlock()
u, err := url.Parse(r.URL)
if err != nil {
return err
@ -459,77 +400,98 @@ func (r *ChartRepository) DownloadIndex(w io.Writer) (err error) {
return nil
}
// Revision returns the revision of the ChartRepository's Index. It assumes
// the Index is stable sorted.
func (r *ChartRepository) Revision(algorithm digest.Algorithm) digest.Digest {
if !r.HasIndex() {
return ""
}
r.Lock()
defer r.Unlock()
if _, ok := r.revisions[algorithm]; !ok {
if b, _ := yaml.Marshal(r.Index); len(b) > 0 {
r.revisions[algorithm] = algorithm.FromBytes(b)
}
}
return r.revisions[algorithm]
}
// Digest returns the digest of the file at the ChartRepository's Path.
func (r *ChartRepository) Digest(algorithm digest.Algorithm) digest.Digest {
if !r.HasFile() {
return ""
}
r.Lock()
defer r.Unlock()
if _, ok := r.digests[algorithm]; !ok {
if f, err := os.Open(r.Path); err == nil {
defer f.Close()
rd := io.LimitReader(f, helm.MaxIndexSize)
if d, err := algorithm.FromReader(rd); err == nil {
r.digests[algorithm] = d
}
}
}
return r.digests[algorithm]
}
// HasIndex returns true if the Index is not nil.
func (r *ChartRepository) HasIndex() bool {
r.RLock()
defer r.RUnlock()
return r.Index != nil
}
// HasCacheFile returns true if CachePath is not empty.
func (r *ChartRepository) HasCacheFile() bool {
// HasFile returns true if Path exists and is a regular file.
func (r *ChartRepository) HasFile() bool {
r.RLock()
defer r.RUnlock()
return r.CachePath != ""
}
// Unload can be used to signal the Go garbage collector the Index can
// be freed from memory if the ChartRepository object is expected to
// continue to exist in the stack for some time.
func (r *ChartRepository) Unload() {
if r == nil {
return
}
r.Lock()
defer r.Unlock()
r.Index = nil
}
// Clear caches the index in memory before unloading it.
// It cleans up temporary files and directories created by the repository.
func (r *ChartRepository) Clear() error {
var errs []error
if err := r.CacheIndexInMemory(); err != nil {
errs = append(errs, err)
}
r.Unload()
if err := r.RemoveCache(); err != nil {
errs = append(errs, err)
}
return kerrors.NewAggregate(errs)
}
// SetMemCache sets the cache to use for this repository.
func (r *ChartRepository) SetMemCache(key string, c *cache.Cache, ttl time.Duration, rec RecordMetricsFunc) {
r.IndexKey = key
r.IndexCache = c
r.IndexTTL = ttl
r.RecordIndexCacheMetric = rec
}
// RemoveCache removes the CachePath if Cached.
func (r *ChartRepository) RemoveCache() error {
if r == nil {
return nil
}
r.Lock()
defer r.Unlock()
if r.Cached {
if err := os.Remove(r.CachePath); err != nil && !os.IsNotExist(err) {
return err
if r.Path != "" {
if stat, err := os.Lstat(r.Path); err == nil {
return stat.Mode().IsRegular()
}
r.CachePath = ""
r.Cached = false
}
return false
}
// Clear clears the Index and removes the file at Path, if cached.
func (r *ChartRepository) Clear() error {
r.Lock()
defer r.Unlock()
r.Index = nil
if r.cached {
if err := os.Remove(r.Path); err != nil {
return fmt.Errorf("failed to remove cached index: %w", err)
}
r.Path = ""
r.cached = false
}
r.invalidate()
return nil
}
// Invalidate clears any cached digests and revisions.
func (r *ChartRepository) Invalidate() {
r.Lock()
defer r.Unlock()
r.invalidate()
}
func (r *ChartRepository) invalidate() {
r.digests = make(map[digest.Algorithm]digest.Digest, 0)
r.revisions = make(map[digest.Algorithm]digest.Digest, 0)
}
// VerifyChart verifies the chart against a signature.
// It returns an error on failure.
func (r *ChartRepository) VerifyChart(_ context.Context, _ *repo.ChartVersion) error {

View File

@ -18,20 +18,22 @@ package repository
import (
"bytes"
"crypto/sha256"
"errors"
"fmt"
"net/url"
"os"
"path/filepath"
"sync"
"testing"
"time"
"github.com/fluxcd/source-controller/internal/cache"
"github.com/fluxcd/source-controller/internal/helm"
. "github.com/onsi/gomega"
"github.com/opencontainers/go-digest"
"helm.sh/helm/v3/pkg/chart"
helmgetter "helm.sh/helm/v3/pkg/getter"
"helm.sh/helm/v3/pkg/repo"
"github.com/fluxcd/source-controller/internal/helm"
)
var now = time.Now()
@ -55,6 +57,136 @@ func (g *mockGetter) Get(u string, _ ...helmgetter.Option) (*bytes.Buffer, error
return bytes.NewBuffer(r), nil
}
// Index load tests are derived from https://github.com/helm/helm/blob/v3.3.4/pkg/repo/index_test.go#L108
// to ensure parity with Helm behaviour.
func TestIndexFromFile(t *testing.T) {
g := NewWithT(t)
// Create an index file that exceeds the max index size.
tmpDir := t.TempDir()
bigIndexFile := filepath.Join(tmpDir, "index.yaml")
data := make([]byte, helm.MaxIndexSize+10)
g.Expect(os.WriteFile(bigIndexFile, data, 0o640)).ToNot(HaveOccurred())
tests := []struct {
name string
filename string
wantErr string
}{
{
name: "regular index file",
filename: testFile,
},
{
name: "chartmuseum index file",
filename: chartmuseumTestFile,
},
{
name: "error if index size exceeds max size",
filename: bigIndexFile,
wantErr: "exceeds the maximum index file size",
},
}
for _, tt := range tests {
tt := tt
t.Run(tt.name, func(t *testing.T) {
g := NewWithT(t)
i, err := IndexFromFile(tt.filename)
if tt.wantErr != "" {
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring(tt.wantErr))
return
}
g.Expect(err).ToNot(HaveOccurred())
verifyLocalIndex(t, i)
})
}
}
func TestIndexFromBytes(t *testing.T) {
tests := []struct {
name string
b []byte
wantName string
wantVersion string
wantDigest string
wantErr string
}{
{
name: "index",
b: []byte(`
apiVersion: v1
entries:
nginx:
- urls:
- https://kubernetes-charts.storage.googleapis.com/nginx-0.2.0.tgz
name: nginx
description: string
version: 0.2.0
home: https://github.com/something/else
digest: "sha256:1234567890abcdef"
`),
wantName: "nginx",
wantVersion: "0.2.0",
wantDigest: "sha256:1234567890abcdef",
},
{
name: "index without API version",
b: []byte(`entries:
nginx:
- name: nginx`),
wantErr: "no API version specified",
},
{
name: "index with duplicate entry",
b: []byte(`apiVersion: v1
entries:
nginx:
- name: nginx"
nginx:
- name: nginx`),
wantErr: "key \"nginx\" already set in map",
},
}
for _, tt := range tests {
tt := tt
t.Run(tt.name, func(t *testing.T) {
g := NewWithT(t)
t.Parallel()
i, err := IndexFromBytes(tt.b)
if tt.wantErr != "" {
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring(tt.wantErr))
g.Expect(i).To(BeNil())
return
}
g.Expect(err).ToNot(HaveOccurred())
g.Expect(i).ToNot(BeNil())
got, err := i.Get(tt.wantName, tt.wantVersion)
g.Expect(err).ToNot(HaveOccurred())
g.Expect(got.Digest).To(Equal(tt.wantDigest))
})
}
}
func TestIndexFromBytes_Unordered(t *testing.T) {
b, err := os.ReadFile(unorderedTestFile)
if err != nil {
t.Fatal(err)
}
i, err := IndexFromBytes(b)
if err != nil {
t.Fatal(err)
}
verifyLocalIndex(t, i)
}
func TestNewChartRepository(t *testing.T) {
repositoryURL := "https://example.com"
providers := helmgetter.Providers{
@ -68,7 +200,7 @@ func TestNewChartRepository(t *testing.T) {
t.Run("should construct chart repository", func(t *testing.T) {
g := NewWithT(t)
r, err := NewChartRepository(repositoryURL, "", providers, nil, options)
r, err := NewChartRepository(repositoryURL, "", providers, nil, options...)
g.Expect(err).ToNot(HaveOccurred())
g.Expect(r).ToNot(BeNil())
g.Expect(r.URL).To(Equal(repositoryURL))
@ -95,7 +227,7 @@ func TestNewChartRepository(t *testing.T) {
})
}
func TestChartRepository_Get(t *testing.T) {
func TestChartRepository_GetChartVersion(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
@ -252,6 +384,31 @@ func TestChartRepository_DownloadChart(t *testing.T) {
}
}
func TestChartRepository_CacheIndex(t *testing.T) {
g := NewWithT(t)
mg := mockGetter{Response: []byte("foo")}
r := newChartRepository()
r.URL = "https://example.com"
r.Client = &mg
r.revisions["key"] = "value"
r.digests["key"] = "value"
err := r.CacheIndex()
g.Expect(err).To(Not(HaveOccurred()))
g.Expect(r.Path).ToNot(BeEmpty())
t.Cleanup(func() { _ = os.Remove(r.Path) })
g.Expect(r.Path).To(BeARegularFile())
b, _ := os.ReadFile(r.Path)
g.Expect(b).To(Equal(mg.Response))
g.Expect(r.revisions).To(BeEmpty())
g.Expect(r.digests).To(BeEmpty())
}
func TestChartRepository_DownloadIndex(t *testing.T) {
g := NewWithT(t)
@ -260,8 +417,9 @@ func TestChartRepository_DownloadIndex(t *testing.T) {
mg := mockGetter{Response: b}
r := &ChartRepository{
URL: "https://example.com",
Client: &mg,
URL: "https://example.com",
Client: &mg,
RWMutex: &sync.RWMutex{},
}
buf := bytes.NewBuffer([]byte{})
@ -271,258 +429,166 @@ func TestChartRepository_DownloadIndex(t *testing.T) {
g.Expect(err).To(BeNil())
}
func TestChartRepository_LoadIndexFromBytes(t *testing.T) {
tests := []struct {
name string
b []byte
wantName string
wantVersion string
wantDigest string
wantErr string
}{
{
name: "index",
b: []byte(`
apiVersion: v1
entries:
nginx:
- urls:
- https://kubernetes-charts.storage.googleapis.com/nginx-0.2.0.tgz
name: nginx
description: string
version: 0.2.0
home: https://github.com/something/else
digest: "sha256:1234567890abcdef"
`),
wantName: "nginx",
wantVersion: "0.2.0",
wantDigest: "sha256:1234567890abcdef",
},
{
name: "index without API version",
b: []byte(`entries:
nginx:
- name: nginx`),
wantErr: "no API version specified",
},
{
name: "index with duplicate entry",
b: []byte(`apiVersion: v1
entries:
nginx:
- name: nginx"
nginx:
- name: nginx`),
wantErr: "key \"nginx\" already set in map",
},
}
for _, tt := range tests {
tt := tt
t.Run(tt.name, func(t *testing.T) {
g := NewWithT(t)
t.Parallel()
r := newChartRepository()
err := r.LoadIndexFromBytes(tt.b)
if tt.wantErr != "" {
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring(tt.wantErr))
g.Expect(r.Index).To(BeNil())
return
}
g.Expect(err).ToNot(HaveOccurred())
g.Expect(r.Index).ToNot(BeNil())
got, err := r.Index.Get(tt.wantName, tt.wantVersion)
g.Expect(err).ToNot(HaveOccurred())
g.Expect(got.Digest).To(Equal(tt.wantDigest))
})
}
}
func TestChartRepository_LoadIndexFromBytes_Unordered(t *testing.T) {
b, err := os.ReadFile(unorderedTestFile)
if err != nil {
t.Fatal(err)
}
r := newChartRepository()
err = r.LoadIndexFromBytes(b)
if err != nil {
t.Fatal(err)
}
verifyLocalIndex(t, r.Index)
}
// Index load tests are derived from https://github.com/helm/helm/blob/v3.3.4/pkg/repo/index_test.go#L108
// to ensure parity with Helm behaviour.
func TestChartRepository_LoadIndexFromFile(t *testing.T) {
g := NewWithT(t)
// Create an index file that exceeds the max index size.
tmpDir := t.TempDir()
bigIndexFile := filepath.Join(tmpDir, "index.yaml")
data := make([]byte, helm.MaxIndexSize+10)
g.Expect(os.WriteFile(bigIndexFile, data, 0o640)).ToNot(HaveOccurred())
tests := []struct {
name string
filename string
wantErr string
}{
{
name: "regular index file",
filename: testFile,
},
{
name: "chartmuseum index file",
filename: chartmuseumTestFile,
},
{
name: "error if index size exceeds max size",
filename: bigIndexFile,
wantErr: "size of index 'index.yaml' exceeds",
},
}
for _, tt := range tests {
tt := tt
t.Run(tt.name, func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
err := r.LoadFromFile(tt.filename)
if tt.wantErr != "" {
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring(tt.wantErr))
return
}
g.Expect(err).ToNot(HaveOccurred())
verifyLocalIndex(t, r.Index)
})
}
}
func TestChartRepository_CacheIndex(t *testing.T) {
g := NewWithT(t)
mg := mockGetter{Response: []byte("foo")}
expectSum := fmt.Sprintf("%x", sha256.Sum256(mg.Response))
r := newChartRepository()
r.URL = "https://example.com"
r.Client = &mg
sum, err := r.CacheIndex()
g.Expect(err).To(Not(HaveOccurred()))
g.Expect(r.CachePath).ToNot(BeEmpty())
defer os.RemoveAll(r.CachePath)
g.Expect(r.CachePath).To(BeARegularFile())
b, _ := os.ReadFile(r.CachePath)
g.Expect(b).To(Equal(mg.Response))
g.Expect(sum).To(BeEquivalentTo(expectSum))
}
func TestChartRepository_StrategicallyLoadIndex(t *testing.T) {
g := NewWithT(t)
t.Run("loads from path", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
r.Index = repo.NewIndexFile()
g.Expect(r.StrategicallyLoadIndex()).To(Succeed())
g.Expect(r.CachePath).To(BeEmpty())
g.Expect(r.Cached).To(BeFalse())
i := filepath.Join(t.TempDir(), "index.yaml")
g.Expect(os.WriteFile(i, []byte(`apiVersion: v1`), 0o644)).To(Succeed())
r.Index = nil
r.CachePath = "/invalid/cache/index/path.yaml"
err := r.StrategicallyLoadIndex()
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring("/invalid/cache/index/path.yaml: no such file or directory"))
g.Expect(r.Cached).To(BeFalse())
r := newChartRepository()
r.Path = i
r.CachePath = ""
r.Client = &mockGetter{}
err = r.StrategicallyLoadIndex()
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring("no API version specified"))
g.Expect(r.Cached).To(BeTrue())
g.Expect(r.RemoveCache()).To(Succeed())
}
func TestChartRepository_CacheIndexInMemory(t *testing.T) {
g := NewWithT(t)
interval, _ := time.ParseDuration("5s")
memCache := cache.New(1, interval)
indexPath := "/multi-tenent-safe/mock/index.yaml"
r := newChartRepository()
r.Index = repo.NewIndexFile()
indexFile := *r.Index
g.Expect(
indexFile.MustAdd(
&chart.Metadata{
Name: "grafana",
Version: "6.17.4",
},
"grafana-6.17.4.tgz",
"http://example.com/charts",
"sha256:1234567890abc",
)).To(Succeed())
indexFile.WriteFile(indexPath, 0o640)
ttl, _ := time.ParseDuration("1m")
r.SetMemCache(indexPath, memCache, ttl, func(event string) {
fmt.Println(event)
err := r.StrategicallyLoadIndex()
g.Expect(err).To(Succeed())
g.Expect(r.Index).ToNot(BeNil())
})
t.Run("loads from client", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
r.Client = &mockGetter{
Response: []byte(`apiVersion: v1`),
}
t.Cleanup(func() {
_ = os.Remove(r.Path)
})
err := r.StrategicallyLoadIndex()
g.Expect(err).To(Succeed())
g.Expect(r.Path).ToNot(BeEmpty())
g.Expect(r.Index).ToNot(BeNil())
})
t.Run("skips if index is already loaded", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
r.Index = repo.NewIndexFile()
g.Expect(r.StrategicallyLoadIndex()).To(Succeed())
})
r.CacheIndexInMemory()
_, cacheHit := r.IndexCache.Get(indexPath)
g.Expect(cacheHit).To(Equal(true))
r.Unload()
g.Expect(r.Index).To(BeNil())
g.Expect(r.StrategicallyLoadIndex()).To(Succeed())
g.Expect(r.Index.Entries["grafana"][0].Digest).To(Equal("sha256:1234567890abc"))
}
func TestChartRepository_LoadFromCache(t *testing.T) {
tests := []struct {
name string
cachePath string
wantErr string
}{
{
name: "cache path",
cachePath: chartmuseumTestFile,
},
{
name: "invalid cache path",
cachePath: "invalid",
wantErr: "stat invalid: no such file",
},
{
name: "no cache path",
cachePath: "",
wantErr: "no cache path set",
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
g := NewWithT(t)
func TestChartRepository_LoadFromPath(t *testing.T) {
t.Run("loads index", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
r.CachePath = tt.cachePath
err := r.LoadFromCache()
if tt.wantErr != "" {
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring(tt.wantErr))
g.Expect(r.Index).To(BeNil())
return
}
i := filepath.Join(t.TempDir(), "index.yaml")
g.Expect(os.WriteFile(i, []byte(`apiVersion: v1`), 0o644)).To(Succeed())
g.Expect(err).ToNot(HaveOccurred())
verifyLocalIndex(t, r.Index)
})
}
r := newChartRepository()
r.Path = i
r.revisions["key"] = "value"
g.Expect(r.LoadFromPath()).To(Succeed())
g.Expect(r.Index).ToNot(BeNil())
g.Expect(r.revisions).To(BeEmpty())
})
t.Run("no cache path", func(t *testing.T) {
g := NewWithT(t)
err := newChartRepository().LoadFromPath()
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring("no cache path"))
})
t.Run("index load error", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
r.Path = filepath.Join(t.TempDir(), "index.yaml")
err := r.LoadFromPath()
g.Expect(err).To(HaveOccurred())
g.Expect(errors.Is(err, os.ErrNotExist)).To(BeTrue())
})
}
func TestChartRepository_Revision(t *testing.T) {
t.Run("with algorithm", func(t *testing.T) {
r := newChartRepository()
r.Index = repo.NewIndexFile()
for _, algo := range []digest.Algorithm{digest.SHA256, digest.SHA512} {
t.Run(algo.String(), func(t *testing.T) {
g := NewWithT(t)
d := r.Revision(algo)
g.Expect(d).ToNot(BeEmpty())
g.Expect(d.Algorithm()).To(Equal(algo))
g.Expect(r.revisions[algo]).To(Equal(d))
})
}
})
t.Run("without index", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
g.Expect(r.Revision(digest.SHA256)).To(BeEmpty())
})
t.Run("from cache", func(t *testing.T) {
g := NewWithT(t)
algo := digest.SHA256
expect := digest.Digest("sha256:fake")
r := newChartRepository()
r.Index = repo.NewIndexFile()
r.revisions[algo] = expect
g.Expect(r.Revision(algo)).To(Equal(expect))
})
}
func TestChartRepository_Digest(t *testing.T) {
t.Run("with algorithm", func(t *testing.T) {
g := NewWithT(t)
p := filepath.Join(t.TempDir(), "index.yaml")
g.Expect(repo.NewIndexFile().WriteFile(p, 0o644)).To(Succeed())
r := newChartRepository()
r.Path = p
for _, algo := range []digest.Algorithm{digest.SHA256, digest.SHA512} {
t.Run(algo.String(), func(t *testing.T) {
g := NewWithT(t)
d := r.Digest(algo)
g.Expect(d).ToNot(BeEmpty())
g.Expect(d.Algorithm()).To(Equal(algo))
g.Expect(r.digests[algo]).To(Equal(d))
})
}
})
t.Run("without path", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
g.Expect(r.Digest(digest.SHA256)).To(BeEmpty())
})
t.Run("from cache", func(t *testing.T) {
g := NewWithT(t)
algo := digest.SHA256
expect := digest.Digest("sha256:fake")
i := filepath.Join(t.TempDir(), "index.yaml")
g.Expect(os.WriteFile(i, []byte(`apiVersion: v1`), 0o644)).To(Succeed())
r := newChartRepository()
r.Path = i
r.digests[algo] = expect
g.Expect(r.Digest(algo)).To(Equal(expect))
})
}
func TestChartRepository_HasIndex(t *testing.T) {
@ -534,23 +600,88 @@ func TestChartRepository_HasIndex(t *testing.T) {
g.Expect(r.HasIndex()).To(BeTrue())
}
func TestChartRepository_HasCacheFile(t *testing.T) {
func TestChartRepository_HasFile(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
g.Expect(r.HasCacheFile()).To(BeFalse())
r.CachePath = "foo"
g.Expect(r.HasCacheFile()).To(BeTrue())
g.Expect(r.HasFile()).To(BeFalse())
i := filepath.Join(t.TempDir(), "index.yaml")
g.Expect(os.WriteFile(i, []byte(`apiVersion: v1`), 0o644)).To(Succeed())
r.Path = i
g.Expect(r.HasFile()).To(BeTrue())
}
func TestChartRepository_UnloadIndex(t *testing.T) {
func TestChartRepository_Clear(t *testing.T) {
t.Run("without index", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
g.Expect(r.Clear()).To(Succeed())
})
t.Run("with index", func(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
r.Index = repo.NewIndexFile()
r.revisions["key"] = "value"
g.Expect(r.Clear()).To(Succeed())
g.Expect(r.Index).To(BeNil())
g.Expect(r.revisions).To(BeEmpty())
})
t.Run("with index and cached path", func(t *testing.T) {
g := NewWithT(t)
f, err := os.CreateTemp(t.TempDir(), "index-*.yaml")
g.Expect(err).ToNot(HaveOccurred())
g.Expect(f.Close()).To(Succeed())
r := newChartRepository()
r.Path = f.Name()
r.Index = repo.NewIndexFile()
r.digests["key"] = "value"
r.revisions["key"] = "value"
r.cached = true
g.Expect(r.Clear()).To(Succeed())
g.Expect(r.Index).To(BeNil())
g.Expect(r.Path).To(BeEmpty())
g.Expect(r.digests).To(BeEmpty())
g.Expect(r.revisions).To(BeEmpty())
g.Expect(r.cached).To(BeFalse())
})
t.Run("with path", func(t *testing.T) {
g := NewWithT(t)
f, err := os.CreateTemp(t.TempDir(), "index-*.yaml")
g.Expect(err).ToNot(HaveOccurred())
g.Expect(f.Close()).To(Succeed())
r := newChartRepository()
r.Path = f.Name()
r.digests["key"] = "value"
g.Expect(r.Clear()).To(Succeed())
g.Expect(r.Path).ToNot(BeEmpty())
g.Expect(r.Path).To(BeARegularFile())
g.Expect(r.digests).To(BeEmpty())
})
}
func TestChartRepository_Invalidate(t *testing.T) {
g := NewWithT(t)
r := newChartRepository()
g.Expect(r.HasIndex()).To(BeFalse())
r.Index = repo.NewIndexFile()
r.Unload()
g.Expect(r.Index).To(BeNil())
r.digests["key"] = "value"
r.revisions["key"] = "value"
r.Invalidate()
g.Expect(r.digests).To(BeEmpty())
g.Expect(r.revisions).To(BeEmpty())
}
func verifyLocalIndex(t *testing.T, i *repo.IndexFile) {
@ -622,27 +753,3 @@ func verifyLocalIndex(t *testing.T, i *repo.IndexFile) {
g.Expect(tt.Keywords).To(ContainElements(expect.Keywords))
}
}
func TestChartRepository_RemoveCache(t *testing.T) {
g := NewWithT(t)
tmpFile, err := os.CreateTemp("", "remove-cache-")
g.Expect(err).ToNot(HaveOccurred())
defer os.Remove(tmpFile.Name())
r := newChartRepository()
r.CachePath = tmpFile.Name()
r.Cached = true
g.Expect(r.RemoveCache()).To(Succeed())
g.Expect(r.CachePath).To(BeEmpty())
g.Expect(r.Cached).To(BeFalse())
g.Expect(tmpFile.Name()).ToNot(BeAnExistingFile())
r.CachePath = tmpFile.Name()
r.Cached = true
g.Expect(r.RemoveCache()).To(Succeed())
g.Expect(r.CachePath).To(BeEmpty())
g.Expect(r.Cached).To(BeFalse())
}

221
internal/index/digest.go Normal file
View File

@ -0,0 +1,221 @@
/*
Copyright 2022 The Flux 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 index
import (
"fmt"
"io"
"sort"
"strings"
"sync"
"github.com/opencontainers/go-digest"
)
// Digester is a simple string key value index that can be used to calculate
// digests of the index. The digests are cached, and only recalculated if the
// index has changed.
type Digester struct {
// index is the map of keys and their associated values.
index map[string]string
// digests is a cache of digests calculated for the index.
digests map[digest.Algorithm]digest.Digest
mu sync.RWMutex
}
// DigesterOption is a functional option for configuring a digester.
type DigesterOption func(*Digester)
// WithIndex returns a DigesterOption that sets the index to the provided map.
// The map is copied, so any changes to the map after the option is applied
// will not be reflected in the index.
func WithIndex(i map[string]string) DigesterOption {
return func(d *Digester) {
if i != nil {
d.mu.Lock()
defer d.mu.Unlock()
if d.index == nil {
d.index = make(map[string]string, len(i))
}
for k, v := range i {
d.index[k] = v
}
d.reset()
}
}
}
// NewDigester returns a new digest index with an empty initialized index.
func NewDigester(opts ...DigesterOption) *Digester {
d := &Digester{
digests: make(map[digest.Algorithm]digest.Digest, 0),
index: make(map[string]string, 0),
}
for _, opt := range opts {
opt(d)
}
return d
}
// Add adds the key and digest to the index.
func (i *Digester) Add(key, value string) {
i.mu.Lock()
defer i.mu.Unlock()
i.index[key] = value
i.reset()
}
// Delete removes the key from the index.
func (i *Digester) Delete(key string) {
i.mu.Lock()
defer i.mu.Unlock()
if _, ok := i.index[key]; ok {
delete(i.index, key)
i.reset()
}
}
// Get returns the digest for the key, or an empty digest if the key is not
// found.
func (i *Digester) Get(key string) string {
i.mu.RLock()
defer i.mu.RUnlock()
return i.index[key]
}
// Has returns true if the index contains the key.
func (i *Digester) Has(key string) bool {
i.mu.RLock()
defer i.mu.RUnlock()
_, ok := i.index[key]
return ok
}
// Index returns a copy of the index.
func (i *Digester) Index() map[string]string {
i.mu.RLock()
defer i.mu.RUnlock()
index := make(map[string]string, len(i.index))
for k, v := range i.index {
index[k] = v
}
return index
}
// Len returns the number of keys in the index.
func (i *Digester) Len() int {
i.mu.RLock()
defer i.mu.RUnlock()
return len(i.index)
}
// String returns a string representation of the index. The keys are stable
// sorted, and the string representation of the key/value pairs is written,
// each pair on a newline with a space between them.
func (i *Digester) String() string {
i.mu.RLock()
defer i.mu.RUnlock()
keys := i.sortedKeys()
var b strings.Builder
for _, k := range keys {
b.Grow(len(k) + len(i.index[k]) + 2)
writeLine(&b, k, i.index[k])
}
return b.String()
}
// WriteTo writes the index to the writer. The keys are stable sorted, and the
// string representation of the key/value pairs is written, each pair on a
// newline with a space between them.
func (i *Digester) WriteTo(w io.Writer) (int64, error) {
i.mu.RLock()
defer i.mu.RUnlock()
keys := i.sortedKeys()
var n int64
for _, k := range keys {
nn, err := writeLine(w, k, i.index[k])
n += int64(nn)
if err != nil {
return n, err
}
}
return n, nil
}
// Digest returns the digest of the index using the provided algorithm.
// If the index has not changed since the last call to Digest, the cached
// digest is returned.
// For verifying the index against a known digest, use Verify.
func (i *Digester) Digest(a digest.Algorithm) digest.Digest {
i.mu.Lock()
defer i.mu.Unlock()
if _, ok := i.digests[a]; !ok {
digester := a.Digester()
keys := i.sortedKeys()
for _, k := range keys {
_, _ = writeLine(digester.Hash(), k, i.index[k])
}
i.digests[a] = digester.Digest()
}
return i.digests[a]
}
// Verify returns true if the index matches the provided digest.
func (i *Digester) Verify(d digest.Digest) bool {
i.mu.RLock()
defer i.mu.RUnlock()
verifier := d.Verifier()
keys := i.sortedKeys()
for _, k := range keys {
_, _ = writeLine(verifier, k, i.index[k])
}
return verifier.Verified()
}
// sortedKeys returns a slice of the keys in the index, sorted alphabetically.
func (i *Digester) sortedKeys() []string {
keys := make([]string, 0, len(i.index))
for k := range i.index {
keys = append(keys, k)
}
sort.Strings(keys)
return keys
}
// reset clears the digests cache.
func (i *Digester) reset() {
i.digests = make(map[digest.Algorithm]digest.Digest, 0)
}
// writeLine writes the key and digest to the writer, separated by a space and
// terminating with a newline.
func writeLine(w io.Writer, key, value string) (int, error) {
return fmt.Fprintf(w, "%s %s\n", key, value)
}

View File

@ -0,0 +1,346 @@
/*
Copyright 2022 The Flux 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 index
import (
"bytes"
"errors"
"testing"
. "github.com/onsi/gomega"
"github.com/opencontainers/go-digest"
)
func TestWithIndex(t *testing.T) {
t.Run("sets the index", func(t *testing.T) {
g := NewWithT(t)
i := map[string]string{"foo": "bar"}
d := &Digester{}
WithIndex(i)(d)
g.Expect(d.index).To(Equal(i))
})
t.Run("resets the digests", func(t *testing.T) {
g := NewWithT(t)
i := map[string]string{"foo": "bar"}
d := &Digester{
digests: map[digest.Algorithm]digest.Digest{
digest.SHA256: "sha256:foo",
},
}
WithIndex(i)(d)
g.Expect(d.digests).To(BeEmpty())
})
}
func TestNewDigester(t *testing.T) {
t.Run("default", func(t *testing.T) {
g := NewWithT(t)
d := NewDigester()
g.Expect(d).ToNot(BeNil())
g.Expect(d.index).ToNot(BeNil())
g.Expect(d.digests).ToNot(BeNil())
})
t.Run("with index", func(t *testing.T) {
g := NewWithT(t)
i := map[string]string{"foo": "bar"}
d := NewDigester(WithIndex(i))
g.Expect(d).ToNot(BeNil())
g.Expect(d.index).To(Equal(i))
g.Expect(d.digests).ToNot(BeNil())
})
}
func TestDigester_Add(t *testing.T) {
t.Run("adds", func(t *testing.T) {
g := NewWithT(t)
d := NewDigester()
d.Add("foo", "bar")
g.Expect(d.index).To(HaveKeyWithValue("foo", "bar"))
})
t.Run("overwrites", func(t *testing.T) {
g := NewWithT(t)
d := NewDigester()
d.Add("foo", "bar")
d.Add("foo", "baz")
g.Expect(d.index).To(HaveKeyWithValue("foo", "baz"))
})
t.Run("resets digests", func(t *testing.T) {
g := NewWithT(t)
d := &Digester{
index: map[string]string{},
digests: map[digest.Algorithm]digest.Digest{
digest.SHA256: "sha256:foo",
},
}
d.Add("foo", "bar")
g.Expect(d.digests).To(BeEmpty())
})
}
func TestDigester_Delete(t *testing.T) {
t.Run("deletes", func(t *testing.T) {
g := NewWithT(t)
d := NewDigester()
d.Add("foo", "bar")
d.Delete("foo")
g.Expect(d.index).ToNot(HaveKey("foo"))
})
t.Run("resets digests", func(t *testing.T) {
g := NewWithT(t)
d := &Digester{
index: map[string]string{
"foo": "bar",
},
digests: map[digest.Algorithm]digest.Digest{
digest.SHA256: "sha256:foo",
},
}
d.Delete("nop")
g.Expect(d.digests).To(HaveLen(1))
d.Delete("foo")
g.Expect(d.digests).To(BeEmpty())
})
}
func TestDigester_Get(t *testing.T) {
g := NewWithT(t)
d := NewDigester()
d.Add("foo", "bar")
g.Expect(d.Get("foo")).To(Equal("bar"))
g.Expect(d.Get("bar")).To(BeEmpty())
}
func TestDigester_Has(t *testing.T) {
g := NewWithT(t)
d := NewDigester()
d.Add("foo", "bar")
g.Expect(d.Has("foo")).To(BeTrue())
g.Expect(d.Has("bar")).To(BeFalse())
}
func TestDigester_Index(t *testing.T) {
g := NewWithT(t)
i := map[string]string{
"foo": "bar",
"bar": "baz",
}
d := NewDigester(WithIndex(i))
iCopy := d.Index()
g.Expect(iCopy).To(Equal(i))
g.Expect(iCopy).ToNot(BeIdenticalTo(i))
}
func TestDigester_Len(t *testing.T) {
g := NewWithT(t)
d := NewDigester(WithIndex(map[string]string{
"foo": "bar",
"bar": "baz",
}))
g.Expect(d.Len()).To(Equal(2))
}
func TestDigester_String(t *testing.T) {
g := NewWithT(t)
d := NewDigester(WithIndex(map[string]string{
"foo": "bar",
"bar": "baz",
}))
g.Expect(d.String()).To(Equal(`bar baz
foo bar
`))
}
func TestDigester_WriteTo(t *testing.T) {
t.Run("writes", func(t *testing.T) {
g := NewWithT(t)
d := NewDigester(WithIndex(map[string]string{
"foo": "bar",
"bar": "baz",
}))
expect := `bar baz
foo bar
`
var buf bytes.Buffer
n, err := d.WriteTo(&buf)
g.Expect(n).To(Equal(int64(len(expect))))
g.Expect(err).ToNot(HaveOccurred())
g.Expect(buf.String()).To(Equal(expect))
})
t.Run("errors", func(t *testing.T) {
g := NewWithT(t)
d := NewDigester(WithIndex(map[string]string{
"foo": "bar",
"bar": "baz",
}))
w := &fakeWriter{
err: errors.New("write error"),
written: 5,
}
n, err := d.WriteTo(w)
g.Expect(err).To(HaveOccurred())
g.Expect(errors.Is(err, w.err)).To(BeTrue())
g.Expect(n).To(Equal(int64(w.written)))
})
}
func TestDigester_Digest(t *testing.T) {
t.Run("returns digest", func(t *testing.T) {
g := NewWithT(t)
d := NewDigester(WithIndex(map[string]string{
"foo": "bar",
"bar": "baz",
}))
expect := digest.SHA256.FromString(d.String())
g.Expect(d.Digest(digest.SHA256)).To(Equal(expect))
g.Expect(d.digests).To(HaveKeyWithValue(digest.SHA256, expect))
})
t.Run("returns cached digest", func(t *testing.T) {
g := NewWithT(t)
d := &Digester{
index: map[string]string{
"foo": "bar",
"bar": "baz",
},
digests: map[digest.Algorithm]digest.Digest{
digest.SHA256: "sha256:foo",
},
}
g.Expect(d.Digest(digest.SHA256)).To(Equal(d.digests[digest.SHA256]))
})
}
func TestDigester_Verify(t *testing.T) {
g := NewWithT(t)
d := NewDigester(WithIndex(map[string]string{
"foo": "bar",
}))
g.Expect(d.Verify(d.Digest(digest.SHA256))).To(BeTrue())
g.Expect(d.Verify(digest.SHA256.FromString("different"))).To(BeFalse())
}
func TestDigester_sortedKeys(t *testing.T) {
g := NewWithT(t)
d := NewDigester(WithIndex(map[string]string{
"c/d/e": "bar",
"a/b/c": "baz",
"f/g/h": "foo",
}))
g.Expect(d.sortedKeys()).To(Equal([]string{
"a/b/c",
"c/d/e",
"f/g/h",
}))
}
func TestDigester_reset(t *testing.T) {
g := NewWithT(t)
d := NewDigester()
d.digests = map[digest.Algorithm]digest.Digest{
digest.SHA256: "sha256:foo",
}
d.reset()
g.Expect(d.digests).To(BeEmpty())
}
func Test_writeLine(t *testing.T) {
t.Run("writes", func(t *testing.T) {
g := NewWithT(t)
var buf bytes.Buffer
n, err := writeLine(&buf, "foo", "bar")
g.Expect(n).To(Equal(8))
g.Expect(err).ToNot(HaveOccurred())
g.Expect(buf.String()).To(Equal(`foo bar
`))
})
t.Run("errors", func(t *testing.T) {
g := NewWithT(t)
w := &fakeWriter{
err: errors.New("write error"),
written: 5,
}
n, err := writeLine(w, "foo", "bar")
g.Expect(err).To(HaveOccurred())
g.Expect(errors.Is(err, w.err)).To(BeTrue())
g.Expect(n).To(Equal(w.written))
})
}
type fakeWriter struct {
written int
err error
}
func (f *fakeWriter) Write(p []byte) (n int, err error) {
return f.written, f.err
}

19
main.go
View File

@ -44,6 +44,8 @@ import (
"github.com/fluxcd/pkg/runtime/logger"
"github.com/fluxcd/pkg/runtime/pprof"
"github.com/fluxcd/pkg/runtime/probes"
"github.com/fluxcd/source-controller/internal/digest"
"github.com/fluxcd/source-controller/internal/features"
"github.com/fluxcd/source-controller/internal/helm/registry"
@ -102,6 +104,7 @@ func main() {
helmCachePurgeInterval string
artifactRetentionTTL time.Duration
artifactRetentionRecords int
artifactDigestAlgo string
)
flag.StringVar(&metricsAddr, "metrics-addr", envOrDefault("METRICS_ADDR", ":8080"),
@ -137,9 +140,11 @@ func main() {
flag.StringSliceVar(&git.HostKeyAlgos, "ssh-hostkey-algos", []string{},
"The list of hostkey algorithms to use for ssh connections, arranged from most preferred to the least.")
flag.DurationVar(&artifactRetentionTTL, "artifact-retention-ttl", 60*time.Second,
"The duration of time that artifacts from previous reconcilations will be kept in storage before being garbage collected.")
"The duration of time that artifacts from previous reconciliations will be kept in storage before being garbage collected.")
flag.IntVar(&artifactRetentionRecords, "artifact-retention-records", 2,
"The maximum number of artifacts to be kept in storage after a garbage collection.")
flag.StringVar(&artifactDigestAlgo, "artifact-digest-algo", digest.Canonical.String(),
"The algorithm to use to calculate the digest of artifacts.")
clientOptions.BindFlags(flag.CommandLine)
logOptions.BindFlags(flag.CommandLine)
@ -159,7 +164,15 @@ func main() {
os.Exit(1)
}
// Set upper bound file size limits Helm
if artifactDigestAlgo != digest.Canonical.String() {
algo, err := digest.AlgorithmForName(artifactDigestAlgo)
if err != nil {
setupLog.Error(err, "unable to configure canonical digest algorithm")
os.Exit(1)
}
digest.Canonical = algo
}
helm.MaxIndexSize = helmIndexLimit
helm.MaxChartSize = helmChartLimit
helm.MaxChartFileSize = helmChartFileLimit
@ -169,7 +182,7 @@ func main() {
watchNamespace = os.Getenv("RUNTIME_NAMESPACE")
}
disableCacheFor := []ctrlclient.Object{}
var disableCacheFor []ctrlclient.Object
shouldCache, err := features.Enabled(features.CacheSecretsAndConfigMaps)
if err != nil {
setupLog.Error(err, "unable to check feature gate "+features.CacheSecretsAndConfigMaps)