From a2959a28c5d15583d4af1827108ac291f8fe495e Mon Sep 17 00:00:00 2001 From: Nathan Romriell Date: Thu, 30 Nov 2023 07:03:34 -0800 Subject: [PATCH] fix(repo-server): excess git requests, resolveReferencedSources and runManifestGenAsync not using cache (Issue #14725) (#16410) * fix(repo-server): excess git requests part 1, resolveReferencedSources and runManifestGenAsync Signed-off-by: nromriell * fix: remove unnecessary settings instantiation Signed-off-by: nromriell --------- Signed-off-by: nromriell --- reposerver/cache/mocks/reposervercache.go | 71 ++++ reposerver/repository/repository.go | 11 +- reposerver/repository/repository_test.go | 449 +++++++++++++++++----- util/cache/mocks/cacheclient.go | 65 ++++ 4 files changed, 488 insertions(+), 108 deletions(-) create mode 100644 reposerver/cache/mocks/reposervercache.go create mode 100644 util/cache/mocks/cacheclient.go diff --git a/reposerver/cache/mocks/reposervercache.go b/reposerver/cache/mocks/reposervercache.go new file mode 100644 index 0000000000000..0e49b5816178e --- /dev/null +++ b/reposerver/cache/mocks/reposervercache.go @@ -0,0 +1,71 @@ +package mocks + +import ( + "testing" + "time" + + "github.com/alicebob/miniredis/v2" + cacheutil "github.com/argoproj/argo-cd/v2/util/cache" + cacheutilmocks "github.com/argoproj/argo-cd/v2/util/cache/mocks" + "github.com/redis/go-redis/v9" + "github.com/stretchr/testify/mock" +) + +type MockCacheType int + +const ( + MockCacheTypeRedis MockCacheType = iota + MockCacheTypeInMem +) + +type MockRepoCache struct { + mock.Mock + RedisClient *cacheutilmocks.MockCacheClient + StopRedisCallback func() +} + +type MockCacheOptions struct { + RepoCacheExpiration time.Duration + RevisionCacheExpiration time.Duration + ReadDelay time.Duration + WriteDelay time.Duration +} + +type CacheCallCounts struct { + ExternalSets int + ExternalGets int + ExternalDeletes int +} + +// Checks that the cache was called the expected number of times +func (mockCache *MockRepoCache) AssertCacheCalledTimes(t *testing.T, calls *CacheCallCounts) { + mockCache.RedisClient.AssertNumberOfCalls(t, "Get", calls.ExternalGets) + mockCache.RedisClient.AssertNumberOfCalls(t, "Set", calls.ExternalSets) + mockCache.RedisClient.AssertNumberOfCalls(t, "Delete", calls.ExternalDeletes) +} + +func (mockCache *MockRepoCache) ConfigureDefaultCallbacks() { + mockCache.RedisClient.On("Get", mock.Anything, mock.Anything).Return(nil) + mockCache.RedisClient.On("Set", mock.Anything).Return(nil) + mockCache.RedisClient.On("Delete", mock.Anything).Return(nil) +} + +func NewInMemoryRedis() (*redis.Client, func()) { + cacheutil.NewInMemoryCache(5 * time.Second) + mr, err := miniredis.Run() + if err != nil { + panic(err) + } + return redis.NewClient(&redis.Options{Addr: mr.Addr()}), mr.Close +} + +func NewMockRepoCache(cacheOpts *MockCacheOptions) *MockRepoCache { + redisClient, stopRedis := NewInMemoryRedis() + redisCacheClient := &cacheutilmocks.MockCacheClient{ + ReadDelay: cacheOpts.ReadDelay, + WriteDelay: cacheOpts.WriteDelay, + BaseCache: cacheutil.NewRedisCache(redisClient, cacheOpts.RepoCacheExpiration, cacheutil.RedisCompressionNone)} + newMockCache := &MockRepoCache{RedisClient: redisCacheClient, StopRedisCallback: stopRedis} + newMockCache.ConfigureDefaultCallbacks() + return newMockCache +} diff --git a/reposerver/repository/repository.go b/reposerver/repository/repository.go index f66bc71ac4621..f5a934ee12c5c 100644 --- a/reposerver/repository/repository.go +++ b/reposerver/repository/repository.go @@ -300,6 +300,7 @@ func (s *Service) runRepoOperation( var gitClient git.Client var helmClient helm.Client var err error + gitClientOpts := git.WithCache(s.cache, !settings.noRevisionCache && !settings.noCache) revision = textutils.FirstNonEmpty(revision, source.TargetRevision) unresolvedRevision := revision if source.IsHelm() { @@ -308,13 +309,13 @@ func (s *Service) runRepoOperation( return err } } else { - gitClient, revision, err = s.newClientResolveRevision(repo, revision, git.WithCache(s.cache, !settings.noRevisionCache && !settings.noCache)) + gitClient, revision, err = s.newClientResolveRevision(repo, revision, gitClientOpts) if err != nil { return err } } - repoRefs, err := resolveReferencedSources(hasMultipleSources, source.Helm, refSources, s.newClientResolveRevision) + repoRefs, err := resolveReferencedSources(hasMultipleSources, source.Helm, refSources, s.newClientResolveRevision, gitClientOpts) if err != nil { return err } @@ -463,7 +464,7 @@ type gitClientGetter func(repo *v1alpha1.Repository, revision string, opts ...gi // // Much of this logic is duplicated in runManifestGenAsync. If making changes here, check whether runManifestGenAsync // should be updated. -func resolveReferencedSources(hasMultipleSources bool, source *v1alpha1.ApplicationSourceHelm, refSources map[string]*v1alpha1.RefTarget, newClientResolveRevision gitClientGetter) (map[string]string, error) { +func resolveReferencedSources(hasMultipleSources bool, source *v1alpha1.ApplicationSourceHelm, refSources map[string]*v1alpha1.RefTarget, newClientResolveRevision gitClientGetter, gitClientOpts git.ClientOpts) (map[string]string, error) { repoRefs := make(map[string]string) if !hasMultipleSources || source == nil { return repoRefs, nil @@ -490,7 +491,7 @@ func resolveReferencedSources(hasMultipleSources bool, source *v1alpha1.Applicat normalizedRepoURL := git.NormalizeGitURL(refSourceMapping.Repo.Repo) _, ok = repoRefs[normalizedRepoURL] if !ok { - _, referencedCommitSHA, err := newClientResolveRevision(&refSourceMapping.Repo, refSourceMapping.TargetRevision) + _, referencedCommitSHA, err := newClientResolveRevision(&refSourceMapping.Repo, refSourceMapping.TargetRevision, gitClientOpts) if err != nil { log.Errorf("Failed to get git client for repo %s: %v", refSourceMapping.Repo.Repo, err) return nil, fmt.Errorf("failed to get git client for repo %s", refSourceMapping.Repo.Repo) @@ -728,7 +729,7 @@ func (s *Service) runManifestGenAsync(ctx context.Context, repoRoot, commitSHA, return } } else { - gitClient, referencedCommitSHA, err := s.newClientResolveRevision(&refSourceMapping.Repo, refSourceMapping.TargetRevision) + gitClient, referencedCommitSHA, err := s.newClientResolveRevision(&refSourceMapping.Repo, refSourceMapping.TargetRevision, git.WithCache(s.cache, !q.NoRevisionCache && !q.NoCache)) if err != nil { log.Errorf("Failed to get git client for repo %s: %v", refSourceMapping.Repo.Repo, err) ch.errCh <- fmt.Errorf("failed to get git client for repo %s", refSourceMapping.Repo.Repo) diff --git a/reposerver/repository/repository_test.go b/reposerver/repository/repository_test.go index a24eb5008c47b..74d09dd6f86b8 100644 --- a/reposerver/repository/repository_test.go +++ b/reposerver/repository/repository_test.go @@ -1,6 +1,7 @@ package repository import ( + "bytes" "context" "encoding/json" "errors" @@ -17,6 +18,7 @@ import ( "testing" "time" + cacheutil "github.com/argoproj/argo-cd/v2/util/cache" log "github.com/sirupsen/logrus" "k8s.io/apimachinery/pkg/api/resource" @@ -28,13 +30,14 @@ import ( "k8s.io/apimachinery/pkg/runtime" "sigs.k8s.io/yaml" + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1" argoappv1 "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1" "github.com/argoproj/argo-cd/v2/reposerver/apiclient" "github.com/argoproj/argo-cd/v2/reposerver/cache" + repositorymocks "github.com/argoproj/argo-cd/v2/reposerver/cache/mocks" "github.com/argoproj/argo-cd/v2/reposerver/metrics" fileutil "github.com/argoproj/argo-cd/v2/test/fixture/path" "github.com/argoproj/argo-cd/v2/util/argo" - cacheutil "github.com/argoproj/argo-cd/v2/util/cache" dbmocks "github.com/argoproj/argo-cd/v2/util/db/mocks" "github.com/argoproj/argo-cd/v2/util/git" gitmocks "github.com/argoproj/argo-cd/v2/util/git/mocks" @@ -51,12 +54,49 @@ gpg: Good signature from "GitHub (web-flow commit signing) " type clientFunc func(*gitmocks.Client, *helmmocks.Client, *iomocks.TempPaths) -func newServiceWithMocks(root string, signed bool) (*Service, *gitmocks.Client) { +type repoCacheMocks struct { + mock.Mock + cacheutilCache *cacheutil.Cache + cache *cache.Cache + mockCache *repositorymocks.MockRepoCache +} + +type newGitRepoHelmChartOptions struct { + chartName string + chartVersion string + // valuesFiles is a map of the values file name to the key/value pairs to be written to the file + valuesFiles map[string]map[string]string +} + +type newGitRepoOptions struct { + path string + createPath bool + remote string + addEmptyCommit bool + helmChartOptions newGitRepoHelmChartOptions +} + +func newCacheMocks() *repoCacheMocks { + mockRepoCache := repositorymocks.NewMockRepoCache(&repositorymocks.MockCacheOptions{ + RepoCacheExpiration: 1 * time.Minute, + RevisionCacheExpiration: 1 * time.Minute, + ReadDelay: 0, + WriteDelay: 0, + }) + cacheutilCache := cacheutil.NewCache(mockRepoCache.RedisClient) + return &repoCacheMocks{ + cacheutilCache: cacheutilCache, + cache: cache.NewCache(cacheutilCache, 1*time.Minute, 1*time.Minute), + mockCache: mockRepoCache, + } +} + +func newServiceWithMocks(t *testing.T, root string, signed bool) (*Service, *gitmocks.Client, *repoCacheMocks) { root, err := filepath.Abs(root) if err != nil { panic(err) } - return newServiceWithOpt(func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { + return newServiceWithOpt(t, func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { gitClient.On("Init").Return(nil) gitClient.On("Fetch", mock.Anything).Return(nil) gitClient.On("Checkout", mock.Anything, mock.Anything).Return(nil) @@ -73,7 +113,7 @@ func newServiceWithMocks(root string, signed bool) (*Service, *gitmocks.Client) chart := "my-chart" oobChart := "out-of-bounds-chart" version := "1.1.0" - helmClient.On("GetIndex", true).Return(&helm.Index{Entries: map[string]helm.Entries{ + helmClient.On("GetIndex", mock.AnythingOfType("bool")).Return(&helm.Index{Entries: map[string]helm.Entries{ chart: {{Version: "1.0.0"}, {Version: version}}, oobChart: {{Version: "1.0.0"}, {Version: version}}, }}, nil) @@ -89,18 +129,16 @@ func newServiceWithMocks(root string, signed bool) (*Service, *gitmocks.Client) }, root) } -func newServiceWithOpt(cf clientFunc, root string) (*Service, *gitmocks.Client) { +func newServiceWithOpt(t *testing.T, cf clientFunc, root string) (*Service, *gitmocks.Client, *repoCacheMocks) { helmClient := &helmmocks.Client{} gitClient := &gitmocks.Client{} paths := &iomocks.TempPaths{} cf(gitClient, helmClient, paths) - service := NewService(metrics.NewMetricsServer(), cache.NewCache( - cacheutil.NewCache(cacheutil.NewInMemoryCache(1*time.Minute)), - 1*time.Minute, - 1*time.Minute, - ), RepoServerInitConstants{ParallelismLimit: 1}, argo.NewResourceTracking(), &git.NoopCredsStore{}, root) + cacheMocks := newCacheMocks() + t.Cleanup(cacheMocks.mockCache.StopRedisCallback) + service := NewService(metrics.NewMetricsServer(), cacheMocks.cache, RepoServerInitConstants{ParallelismLimit: 1}, argo.NewResourceTracking(), &git.NoopCredsStore{}, root) - service.newGitClient = func(rawRepoURL string, root string, creds git.Creds, insecure bool, enableLfs bool, prosy string, opts ...git.ClientOpts) (client git.Client, e error) { + service.newGitClient = func(rawRepoURL string, root string, creds git.Creds, insecure bool, enableLfs bool, proxy string, opts ...git.ClientOpts) (client git.Client, e error) { return gitClient, nil } service.newHelmClient = func(repoURL string, creds helm.Creds, enableOci bool, proxy string, opts ...helm.ClientOpts) helm.Client { @@ -110,20 +148,20 @@ func newServiceWithOpt(cf clientFunc, root string) (*Service, *gitmocks.Client) return io.NopCloser } service.gitRepoPaths = paths - return service, gitClient + return service, gitClient, cacheMocks } -func newService(root string) *Service { - service, _ := newServiceWithMocks(root, false) +func newService(t *testing.T, root string) *Service { + service, _, _ := newServiceWithMocks(t, root, false) return service } -func newServiceWithSignature(root string) *Service { - service, _ := newServiceWithMocks(root, true) +func newServiceWithSignature(t *testing.T, root string) *Service { + service, _, _ := newServiceWithMocks(t, root, true) return service } -func newServiceWithCommitSHA(root, revision string) *Service { +func newServiceWithCommitSHA(t *testing.T, root, revision string) *Service { var revisionErr error commitSHARegex := regexp.MustCompile("^[0-9A-Fa-f]{40}$") @@ -131,7 +169,7 @@ func newServiceWithCommitSHA(root, revision string) *Service { revisionErr = errors.New("not a commit SHA") } - service, gitClient := newServiceWithOpt(func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { + service, gitClient, _ := newServiceWithOpt(t, func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { gitClient.On("Init").Return(nil) gitClient.On("Fetch", mock.Anything).Return(nil) gitClient.On("Checkout", mock.Anything, mock.Anything).Return(nil) @@ -150,7 +188,7 @@ func newServiceWithCommitSHA(root, revision string) *Service { } func TestGenerateYamlManifestInDir(t *testing.T) { - service := newService("../../manifests/base") + service := newService(t, "../../manifests/base") src := argoappv1.ApplicationSource{Path: "."} q := apiclient.ManifestRequest{ @@ -247,7 +285,7 @@ func TestGenerateManifests_MissingSymlinkDestination(t *testing.T) { } func TestGenerateManifests_K8SAPIResetCache(t *testing.T) { - service := newService("../../manifests/base") + service := newService(t, "../../manifests/base") src := argoappv1.ApplicationSource{Path: "."} q := apiclient.ManifestRequest{ @@ -275,7 +313,7 @@ func TestGenerateManifests_K8SAPIResetCache(t *testing.T) { } func TestGenerateManifests_EmptyCache(t *testing.T) { - service := newService("../../manifests/base") + service, gitMocks, mockCache := newServiceWithMocks(t, "../../manifests/base", false) src := argoappv1.ApplicationSource{Path: "."} q := apiclient.ManifestRequest{ @@ -291,11 +329,85 @@ func TestGenerateManifests_EmptyCache(t *testing.T) { res, err := service.GenerateManifest(context.Background(), &q) assert.NoError(t, err) assert.True(t, len(res.Manifests) > 0) + mockCache.mockCache.AssertCacheCalledTimes(t, &repositorymocks.CacheCallCounts{ + ExternalSets: 2, + ExternalGets: 2, + ExternalDeletes: 1}) + gitMocks.AssertCalled(t, "LsRemote", mock.Anything) + gitMocks.AssertCalled(t, "Fetch", mock.Anything) +} + +// Test that calling manifest generation on source helm reference helm files that when the revision is cached it does not call ls-remote +func TestGenerateManifestsHelmWithRefs_CachedNoLsRemote(t *testing.T) { + dir := t.TempDir() + repopath := fmt.Sprintf("%s/tmprepo", dir) + cacheMocks := newCacheMocks() + t.Cleanup(func() { + cacheMocks.mockCache.StopRedisCallback() + err := filepath.WalkDir(dir, + func(path string, di fs.DirEntry, err error) error { + if err == nil { + return os.Chmod(path, 0777) + } + return err + }) + if err != nil { + t.Fatal(err) + } + }) + service := NewService(metrics.NewMetricsServer(), cacheMocks.cache, RepoServerInitConstants{ParallelismLimit: 1}, argo.NewResourceTracking(), &git.NoopCredsStore{}, repopath) + var gitClient git.Client + var err error + service.newGitClient = func(rawRepoURL string, root string, creds git.Creds, insecure bool, enableLfs bool, proxy string, opts ...git.ClientOpts) (client git.Client, e error) { + opts = append(opts, git.WithEventHandlers(git.EventHandlers{ + // Primary check, we want to make sure ls-remote is not called when the item is in cache + OnLsRemote: func(repo string) func() { + return func() { + assert.Fail(t, "LsRemote should not be called when the item is in cache") + } + }, + })) + gitClient, err = git.NewClientExt(rawRepoURL, root, creds, insecure, enableLfs, proxy, opts...) + return gitClient, err + } + repoRemote := fmt.Sprintf("file://%s", repopath) + revision := initGitRepo(t, newGitRepoOptions{ + path: repopath, + createPath: true, + remote: repoRemote, + helmChartOptions: newGitRepoHelmChartOptions{ + chartName: "my-chart", + chartVersion: "v1.0.0", + valuesFiles: map[string]map[string]string{"test.yaml": {"testval": "test"}}}, + }) + src := argoappv1.ApplicationSource{RepoURL: repoRemote, Path: ".", TargetRevision: "HEAD", Helm: &argoappv1.ApplicationSourceHelm{ + ValueFiles: []string{"$ref/test.yaml"}, + }} + repo := &argoappv1.Repository{ + Repo: repoRemote, + } + q := apiclient.ManifestRequest{ + Repo: repo, + Revision: "HEAD", + HasMultipleSources: true, + ApplicationSource: &src, + ProjectName: "default", + ProjectSourceRepos: []string{"*"}, + RefSources: map[string]*argoappv1.RefTarget{"$ref": {TargetRevision: "HEAD", Repo: *repo}}, + } + err = cacheMocks.cacheutilCache.SetItem(fmt.Sprintf("git-refs|%s", repoRemote), [][2]string{{"HEAD", revision}}, 30*time.Second, false) + assert.NoError(t, err) + _, err = service.GenerateManifest(context.Background(), &q) + assert.NoError(t, err) + cacheMocks.mockCache.AssertCacheCalledTimes(t, &repositorymocks.CacheCallCounts{ + ExternalSets: 2, + ExternalGets: 5}) } // ensure we can use a semver constraint range (>= 1.0.0) and get back the correct chart (1.0.0) func TestHelmManifestFromChartRepo(t *testing.T) { - service := newService(".") + root := t.TempDir() + service, gitMocks, mockCache := newServiceWithMocks(t, root, false) source := &argoappv1.ApplicationSource{Chart: "my-chart", TargetRevision: ">= 1.0.0"} request := &apiclient.ManifestRequest{Repo: &argoappv1.Repository{}, ApplicationSource: source, NoCache: true, ProjectName: "something", ProjectSourceRepos: []string{"*"}} @@ -309,10 +421,14 @@ func TestHelmManifestFromChartRepo(t *testing.T) { Revision: "1.1.0", SourceType: "Helm", }, response) + mockCache.mockCache.AssertCacheCalledTimes(t, &repositorymocks.CacheCallCounts{ + ExternalSets: 1, + ExternalGets: 0}) + gitMocks.AssertNotCalled(t, "LsRemote", mock.Anything) } func TestHelmChartReferencingExternalValues(t *testing.T) { - service := newService(".") + service := newService(t, ".") spec := argoappv1.ApplicationSpec{ Sources: []argoappv1.ApplicationSource{ {RepoURL: "https://helm.example.com", Chart: "my-chart", TargetRevision: ">= 1.0.0", Helm: &argoappv1.ApplicationSourceHelm{ @@ -342,7 +458,7 @@ func TestHelmChartReferencingExternalValues(t *testing.T) { } func TestHelmChartReferencingExternalValues_OutOfBounds_Symlink(t *testing.T) { - service := newService(".") + service := newService(t, ".") err := os.Mkdir("testdata/oob-symlink", 0755) require.NoError(t, err) t.Cleanup(func() { @@ -376,7 +492,7 @@ func TestHelmChartReferencingExternalValues_OutOfBounds_Symlink(t *testing.T) { } func TestGenerateManifestsUseExactRevision(t *testing.T) { - service, gitClient := newServiceWithMocks(".", false) + service, gitClient, _ := newServiceWithMocks(t, ".", false) src := argoappv1.ApplicationSource{Path: "./testdata/recurse", Directory: &argoappv1.ApplicationSourceDirectory{Recurse: true}} @@ -390,7 +506,7 @@ func TestGenerateManifestsUseExactRevision(t *testing.T) { } func TestRecurseManifestsInDir(t *testing.T) { - service := newService(".") + service := newService(t, ".") src := argoappv1.ApplicationSource{Path: "./testdata/recurse", Directory: &argoappv1.ApplicationSourceDirectory{Recurse: true}} @@ -403,7 +519,7 @@ func TestRecurseManifestsInDir(t *testing.T) { } func TestInvalidManifestsInDir(t *testing.T) { - service := newService(".") + service := newService(t, ".") src := argoappv1.ApplicationSource{Path: "./testdata/invalid-manifests", Directory: &argoappv1.ApplicationSourceDirectory{Recurse: true}} @@ -414,7 +530,7 @@ func TestInvalidManifestsInDir(t *testing.T) { } func TestInvalidMetadata(t *testing.T) { - service := newService(".") + service := newService(t, ".") src := argoappv1.ApplicationSource{Path: "./testdata/invalid-metadata", Directory: &argoappv1.ApplicationSourceDirectory{Recurse: true}} q := apiclient.ManifestRequest{Repo: &argoappv1.Repository{}, ApplicationSource: &src, AppLabelKey: "test", AppName: "invalid-metadata", TrackingMethod: "annotation+label"} @@ -424,7 +540,7 @@ func TestInvalidMetadata(t *testing.T) { } func TestNilMetadataAccessors(t *testing.T) { - service := newService(".") + service := newService(t, ".") expected := "{\"apiVersion\":\"v1\",\"kind\":\"ConfigMap\",\"metadata\":{\"annotations\":{\"argocd.argoproj.io/tracking-id\":\"nil-metadata-accessors:/ConfigMap:/my-map\"},\"labels\":{\"test\":\"nil-metadata-accessors\"},\"name\":\"my-map\"},\"stringData\":{\"foo\":\"bar\"}}" src := argoappv1.ApplicationSource{Path: "./testdata/nil-metadata-accessors", Directory: &argoappv1.ApplicationSourceDirectory{Recurse: true}} @@ -436,7 +552,7 @@ func TestNilMetadataAccessors(t *testing.T) { } func TestGenerateJsonnetManifestInDir(t *testing.T) { - service := newService(".") + service := newService(t, ".") q := apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -459,7 +575,7 @@ func TestGenerateJsonnetManifestInDir(t *testing.T) { } func TestGenerateJsonnetManifestInRootDir(t *testing.T) { - service := newService("testdata/jsonnet-1") + service := newService(t, "testdata/jsonnet-1") q := apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -482,7 +598,7 @@ func TestGenerateJsonnetManifestInRootDir(t *testing.T) { } func TestGenerateJsonnetLibOutside(t *testing.T) { - service := newService(".") + service := newService(t, ".") q := apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -553,7 +669,7 @@ func TestManifestGenErrorCacheByNumRequests(t *testing.T) { for _, tt := range tests { testName := fmt.Sprintf("gen-attempts-%d-pause-%d-total-%d", tt.PauseGenerationAfterFailedGenerationAttempts, tt.PauseGenerationOnFailureForRequests, tt.TotalCacheInvocations) t.Run(testName, func(t *testing.T) { - service := newService(".") + service := newService(t, ".") service.initConstants = RepoServerInitConstants{ ParallelismLimit: 1, @@ -631,7 +747,7 @@ func TestManifestGenErrorCacheFileContentsChange(t *testing.T) { tmpDir := t.TempDir() - service := newService(tmpDir) + service := newService(t, tmpDir) service.initConstants = RepoServerInitConstants{ ParallelismLimit: 1, @@ -701,7 +817,7 @@ func TestManifestGenErrorCacheByMinutesElapsed(t *testing.T) { for _, tt := range tests { testName := fmt.Sprintf("pause-time-%d", tt.PauseGenerationOnFailureForMinutes) t.Run(testName, func(t *testing.T) { - service := newService(".") + service := newService(t, ".") // Here we simulate the passage of time by overriding the now() function of Service currentTime := time.Now() @@ -771,7 +887,7 @@ func TestManifestGenErrorCacheByMinutesElapsed(t *testing.T) { func TestManifestGenErrorCacheRespectsNoCache(t *testing.T) { - service := newService(".") + service := newService(t, ".") service.initConstants = RepoServerInitConstants{ ParallelismLimit: 1, @@ -828,7 +944,7 @@ func TestManifestGenErrorCacheRespectsNoCache(t *testing.T) { } func TestGenerateHelmWithValues(t *testing.T) { - service := newService("../../util/helm/testdata/redis") + service := newService(t, "../../util/helm/testdata/redis") res, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -865,7 +981,7 @@ func TestGenerateHelmWithValues(t *testing.T) { } func TestHelmWithMissingValueFiles(t *testing.T) { - service := newService("../../util/helm/testdata/redis") + service := newService(t, "../../util/helm/testdata/redis") missingValuesFile := "values-prod-overrides.yaml" req := &apiclient.ManifestRequest{ @@ -893,7 +1009,7 @@ func TestHelmWithMissingValueFiles(t *testing.T) { } func TestGenerateHelmWithEnvVars(t *testing.T) { - service := newService("../../util/helm/testdata/redis") + service := newService(t, "../../util/helm/testdata/redis") res, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -930,7 +1046,7 @@ func TestGenerateHelmWithEnvVars(t *testing.T) { // The requested value file (`../minio/values.yaml`) is outside the app path (`./util/helm/testdata/redis`), however // since the requested value is still under the repo directory (`~/go/src/github.com/argoproj/argo-cd`), it is allowed func TestGenerateHelmWithValuesDirectoryTraversal(t *testing.T) { - service := newService("../../util/helm/testdata") + service := newService(t, "../../util/helm/testdata") _, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, AppName: "test", @@ -947,7 +1063,7 @@ func TestGenerateHelmWithValuesDirectoryTraversal(t *testing.T) { assert.NoError(t, err) // Test the case where the path is "." - service = newService("./testdata") + service = newService(t, "./testdata") _, err = service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, AppName: "test", @@ -961,7 +1077,7 @@ func TestGenerateHelmWithValuesDirectoryTraversal(t *testing.T) { } func TestChartRepoWithOutOfBoundsSymlink(t *testing.T) { - service := newService(".") + service := newService(t, ".") source := &argoappv1.ApplicationSource{Chart: "out-of-bounds-chart", TargetRevision: ">= 1.0.0"} request := &apiclient.ManifestRequest{Repo: &argoappv1.Repository{}, ApplicationSource: source, NoCache: true} _, err := service.GenerateManifest(context.Background(), request) @@ -971,7 +1087,7 @@ func TestChartRepoWithOutOfBoundsSymlink(t *testing.T) { // This is a Helm first-class app with a values file inside the repo directory // (`~/go/src/github.com/argoproj/argo-cd/reposerver/repository`), so it is allowed func TestHelmManifestFromChartRepoWithValueFile(t *testing.T) { - service := newService(".") + service := newService(t, ".") source := &argoappv1.ApplicationSource{ Chart: "my-chart", TargetRevision: ">= 1.0.0", @@ -1000,7 +1116,7 @@ func TestHelmManifestFromChartRepoWithValueFile(t *testing.T) { // This is a Helm first-class app with a values file outside the repo directory // (`~/go/src/github.com/argoproj/argo-cd/reposerver/repository`), so it is not allowed func TestHelmManifestFromChartRepoWithValueFileOutsideRepo(t *testing.T) { - service := newService(".") + service := newService(t, ".") source := &argoappv1.ApplicationSource{ Chart: "my-chart", TargetRevision: ">= 1.0.0", @@ -1015,7 +1131,7 @@ func TestHelmManifestFromChartRepoWithValueFileOutsideRepo(t *testing.T) { func TestHelmManifestFromChartRepoWithValueFileLinks(t *testing.T) { t.Run("Valid symlink", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") source := &argoappv1.ApplicationSource{ Chart: "my-chart", TargetRevision: ">= 1.0.0", @@ -1031,7 +1147,7 @@ func TestHelmManifestFromChartRepoWithValueFileLinks(t *testing.T) { } func TestGenerateHelmWithURL(t *testing.T) { - service := newService("../../util/helm/testdata/redis") + service := newService(t, "../../util/helm/testdata/redis") _, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -1054,7 +1170,7 @@ func TestGenerateHelmWithURL(t *testing.T) { // (`~/go/src/github.com/argoproj/argo-cd/util/helm/testdata/redis`), so it is blocked func TestGenerateHelmWithValuesDirectoryTraversalOutsideRepo(t *testing.T) { t.Run("Values file with relative path pointing outside repo root", func(t *testing.T) { - service := newService("../../util/helm/testdata/redis") + service := newService(t, "../../util/helm/testdata/redis") _, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, AppName: "test", @@ -1073,7 +1189,7 @@ func TestGenerateHelmWithValuesDirectoryTraversalOutsideRepo(t *testing.T) { }) t.Run("Values file with relative path pointing inside repo root", func(t *testing.T) { - service := newService("./testdata") + service := newService(t, "./testdata") _, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, AppName: "test", @@ -1091,7 +1207,7 @@ func TestGenerateHelmWithValuesDirectoryTraversalOutsideRepo(t *testing.T) { }) t.Run("Values file with absolute path stays within repo root", func(t *testing.T) { - service := newService("./testdata") + service := newService(t, "./testdata") _, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, AppName: "test", @@ -1109,7 +1225,7 @@ func TestGenerateHelmWithValuesDirectoryTraversalOutsideRepo(t *testing.T) { }) t.Run("Values file with absolute path using back-references outside repo root", func(t *testing.T) { - service := newService("./testdata") + service := newService(t, "./testdata") _, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, AppName: "test", @@ -1128,7 +1244,7 @@ func TestGenerateHelmWithValuesDirectoryTraversalOutsideRepo(t *testing.T) { }) t.Run("Remote values file from forbidden protocol", func(t *testing.T) { - service := newService("./testdata") + service := newService(t, "./testdata") _, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, AppName: "test", @@ -1147,7 +1263,7 @@ func TestGenerateHelmWithValuesDirectoryTraversalOutsideRepo(t *testing.T) { }) t.Run("Remote values file from custom allowed protocol", func(t *testing.T) { - service := newService("./testdata") + service := newService(t, "./testdata") _, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, AppName: "test", @@ -1168,7 +1284,7 @@ func TestGenerateHelmWithValuesDirectoryTraversalOutsideRepo(t *testing.T) { // File parameter should not allow traversal outside of the repository root func TestGenerateHelmWithAbsoluteFileParameter(t *testing.T) { - service := newService("../..") + service := newService(t, "../..") file, err := os.CreateTemp("", "external-secret.txt") assert.NoError(t, err) @@ -1209,7 +1325,7 @@ func TestGenerateHelmWithAbsoluteFileParameter(t *testing.T) { // directory (`~/go/src/github.com/argoproj/argo-cd`), it is allowed. It is used as a means of // providing direct content to a helm chart via a specific key. func TestGenerateHelmWithFileParameter(t *testing.T) { - service := newService("../../util/helm/testdata") + service := newService(t, "../../util/helm/testdata") res, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -1234,7 +1350,7 @@ func TestGenerateHelmWithFileParameter(t *testing.T) { } func TestGenerateNullList(t *testing.T) { - service := newService(".") + service := newService(t, ".") t.Run("null list", func(t *testing.T) { res1, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ @@ -1302,7 +1418,7 @@ func TestGenerateFromUTF16(t *testing.T) { } func TestListApps(t *testing.T) { - service := newService("./testdata") + service := newService(t, "./testdata") res, err := service.ListApps(context.Background(), &apiclient.ListAppsRequest{Repo: &argoappv1.Repository{}}) assert.NoError(t, err) @@ -1329,7 +1445,7 @@ func TestListApps(t *testing.T) { } func TestGetAppDetailsHelm(t *testing.T) { - service := newService("../../util/helm/testdata/dependency") + service := newService(t, "../../util/helm/testdata/dependency") res, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1344,8 +1460,26 @@ func TestGetAppDetailsHelm(t *testing.T) { assert.Equal(t, "Helm", res.Type) assert.EqualValues(t, []string{"values-production.yaml", "values.yaml"}, res.Helm.ValueFiles) } + +func TestGetAppDetailsHelmUsesCache(t *testing.T) { + service := newService(t, "../../util/helm/testdata/dependency") + + res, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ + Repo: &argoappv1.Repository{}, + Source: &argoappv1.ApplicationSource{ + Path: ".", + }, + }) + + assert.NoError(t, err) + assert.NotNil(t, res.Helm) + + assert.Equal(t, "Helm", res.Type) + assert.EqualValues(t, []string{"values-production.yaml", "values.yaml"}, res.Helm.ValueFiles) +} + func TestGetAppDetailsHelm_WithNoValuesFile(t *testing.T) { - service := newService("../../util/helm/testdata/api-versions") + service := newService(t, "../../util/helm/testdata/api-versions") res, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1363,7 +1497,7 @@ func TestGetAppDetailsHelm_WithNoValuesFile(t *testing.T) { } func TestGetAppDetailsKustomize(t *testing.T) { - service := newService("../../util/kustomize/testdata/kustomization_yaml") + service := newService(t, "../../util/kustomize/testdata/kustomization_yaml") res, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1380,7 +1514,7 @@ func TestGetAppDetailsKustomize(t *testing.T) { } func TestGetHelmCharts(t *testing.T) { - service := newService("../..") + service := newService(t, "../..") res, err := service.GetHelmCharts(context.Background(), &apiclient.HelmChartsRequest{Repo: &argoappv1.Repository{}}) // fix flakiness @@ -1401,7 +1535,7 @@ func TestGetHelmCharts(t *testing.T) { } func TestGetRevisionMetadata(t *testing.T) { - service, gitClient := newServiceWithMocks("../..", false) + service, gitClient, _ := newServiceWithMocks(t, "../..", false) now := time.Now() gitClient.On("RevisionMetadata", mock.Anything).Return(&git.RevisionMetadata{ @@ -1469,7 +1603,7 @@ func TestGetRevisionMetadata(t *testing.T) { func TestGetSignatureVerificationResult(t *testing.T) { // Commit with signature and verification requested { - service := newServiceWithSignature("../../manifests/base") + service := newServiceWithSignature(t, "../../manifests/base") src := argoappv1.ApplicationSource{Path: "."} q := apiclient.ManifestRequest{ @@ -1486,7 +1620,7 @@ func TestGetSignatureVerificationResult(t *testing.T) { } // Commit with signature and verification not requested { - service := newServiceWithSignature("../../manifests/base") + service := newServiceWithSignature(t, "../../manifests/base") src := argoappv1.ApplicationSource{Path: "."} q := apiclient.ManifestRequest{Repo: &argoappv1.Repository{}, ApplicationSource: &src, ProjectName: "something", @@ -1498,7 +1632,7 @@ func TestGetSignatureVerificationResult(t *testing.T) { } // Commit without signature and verification requested { - service := newService("../../manifests/base") + service := newService(t, "../../manifests/base") src := argoappv1.ApplicationSource{Path: "."} q := apiclient.ManifestRequest{Repo: &argoappv1.Repository{}, ApplicationSource: &src, VerifySignature: true, ProjectName: "something", @@ -1510,7 +1644,7 @@ func TestGetSignatureVerificationResult(t *testing.T) { } // Commit without signature and verification not requested { - service := newService("../../manifests/base") + service := newService(t, "../../manifests/base") src := argoappv1.ApplicationSource{Path: "."} q := apiclient.ManifestRequest{Repo: &argoappv1.Repository{}, ApplicationSource: &src, VerifySignature: true, ProjectName: "something", @@ -1543,7 +1677,7 @@ func Test_newEnv(t *testing.T) { } func TestService_newHelmClientResolveRevision(t *testing.T) { - service := newService(".") + service := newService(t, ".") t.Run("EmptyRevision", func(t *testing.T) { _, _, err := service.newHelmClientResolveRevision(&argoappv1.Repository{}, "", "", true) @@ -1557,7 +1691,7 @@ func TestService_newHelmClientResolveRevision(t *testing.T) { func TestGetAppDetailsWithAppParameterFile(t *testing.T) { t.Run("No app name set and app specific file exists", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "multi", func(t *testing.T, path string) { details, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1570,7 +1704,7 @@ func TestGetAppDetailsWithAppParameterFile(t *testing.T) { }) }) t.Run("No app specific override", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "single-global", func(t *testing.T, path string) { details, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1584,7 +1718,7 @@ func TestGetAppDetailsWithAppParameterFile(t *testing.T) { }) }) t.Run("Only app specific override", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "single-app-only", func(t *testing.T, path string) { details, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1598,7 +1732,7 @@ func TestGetAppDetailsWithAppParameterFile(t *testing.T) { }) }) t.Run("App specific override", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "multi", func(t *testing.T, path string) { details, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1612,7 +1746,7 @@ func TestGetAppDetailsWithAppParameterFile(t *testing.T) { }) }) t.Run("App specific overrides containing non-mergeable field", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "multi", func(t *testing.T, path string) { details, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1626,7 +1760,7 @@ func TestGetAppDetailsWithAppParameterFile(t *testing.T) { }) }) t.Run("Broken app-specific overrides", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "multi", func(t *testing.T, path string) { _, err := service.GetAppDetails(context.Background(), &apiclient.RepoServerAppDetailsQuery{ Repo: &argoappv1.Repository{}, @@ -1668,7 +1802,7 @@ func runWithTempTestdata(t *testing.T, path string, runner func(t *testing.T, pa func TestGenerateManifestsWithAppParameterFile(t *testing.T) { t.Run("Single global override", func(t *testing.T) { runWithTempTestdata(t, "single-global", func(t *testing.T, path string) { - service := newService(".") + service := newService(t, ".") manifests, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, ApplicationSource: &argoappv1.ApplicationSource{ @@ -1699,7 +1833,7 @@ func TestGenerateManifestsWithAppParameterFile(t *testing.T) { t.Run("Single global override Helm", func(t *testing.T) { runWithTempTestdata(t, "single-global-helm", func(t *testing.T, path string) { - service := newService(".") + service := newService(t, ".") manifests, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, ApplicationSource: &argoappv1.ApplicationSource{ @@ -1729,7 +1863,7 @@ func TestGenerateManifestsWithAppParameterFile(t *testing.T) { }) t.Run("Application specific override", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "single-app-only", func(t *testing.T, path string) { manifests, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -1760,8 +1894,29 @@ func TestGenerateManifestsWithAppParameterFile(t *testing.T) { }) }) + t.Run("Multi-source with source as ref only does not generate manifests", func(t *testing.T) { + service := newService(t, ".") + runWithTempTestdata(t, "single-app-only", func(t *testing.T, path string) { + manifests, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ + Repo: &argoappv1.Repository{}, + ApplicationSource: &argoappv1.ApplicationSource{ + Path: "", + Chart: "", + Ref: "test", + }, + AppName: "testapp-multi-ref-only", + ProjectName: "something", + ProjectSourceRepos: []string{"*"}, + HasMultipleSources: true, + }) + assert.NoError(t, err) + assert.Empty(t, manifests.Manifests) + assert.NotEmpty(t, manifests.Revision) + }) + }) + t.Run("Application specific override for other app", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "single-app-only", func(t *testing.T, path string) { manifests, err := service.GenerateManifest(context.Background(), &apiclient.ManifestRequest{ Repo: &argoappv1.Repository{}, @@ -1793,7 +1948,7 @@ func TestGenerateManifestsWithAppParameterFile(t *testing.T) { }) t.Run("Override info does not appear in cache key", func(t *testing.T) { - service := newService(".") + service := newService(t, ".") runWithTempTestdata(t, "single-global", func(t *testing.T, path string) { source := &argoappv1.ApplicationSource{ Path: path, @@ -1843,7 +1998,7 @@ func TestGenerateManifestWithAnnotatedAndRegularGitTagHashes(t *testing.T) { ProjectSourceRepos: []string{"*"}, }, wantError: false, - service: newServiceWithCommitSHA(".", regularGitTagHash), + service: newServiceWithCommitSHA(t, ".", regularGitTagHash), }, { @@ -1859,7 +2014,7 @@ func TestGenerateManifestWithAnnotatedAndRegularGitTagHashes(t *testing.T) { ProjectSourceRepos: []string{"*"}, }, wantError: false, - service: newServiceWithCommitSHA(".", annotatedGitTaghash), + service: newServiceWithCommitSHA(t, ".", annotatedGitTaghash), }, { @@ -1875,7 +2030,7 @@ func TestGenerateManifestWithAnnotatedAndRegularGitTagHashes(t *testing.T) { ProjectSourceRepos: []string{"*"}, }, wantError: true, - service: newServiceWithCommitSHA(".", invalidGitTaghash), + service: newServiceWithCommitSHA(t, ".", invalidGitTaghash), }, } for _, tt := range tests { @@ -1900,7 +2055,7 @@ func TestGenerateManifestWithAnnotatedAndRegularGitTagHashes(t *testing.T) { func TestGenerateManifestWithAnnotatedTagsAndMultiSourceApp(t *testing.T) { annotatedGitTaghash := "95249be61b028d566c29d47b19e65c5603388a41" - service := newServiceWithCommitSHA(".", annotatedGitTaghash) + service := newServiceWithCommitSHA(t, ".", annotatedGitTaghash) refSources := map[string]*argoappv1.RefTarget{} @@ -2485,7 +2640,7 @@ func Test_findManifests(t *testing.T) { } func TestTestRepoOCI(t *testing.T) { - service := newService(".") + service := newService(t, ".") _, err := service.TestRepository(context.Background(), &apiclient.TestRepositoryRequest{ Repo: &argoappv1.Repository{ Repo: "https://demo.goharbor.io", @@ -2510,7 +2665,7 @@ func Test_getHelmDependencyRepos(t *testing.T) { func TestResolveRevision(t *testing.T) { - service := newService(".") + service := newService(t, ".") repo := &argoappv1.Repository{Repo: "https://github.com/argoproj/argo-cd"} app := &argoappv1.Application{Spec: argoappv1.ApplicationSpec{Source: &argoappv1.ApplicationSource{}}} resolveRevisionResponse, err := service.ResolveRevision(context.Background(), &apiclient.ResolveRevisionRequest{ @@ -2532,7 +2687,7 @@ func TestResolveRevision(t *testing.T) { func TestResolveRevisionNegativeScenarios(t *testing.T) { - service := newService(".") + service := newService(t, ".") repo := &argoappv1.Repository{Repo: "https://github.com/argoproj/argo-cd"} app := &argoappv1.Application{Spec: argoappv1.ApplicationSpec{Source: &argoappv1.ApplicationSource{}}} resolveRevisionResponse, err := service.ResolveRevision(context.Background(), &apiclient.ResolveRevisionRequest{ @@ -2579,19 +2734,57 @@ func TestDirectoryPermissionInitializer(t *testing.T) { require.Error(t, err) } -func initGitRepo(repoPath string, remote string) error { - if err := os.Mkdir(repoPath, 0755); err != nil { - return err +func addHelmToGitRepo(t *testing.T, options newGitRepoOptions) { + err := os.WriteFile(filepath.Join(options.path, "Chart.yaml"), []byte("name: test\nversion: v1.0.0"), 0777) + assert.NoError(t, err) + for valuesFileName, values := range options.helmChartOptions.valuesFiles { + valuesFileContents, err := yaml.Marshal(values) + assert.NoError(t, err) + err = os.WriteFile(filepath.Join(options.path, valuesFileName), valuesFileContents, 0777) + assert.NoError(t, err) + } + assert.NoError(t, err) + cmd := exec.Command("git", "add", "-A") + cmd.Dir = options.path + assert.NoError(t, cmd.Run()) + cmd = exec.Command("git", "commit", "-m", "Initial commit") + cmd.Dir = options.path + assert.NoError(t, cmd.Run()) +} + +func initGitRepo(t *testing.T, options newGitRepoOptions) (revision string) { + if options.createPath { + assert.NoError(t, os.Mkdir(options.path, 0755)) + } + + cmd := exec.Command("git", "init", options.path) + cmd.Dir = options.path + assert.NoError(t, cmd.Run()) + + if options.remote != "" { + cmd = exec.Command("git", "remote", "add", "origin", options.path) + cmd.Dir = options.path + assert.NoError(t, cmd.Run()) } - cmd := exec.Command("git", "init", repoPath) - cmd.Dir = repoPath - if err := cmd.Run(); err != nil { - return err + commitAdded := options.addEmptyCommit || options.helmChartOptions.chartName != "" + if options.addEmptyCommit { + cmd = exec.Command("git", "commit", "-m", "Initial commit", "--allow-empty") + cmd.Dir = options.path + assert.NoError(t, cmd.Run()) + } else if options.helmChartOptions.chartName != "" { + addHelmToGitRepo(t, options) } - cmd = exec.Command("git", "remote", "add", "origin", remote) - cmd.Dir = repoPath - return cmd.Run() + + if commitAdded { + var revB bytes.Buffer + cmd = exec.Command("git", "rev-parse", "HEAD", options.path) + cmd.Dir = options.path + cmd.Stdout = &revB + assert.NoError(t, cmd.Run()) + revision = strings.Split(revB.String(), "\n")[0] + } + return revision } func TestInit(t *testing.T) { @@ -2604,16 +2797,16 @@ func TestInit(t *testing.T) { }) repoPath := path.Join(dir, "repo1") - require.NoError(t, initGitRepo(repoPath, "https://github.com/argo-cd/test-repo1")) + initGitRepo(t, newGitRepoOptions{path: repoPath, remote: "https://github.com/argo-cd/test-repo1", createPath: true, addEmptyCommit: false}) - service := newService(".") + service := newService(t, ".") service.rootDir = dir require.NoError(t, service.Init()) _, err := os.ReadDir(dir) require.Error(t, err) - require.NoError(t, initGitRepo(path.Join(dir, "repo2"), "https://github.com/argo-cd/test-repo2")) + initGitRepo(t, newGitRepoOptions{path: path.Join(dir, "repo2"), remote: "https://github.com/argo-cd/test-repo2", createPath: true, addEmptyCommit: false}) } // TestCheckoutRevisionCanGetNonstandardRefs shows that we can fetch a revision that points to a non-standard ref. In @@ -2926,7 +3119,7 @@ func TestErrorGetGitDirectories(t *testing.T) { want *apiclient.GitDirectoriesResponse wantErr assert.ErrorAssertionFunc }{ - {name: "InvalidRepo", fields: fields{service: newService(".")}, args: args{ + {name: "InvalidRepo", fields: fields{service: newService(t, ".")}, args: args{ ctx: context.TODO(), request: &apiclient.GitDirectoriesRequest{ Repo: nil, @@ -2935,7 +3128,7 @@ func TestErrorGetGitDirectories(t *testing.T) { }, }, want: nil, wantErr: assert.Error}, {name: "InvalidResolveRevision", fields: fields{service: func() *Service { - s, _ := newServiceWithOpt(func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { + s, _, _ := newServiceWithOpt(t, func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { gitClient.On("Checkout", mock.Anything, mock.Anything).Return(nil) gitClient.On("LsRemote", mock.Anything).Return("", fmt.Errorf("ah error")) paths.On("GetPath", mock.Anything).Return(".", nil) @@ -2966,7 +3159,7 @@ func TestErrorGetGitDirectories(t *testing.T) { func TestGetGitDirectories(t *testing.T) { // test not using the cache root := "./testdata/git-files-dirs" - s, _ := newServiceWithOpt(func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { + s, _, cacheMocks := newServiceWithOpt(t, func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { gitClient.On("Init").Return(nil) gitClient.On("Fetch", mock.Anything).Return(nil) gitClient.On("Checkout", mock.Anything, mock.Anything).Once().Return(nil) @@ -2989,6 +3182,10 @@ func TestGetGitDirectories(t *testing.T) { directories, err = s.GetGitDirectories(context.TODO(), dirRequest) assert.Nil(t, err) assert.ElementsMatch(t, []string{"app", "app/bar", "app/foo/bar", "somedir", "app/foo"}, directories.GetPaths()) + cacheMocks.mockCache.AssertCacheCalledTimes(t, &repositorymocks.CacheCallCounts{ + ExternalSets: 1, + ExternalGets: 2, + }) } func TestErrorGetGitFiles(t *testing.T) { @@ -3006,7 +3203,7 @@ func TestErrorGetGitFiles(t *testing.T) { want *apiclient.GitFilesResponse wantErr assert.ErrorAssertionFunc }{ - {name: "InvalidRepo", fields: fields{service: newService(".")}, args: args{ + {name: "InvalidRepo", fields: fields{service: newService(t, ".")}, args: args{ ctx: context.TODO(), request: &apiclient.GitFilesRequest{ Repo: nil, @@ -3015,7 +3212,7 @@ func TestErrorGetGitFiles(t *testing.T) { }, }, want: nil, wantErr: assert.Error}, {name: "InvalidResolveRevision", fields: fields{service: func() *Service { - s, _ := newServiceWithOpt(func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { + s, _, _ := newServiceWithOpt(t, func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { gitClient.On("Checkout", mock.Anything, mock.Anything).Return(nil) gitClient.On("LsRemote", mock.Anything).Return("", fmt.Errorf("ah error")) paths.On("GetPath", mock.Anything).Return(".", nil) @@ -3048,7 +3245,7 @@ func TestGetGitFiles(t *testing.T) { files := []string{"./testdata/git-files-dirs/somedir/config.yaml", "./testdata/git-files-dirs/config.yaml", "./testdata/git-files-dirs/config.yaml", "./testdata/git-files-dirs/app/foo/bar/config.yaml"} root := "" - s, _ := newServiceWithOpt(func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { + s, _, cacheMocks := newServiceWithOpt(t, func(gitClient *gitmocks.Client, helmClient *helmmocks.Client, paths *iomocks.TempPaths) { gitClient.On("Init").Return(nil) gitClient.On("Fetch", mock.Anything).Return(nil) gitClient.On("Checkout", mock.Anything, mock.Anything).Once().Return(nil) @@ -3081,6 +3278,10 @@ func TestGetGitFiles(t *testing.T) { fileResponse, err = s.GetGitFiles(context.TODO(), filesRequest) assert.Nil(t, err) assert.Equal(t, expected, fileResponse.GetMap()) + cacheMocks.mockCache.AssertCacheCalledTimes(t, &repositorymocks.CacheCallCounts{ + ExternalSets: 1, + ExternalGets: 2, + }) } func Test_getRepoSanitizerRegex(t *testing.T) { @@ -3090,3 +3291,45 @@ func Test_getRepoSanitizerRegex(t *testing.T) { msg = r.ReplaceAllString("error message containing /tmp/_argocd-repo/SENSITIVE/with/trailing/path and other stuff", "") assert.Equal(t, "error message containing /with/trailing/path and other stuff", msg) } + +func TestGetRevisionChartDetails(t *testing.T) { + t.Run("Test revision semvar", func(t *testing.T) { + root := t.TempDir() + service := newService(t, root) + _, err := service.GetRevisionChartDetails(context.Background(), &apiclient.RepoServerRevisionChartDetailsRequest{ + Repo: &v1alpha1.Repository{ + Repo: fmt.Sprintf("file://%s", root), + Name: "test-repo-name", + Type: "helm", + }, + Name: "test-name", + Revision: "test-revision", + }) + assert.ErrorContains(t, err, "invalid revision") + }) + + t.Run("Test GetRevisionChartDetails", func(t *testing.T) { + root := t.TempDir() + service := newService(t, root) + repoUrl := fmt.Sprintf("file://%s", root) + err := service.cache.SetRevisionChartDetails(repoUrl, "my-chart", "1.1.0", &argoappv1.ChartDetails{ + Description: "test-description", + Home: "test-home", + Maintainers: []string{"test-maintainer"}, + }) + assert.NoError(t, err) + chartDetails, err := service.GetRevisionChartDetails(context.Background(), &apiclient.RepoServerRevisionChartDetailsRequest{ + Repo: &v1alpha1.Repository{ + Repo: fmt.Sprintf("file://%s", root), + Name: "test-repo-name", + Type: "helm", + }, + Name: "my-chart", + Revision: "1.1.0", + }) + assert.NoError(t, err) + assert.Equal(t, "test-description", chartDetails.Description) + assert.Equal(t, "test-home", chartDetails.Home) + assert.Equal(t, []string{"test-maintainer"}, chartDetails.Maintainers) + }) +} diff --git a/util/cache/mocks/cacheclient.go b/util/cache/mocks/cacheclient.go new file mode 100644 index 0000000000000..e653847ec49a8 --- /dev/null +++ b/util/cache/mocks/cacheclient.go @@ -0,0 +1,65 @@ +package mocks + +import ( + "context" + "time" + + cache "github.com/argoproj/argo-cd/v2/util/cache" + "github.com/stretchr/testify/mock" +) + +type MockCacheClient struct { + mock.Mock + BaseCache cache.CacheClient + ReadDelay time.Duration + WriteDelay time.Duration +} + +func (c *MockCacheClient) Set(item *cache.Item) error { + args := c.Called(item) + if len(args) > 0 && args.Get(0) != nil { + return args.Get(0).(error) + } + if c.WriteDelay > 0 { + time.Sleep(c.WriteDelay) + } + return c.BaseCache.Set(item) +} + +func (c *MockCacheClient) Get(key string, obj interface{}) error { + args := c.Called(key, obj) + if len(args) > 0 && args.Get(0) != nil { + return args.Get(0).(error) + } + if c.ReadDelay > 0 { + time.Sleep(c.ReadDelay) + } + return c.BaseCache.Get(key, obj) +} + +func (c *MockCacheClient) Delete(key string) error { + args := c.Called(key) + if len(args) > 0 && args.Get(0) != nil { + return args.Get(0).(error) + } + if c.WriteDelay > 0 { + time.Sleep(c.WriteDelay) + } + return c.BaseCache.Delete(key) +} + +func (c *MockCacheClient) OnUpdated(ctx context.Context, key string, callback func() error) error { + args := c.Called(ctx, key, callback) + if len(args) > 0 && args.Get(0) != nil { + return args.Get(0).(error) + } + return c.BaseCache.OnUpdated(ctx, key, callback) +} + +func (c *MockCacheClient) NotifyUpdated(key string) error { + args := c.Called(key) + if len(args) > 0 && args.Get(0) != nil { + return args.Get(0).(error) + } + return c.BaseCache.NotifyUpdated(key) +}