Skip to content

Commit

Permalink
feat: upload and download logs/artifact from blob storage configured …
Browse files Browse the repository at this point in the history
…in external cluster (#4138)

* blob storage cm secret inducing internally and removed AWS creds set as env variable(will be done in ci-runner)

* some refactoring and UseExternalClusterBlob flag introduced in cicdconfig

* added support fr fetching logs and artifacts from external cluster blob storage

* wire

* minor fix

* import fix

* code refactoring

* info comment

* fix

* comments added

* comments fix

* decodeSecretKey

* code review oncorporation l1

* code review incorporation l2

* code review incorporation l3

* code review incorporation l4

* code review incorporation l5

* code review incorporation l6

* minor fix for job tyoe

* minor fix

* code review incorporation

* code review incorporation

* code review incorporation

* minor fix after debugging error

* common lib version upgrade from 0.0.3 to 0.0.4
  • Loading branch information
prakash100198 committed Oct 31, 2023
1 parent d05fd2e commit 9e5a4ac
Show file tree
Hide file tree
Showing 19 changed files with 402 additions and 26 deletions.
3 changes: 3 additions & 0 deletions Wire.go
Original file line number Diff line number Diff line change
Expand Up @@ -572,6 +572,9 @@ func InitializeApp() (*App, error) {
pipeline.NewCdHandlerImpl,
wire.Bind(new(pipeline.CdHandler), new(*pipeline.CdHandlerImpl)),

pipeline.NewBlobStorageConfigServiceImpl,
wire.Bind(new(pipeline.BlobStorageConfigService), new(*pipeline.BlobStorageConfigServiceImpl)),

pipeline.NewWorkflowDagExecutorImpl,
wire.Bind(new(pipeline.WorkflowDagExecutor), new(*pipeline.WorkflowDagExecutorImpl)),
appClone.NewAppCloneServiceImpl,
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ require (
github.com/davecgh/go-spew v1.1.1
github.com/deckarep/golang-set v1.8.0
github.com/devtron-labs/authenticator v0.4.31
github.com/devtron-labs/common-lib v0.0.3
github.com/devtron-labs/common-lib v0.0.4
github.com/devtron-labs/protos v0.0.0-20230503113602-282404f70fd2
github.com/evanphx/json-patch v5.6.0+incompatible
github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -293,8 +293,8 @@ github.com/denisenkom/go-mssqldb v0.0.0-20190707035753-2be1aa521ff4 h1:YcpmyvADG
github.com/denisenkom/go-mssqldb v0.0.0-20190707035753-2be1aa521ff4/go.mod h1:zAg7JM8CkOJ43xKXIj7eRO9kmWm/TW578qo+oDO6tuM=
github.com/devtron-labs/authenticator v0.4.31 h1:CEMLek3JnMuH9ULsC6BHNJr+NiyGzBd4lgdSxH2IGnc=
github.com/devtron-labs/authenticator v0.4.31/go.mod h1:ozNfT8WcruiSgnUbyp48WVfc41++W6xYXhKFp67lNTU=
github.com/devtron-labs/common-lib v0.0.3 h1:9I1czMTU6+IeNpIPBydVibTRu8H9JQmsNi7JXTEgI/8=
github.com/devtron-labs/common-lib v0.0.3/go.mod h1:x6OdUIo2z9kxXtBfz7fJEfD4s8kiAtEmlApozOf7ECM=
github.com/devtron-labs/common-lib v0.0.4 h1:kKdhQnzy9cK+fDdcbh2JmjLGfzRF67VqtsbTHKyK47Q=
github.com/devtron-labs/common-lib v0.0.4/go.mod h1:x6OdUIo2z9kxXtBfz7fJEfD4s8kiAtEmlApozOf7ECM=
github.com/devtron-labs/protos v0.0.0-20230503113602-282404f70fd2 h1:/IEIsJTxDZ3hv8uOoCaqdWCXqcv7nCAgX9AP/v84dUY=
github.com/devtron-labs/protos v0.0.0-20230503113602-282404f70fd2/go.mod h1:l85jxWHlcSo910hdUfRycL40yGzC6glE93V1sVxVPto=
github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
Expand Down
13 changes: 13 additions & 0 deletions internal/sql/repository/pipelineConfig/CdWorfkflowRepository.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,9 @@ const (
WorkflowSucceeded = "Succeeded"
WorkflowTimedOut = "TimedOut"
WorkflowUnableToFetchState = "UnableToFetch"
WorkflowTypeDeploy = "DEPLOY"
WorkflowTypePre = "PRE"
WorkflowTypePost = "POST"
)

func (a WorkflowStatus) String() string {
Expand Down Expand Up @@ -166,6 +169,16 @@ type CdWorkflowRunner struct {
sql.AuditLog
}

func (c *CdWorkflowRunner) IsExternalRun() bool {
var isExtCluster bool
if c.WorkflowType == WorkflowTypePre {
isExtCluster = c.CdWorkflow.Pipeline.RunPreStageInEnv
} else if c.WorkflowType == WorkflowTypePost {
isExtCluster = c.CdWorkflow.Pipeline.RunPostStageInEnv
}
return isExtCluster
}

type CiPipelineMaterialResponse struct {
Id int `json:"id"`
GitMaterialId int `json:"gitMaterialId"`
Expand Down
14 changes: 9 additions & 5 deletions internal/sql/repository/pipelineConfig/CiWorkflowRepository.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,9 +72,13 @@ type CiWorkflow struct {
CiBuildType string `sql:"ci_build_type"`
EnvironmentId int `sql:"environment_id"`
ImagePathReservationId int `sql:"image_path_reservation_id"`
ReferenceCiWorkflowId int `sql:"ref_ci_workflow_id"`
ParentCiWorkFlowId int `sql:"parent_ci_workflow_id"`
CiPipeline *CiPipeline
ReferenceCiWorkflowId int `sql:"ref_ci_workflow_id"`
ParentCiWorkFlowId int `sql:"parent_ci_workflow_id"`
CiPipeline *CiPipeline
}

func (r *CiWorkflow) IsExternalRunInJobType() bool {
return r.EnvironmentId != 0
}

type WorkflowWithArtifact struct {
Expand All @@ -101,8 +105,8 @@ type WorkflowWithArtifact struct {
EnvironmentId int `json:"environmentId"`
EnvironmentName string `json:"environmentName"`
ImagePathReservationId int `json:"image_path_reservation_id"`
RefCiWorkflowId int `json:"referenceCiWorkflowId"`
ParentCiWorkflowId int `json:"parent_ci_workflow_id"`
RefCiWorkflowId int `json:"referenceCiWorkflowId"`
ParentCiWorkflowId int `json:"parent_ci_workflow_id"`
}

type GitCommit struct {
Expand Down
32 changes: 32 additions & 0 deletions pkg/cluster/ClusterService.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,9 @@ type ClusterService interface {
ConnectClustersInBatch(clusters []*ClusterBean, clusterExistInDb bool)
ConvertClusterBeanToCluster(clusterBean *ClusterBean, userId int32) *repository.Cluster
ConvertClusterBeanObjectToCluster(bean *ClusterBean) *v1alpha1.Cluster

GetClusterConfigByClusterId(clusterId int) (*k8s.ClusterConfig, error)
GetClusterConfigByEnvId(envId int) (*k8s.ClusterConfig, error)
}

type ClusterServiceImpl struct {
Expand Down Expand Up @@ -1095,3 +1098,32 @@ func (impl ClusterServiceImpl) ConvertClusterBeanObjectToCluster(bean *ClusterBe
}
return cl
}

func (impl ClusterServiceImpl) GetClusterConfigByClusterId(clusterId int) (*k8s.ClusterConfig, error) {
clusterBean, err := impl.FindById(clusterId)
if err != nil {
impl.logger.Errorw("error in getting clusterBean by cluster id", "err", err, "clusterId", clusterId)
return nil, err
}
rq := *clusterBean
clusterConfig, err := rq.GetClusterConfig()
if err != nil {
impl.logger.Errorw("error in getting cluster config", "err", err, "clusterId", clusterBean.Id)
return nil, err
}
return clusterConfig, nil
}

func (impl ClusterServiceImpl) GetClusterConfigByEnvId(envId int) (*k8s.ClusterConfig, error) {
envBean, err := impl.environmentService.FindById(envId)
if err != nil {
impl.logger.Errorw("error in getting envBean by envId", "err", err, "envId", envId)
return nil, err
}
clusterConfig, err := impl.GetClusterConfigByClusterId(envBean.ClusterId)
if err != nil {
impl.logger.Errorw("error in getting cluster config by env id", "err", err, "envId", envId)
return nil, err
}
return clusterConfig, nil
}
2 changes: 1 addition & 1 deletion pkg/generateManifest/DeployementTemplateService_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import (
"context"
"errors"
client2 "github.com/devtron-labs/authenticator/client"
"github.com/devtron-labs/common-lib/utils/k8s"
"github.com/devtron-labs/devtron/api/bean"
client "github.com/devtron-labs/devtron/api/helm-app"
mocks4 "github.com/devtron-labs/devtron/api/helm-app/mocks"
Expand All @@ -16,7 +17,6 @@ import (
"github.com/devtron-labs/devtron/pkg/chart/mocks"
chartRepoRepository "github.com/devtron-labs/devtron/pkg/chartRepo/repository"
mocks5 "github.com/devtron-labs/devtron/pkg/chartRepo/repository/mocks"
"github.com/devtron-labs/devtron/util/k8s"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"testing"
Expand Down
99 changes: 99 additions & 0 deletions pkg/pipeline/BlobStorageConfigService.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
package pipeline

import (
"context"
"fmt"
blob_storage "github.com/devtron-labs/common-lib/blob-storage"
"github.com/devtron-labs/common-lib/utils/k8s"
bean2 "github.com/devtron-labs/devtron/pkg/pipeline/bean"
"github.com/devtron-labs/devtron/pkg/pipeline/types"
"go.uber.org/zap"
v12 "k8s.io/apimachinery/pkg/apis/meta/v1"
"strconv"
)

type BlobStorageConfigService interface {
FetchCmAndSecretBlobConfigFromExternalCluster(clusterConfig *k8s.ClusterConfig, namespace string) (*bean2.CmBlobStorageConfig, *bean2.SecretBlobStorageConfig, error)
}
type BlobStorageConfigServiceImpl struct {
Logger *zap.SugaredLogger
k8sUtil *k8s.K8sUtil
ciCdConfig *types.CiCdConfig
}

func NewBlobStorageConfigServiceImpl(Logger *zap.SugaredLogger, k8sUtil *k8s.K8sUtil, ciCdConfig *types.CiCdConfig) *BlobStorageConfigServiceImpl {
return &BlobStorageConfigServiceImpl{
Logger: Logger,
k8sUtil: k8sUtil,
ciCdConfig: ciCdConfig,
}
}

func (impl *BlobStorageConfigServiceImpl) FetchCmAndSecretBlobConfigFromExternalCluster(clusterConfig *k8s.ClusterConfig, namespace string) (*bean2.CmBlobStorageConfig, *bean2.SecretBlobStorageConfig, error) {
cmConfig := &bean2.CmBlobStorageConfig{}
secretConfig := &bean2.SecretBlobStorageConfig{}
_, _, kubeClient, err := impl.k8sUtil.GetK8sConfigAndClients(clusterConfig)
if err != nil {
impl.Logger.Errorw("FetchCmAndSecretBlobConfigFromExternalCluster, error in getting kubeClient by cluster config", "err", err)
return cmConfig, secretConfig, err
}
cv1 := kubeClient.CoreV1()
ctx := context.Background()
opts := v12.GetOptions{}
cmName := impl.ciCdConfig.ExtBlobStorageCmName
secretName := impl.ciCdConfig.ExtBlobStorageSecretName
cm, err := cv1.ConfigMaps(namespace).Get(ctx, cmName, opts)
if err != nil {
impl.Logger.Errorw("error in getting config map in external cluster", "err", err, "blobStorageCmName", impl.ciCdConfig.ExtBlobStorageCmName, "clusterName", clusterConfig.ClusterName)
return cmConfig, secretConfig, err
}
secret, err := cv1.Secrets(namespace).Get(ctx, secretName, opts)
if err != nil {
impl.Logger.Errorw("error in getting secret in external cluster", "err", err, "blobStorageSecretName", impl.ciCdConfig.ExtBlobStorageSecretName, "clusterName", clusterConfig.ClusterName)
return cmConfig, secretConfig, err
}
//for IAM configured in S3 in external cluster, get logs/artifact will not work
if cm.Data != nil && secret.Data != nil {
err = cmConfig.SetCmBlobStorageConfig(cm.Data)
if err != nil {
fmt.Println("error marshalling external blob storage cm data to struct:", err)
return cmConfig, secretConfig, err
}
err = secretConfig.SetSecretBlobStorageConfig(secret.Data)
if err != nil {
fmt.Println("error marshalling external blob storage secret data to struct:", err)
return cmConfig, secretConfig, err
}
}
if cm.Data == nil {
fmt.Println("Data field not found in config map")
}
if secret.Data == nil {
fmt.Println("Data field not found in secret")
}
impl.Logger.Infow("fetching cm and secret from external cluster cloud provider", "ext cluster config: ", cmConfig)
return cmConfig, secretConfig, nil
}

func updateRequestWithExtClusterCmAndSecret(request *blob_storage.BlobStorageRequest, cmConfig *bean2.CmBlobStorageConfig, secretConfig *bean2.SecretBlobStorageConfig) *blob_storage.BlobStorageRequest {
request.StorageType = cmConfig.CloudProvider

request.AwsS3BaseConfig.AccessKey = cmConfig.S3AccessKey
request.AwsS3BaseConfig.EndpointUrl = cmConfig.S3Endpoint
request.AwsS3BaseConfig.Passkey = types.DecodeSecretKey(secretConfig.S3SecretKey)
isInSecure, _ := strconv.ParseBool(cmConfig.S3EndpointInsecure)
request.AwsS3BaseConfig.IsInSecure = isInSecure
request.AwsS3BaseConfig.BucketName = cmConfig.CdDefaultBuildLogsBucket
request.AwsS3BaseConfig.Region = cmConfig.CdDefaultCdLogsBucketRegion
s3BucketVersioned, _ := strconv.ParseBool(cmConfig.S3BucketVersioned)
request.AwsS3BaseConfig.VersioningEnabled = s3BucketVersioned

request.AzureBlobBaseConfig.AccountName = cmConfig.AzureAccountName
request.AzureBlobBaseConfig.AccountKey = types.DecodeSecretKey(secretConfig.AzureAccountKey)
request.AzureBlobBaseConfig.BlobContainerName = cmConfig.AzureBlobContainerCiLog

request.GcpBlobBaseConfig.CredentialFileJsonData = types.DecodeSecretKey(secretConfig.GcpBlobStorageCredentialJson)
request.GcpBlobBaseConfig.BucketName = cmConfig.CdDefaultBuildLogsBucket

return request
}
46 changes: 41 additions & 5 deletions pkg/pipeline/CdHandler.go
Original file line number Diff line number Diff line change
Expand Up @@ -120,9 +120,11 @@ type CdHandlerImpl struct {
k8sUtil *k8s.K8sUtil
workflowService WorkflowService
config *types.CdConfig
clusterService cluster.ClusterService
blobConfigStorageService BlobStorageConfigService
}

func NewCdHandlerImpl(Logger *zap.SugaredLogger, userService user.UserService, cdWorkflowRepository pipelineConfig.CdWorkflowRepository, ciLogService CiLogService, ciArtifactRepository repository.CiArtifactRepository, ciPipelineMaterialRepository pipelineConfig.CiPipelineMaterialRepository, pipelineRepository pipelineConfig.PipelineRepository, envRepository repository2.EnvironmentRepository, ciWorkflowRepository pipelineConfig.CiWorkflowRepository, helmAppService client.HelmAppService, pipelineOverrideRepository chartConfig.PipelineOverrideRepository, workflowDagExecutor WorkflowDagExecutor, appListingService app.AppListingService, appListingRepository repository.AppListingRepository, pipelineStatusTimelineRepository pipelineConfig.PipelineStatusTimelineRepository, application application.ServiceClient, argoUserService argo.ArgoUserService, deploymentEventHandler app.DeploymentEventHandler, eventClient client2.EventClient, pipelineStatusTimelineResourcesService status.PipelineStatusTimelineResourcesService, pipelineStatusSyncDetailService status.PipelineStatusSyncDetailService, pipelineStatusTimelineService status.PipelineStatusTimelineService, appService app.AppService, appStatusService app_status.AppStatusService, enforcerUtil rbac.EnforcerUtil, installedAppRepository repository3.InstalledAppRepository, installedAppVersionHistoryRepository repository3.InstalledAppVersionHistoryRepository, appRepository app2.AppRepository, resourceGroupService resourceGroup2.ResourceGroupService, imageTaggingService ImageTaggingService, k8sUtil *k8s.K8sUtil, workflowService WorkflowService) *CdHandlerImpl {
func NewCdHandlerImpl(Logger *zap.SugaredLogger, userService user.UserService, cdWorkflowRepository pipelineConfig.CdWorkflowRepository, ciLogService CiLogService, ciArtifactRepository repository.CiArtifactRepository, ciPipelineMaterialRepository pipelineConfig.CiPipelineMaterialRepository, pipelineRepository pipelineConfig.PipelineRepository, envRepository repository2.EnvironmentRepository, ciWorkflowRepository pipelineConfig.CiWorkflowRepository, helmAppService client.HelmAppService, pipelineOverrideRepository chartConfig.PipelineOverrideRepository, workflowDagExecutor WorkflowDagExecutor, appListingService app.AppListingService, appListingRepository repository.AppListingRepository, pipelineStatusTimelineRepository pipelineConfig.PipelineStatusTimelineRepository, application application.ServiceClient, argoUserService argo.ArgoUserService, deploymentEventHandler app.DeploymentEventHandler, eventClient client2.EventClient, pipelineStatusTimelineResourcesService status.PipelineStatusTimelineResourcesService, pipelineStatusSyncDetailService status.PipelineStatusSyncDetailService, pipelineStatusTimelineService status.PipelineStatusTimelineService, appService app.AppService, appStatusService app_status.AppStatusService, enforcerUtil rbac.EnforcerUtil, installedAppRepository repository3.InstalledAppRepository, installedAppVersionHistoryRepository repository3.InstalledAppVersionHistoryRepository, appRepository app2.AppRepository, resourceGroupService resourceGroup2.ResourceGroupService, imageTaggingService ImageTaggingService, k8sUtil *k8s.K8sUtil, workflowService WorkflowService, clusterService cluster.ClusterService, blobConfigStorageService BlobStorageConfigService) *CdHandlerImpl {
cdh := &CdHandlerImpl{
Logger: Logger,
userService: userService,
Expand Down Expand Up @@ -156,6 +158,8 @@ func NewCdHandlerImpl(Logger *zap.SugaredLogger, userService user.UserService, c
imageTaggingService: imageTaggingService,
k8sUtil: k8sUtil,
workflowService: workflowService,
clusterService: clusterService,
blobConfigStorageService: blobConfigStorageService,
}
config, err := types.GetCdConfig()
if err != nil {
Expand Down Expand Up @@ -914,7 +918,7 @@ func (impl *CdHandlerImpl) getWorkflowLogs(pipelineId int, cdWorkflow *pipelineC
return nil, nil, errors.New("logs-not-stored-in-repository")
} else if string(v1alpha1.NodeSucceeded) == cdWorkflow.Status || string(v1alpha1.NodeError) == cdWorkflow.Status || string(v1alpha1.NodeFailed) == cdWorkflow.Status || cdWorkflow.Status == WorkflowCancel {
impl.Logger.Debugw("pod is not live ", "err", err)
return impl.getLogsFromRepository(pipelineId, cdWorkflow)
return impl.getLogsFromRepository(pipelineId, cdWorkflow, clusterConfig, runStageInEnv)
}
impl.Logger.Errorw("err on fetch workflow logs", "err", err)
return nil, nil, err
Expand All @@ -923,8 +927,8 @@ func (impl *CdHandlerImpl) getWorkflowLogs(pipelineId int, cdWorkflow *pipelineC
return logReader, cleanUp, err
}

func (impl *CdHandlerImpl) getLogsFromRepository(pipelineId int, cdWorkflow *pipelineConfig.CdWorkflowRunner) (*bufio.Reader, func() error, error) {
impl.Logger.Debug("getting historic logs")
func (impl *CdHandlerImpl) getLogsFromRepository(pipelineId int, cdWorkflow *pipelineConfig.CdWorkflowRunner, clusterConfig *k8s.ClusterConfig, isExt bool) (*bufio.Reader, func() error, error) {
impl.Logger.Debug("getting historic logs", "pipelineId", pipelineId)

cdConfig, err := impl.cdWorkflowRepository.FindConfigByPipelineId(pipelineId)
if err != nil && !util.IsErrNoRows(err) {
Expand Down Expand Up @@ -965,6 +969,19 @@ func (impl *CdHandlerImpl) getLogsFromRepository(pipelineId int, cdWorkflow *pip
CredentialFileJsonData: impl.config.BlobStorageGcpCredentialJson,
},
}
useExternalBlobStorage := isExternalBlobStorageEnabled(isExt, impl.config.UseBlobStorageConfigInCdWorkflow)
if useExternalBlobStorage {
//fetch extClusterBlob cm and cs from k8s client, if they are present then read creds
//from them else return.
cmConfig, secretConfig, err := impl.blobConfigStorageService.FetchCmAndSecretBlobConfigFromExternalCluster(clusterConfig, cdWorkflow.Namespace)
if err != nil {
impl.Logger.Errorw("error in fetching config map and secret from external cluster", "err", err, "clusterConfig", clusterConfig)
return nil, nil, err
}
rq := &cdLogRequest
rq.SetBuildLogRequest(cmConfig, secretConfig)
}

impl.Logger.Infow("s3 log req ", "req", cdLogRequest)
oldLogsStream, cleanUp, err := impl.ciLogService.FetchLogs(impl.config.BaseLogLocationPath, cdLogRequest)
if err != nil {
Expand All @@ -974,6 +991,10 @@ func (impl *CdHandlerImpl) getLogsFromRepository(pipelineId int, cdWorkflow *pip
logReader := bufio.NewReader(oldLogsStream)
return logReader, cleanUp, err
}
func isExternalBlobStorageEnabled(isExternalRun bool, useBlobStorageConfigInCdWorkflow bool) bool {
//TODO impl.config.UseBlobStorageConfigInCdWorkflow fetches the live status, we need to check from db as well, we should put useExternalBlobStorage in db
return isExternalRun && !useBlobStorageConfigInCdWorkflow
}

func (impl *CdHandlerImpl) FetchCdWorkflowDetails(appId int, environmentId int, pipelineId int, buildId int) (types.WorkflowResponse, error) {
workflowR, err := impl.cdWorkflowRepository.FindWorkflowRunnerById(buildId)
Expand Down Expand Up @@ -1066,7 +1087,7 @@ func (impl *CdHandlerImpl) DownloadCdWorkflowArtifacts(pipelineId int, buildId i
impl.Logger.Errorw("unable to fetch ciWorkflow", "err", err)
return nil, err
}

useExternalBlobStorage := isExternalBlobStorageEnabled(wfr.IsExternalRun(), impl.config.UseBlobStorageConfigInCdWorkflow)
if !wfr.BlobStorageEnabled {
return nil, errors.New("logs-not-stored-in-repository")
}
Expand Down Expand Up @@ -1116,6 +1137,21 @@ func (impl *CdHandlerImpl) DownloadCdWorkflowArtifacts(pipelineId int, buildId i
AwsS3BaseConfig: awsS3BaseConfig,
GcpBlobBaseConfig: gcpBlobBaseConfig,
}
if useExternalBlobStorage {
clusterConfig, err := impl.clusterService.GetClusterConfigByClusterId(wfr.CdWorkflow.Pipeline.Environment.ClusterId)
if err != nil {
impl.Logger.Errorw("GetClusterConfigByClusterId, error in fetching clusterConfig", "err", err, "clusterId", wfr.CdWorkflow.Pipeline.Environment.ClusterId)
return nil, err
}
//fetch extClusterBlob cm and cs from k8s client, if they are present then read creds
//from them else return.
cmConfig, secretConfig, err := impl.blobConfigStorageService.FetchCmAndSecretBlobConfigFromExternalCluster(clusterConfig, wfr.Namespace)
if err != nil {
impl.Logger.Errorw("error in fetching config map and secret from external cluster", "err", err, "clusterConfig", clusterConfig)
return nil, err
}
request = updateRequestWithExtClusterCmAndSecret(request, cmConfig, secretConfig)
}
_, numBytes, err := blobStorageService.Get(request)
if err != nil {
impl.Logger.Errorw("error occurred while downloading file", "request", request, "error", err)
Expand Down

0 comments on commit 9e5a4ac

Please sign in to comment.