refactor reconciler to use fluxcd/pkg/git

Signed-off-by: Sanskar Jaiswal <jaiswalsanskar078@gmail.com>
This commit is contained in:
Sanskar Jaiswal 2022-09-02 16:19:04 +05:30 committed by Paulo Gomes
parent a7dba5331b
commit 0296f7acf0
No known key found for this signature in database
GPG Key ID: 9995233870E99BEE
8 changed files with 99 additions and 805 deletions

View File

@ -1,76 +0,0 @@
/*
Copyright 2020, 2021 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 controllers
import (
"errors"
"testing"
)
func TestLibgit2ErrorTidy(t *testing.T) {
// this is what GitLab sends if the deploy key doesn't have write access
gitlabMessage := `remote:
remote: ========================================================================
remote:
remote: This deploy key does not have write access to this project.
remote:
remote: ========================================================================
remote:
`
expectedReformat := "remote: This deploy key does not have write access to this project."
err := errors.New(gitlabMessage)
err = libgit2PushError(err)
reformattedMessage := err.Error()
if reformattedMessage != expectedReformat {
t.Errorf("expected %q, got %q", expectedReformat, reformattedMessage)
}
}
func TestLibgit2Multiline(t *testing.T) {
// this is a hypothetical error message, in which the useful
// content spans more than one line
multilineMessage := `remote:
remote: ========================================================================
remote:
remote: This deploy key does not have write access to this project.
remote: You will need to create a new deploy key.
remote:
remote: ========================================================================
remote:
`
expectedReformat := "remote: This deploy key does not have write access to this project. You will need to create a new deploy key."
err := errors.New(multilineMessage)
err = libgit2PushError(err)
reformattedMessage := err.Error()
if reformattedMessage != expectedReformat {
t.Errorf("expected %q, got %q", expectedReformat, reformattedMessage)
}
}
func TestLibgit2ErrorUnchanged(t *testing.T) {
// this is (roughly) what GitHub sends if the deploy key doesn't have write access
regularMessage := `remote: ERROR: deploy key does not have permissions`
expectedReformat := regularMessage
err := errors.New(regularMessage)
err = libgit2PushError(err)
reformattedMessage := err.Error()
if reformattedMessage != expectedReformat {
t.Errorf("expected %q, got %q", expectedReformat, reformattedMessage)
}
}

View File

@ -1,318 +0,0 @@
package controllers
import (
"context"
"fmt"
"os"
"path/filepath"
"testing"
"time"
"github.com/go-logr/logr"
libgit2 "github.com/libgit2/git2go/v33"
. "github.com/onsi/gomega"
"k8s.io/apimachinery/pkg/types"
"github.com/fluxcd/pkg/gittestserver"
"github.com/fluxcd/source-controller/pkg/git"
)
func populateRepoFromFixture(repo *libgit2.Repository, fixture string) error {
absFixture, err := filepath.Abs(fixture)
if err != nil {
return err
}
if err := filepath.Walk(absFixture, func(path string, info os.FileInfo, err error) error {
if err != nil {
return err
}
if info.IsDir() {
return os.MkdirAll(filepath.Join(path[len(fixture):]), info.Mode())
}
// copy symlinks as-is, so I can test what happens with broken symlinks
if info.Mode()&os.ModeSymlink > 0 {
target, err := os.Readlink(path)
if err != nil {
return err
}
return os.Symlink(target, path[len(fixture):])
}
fileBytes, err := os.ReadFile(path)
if err != nil {
return err
}
ff, err := os.Create(path[len(fixture):])
if err != nil {
return err
}
defer ff.Close()
_, err = ff.Write(fileBytes)
return err
}); err != nil {
return err
}
sig := &libgit2.Signature{
Name: "Testbot",
Email: "test@example.com",
When: time.Now(),
}
if _, err := commitWorkDir(repo, "main", "Initial revision from "+fixture, sig); err != nil {
return err
}
return nil
}
func TestRepoForFixture(t *testing.T) {
tmp, err := os.MkdirTemp("", "flux-test")
if err != nil {
t.Fatal(err)
}
defer os.RemoveAll(tmp)
repo, err := initGitRepoPlain("testdata/pathconfig", tmp)
if err != nil {
t.Error(err)
}
repo.Free()
}
func TestIgnoreBrokenSymlink(t *testing.T) {
// init a git repo in the filesystem so we can operate on files there
tmp, err := os.MkdirTemp("", "flux-test")
if err != nil {
t.Fatal(err)
}
defer os.RemoveAll(tmp)
repo, err := initGitRepoPlain("testdata/brokenlink", tmp)
if err != nil {
t.Fatal(err)
}
_, err = commitChangedManifests(logr.Discard(), repo, tmp, nil, nil, "unused")
if err != errNoChanges {
t.Fatalf("expected no changes but got: %v", err)
}
}
// this is a hook script that will reject a ref update for a branch
// that's not `main`
const rejectBranch = `
if [ "$1" != "refs/heads/main" ]; then
echo "*** Rejecting push to non-main branch $1" >&2
exit 1
fi
`
func TestPushRejected(t *testing.T) {
// Check that pushing to a repository which rejects a ref update
// results in an error. Why would a repo reject an update? If yu
// use e.g., branch protection in GitHub, this is what happens --
// see
// https://github.com/fluxcd/image-automation-controller/issues/194.
branch := "push-branch"
gitServer, err := gittestserver.NewTempGitServer()
if err != nil {
t.Fatal(err)
}
gitServer.AutoCreate()
gitServer.InstallUpdateHook(rejectBranch)
if err = gitServer.StartHTTP(); err != nil {
t.Fatal(err)
}
// We use "test" as the branch to init repo, to avoid potential conflicts
// with the default branch(main/master) of the system this test is running
// on. If, for e.g., we used main as the branch and the default branch is
// supposed to be main, this will fail as this would try to create a branch
// named main explicitly.
if err = initGitRepo(gitServer, "testdata/appconfig", "test", "/appconfig.git"); err != nil {
t.Fatal(err)
}
repoURL := gitServer.HTTPAddressWithCredentials() + "/appconfig.git"
cloneCtx, cancel := context.WithTimeout(ctx, time.Second*10)
defer cancel()
repo, err := clone(cloneCtx, repoURL, "test", nil)
if err != nil {
t.Fatal(err)
}
defer repo.Free()
cleanup, err := configureTransportOptsForRepo(repo, nil)
if err != nil {
t.Fatal(err)
}
defer cleanup()
// This is here to guard against push in general being broken
err = push(context.TODO(), repo.Workdir(), "test", repoAccess{})
if err != nil {
t.Fatal(err)
}
// This is not under test, but needed for the next bit
if err = repo.SetHead(fmt.Sprintf("refs/heads/%s", branch)); err != nil {
t.Fatal(err)
}
// This is supposed to fail, because the hook rejects the branch
// pushed to.
err = push(context.TODO(), repo.Workdir(), branch, repoAccess{})
if err == nil {
t.Error("push to a forbidden branch is expected to fail, but succeeded")
}
}
func TestEarlyEOF(t *testing.T) {
g := NewWithT(t)
gitServer, err := gittestserver.NewTempGitServer()
g.Expect(err).ToNot(HaveOccurred())
defer os.RemoveAll(gitServer.Root())
username := "norris"
password := "chuck"
gitServer.
AutoCreate().
KeyDir(filepath.Join(t.TempDir(), "keys")).
Auth(username, password).
ReadOnly(true)
err = gitServer.StartHTTP()
g.Expect(err).ToNot(HaveOccurred())
err = initGitRepo(gitServer, "testdata/appconfig", "test", "/appconfig.git")
g.Expect(err).ToNot(HaveOccurred())
repoURL := gitServer.HTTPAddressWithCredentials() + "/appconfig.git"
cloneCtx, cancel := context.WithTimeout(ctx, time.Second*10)
defer cancel()
access := repoAccess{
auth: &git.AuthOptions{
Username: username,
Password: password,
},
}
repo, err := clone(cloneCtx, repoURL, "test", access.auth)
g.Expect(err).ToNot(HaveOccurred())
defer repo.Free()
cleanup, err := configureTransportOptsForRepo(repo, access.auth)
g.Expect(err).ToNot(HaveOccurred())
defer cleanup()
err = push(context.TODO(), repo.Workdir(), "test", access)
g.Expect(err).To(HaveOccurred())
g.Expect(err.Error()).To(ContainSubstring("early EOF (the SSH key may not have write access to the repository)"))
}
func Test_switchToBranch(t *testing.T) {
g := NewWithT(t)
gitServer, err := gittestserver.NewTempGitServer()
g.Expect(err).ToNot(HaveOccurred())
gitServer.AutoCreate()
g.Expect(gitServer.StartHTTP()).To(Succeed())
// We use "test" as the branch to init repo, to avoid potential conflicts
// with the default branch(main/master) of the system this test is running
// on. If, for e.g., we used main as the branch and the default branch is
// supposed to be main, this will fail as this would try to create a branch
// named main explicitly.
branch := "test"
g.Expect(initGitRepo(gitServer, "testdata/appconfig", branch, "/appconfig.git")).To(Succeed())
repoURL := gitServer.HTTPAddressWithCredentials() + "/appconfig.git"
cloneCtx, cancel := context.WithTimeout(ctx, time.Second*10)
defer cancel()
repo, err := clone(cloneCtx, repoURL, branch, nil)
g.Expect(err).ToNot(HaveOccurred())
defer repo.Free()
head, err := repo.Head()
g.Expect(err).ToNot(HaveOccurred())
defer head.Free()
target := head.Target()
// register transport options and update remote to transport url
cleanup, err := configureTransportOptsForRepo(repo, nil)
if err != nil {
t.Fatal(err)
}
defer cleanup()
// calling switchToBranch with a branch that doesn't exist on origin
// should result in the branch being created and switched to.
branch = "not-on-origin"
switchToBranch(repo, context.TODO(), branch, repoAccess{})
head, err = repo.Head()
g.Expect(err).ToNot(HaveOccurred())
name, err := head.Branch().Name()
g.Expect(err).ToNot(HaveOccurred())
g.Expect(name).To(Equal(branch))
cc, err := repo.LookupCommit(head.Target())
g.Expect(err).ToNot(HaveOccurred())
defer cc.Free()
g.Expect(cc.Id().String()).To(Equal(target.String()))
// create a branch with the HEAD commit and push it to origin
branch = "exists-on-origin"
_, err = repo.CreateBranch(branch, cc, false)
g.Expect(err).ToNot(HaveOccurred())
origin, err := repo.Remotes.Lookup(originRemote)
g.Expect(err).ToNot(HaveOccurred())
defer origin.Free()
g.Expect(origin.Push(
[]string{fmt.Sprintf("refs/heads/%s:refs/heads/%s", branch, branch)}, &libgit2.PushOptions{},
)).To(Succeed())
// push a new commit to the branch. this is done to test whether we properly
// sync our local branch with the remote branch, before switching.
policyKey := types.NamespacedName{
Name: "policy",
Namespace: "ns",
}
commitID := commitInRepo(g, repoURL, branch, "Install setter marker", func(tmp string) {
g.Expect(replaceMarker(tmp, policyKey)).To(Succeed())
})
// calling switchToBranch with a branch that exists should make sure to fetch latest
// for that branch from origin, and then switch to it.
switchToBranch(repo, context.TODO(), branch, repoAccess{})
head, err = repo.Head()
g.Expect(err).ToNot(HaveOccurred())
name, err = head.Branch().Name()
g.Expect(err).ToNot(HaveOccurred())
g.Expect(name).To(Equal(branch))
g.Expect(head.Target().String()).To(Equal(commitID.String()))
// push a commit after switching to the branch, to check if the local
// branch is synced with origin.
replaceMarker(repo.Workdir(), policyKey)
sig := &libgit2.Signature{
Name: "Testbot",
Email: "test@example.com",
When: time.Now(),
}
_, err = commitWorkDir(repo, branch, "update policy", sig)
g.Expect(err).ToNot(HaveOccurred())
g.Expect(push(context.TODO(), repo.Workdir(), branch, repoAccess{})).To(Succeed())
}

View File

@ -19,21 +19,18 @@ package controllers
import (
"bytes"
"context"
"errors"
"fmt"
"math"
"net/url"
"os"
"path/filepath"
"strings"
"text/template"
"time"
"github.com/Masterminds/sprig/v3"
"github.com/ProtonMail/go-crypto/openpgp"
"github.com/ProtonMail/go-crypto/openpgp/packet"
securejoin "github.com/cyphar/filepath-securejoin"
"github.com/go-logr/logr"
libgit2 "github.com/libgit2/git2go/v33"
corev1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"
@ -52,15 +49,14 @@ import (
imagev1_reflect "github.com/fluxcd/image-reflector-controller/api/v1beta1"
apiacl "github.com/fluxcd/pkg/apis/acl"
"github.com/fluxcd/pkg/apis/meta"
"github.com/fluxcd/pkg/git"
libgit2pkg "github.com/fluxcd/pkg/git/libgit2"
"github.com/fluxcd/pkg/runtime/acl"
helper "github.com/fluxcd/pkg/runtime/controller"
"github.com/fluxcd/pkg/runtime/events"
"github.com/fluxcd/pkg/runtime/logger"
"github.com/fluxcd/pkg/runtime/predicates"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
"github.com/fluxcd/source-controller/pkg/git"
"github.com/fluxcd/source-controller/pkg/git/libgit2/managed"
gitstrat "github.com/fluxcd/source-controller/pkg/git/strategy"
imagev1 "github.com/fluxcd/image-automation-controller/api/v1beta1"
"github.com/fluxcd/image-automation-controller/pkg/update"
@ -223,7 +219,7 @@ func (r *ImageUpdateAutomationReconciler) Reconcile(ctx context.Context, req ctr
} else if r := origin.Spec.Reference; r != nil {
ref = r
tracelog.Info("using git repository ref from GitRepository spec", "ref", ref)
} // else remain as `nil`, which is an acceptable value for cloneInto, later.
} // else remain as `nil` and git.DefaultBranch will be used.
var pushBranch string
if gitSpec.Push != nil {
@ -254,54 +250,39 @@ func (r *ImageUpdateAutomationReconciler) Reconcile(ctx context.Context, req ctr
debuglog.Info("attempting to clone git repository", "gitrepository", originName, "ref", ref, "working", tmp)
access, err := r.getRepoAccess(ctx, &origin)
authOpts, err := r.getAuthOpts(ctx, &origin)
if err != nil {
return failWithError(err)
}
// We set the TransportOptionsURL of this set of authentication options here by constructing
// a unique URL that won't clash in a multi tenant environment. This unique URL is used by
// libgit2 managed transports. This enables us to bypass the inbuilt credentials callback in
// libgit2, which is inflexible and unstable.
// NB: The Transport Options URL must be unique, therefore it must use the object under
// reconciliation details, instead of the repository it depends on.
if strings.HasPrefix(origin.Spec.URL, "http") {
access.auth.TransportOptionsURL = fmt.Sprintf("http://%s/%s/%d", auto.Name, auto.UID, auto.Generation)
} else if strings.HasPrefix(origin.Spec.URL, "ssh") {
access.auth.TransportOptionsURL = fmt.Sprintf("ssh://%s/%s/%d", auto.Name, auto.UID, auto.Generation)
} else {
return failWithError(fmt.Errorf("git repository URL '%s' has invalid transport type, supported types are: http, https, ssh", origin.Spec.URL))
lgc, err := libgit2pkg.NewClient(tmp, authOpts)
if err != nil {
return failWithError(err)
}
defer lgc.Close()
opts := git.CloneOptions{}
if ref != nil {
opts.Tag = ref.Tag
opts.SemVer = ref.SemVer
opts.Commit = ref.Commit
opts.Branch = ref.Branch
}
// Use the git operations timeout for the repo.
cloneCtx, cancel := context.WithTimeout(ctx, origin.Spec.Timeout.Duration)
defer cancel()
var repo *libgit2.Repository
if repo, err = cloneInto(cloneCtx, access, ref, tmp); err != nil {
if _, err := lgc.Clone(cloneCtx, origin.Spec.URL, opts); err != nil {
return failWithError(err)
}
defer repo.Free()
// Checkout removes TransportOptions before returning, therefore this
// must happen after cloneInto.
// TODO(pjbgf): Git consolidation should improve the API workflow.
managed.AddTransportOptions(access.auth.TransportOptionsURL, managed.TransportOptions{
TargetURL: origin.Spec.URL,
AuthOpts: access.auth,
ProxyOptions: &libgit2.ProxyOptions{Type: libgit2.ProxyTypeAuto},
Context: cloneCtx,
})
defer managed.RemoveTransportOptions(access.auth.TransportOptionsURL)
// When there's a push spec, the pushed-to branch is where commits
// shall be made
if gitSpec.Push != nil && !(ref != nil && ref.Branch == pushBranch) {
// Use the git operations timeout for the repo.
fetchCtx, cancel := context.WithTimeout(ctx, origin.Spec.Timeout.Duration)
defer cancel()
if err := switchToBranch(repo, fetchCtx, pushBranch, access); err != nil && err != errRemoteBranchMissing {
if err := lgc.SwitchBranch(fetchCtx, pushBranch); err != nil {
return failWithError(err)
}
}
@ -366,14 +347,18 @@ func (r *ImageUpdateAutomationReconciler) Reconcile(ctx context.Context, req ctr
// The status message depends on what happens next. Since there's
// more than one way to succeed, there's some if..else below, and
// early returns only on failure.
signature := &libgit2.Signature{
Name: gitSpec.Commit.Author.Name,
Email: gitSpec.Commit.Author.Email,
When: time.Now(),
}
if rev, err := commitChangedManifests(tracelog, repo, tmp, signingEntity, signature, message); err != nil {
if err != errNoChanges {
if rev, err := lgc.Commit(
git.Commit{
Author: git.Signature{
Name: gitSpec.Commit.Author.Name,
Email: gitSpec.Commit.Author.Email,
When: time.Now(),
},
Message: message,
},
git.WithSigner(signingEntity),
); err != nil {
if err != git.ErrNoStagedFiles {
return failWithError(err)
}
@ -387,7 +372,7 @@ func (r *ImageUpdateAutomationReconciler) Reconcile(ctx context.Context, req ctr
// Use the git operations timeout for the repo.
pushCtx, cancel := context.WithTimeout(ctx, origin.Spec.Timeout.Duration)
defer cancel()
if err := push(pushCtx, tmp, pushBranch, access); err != nil {
if err := lgc.Push(pushCtx); err != nil {
return failWithError(err)
}
@ -512,11 +497,8 @@ type repoAccess struct {
url string
}
func (r *ImageUpdateAutomationReconciler) getRepoAccess(ctx context.Context, repository *sourcev1.GitRepository) (repoAccess, error) {
var access repoAccess
access.url = repository.Spec.URL
access.auth = &git.AuthOptions{}
func (r *ImageUpdateAutomationReconciler) getAuthOpts(ctx context.Context, repository *sourcev1.GitRepository) (*git.AuthOptions, error) {
var data map[string][]byte
if repository.Spec.SecretRef != nil {
name := types.NamespacedName{
Namespace: repository.GetNamespace(),
@ -526,181 +508,22 @@ func (r *ImageUpdateAutomationReconciler) getRepoAccess(ctx context.Context, rep
secret := &corev1.Secret{}
err := r.Client.Get(ctx, name, secret)
if err != nil {
err = fmt.Errorf("auth secret error: %w", err)
return access, err
return nil, fmt.Errorf("failed to get secret '%s': %w", name.String(), err)
}
data = secret.Data
}
access.auth, err = git.AuthOptionsFromSecret(access.url, secret)
if err != nil {
err = fmt.Errorf("auth error: %w", err)
return access, err
}
}
return access, nil
}
// cloneInto clones the upstream repository at the `ref` given (which
// can be `nil`). It returns a `*libgit2.Repository` since that is used
// for committing changes.
func cloneInto(ctx context.Context, access repoAccess, ref *sourcev1.GitRepositoryRef,
path string) (*libgit2.Repository, error) {
opts := git.CheckoutOptions{}
if ref != nil {
opts.Tag = ref.Tag
opts.SemVer = ref.SemVer
opts.Commit = ref.Commit
opts.Branch = ref.Branch
}
checkoutStrat, err := gitstrat.CheckoutStrategyForImplementation(ctx, sourcev1.LibGit2Implementation, opts)
if err == nil {
_, err = checkoutStrat.Checkout(ctx, path, access.url, access.auth)
}
u, err := url.Parse(repository.Spec.URL)
if err != nil {
return nil, err
return nil, fmt.Errorf("failed to parse URL '%s': %w", repository.Spec.URL, err)
}
return libgit2.OpenRepository(path)
}
func headCommit(repo *libgit2.Repository) (*libgit2.Commit, error) {
head, err := repo.Head()
opts, err := git.NewAuthOptions(*u, data)
if err != nil {
return nil, err
}
defer head.Free()
c, err := repo.LookupCommit(head.Target())
if err != nil {
return nil, err
}
return c, nil
}
var errNoChanges error = errors.New("no changes made to working directory")
func commitChangedManifests(tracelog logr.Logger, repo *libgit2.Repository, absRepoPath string, ent *openpgp.Entity, sig *libgit2.Signature, message string) (string, error) {
sl, err := repo.StatusList(&libgit2.StatusOptions{
Show: libgit2.StatusShowIndexAndWorkdir,
})
if err != nil {
return "", err
}
defer sl.Free()
count, err := sl.EntryCount()
if err != nil {
return "", err
return nil, fmt.Errorf("failed to configure authentication options: %w", err)
}
if count == 0 {
return "", errNoChanges
}
var parentC []*libgit2.Commit
head, err := headCommit(repo)
if err == nil {
defer head.Free()
parentC = append(parentC, head)
}
index, err := repo.Index()
if err != nil {
return "", err
}
defer index.Free()
// add to index any files that are not within .git/
if err = filepath.Walk(repo.Workdir(),
func(path string, info os.FileInfo, err error) error {
if err != nil {
return err
}
rel, err := filepath.Rel(repo.Workdir(), path)
if err != nil {
return err
}
f, err := os.Stat(path)
if err != nil {
return err
}
if f.IsDir() || strings.HasPrefix(rel, ".git") || rel == "." {
return nil
}
if err := index.AddByPath(rel); err != nil {
tracelog.Info("adding file", "file", rel)
return err
}
return nil
}); err != nil {
return "", err
}
if err := index.Write(); err != nil {
return "", err
}
treeID, err := index.WriteTree()
if err != nil {
return "", err
}
tree, err := repo.LookupTree(treeID)
if err != nil {
return "", err
}
defer tree.Free()
commitID, err := repo.CreateCommit("HEAD", sig, sig, message, tree, parentC...)
if err != nil {
return "", err
}
// return unsigned commit if pgp entity is not provided
if ent == nil {
return commitID.String(), nil
}
commit, err := repo.LookupCommit(commitID)
if err != nil {
return "", err
}
defer commit.Free()
signedCommitID, err := commit.WithSignatureUsing(func(commitContent string) (string, string, error) {
cipherText := new(bytes.Buffer)
err := openpgp.ArmoredDetachSignText(cipherText, ent, strings.NewReader(commitContent), &packet.Config{})
if err != nil {
return "", "", errors.New("error signing payload")
}
return cipherText.String(), "", nil
})
if err != nil {
return "", err
}
signedCommit, err := repo.LookupCommit(signedCommitID)
if err != nil {
return "", err
}
defer signedCommit.Free()
newHead, err := repo.Head()
if err != nil {
return "", err
}
defer newHead.Free()
ref, err := repo.References.Create(
newHead.Name(),
signedCommit.Id(),
true,
"repoint to signed commit",
)
if err != nil {
return "", err
}
defer ref.Free()
return signedCommitID.String(), nil
return opts, nil
}
// getSigningEntity retrieves an OpenPGP entity referenced by the
@ -733,175 +556,6 @@ func (r *ImageUpdateAutomationReconciler) getSigningEntity(ctx context.Context,
return entities[0], nil
}
var errRemoteBranchMissing = errors.New("remote branch missing")
// switchToBranch switches to a branch after fetching latest from upstream.
// If the branch does not exist, it is created using the head as the starting point.
func switchToBranch(repo *libgit2.Repository, ctx context.Context, branch string, access repoAccess) error {
origin, err := repo.Remotes.Lookup(originRemote)
if err != nil {
return fmt.Errorf("cannot lookup remote: %w", err)
}
defer origin.Free()
// Override callbacks with dummy ones as they are not needed within Managed Transport.
// However, not setting them may lead to git2go panicing.
callbacks := managed.RemoteCallbacks()
// Force the fetching of the remote branch.
err = origin.Fetch([]string{branch}, &libgit2.FetchOptions{
RemoteCallbacks: callbacks,
}, "")
if err != nil {
return fmt.Errorf("cannot fetch remote branch: %w", err)
}
remoteBranch, err := repo.References.Lookup(fmt.Sprintf("refs/remotes/origin/%s", branch))
if err != nil && !libgit2.IsErrorCode(err, libgit2.ErrorCodeNotFound) {
return err
}
if remoteBranch != nil {
defer remoteBranch.Free()
}
err = nil
var commit *libgit2.Commit
// tries to get tip commit from remote branch, if it exists.
// otherwise gets the commit that local head is pointing to.
if remoteBranch != nil {
commit, err = repo.LookupCommit(remoteBranch.Target())
} else {
head, err := repo.Head()
if err != nil {
return fmt.Errorf("cannot get repo head: %w", err)
}
defer head.Free()
commit, err = repo.LookupCommit(head.Target())
}
if err != nil {
return fmt.Errorf("cannot find the head commit: %w", err)
}
defer commit.Free()
localBranch, err := repo.References.Lookup(fmt.Sprintf("refs/heads/%s", branch))
if err != nil && !libgit2.IsErrorCode(err, libgit2.ErrorCodeNotFound) {
return fmt.Errorf("cannot lookup branch '%s': %w", branch, err)
}
if localBranch == nil {
lb, err := repo.CreateBranch(branch, commit, false)
if err != nil {
return fmt.Errorf("cannot create branch '%s': %w", branch, err)
}
defer lb.Free()
// We could've done something like:
// localBranch = lb.Reference
// But for some reason, calling `lb.Free()` AND using it, causes a really
// nasty crash. Since, we can't avoid calling `lb.Free()`, in order to prevent
// memory leaks, we don't use `lb` and instead manually lookup the ref.
localBranch, err = repo.References.Lookup(fmt.Sprintf("refs/heads/%s", branch))
if err != nil {
return fmt.Errorf("cannot lookup branch '%s': %w", branch, err)
}
}
defer localBranch.Free()
tree, err := repo.LookupTree(commit.TreeId())
if err != nil {
return fmt.Errorf("cannot lookup tree for branch '%s': %w", branch, err)
}
defer tree.Free()
err = repo.CheckoutTree(tree, &libgit2.CheckoutOpts{
// the remote branch should take precedence if it exists at this point in time.
Strategy: libgit2.CheckoutForce,
})
if err != nil {
return fmt.Errorf("cannot checkout tree for branch '%s': %w", branch, err)
}
ref, err := localBranch.SetTarget(commit.Id(), "")
if err != nil {
return fmt.Errorf("cannot update branch '%s' to be at target commit: %w", branch, err)
}
ref.Free()
return repo.SetHead("refs/heads/" + branch)
}
// push pushes the branch given to the origin using the git library
// indicated by `impl`. It's passed both the path to the repo and a
// libgit2.Repository value, since the latter may as well be used if the
// implementation is libgit2.
func push(ctx context.Context, path, branch string, access repoAccess) error {
repo, err := libgit2.OpenRepository(path)
if err != nil {
return err
}
defer repo.Free()
origin, err := repo.Remotes.Lookup(originRemote)
if err != nil {
return err
}
defer origin.Free()
// Override callbacks with dummy ones as they are not needed within Managed Transport.
// However, not setting them may lead to git2go panicing.
callbacks := managed.RemoteCallbacks()
// calling repo.Push will succeed even if a reference update is
// rejected; to detect this case, this callback is supplied.
var callbackErr error
callbacks.PushUpdateReferenceCallback = func(refname, status string) error {
if status != "" {
callbackErr = fmt.Errorf("ref %s rejected: %s", refname, status)
}
return nil
}
err = origin.Push([]string{fmt.Sprintf("refs/heads/%s:refs/heads/%s", branch, branch)}, &libgit2.PushOptions{
RemoteCallbacks: callbacks,
ProxyOptions: libgit2.ProxyOptions{Type: libgit2.ProxyTypeAuto},
})
if err != nil {
if strings.Contains(err.Error(), "early EOF") {
return fmt.Errorf("%w (the SSH key may not have write access to the repository)", err)
}
return libgit2PushError(err)
}
return callbackErr
}
func libgit2PushError(err error) error {
if err == nil {
return err
}
// libgit2 returns the whole output from stderr, and we only need
// the message. GitLab likes to return a banner, so as an
// heuristic, strip any lines that are just "remote:" and spaces
// or fencing.
msg := err.Error()
lines := strings.Split(msg, "\n")
if len(lines) == 1 {
return err
}
var b strings.Builder
// the following removes the prefix "remote:" from each line; to
// retain a bit of fidelity to the original error, start with it.
b.WriteString("remote: ")
var appending bool
for _, line := range lines {
m := strings.TrimPrefix(line, "remote:")
if m = strings.Trim(m, " \t="); m != "" {
if appending {
b.WriteString(" ")
}
b.WriteString(m)
appending = true
}
}
return errors.New(b.String())
}
// --- events, metrics
func (r *ImageUpdateAutomationReconciler) event(ctx context.Context, auto imagev1.ImageUpdateAutomation, severity, msg string) {

View File

@ -30,9 +30,9 @@ import (
ctrl "sigs.k8s.io/controller-runtime"
imagev1_reflect "github.com/fluxcd/image-reflector-controller/api/v1beta1"
"github.com/fluxcd/pkg/git/libgit2/transport"
"github.com/fluxcd/pkg/runtime/testenv"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
"github.com/fluxcd/source-controller/pkg/git/libgit2/managed"
imagev1 "github.com/fluxcd/image-automation-controller/api/v1beta1"
// +kubebuilder:scaffold:imports
@ -65,20 +65,22 @@ func TestMain(m *testing.M) {
filepath.Join("testdata", "crds"),
))
managed.InitManagedTransport()
if err := transport.InitManagedTransport(); err != nil {
panic(fmt.Sprintf("failed to initialize libgit2 managed transport: %v", err))
}
controllerName := "image-automation-controller"
if err := (&ImageUpdateAutomationReconciler{
Client: testEnv,
EventRecorder: testEnv.GetEventRecorderFor(controllerName),
}).SetupWithManager(testEnv, ImageUpdateAutomationReconcilerOptions{}); err != nil {
panic(fmt.Sprintf("Failed to start ImageUpdateAutomationReconciler: %v", err))
panic(fmt.Sprintf("failed to start ImageUpdateAutomationReconciler: %v", err))
}
go func() {
fmt.Println("Starting the test environment")
if err := testEnv.Start(ctx); err != nil {
panic(fmt.Sprintf("Failed to start the test environment manager: %v", err))
panic(fmt.Sprintf("failed to start the test environment manager: %v", err))
}
}()
<-testEnv.Manager.Elected()
@ -87,7 +89,7 @@ func TestMain(m *testing.M) {
fmt.Println("Stopping the test environment")
if err := testEnv.Stop(); err != nil {
panic(fmt.Sprintf("Failed to stop the test environment: %v", err))
panic(fmt.Sprintf("failed to stop the test environment: %v", err))
}
os.Exit(code)

View File

@ -50,11 +50,11 @@ import (
imagev1_reflect "github.com/fluxcd/image-reflector-controller/api/v1beta1"
"github.com/fluxcd/pkg/apis/acl"
"github.com/fluxcd/pkg/apis/meta"
"github.com/fluxcd/pkg/git"
"github.com/fluxcd/pkg/git/libgit2/transport"
"github.com/fluxcd/pkg/gittestserver"
"github.com/fluxcd/pkg/ssh"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
"github.com/fluxcd/source-controller/pkg/git"
"github.com/fluxcd/source-controller/pkg/git/libgit2/managed"
imagev1 "github.com/fluxcd/image-automation-controller/api/v1beta1"
"github.com/fluxcd/image-automation-controller/pkg/test"
@ -907,7 +907,7 @@ func configureTransportOptsForRepo(repo *libgit2.Repository, authOpts *git.AuthO
return nil, err
}
transportOptsURL := u.Scheme + "://" + randStringRunes(5)
managed.AddTransportOptions(transportOptsURL, managed.TransportOptions{
transport.AddTransportOptions(transportOptsURL, transport.TransportOptions{
TargetURL: repoURL,
AuthOpts: authOpts,
})
@ -917,7 +917,7 @@ func configureTransportOptsForRepo(repo *libgit2.Repository, authOpts *git.AuthO
return nil, fmt.Errorf("could not set remote origin url: %v", err)
}
return func() {
managed.RemoveTransportOptions(transportOptsURL)
transport.RemoveTransportOptions(transportOptsURL)
repo.Remotes.SetUrl(originRemote, repoURL)
}, nil
}
@ -1188,12 +1188,12 @@ func clone(ctx context.Context, repoURL, branchName string, authOpts *git.AuthOp
}
transportOptsURL := u.Scheme + "://" + randStringRunes(5)
managed.AddTransportOptions(transportOptsURL, managed.TransportOptions{
transport.AddTransportOptions(transportOptsURL, transport.TransportOptions{
TargetURL: repoURL,
Context: ctx,
AuthOpts: authOpts,
})
defer managed.RemoveTransportOptions(transportOptsURL)
defer transport.RemoveTransportOptions(transportOptsURL)
opts := &git2go.CloneOptions{
Bare: false,
@ -1266,6 +1266,21 @@ func waitForNewHead(g *WithT, repo *git2go.Repository, branch, preChangeHash str
}
}
func headCommit(repo *libgit2.Repository) (*libgit2.Commit, error) {
head, err := repo.Head()
if err != nil {
return nil, err
}
defer head.Free()
c, err := repo.LookupCommit(head.Target())
if err != nil {
return nil, err
}
return c, nil
}
func commitIdFromBranch(repo *git2go.Repository, branchName string) string {
commitId := ""
head, err := headFromBranch(repo, branchName)

13
go.mod
View File

@ -12,6 +12,11 @@ replace github.com/fluxcd/image-automation-controller/api => ./api
// - libgit2/git2go#918.
replace github.com/libgit2/git2go/v33 => github.com/fluxcd/git2go/v33 v33.0.9-flux
// This lets us use `go-billy/util.Walk()`, as this function hasn't been released
// in a tagged version yet:
// https://github.com/go-git/go-billy/blob/e0768be422ff616fc042d1d62bfa65962f716ad8/util/walk.go#L59
replace github.com/go-git/go-billy/v5 => github.com/go-git/go-billy/v5 v5.3.2-0.20210603175951-e0768be422ff
require (
github.com/AdaLogics/go-fuzz-headers v0.0.0-20221007124625-37f5449ff7df
github.com/Masterminds/sprig/v3 v3.2.2
@ -21,10 +26,11 @@ require (
github.com/fluxcd/image-reflector-controller/api v0.22.1
github.com/fluxcd/pkg/apis/acl v0.1.0
github.com/fluxcd/pkg/apis/meta v0.17.0
github.com/fluxcd/pkg/git v0.6.1
github.com/fluxcd/pkg/git/libgit2 v0.1.1-0.20220908131225-538bbcd1fc66
github.com/fluxcd/pkg/gittestserver v0.7.0
github.com/fluxcd/pkg/runtime v0.22.0
github.com/fluxcd/pkg/ssh v0.6.0
github.com/fluxcd/source-controller v0.31.0
github.com/fluxcd/source-controller/api v0.31.0
github.com/go-git/go-billy/v5 v5.3.1
github.com/go-git/go-git/v5 v5.4.2
@ -63,6 +69,7 @@ require (
github.com/chai2010/gettext-go v1.0.2 // indirect
github.com/cloudflare/circl v1.1.0 // indirect
github.com/davecgh/go-spew v1.1.1 // indirect
github.com/elazarl/goproxy v0.0.0-20221015165544-a0805db90819 // indirect
github.com/emicklei/go-restful/v3 v3.9.0 // indirect
github.com/emirpasic/gods v1.18.1 // indirect
github.com/evanphx/json-patch v5.6.0+incompatible // indirect
@ -70,6 +77,7 @@ require (
github.com/exponent-io/jsonpath v0.0.0-20151013193312-d6023ce2651d // indirect
github.com/fluxcd/gitkit v0.6.0 // indirect
github.com/fluxcd/pkg/gitutil v0.2.0 // indirect
github.com/fluxcd/pkg/http/transport v0.0.1 // indirect
github.com/fluxcd/pkg/version v0.2.0 // indirect
github.com/fsnotify/fsnotify v1.5.4 // indirect
github.com/go-errors/errors v1.0.1 // indirect
@ -101,7 +109,8 @@ require (
github.com/kevinburke/ssh_config v1.2.0 // indirect
github.com/liggitt/tabwriter v0.0.0-20181228230101-89fcab3d43de // indirect
github.com/mailru/easyjson v0.7.7 // indirect
github.com/matryer/is v1.4.0 // indirect
github.com/mattn/go-colorable v0.1.13 // indirect
github.com/mattn/go-isatty v0.0.16 // indirect
github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect
github.com/mitchellh/copystructure v1.2.0 // indirect
github.com/mitchellh/go-homedir v1.1.0 // indirect

22
go.sum
View File

@ -105,6 +105,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE=
github.com/elazarl/goproxy v0.0.0-20221015165544-a0805db90819 h1:RIB4cRk+lBqKK3Oy0r2gRX4ui7tuhiZq2SuTtTCi0/0=
github.com/elazarl/goproxy v0.0.0-20221015165544-a0805db90819/go.mod h1:Ro8st/ElPeALwNFlcTpWmkr6IoMFfkjXAvTHpevnDsM=
github.com/elazarl/goproxy/ext v0.0.0-20190711103511-473e67f1d7d2/go.mod h1:gNh8nYJoAm43RfaxurUnxr+N1PwuFV3ZMl/efxlIlY8=
github.com/emicklei/go-restful/v3 v3.9.0 h1:XwGDlfxEnQZzuopoqxwSEllNcCOM9DhhFyhFIIGKwxE=
github.com/emicklei/go-restful/v3 v3.9.0/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc=
github.com/emirpasic/gods v1.12.0/go.mod h1:YfzfFFoVP/catgzJb4IKIqXjX78Ha8FMSDh3ymbK86o=
@ -136,18 +138,22 @@ github.com/fluxcd/pkg/apis/acl v0.1.0 h1:EoAl377hDQYL3WqanWCdifauXqXbMyFuK82NnX6
github.com/fluxcd/pkg/apis/acl v0.1.0/go.mod h1:zfEZzz169Oap034EsDhmCAGgnWlcWmIObZjYMusoXS8=
github.com/fluxcd/pkg/apis/meta v0.17.0 h1:Y2dfo1syHZDb9Mexjr2SWdcj1FnxnRXm015hEnhl6wU=
github.com/fluxcd/pkg/apis/meta v0.17.0/go.mod h1:GrOVzWXiu22XjLNgLLe2EBYhQPqZetes5SIADb4bmHE=
github.com/fluxcd/pkg/git v0.6.1 h1:LC5k/5QBgDNoaDMb6ukmKNcxLih/Se09m1x5vLfUZb8=
github.com/fluxcd/pkg/git v0.6.1/go.mod h1:O1YYuMUr5z8gHZrB3xBIMFyOdcCXG7kHUAuAqu6UkeA=
github.com/fluxcd/pkg/git/libgit2 v0.1.1-0.20220908131225-538bbcd1fc66 h1:OqRl5sbUtsOOFkem1ajgmq/78sbXgiAAFeuswRvB/DE=
github.com/fluxcd/pkg/git/libgit2 v0.1.1-0.20220908131225-538bbcd1fc66/go.mod h1:U0BrFOeONG/Au+LTlfc2UC6uRbLnWA8ulEj2liVLuTQ=
github.com/fluxcd/pkg/gittestserver v0.7.0 h1:PRVaEjeC/ePKTusB5Bx/ExM0P6bjroPdG6K2DO7YJUM=
github.com/fluxcd/pkg/gittestserver v0.7.0/go.mod h1:WHqqZQfdePi5M/s1ONMTB4MigktqJhzAFJOZ0KTBw9Y=
github.com/fluxcd/pkg/gitutil v0.2.0 h1:7vvXfq+Ur1/WXEejXY/b2haJ/2Uj5Et5v4V33l+ni1Q=
github.com/fluxcd/pkg/gitutil v0.2.0/go.mod h1:oOq6wzzTJmD/PPIM5GHj+PGtfbrL7cbQKZCDnVvyp+w=
github.com/fluxcd/pkg/http/transport v0.0.1 h1:2iB63xfOOgkH+gdKC5qfYV1TcL546JKOE/7ZZ86hRoc=
github.com/fluxcd/pkg/http/transport v0.0.1/go.mod h1:aDIYfECLVh3KTvM8HvNcpm2ESrVbhteJWEl0AFbcjJk=
github.com/fluxcd/pkg/runtime v0.22.0 h1:4YV/An41b+OGdSWDogwFfHr22CEE/in+lBLEK0fr1yc=
github.com/fluxcd/pkg/runtime v0.22.0/go.mod h1:Cm6jIhltzXIM3CRRY6SFASDn+z2m/1yPqOWwD73c3io=
github.com/fluxcd/pkg/ssh v0.6.0 h1:yRJ866obXCo0JseJCqlGKtF8cumioPcwIC6kMwf8Spg=
github.com/fluxcd/pkg/ssh v0.6.0/go.mod h1:Puf+hWNzFsqdHtjDpsI3PTTvitZu41ObuPFblOdzrrI=
github.com/fluxcd/pkg/version v0.2.0 h1:jG22c59Bsv6vL51N7Bqn8tjHArYOXrjbIkGArlIrv5w=
github.com/fluxcd/pkg/version v0.2.0/go.mod h1:umN1VAOV0sB1JDVwb8eXZzuuqIAEku+y+vcCVBBUIf0=
github.com/fluxcd/source-controller v0.31.0 h1:E19qlOVLNLb9rvbDDe3ZZDPZfmYN6Gs2esS935/lK0U=
github.com/fluxcd/source-controller v0.31.0/go.mod h1:eK0nMfog8Wnhv5aYf39qgmSBfVxthb8AKwea/Bkqnls=
github.com/fluxcd/source-controller/api v0.31.0 h1:4PZQt2XILTUZ/2JOVGzAIpNDXjx8n10skAhuBHa9tVw=
github.com/fluxcd/source-controller/api v0.31.0/go.mod h1:XOf8hJB7jFcAKiOb8HVZcegkBeNSb4g0nxqnNjeVufg=
github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc=
@ -161,9 +167,8 @@ github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6
github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q=
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.2.0/go.mod h1:pmpqyWchKfYfrkb/UVH4otLvyi/5gJlGI4Hb3ZqZ3W0=
github.com/go-git/go-billy/v5 v5.3.1 h1:CPiOUAzKtMRvolEKw+bG1PLRpT7D3LIs3/3ey4Aiu34=
github.com/go-git/go-billy/v5 v5.3.1/go.mod h1:pmpqyWchKfYfrkb/UVH4otLvyi/5gJlGI4Hb3ZqZ3W0=
github.com/go-git/go-billy/v5 v5.3.2-0.20210603175951-e0768be422ff h1:am7s+gMb3IKU9ym6BiErCR1pF1gnbjjl3eVbB7k6BdI=
github.com/go-git/go-billy/v5 v5.3.2-0.20210603175951-e0768be422ff/go.mod h1:pmpqyWchKfYfrkb/UVH4otLvyi/5gJlGI4Hb3ZqZ3W0=
github.com/go-git/go-git-fixtures/v4 v4.2.1 h1:n9gGL1Ct/yIw+nfsfr8s4+sbhT+Ncu2SubfXjIWgci8=
github.com/go-git/go-git-fixtures/v4 v4.2.1/go.mod h1:K8zd3kDUAykwTdDCr+I0per6Y6vMiRR/nnVTBtavnB0=
github.com/go-git/go-git/v5 v5.4.2 h1:BXyZu9t0VkbiHtqrsvdq39UDhGJTl1h55VW6CSC4aY4=
@ -330,15 +335,16 @@ github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN
github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0=
github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc=
github.com/matryer/is v1.2.0 h1:92UTHpy8CDwaJ08GqLDzhhuixiBUUD1p3AU6PHddz4A=
github.com/matryer/is v1.2.0/go.mod h1:2fLPjFQM9rhQ15aVEtbuwhJinnOqrmgXPNdZsdwlWXA=
github.com/matryer/is v1.4.0 h1:sosSmIWwkYITGrxZ25ULNDeKiMNzFSr4V/eqBQP0PeE=
github.com/matryer/is v1.4.0/go.mod h1:8I/i5uYgLzgsgEloJE1U6xx5HkBQpAZvepWuujKwMRU=
github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc=
github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4=
github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA=
github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg=
github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU=
github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94=
github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ=
github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM=
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI=
github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4=
@ -421,6 +427,7 @@ github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1
github.com/prometheus/procfs v0.8.0 h1:ODq8ZFEaYeCaZOJlZZdJA2AbQR98dSHSM1KW/You5mo=
github.com/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0uaxHdg830/4=
github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
github.com/rogpeppe/go-charset v0.0.0-20180617210344-2471d30d28b4/go.mod h1:qgYeAmZ5ZIpBWTGllZSQnw97Dj+woV0toclVaRGI8pc=
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.8.0 h1:FCbCCtXNOY3UtUuHUYaghJg4y7Fd14rXifAYUAtL9R8=
github.com/rogpeppe/go-internal v1.8.0/go.mod h1:WmiCO8CzOY8rg0OYDC4/i/2WRWAB6poM+XZ2dLUbcbE=
@ -658,6 +665,7 @@ golang.org/x/sys v0.0.0-20220412211240-33da011f77ad/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.1.0 h1:kunALQeHf1/185U1i0GOB/fy1IPRDDpuoOOqRReG57U=
golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw=

View File

@ -41,8 +41,8 @@ import (
imagev1 "github.com/fluxcd/image-automation-controller/api/v1beta1"
"github.com/fluxcd/image-automation-controller/internal/features"
"github.com/fluxcd/source-controller/pkg/git"
"github.com/fluxcd/source-controller/pkg/git/libgit2/managed"
"github.com/fluxcd/pkg/git"
"github.com/fluxcd/pkg/git/libgit2/transport"
// +kubebuilder:scaffold:imports
"github.com/fluxcd/image-automation-controller/controllers"
@ -163,7 +163,7 @@ func main() {
}
// +kubebuilder:scaffold:builder
if err = managed.InitManagedTransport(); err != nil {
if err = transport.InitManagedTransport(); err != nil {
setupLog.Error(err, "unable to initialize libgit2 managed transport")
os.Exit(1)
}