Skip to content

Commit

Permalink
feat: Use Argo CD ApplicationSet to manage deployments across CDPipel…
Browse files Browse the repository at this point in the history
…ine (#20)

Added functionality:
- For every CDPipeline ApplicationSet is created
- If CDPipeline is removed, related ApplicationSet is removed too
- For every Stage and Codebase in the CDPipeline -
  the related ApplicationSet generator is created/deleted

Change-Id: I13bf5a98d84773dd026574246a72072ba8f853e9
  • Loading branch information
zmotso committed Jan 17, 2024
1 parent c4ae1a2 commit 2e534ab
Show file tree
Hide file tree
Showing 17 changed files with 2,208 additions and 173 deletions.
11 changes: 11 additions & 0 deletions config/rbac/role.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,17 @@ metadata:
name: manager-role
namespace: placeholder
rules:
- apiGroups:
- argoproj.io
resources:
- applicationsets
verbs:
- create
- get
- list
- patch
- update
- watch
- apiGroups:
- v2.edp.epam.com
resources:
Expand Down
47 changes: 42 additions & 5 deletions controllers/cdpipeline/cdpipeline_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,16 +21,22 @@ import (
"github.com/epam/edp-cd-pipeline-operator/v2/pkg/util/consts"
)

func NewReconcileCDPipeline(c client.Client, scheme *runtime.Scheme) *ReconcileCDPipeline {
func NewReconcileCDPipeline(
c client.Client,
scheme *runtime.Scheme,
createApplicationSet func(ctx context.Context, pipeline *cdPipeApi.CDPipeline) error,
) *ReconcileCDPipeline {
return &ReconcileCDPipeline{
client: c,
scheme: scheme,
client: c,
scheme: scheme,
createApplicationSet: createApplicationSet,
}
}

type ReconcileCDPipeline struct {
client client.Client
scheme *runtime.Scheme
client client.Client
scheme *runtime.Scheme
createApplicationSet func(ctx context.Context, pipeline *cdPipeApi.CDPipeline) error
}

const (
Expand Down Expand Up @@ -70,6 +76,7 @@ func (r *ReconcileCDPipeline) SetupWithManager(mgr ctrl.Manager) error {
//+kubebuilder:rbac:groups=v2.edp.epam.com,namespace=placeholder,resources=cdpipelines,verbs=get;list;watch;create;update;patch;delete
//+kubebuilder:rbac:groups=v2.edp.epam.com,namespace=placeholder,resources=cdpipelines/status,verbs=get;update;patch
//+kubebuilder:rbac:groups=v2.edp.epam.com,namespace=placeholder,resources=cdpipelines/finalizers,verbs=update
//+kubebuilder:rbac:groups=argoproj.io,namespace=placeholder,resources=applicationsets,verbs=get;list;watch;update;patch;create

func (r *ReconcileCDPipeline) Reconcile(ctx context.Context, request reconcile.Request) (reconcile.Result, error) {
log := ctrl.LoggerFrom(ctx)
Expand Down Expand Up @@ -97,6 +104,14 @@ func (r *ReconcileCDPipeline) Reconcile(ctx context.Context, request reconcile.R
return *result, nil
}

if err = r.createApplicationSet(ctx, pipeline); err != nil {
if statusErr := r.setFailedStatus(ctx, pipeline, err); statusErr != nil {
return reconcile.Result{}, statusErr
}

return reconcile.Result{}, fmt.Errorf("failed to create application set: %w", err)
}

if err := r.setFinishStatus(ctx, pipeline); err != nil {
return reconcile.Result{}, err
}
Expand Down Expand Up @@ -181,6 +196,28 @@ func (r *ReconcileCDPipeline) setFinishStatus(ctx context.Context, p *cdPipeApi.
return nil
}

func (r *ReconcileCDPipeline) setFailedStatus(ctx context.Context, p *cdPipeApi.CDPipeline, err error) error {
log := ctrl.LoggerFrom(ctx)

p.Status = cdPipeApi.CDPipelineStatus{
Status: consts.FailedStatus,
Available: false,
LastTimeUpdated: metaV1.Now(),
Username: "system",
Result: cdPipeApi.Error,
DetailedMessage: err.Error(),
Value: consts.FailedStatus,
}

if err = r.client.Status().Update(ctx, p); err != nil {
return fmt.Errorf("failed to update CDPipeline status: %w", err)
}

log.Info("CDPipeline failed status has been updated")

return nil
}

// hasActiveOwnedStages checks if there are any active stages owned by the pipeline.
func (r *ReconcileCDPipeline) hasActiveOwnedStages(ctx context.Context, pipeline *cdPipeApi.CDPipeline) (bool, error) {
stages := &cdPipeApi.StageList{}
Expand Down
27 changes: 9 additions & 18 deletions controllers/cdpipeline/cdpipeline_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,25 +51,16 @@ func createScheme(t *testing.T) *runtime.Scheme {
return scheme
}

func TestNewReconcileCDPipeline_Success(t *testing.T) {
scheme := runtime.NewScheme()
client := fake.NewClientBuilder().Build()

expectedReconcileCdPipeline := &ReconcileCDPipeline{
client: client,
scheme: scheme,
}

reconciledCdPipeline := NewReconcileCDPipeline(client, scheme)
assert.Equal(t, expectedReconcileCdPipeline, reconciledCdPipeline)
func createApplicationSetMock(ctx context.Context, pipeline *cdPipeApi.CDPipeline) error {
return nil
}

func TestReconcile_Success(t *testing.T) {
emptyCdPipeline := emptyCdPipelineInit(t)
scheme := createScheme(t)
client := fake.NewClientBuilder().WithScheme(scheme).WithObjects(emptyCdPipeline).Build()

reconcileCDPipeline := NewReconcileCDPipeline(client, scheme)
reconcileCDPipeline := NewReconcileCDPipeline(client, scheme, createApplicationSetMock)

_, err := reconcileCDPipeline.Reconcile(context.Background(), reconcile.Request{NamespacedName: types.NamespacedName{
Namespace: namespace,
Expand All @@ -93,7 +84,7 @@ func TestReconcile_PipelineIsNotFound(t *testing.T) {
scheme := createScheme(t)
client := fake.NewClientBuilder().WithScheme(scheme).WithObjects(&cdPipeline).Build()

reconcileCDPipeline := NewReconcileCDPipeline(client, scheme)
reconcileCDPipeline := NewReconcileCDPipeline(client, scheme, createApplicationSetMock)

_, err := reconcileCDPipeline.Reconcile(context.Background(), reconcile.Request{NamespacedName: types.NamespacedName{
Namespace: namespace,
Expand All @@ -107,7 +98,7 @@ func TestReconcile_GetCdPipelineError(t *testing.T) {
scheme := runtime.NewScheme()
client := fake.NewClientBuilder().WithScheme(scheme).Build()

reconcileCDPipeline := NewReconcileCDPipeline(client, scheme)
reconcileCDPipeline := NewReconcileCDPipeline(client, scheme, createApplicationSetMock)

_, err := reconcileCDPipeline.Reconcile(context.Background(), reconcile.Request{NamespacedName: types.NamespacedName{
Namespace: namespace,
Expand All @@ -131,7 +122,7 @@ func TestAddFinalizer_DeletionTimestampNotZero(t *testing.T) {
scheme := createScheme(t)
client := fake.NewClientBuilder().WithScheme(scheme).WithObjects(&cdPipeline).Build()

reconcileCdPipeline := NewReconcileCDPipeline(client, scheme)
reconcileCdPipeline := NewReconcileCDPipeline(client, scheme, createApplicationSetMock)

res, err := reconcileCdPipeline.tryToDeletePipeline(ctrl.LoggerInto(context.Background(), logr.Discard()), &cdPipeline)
assert.NoError(t, err)
Expand Down Expand Up @@ -170,7 +161,7 @@ func TestAddFinalizer_PostponeDeletion(t *testing.T) {
scheme := createScheme(t)
client := fake.NewClientBuilder().WithScheme(scheme).WithObjects(&cdPipeline, stage).Build()

reconcileCdPipeline := NewReconcileCDPipeline(client, scheme)
reconcileCdPipeline := NewReconcileCDPipeline(client, scheme, createApplicationSetMock)

res, err := reconcileCdPipeline.tryToDeletePipeline(ctrl.LoggerInto(context.Background(), logr.Discard()), &cdPipeline)
assert.NoError(t, err)
Expand All @@ -190,7 +181,7 @@ func TestAddFinalizer_DeletionTimestampIsZero(t *testing.T) {
scheme := createScheme(t)
client := fake.NewClientBuilder().WithScheme(scheme).WithObjects(cdPipeline).Build()

reconcileCdPipeline := NewReconcileCDPipeline(client, scheme)
reconcileCdPipeline := NewReconcileCDPipeline(client, scheme, createApplicationSetMock)

res, err := reconcileCdPipeline.tryToDeletePipeline(ctrl.LoggerInto(context.Background(), logr.Discard()), cdPipeline)
assert.NoError(t, err)
Expand All @@ -210,7 +201,7 @@ func TestSetFinishStatus_Success(t *testing.T) {
scheme := createScheme(t)
client := fake.NewClientBuilder().WithScheme(scheme).WithObjects(cdPipeline).Build()

reconcileCdPipeline := NewReconcileCDPipeline(client, scheme)
reconcileCdPipeline := NewReconcileCDPipeline(client, scheme, createApplicationSetMock)

err := reconcileCdPipeline.setFinishStatus(context.Background(), cdPipeline)
assert.NoError(t, err)
Expand Down
25 changes: 25 additions & 0 deletions controllers/stage/chain/add_applicationset_generators.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
package chain

import (
"context"
"fmt"

cdPipeApi "github.com/epam/edp-cd-pipeline-operator/v2/api/v1"
)

type applicationSetManager interface {
CreateApplicationSetGenerators(ctx context.Context, stage *cdPipeApi.Stage) error
RemoveApplicationSetGenerators(ctx context.Context, stage *cdPipeApi.Stage) error
}

type AddApplicationSetGenerators struct {
applicationSetManager applicationSetManager
}

func (h AddApplicationSetGenerators) ServeRequest(ctx context.Context, stage *cdPipeApi.Stage) error {
if err := h.applicationSetManager.CreateApplicationSetGenerators(ctx, stage); err != nil {
return fmt.Errorf("failed to create application set generators: %w", err)
}

return nil
}
7 changes: 7 additions & 0 deletions controllers/stage/chain/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (

cdPipeApi "github.com/epam/edp-cd-pipeline-operator/v2/api/v1"
"github.com/epam/edp-cd-pipeline-operator/v2/controllers/stage/chain/handler"
"github.com/epam/edp-cd-pipeline-operator/v2/pkg/argocd"
"github.com/epam/edp-cd-pipeline-operator/v2/pkg/multiclusterclient"
"github.com/epam/edp-cd-pipeline-operator/v2/pkg/rbac"
)
Expand Down Expand Up @@ -52,6 +53,9 @@ func CreateChain(ctx context.Context, c client.Client, stage *cdPipeApi.Stage) (
multiClusterClient: multiClusterCl,
internalClient: c,
},
AddApplicationSetGenerators{
applicationSetManager: argocd.NewArgoApplicationSetManager(c),
},
)

return ch, nil
Expand All @@ -65,6 +69,9 @@ func CreateDeleteChain(ctx context.Context, c client.Client, stage *cdPipeApi.St
DeleteEnvironmentLabelFromCodebaseImageStreams{
client: c,
},
RemoveApplicationSetGenerators{
applicationSetManager: argocd.NewArgoApplicationSetManager(c),
},
)

multiClusterCl, err := multiclusterclient.NewClientProvider(c).GetClusterClient(ctx, stage.Namespace, stage.Spec.ClusterName, client.Options{})
Expand Down
20 changes: 20 additions & 0 deletions controllers/stage/chain/remove_applicationset_generators.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
package chain

import (
"context"
"fmt"

cdPipeApi "github.com/epam/edp-cd-pipeline-operator/v2/api/v1"
)

type RemoveApplicationSetGenerators struct {
applicationSetManager applicationSetManager
}

func (h RemoveApplicationSetGenerators) ServeRequest(ctx context.Context, stage *cdPipeApi.Stage) error {
if err := h.applicationSetManager.RemoveApplicationSetGenerators(ctx, stage); err != nil {
return fmt.Errorf("failed to remove application set generators: %w", err)
}

return nil
}
1 change: 1 addition & 0 deletions controllers/stage/stage_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ func (r *ReconcileStage) SetupWithManager(mgr ctrl.Manager) error {
//+kubebuilder:rbac:groups=v2.edp.epam.com,namespace=placeholder,resources=stages,verbs=get;list;watch;create;update;patch;delete
//+kubebuilder:rbac:groups=v2.edp.epam.com,namespace=placeholder,resources=stages/status,verbs=get;update;patch
//+kubebuilder:rbac:groups=v2.edp.epam.com,namespace=placeholder,resources=stages/finalizers,verbs=update
//+kubebuilder:rbac:groups=argoproj.io,namespace=placeholder,resources=applicationsets,verbs=get;list;watch;update;patch;create

func (r *ReconcileStage) Reconcile(ctx context.Context, request reconcile.Request) (reconcile.Result, error) {
log := ctrl.LoggerFrom(ctx)
Expand Down
46 changes: 26 additions & 20 deletions controllers/stage/stage_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,9 @@ import (
"testing"
"time"

argoApi "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1"
"github.com/go-logr/logr"
projectApi "github.com/openshift/api/project/v1"
projectApi "github.com/openshift/api/project"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
corev1 "k8s.io/api/core/v1"
Expand Down Expand Up @@ -89,13 +90,12 @@ func TestTryToDeleteCDStage_DeletionTimestampIsZero(t *testing.T) {

func TestTryToDeleteCDStage_Success(t *testing.T) {
scheme := runtime.NewScheme()
err := cdPipeApi.AddToScheme(scheme)
require.NoError(t, err)
err = codebaseApi.AddToScheme(scheme)
require.NoError(t, err)
err = corev1.AddToScheme(scheme)
require.NoError(t, err)
require.NoError(t, projectApi.AddToScheme(scheme))

require.NoError(t, cdPipeApi.AddToScheme(scheme))
require.NoError(t, codebaseApi.AddToScheme(scheme))
require.NoError(t, corev1.AddToScheme(scheme))
require.NoError(t, projectApi.Install(scheme))
require.NoError(t, argoApi.AddToScheme(scheme))

stage := &cdPipeApi.Stage{
TypeMeta: metaV1.TypeMeta{},
Expand Down Expand Up @@ -148,7 +148,7 @@ func TestTryToDeleteCDStage_Success(t *testing.T) {
log: logr.Discard(),
}

_, err = reconcileStage.tryToDeleteCDStage(ctrl.LoggerInto(context.Background(), logr.Discard()), stage)
_, err := reconcileStage.tryToDeleteCDStage(ctrl.LoggerInto(context.Background(), logr.Discard()), stage)
assert.NoError(t, err)

previousImageStream, err := cluster.GetCodebaseImageStream(reconcileStage.client, dockerImageName, namespace)
Expand Down Expand Up @@ -240,15 +240,12 @@ func TestSetFinishStatus_Success(t *testing.T) {

func TestReconcileStage_Reconcile_Success(t *testing.T) {
scheme := runtime.NewScheme()
err := cdPipeApi.AddToScheme(scheme)
require.NoError(t, err)
err = codebaseApi.AddToScheme(scheme)
require.NoError(t, err)
err = k8sApi.AddToScheme(scheme)
require.NoError(t, err)
err = corev1.AddToScheme(scheme)
require.NoError(t, err)
require.NoError(t, projectApi.AddToScheme(scheme))

require.NoError(t, cdPipeApi.AddToScheme(scheme))
require.NoError(t, codebaseApi.AddToScheme(scheme))
require.NoError(t, corev1.AddToScheme(scheme))
require.NoError(t, projectApi.Install(scheme))
require.NoError(t, argoApi.AddToScheme(scheme))

stage := &cdPipeApi.Stage{
TypeMeta: metaV1.TypeMeta{},
Expand Down Expand Up @@ -302,7 +299,7 @@ func TestReconcileStage_Reconcile_Success(t *testing.T) {
objectmodifier.NewStageBatchModifier(fakeClient, []objectmodifier.StageModifier{}),
)

_, err = reconcileStage.Reconcile(ctrl.LoggerInto(context.Background(), logr.Discard()), reconcile.Request{NamespacedName: types.NamespacedName{
_, err := reconcileStage.Reconcile(ctrl.LoggerInto(context.Background(), logr.Discard()), reconcile.Request{NamespacedName: types.NamespacedName{
Namespace: namespace,
Name: name,
}})
Expand All @@ -327,11 +324,13 @@ func TestReconcileStage_Reconcile_Success(t *testing.T) {

func TestReconcileStage_ReconcileReconcile_SetOwnerRef(t *testing.T) {
scheme := runtime.NewScheme()

require.NoError(t, corev1.AddToScheme(scheme))
require.NoError(t, cdPipeApi.AddToScheme(scheme))
require.NoError(t, codebaseApi.AddToScheme(scheme))
require.NoError(t, componentApi.AddToScheme(scheme))
require.NoError(t, k8sApi.AddToScheme(scheme))
require.NoError(t, argoApi.AddToScheme(scheme))

edpComponent := &componentApi.EDPComponent{
TypeMeta: metaV1.TypeMeta{},
Expand Down Expand Up @@ -383,7 +382,14 @@ func TestReconcileStage_ReconcileReconcile_SetOwnerRef(t *testing.T) {
},
}

fakeClient := fake.NewClientBuilder().WithScheme(scheme).WithObjects(cdPipeline, image, stage, edpComponent).Build()
appset := &argoApi.ApplicationSet{
ObjectMeta: metaV1.ObjectMeta{
Name: cdPipeline.Name,
Namespace: namespace,
},
}

fakeClient := fake.NewClientBuilder().WithScheme(scheme).WithObjects(cdPipeline, image, stage, edpComponent, appset).Build()

reconcileStage := NewReconcileStage(
fakeClient,
Expand Down
11 changes: 11 additions & 0 deletions deploy-templates/templates/role_kubernetes.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,17 @@ metadata:
{{- include "cd-pipeline-operator.labels" . | nindent 4 }}
name: edp-{{ .Values.name }}
rules:
- apiGroups:
- argoproj.io
resources:
- applicationsets
verbs:
- get
- list
- patch
- update
- watch
- create
- apiGroups:
- '*'
resources:
Expand Down
Loading

0 comments on commit 2e534ab

Please sign in to comment.