Skip to content

Commit

Permalink
feat(event-reporter): multi-sourced apps - resource git details repor…
Browse files Browse the repository at this point in the history
…ting (#352)

* event-reporter: created dedicated func resolveApplicationVersions

* event-reporter: added new types for variables grouping

* event-reporter: removed redundant code comment

* event-reporter(refactoring): params grouping, added new type ArgoTrackingMetadata to pass data between methods

* event-reporter(refactoring): params grouping, added new type ReportedEntityParentApp to pass data between methods

* event-reporter(refactoring): params grouping, added new type ReportedResource to pass data between methods, created new methods to group logic inside getResourceEventPayload

* event-reporter(refactoring): fixed nil pointer issues after refactoring

* event-reporter(refactoring): fixed lint issue

* event-reporter(refactoring): fixed lint issue

* event-reporter: added agroDb to ApplicationEventReporter

* event-reporter: ObjectSource message proto extended to report operationSyncRevisions, destServer, destName, appMultiSourced

* event-reporter / utils: added func GetOperationRevisions

* event-reporter(non-grpc app client): added support of new query params SourcePositions, Revisions for GetManifests request

* event-reporter(getDesiredManifests): updated logic to retrieve appVersion for multi-sourced applications based on sync result revisions

* event-reporter(resource source payload): added new field OperationSyncRevisions

* event-reporter(resource source payload): added new fields DestName, DestServer

* event-reporter(resource source payload): added new field AppMultiSourced

* event-reporter(resource source payload): added AppSourceIdx with source index to which this resource belongs. Also updated logic of retrieving correct git commit info based on AppSourceIdx

* event-reporter: updated changelog

* event-reporter: fixed lint issues

* event-reporter: changes after pr review

* fix lint issue

* reporter: fix lint issue

* reporter: fix lint issue

* updated changelog record

* event-reporter: refactored fields inside ObjectSource

* event-reporter: added runtime version reporting with each event
  • Loading branch information
oleksandr-codefresh authored Dec 3, 2024
1 parent bfe424f commit a26fd08
Show file tree
Hide file tree
Showing 21 changed files with 1,295 additions and 358 deletions.
8 changes: 8 additions & 0 deletions assets/swagger.json
Original file line number Diff line number Diff line change
Expand Up @@ -5294,6 +5294,14 @@
"sourceType": {
"type": "string"
},
"sourcesManifestsStartingIdx": {
"type": "array",
"title": "for multisourced apps will be [0,12,20], so this means that 0-11 - from first app source, 12-19 from second one, 20-x - third one",
"items": {
"type": "integer",
"format": "int32"
}
},
"verifyResult": {
"type": "string",
"title": "Raw response of git verify-commit operation (always the empty string for Helm)"
Expand Down
3 changes: 2 additions & 1 deletion changelog/CHANGELOG-2.12.3-2024.11.14-d7a4c4e7e.md
Original file line number Diff line number Diff line change
@@ -1,2 +1,3 @@
### Changes
- fix(event-reporter): fixed issue when if managed resource in degraded state and all of it child nodes in non-degraded state no error reported
- fix(event-reporter): fixed issue when if managed resource in degraded state and all of it child nodes in non-degraded state no error reported
- fix(repo-server): fixed repeated resources generation for ApplicationSourceTypeDirectory
4 changes: 2 additions & 2 deletions changelog/CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,2 +1,2 @@
### Changes
- fix(event-reporter): fixed issue when if managed resource in degraded state and all of it child nodes in non-degraded state no error reported
### Features
- feat(event-reporter): multisourced apps support improvements: reporting syncOperationRevisions, detecting correct resource sourceIdx, reporting correct git commit info
11 changes: 7 additions & 4 deletions cmd/event-reporter-server/commands/event_reporter_server.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ func NewCommand() *cobra.Command {
codefreshUrl string
codefreshToken string
shardingAlgorithm string
runtimeVersion string
rootpath string
useGrpc bool

Expand Down Expand Up @@ -179,10 +180,11 @@ func NewCommand() *cobra.Command {
ApplicationNamespaces: applicationNamespaces,
ApplicationServiceClient: getApplicationClient(useGrpc, applicationServerAddress, argocdToken, rootpath),
CodefreshConfig: &codefresh.CodefreshConfig{
BaseURL: codefreshUrl,
AuthToken: codefreshToken,
TlsInsecure: codefreshTlsInsecure,
CaCertPath: codefreshTlsCertPath,
BaseURL: codefreshUrl,
AuthToken: codefreshToken,
TlsInsecure: codefreshTlsInsecure,
CaCertPath: codefreshTlsCertPath,
RuntimeVersion: runtimeVersion,
},
RateLimiterOpts: &reporter.RateLimiterOpts{
Enabled: rateLimiterEnabled,
Expand Down Expand Up @@ -236,6 +238,7 @@ func NewCommand() *cobra.Command {
command.Flags().StringVar(&codefreshUrl, "codefresh-url", env.StringFromEnv("CODEFRESH_URL", "https://g.codefresh.io"), "Codefresh API url")
command.Flags().StringVar(&codefreshToken, "codefresh-token", env.StringFromEnv("CODEFRESH_TOKEN", ""), "Codefresh token")
command.Flags().StringVar(&shardingAlgorithm, "sharding-method", env.StringFromEnv(common.EnvEventReporterShardingAlgorithm, common.DefaultEventReporterShardingAlgorithm), "Enables choice of sharding method. Supported sharding methods are : [legacy] ")
command.Flags().StringVar(&runtimeVersion, "codefresh-runtime-version", env.StringFromEnv("CODEFRESH_RUNTIME_VERSION", ""), "Codefresh runtime version to be reported with each event to platform")
command.Flags().StringSliceVar(&applicationNamespaces, "application-namespaces", env.StringsFromEnv("ARGOCD_APPLICATION_NAMESPACES", []string{}, ","), "List of additional namespaces where application resources can be managed in")
command.Flags().BoolVar(&useGrpc, "grpc", env.ParseBoolFromEnv("USE_GRPC", false), "Use grpc for interact with argocd server")
command.Flags().BoolVar(&rateLimiterEnabled, "rate-limiter-enabled", env.ParseBoolFromEnv("RATE_LIMITER_ENABLED", false), "Use rate limiter for prevent queue to be overflowed")
Expand Down
10 changes: 10 additions & 0 deletions event_reporter/application/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,16 @@ func (c *httpApplicationClient) GetManifests(ctx context.Context, in *appclient.
if in.Revision != nil {
params = fmt.Sprintf("%s&revision=%s", params, *in.Revision)
}
if in.SourcePositions != nil && len(in.SourcePositions) > 0 {
for _, sourcePosition := range in.SourcePositions {
params = fmt.Sprintf("%s&sourcePositions=%d", params, sourcePosition)
}
}
if in.Revisions != nil && len(in.Revisions) > 0 {
for _, revision := range in.Revisions {
params = fmt.Sprintf("%s&revisions=%s", params, revision)
}
}
url := fmt.Sprintf("%s/api/v1/applications/%s/manifests%s", c.baseUrl, *in.Name, params)

manifest := &repoapiclient.ManifestResponse{}
Expand Down
6 changes: 4 additions & 2 deletions event_reporter/controller/controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,8 @@ import (
"strings"
"time"

"github.com/argoproj/argo-cd/v2/util/db"

appclient "github.com/argoproj/argo-cd/v2/event_reporter/application"

log "github.com/sirupsen/logrus"
Expand Down Expand Up @@ -43,15 +45,15 @@ type eventReporterController struct {
metricsServer *metrics.MetricsServer
}

func NewEventReporterController(appInformer cache.SharedIndexInformer, cache *servercache.Cache, settingsMgr *settings.SettingsManager, applicationServiceClient appclient.ApplicationClient, appLister applisters.ApplicationLister, codefreshConfig *codefresh.CodefreshConfig, metricsServer *metrics.MetricsServer, featureManager *reporter.FeatureManager, rateLimiterOpts *reporter.RateLimiterOpts) EventReporterController {
func NewEventReporterController(appInformer cache.SharedIndexInformer, cache *servercache.Cache, settingsMgr *settings.SettingsManager, applicationServiceClient appclient.ApplicationClient, appLister applisters.ApplicationLister, codefreshConfig *codefresh.CodefreshConfig, metricsServer *metrics.MetricsServer, featureManager *reporter.FeatureManager, rateLimiterOpts *reporter.RateLimiterOpts, db db.ArgoDB) EventReporterController {
appBroadcaster := reporter.NewBroadcaster(featureManager, metricsServer, rateLimiterOpts)
_, err := appInformer.AddEventHandler(appBroadcaster)
if err != nil {
log.Error(err)
}
return &eventReporterController{
appBroadcaster: appBroadcaster,
applicationEventReporter: reporter.NewApplicationEventReporter(cache, applicationServiceClient, appLister, codefreshConfig, metricsServer),
applicationEventReporter: reporter.NewApplicationEventReporter(cache, applicationServiceClient, appLister, codefreshConfig, metricsServer, db),
cache: cache,
settingsMgr: settingsMgr,
applicationServiceClient: applicationServiceClient,
Expand Down
3 changes: 3 additions & 0 deletions event_reporter/reporter/app_revision_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ func TestGetRevisionsDetails(t *testing.T) {
newAppLister(),
appServiceClient,
&metrics.MetricsServer{},
fakeArgoDb(),
}

Check failure on line 58 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Lint Go code

too few values in struct literal of type applicationEventReporter

Check failure on line 58 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests with -race for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 58 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests with -race for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 58 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 58 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests for Go packages

too few values in struct literal of type applicationEventReporter

result, _ := reporter.getRevisionsDetails(context.Background(), &app, []string{expectedRevision})
Expand Down Expand Up @@ -118,6 +119,7 @@ func TestGetRevisionsDetails(t *testing.T) {
newAppLister(),
appServiceClient,
&metrics.MetricsServer{},
fakeArgoDb(),
}

Check failure on line 123 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Lint Go code

too few values in struct literal of type applicationEventReporter

Check failure on line 123 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests with -race for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 123 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests with -race for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 123 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 123 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests for Go packages

too few values in struct literal of type applicationEventReporter

result, _ := reporter.getRevisionsDetails(context.Background(), &app, []string{expectedRevision1, expectedRevision2})
Expand Down Expand Up @@ -159,6 +161,7 @@ func TestGetRevisionsDetails(t *testing.T) {
newAppLister(),
appServiceClient,
&metrics.MetricsServer{},
fakeArgoDb(),
}

Check failure on line 165 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Lint Go code

too few values in struct literal of type applicationEventReporter

Check failure on line 165 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests with -race for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 165 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests with -race for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 165 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests for Go packages

too few values in struct literal of type applicationEventReporter

Check failure on line 165 in event_reporter/reporter/app_revision_test.go

View workflow job for this annotation

GitHub Actions / Run unit tests for Go packages

too few values in struct literal of type applicationEventReporter

result, _ := reporter.getRevisionsDetails(context.Background(), &app, []string{expectedRevision})
Expand Down
116 changes: 91 additions & 25 deletions event_reporter/reporter/application_event_reporter.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,12 @@ import (
"strings"
"time"

"github.com/argoproj/argo-cd/v2/util/db"

"github.com/argoproj/argo-cd/v2/event_reporter/utils"

argoutils "github.com/argoproj/argo-cd/v2/util/argo"

"github.com/argoproj/argo-cd/v2/reposerver/apiclient"

argocommon "github.com/argoproj/argo-cd/v2/common"
Expand Down Expand Up @@ -40,6 +44,8 @@ type applicationEventReporter struct {
appLister applisters.ApplicationLister
applicationServiceClient appclient.ApplicationClient
metricsServer *metrics.MetricsServer
db db.ArgoDB
runtimeVersion string
}

type ApplicationEventReporter interface {
Expand All @@ -53,13 +59,15 @@ type ApplicationEventReporter interface {
ShouldSendApplicationEvent(ae *appv1.ApplicationWatchEvent) (shouldSend bool, syncStatusChanged bool)
}

func NewApplicationEventReporter(cache *servercache.Cache, applicationServiceClient appclient.ApplicationClient, appLister applisters.ApplicationLister, codefreshConfig *codefresh.CodefreshConfig, metricsServer *metrics.MetricsServer) ApplicationEventReporter {
func NewApplicationEventReporter(cache *servercache.Cache, applicationServiceClient appclient.ApplicationClient, appLister applisters.ApplicationLister, codefreshConfig *codefresh.CodefreshConfig, metricsServer *metrics.MetricsServer, db db.ArgoDB) ApplicationEventReporter {
return &applicationEventReporter{
cache: cache,
applicationServiceClient: applicationServiceClient,
codefreshClient: codefresh.NewCodefreshClient(codefreshConfig),
appLister: appLister,
metricsServer: metricsServer,
db: db,
runtimeVersion: codefreshConfig.RuntimeVersion,
}
}

Expand Down Expand Up @@ -87,15 +95,28 @@ func (s *applicationEventReporter) shouldSendResourceEvent(a *appv1.Application,
return true
}

func (r *applicationEventReporter) getDesiredManifests(ctx context.Context, a *appv1.Application, revision *string, logCtx *log.Entry) (*apiclient.ManifestResponse, bool) {
func (r *applicationEventReporter) getDesiredManifests(
ctx context.Context,
logCtx *log.Entry,
a *appv1.Application,
revision *string,
sourcePositions *[]int64,
revisions *[]string,
) (*apiclient.ManifestResponse, bool) {
// get the desired state manifests of the application
project := a.Spec.GetProject()
desiredManifests, err := r.applicationServiceClient.GetManifests(ctx, &application.ApplicationManifestQuery{
query := application.ApplicationManifestQuery{
Name: &a.Name,
AppNamespace: &a.Namespace,
Revision: revision,
Project: &project,
})
}
if sourcePositions != nil && query.Revisions != nil {
query.SourcePositions = *sourcePositions
query.Revisions = *revisions
}

desiredManifests, err := r.applicationServiceClient.GetManifests(ctx, &query)
if err != nil {
// if it's manifest generation error we need to still report the actual state
// of the resources, but since we can't get the desired state, we will report
Expand Down Expand Up @@ -137,7 +158,7 @@ func (s *applicationEventReporter) StreamApplicationEvents(

logCtx.Info("getting desired manifests")

desiredManifests, manifestGenErr := s.getDesiredManifests(ctx, a, nil, logCtx)
desiredManifests, manifestGenErr := s.getDesiredManifests(ctx, logCtx, a, nil, nil, nil)

applicationVersions := s.resolveApplicationVersions(ctx, a, logCtx)

Expand All @@ -158,17 +179,21 @@ func (s *applicationEventReporter) StreamApplicationEvents(
rs := utils.GetAppAsResource(a)
utils.SetHealthStatusIfMissing(rs)

parentDesiredManifests, manifestGenErr := s.getDesiredManifests(ctx, parentApplicationEntity, nil, logCtx)
parentDesiredManifests, manifestGenErr := s.getDesiredManifests(ctx, logCtx, parentApplicationEntity, nil, nil, nil)

parentAppSyncRevisionsMetadata, err := s.getApplicationRevisionsMetadata(ctx, logCtx, parentApplicationEntity)
if err != nil {
logCtx.WithError(err).Warn("failed to get parent application's revision metadata, resuming")
}

validatedDestination := parentApplicationEntity.Spec.Destination.DeepCopy()
_ = argoutils.ValidateDestination(ctx, validatedDestination, s.db) // resolves server field if missing

err = s.processResource(ctx, *rs, logCtx, eventProcessingStartedAt, parentDesiredManifests, manifestGenErr, a, applicationVersions, &ReportedEntityParentApp{
app: parentApplicationEntity,
appTree: appTree,
revisionsMetadata: parentAppSyncRevisionsMetadata,
app: parentApplicationEntity,
appTree: appTree,
revisionsMetadata: parentAppSyncRevisionsMetadata,
validatedDestination: validatedDestination,
}, argoTrackingMetadata)
if err != nil {
s.metricsServer.IncErroredEventsCounter(metrics.MetricChildAppEventType, metrics.MetricEventUnknownErrorType, a.Name)
Expand All @@ -178,7 +203,7 @@ func (s *applicationEventReporter) StreamApplicationEvents(
} else {
// will get here only for root applications (not managed as a resource by another application)
logCtx.Info("processing as root application")
appEvent, err := s.getApplicationEventPayload(ctx, a, appTree, eventProcessingStartedAt, applicationVersions, argoTrackingMetadata)
appEvent, err := s.getApplicationEventPayload(ctx, a, appTree, eventProcessingStartedAt, applicationVersions, argoTrackingMetadata, s.runtimeVersion)
if err != nil {
s.metricsServer.IncErroredEventsCounter(metrics.MetricParentAppEventType, metrics.MetricEventGetPayloadErrorType, a.Name)
return fmt.Errorf("failed to get application event: %w", err)
Expand All @@ -197,6 +222,9 @@ func (s *applicationEventReporter) StreamApplicationEvents(
s.metricsServer.ObserveEventProcessingDurationHistogramDuration(a.Name, metrics.MetricParentAppEventType, metricTimer.Duration())
}

validatedDestination := a.Spec.Destination.DeepCopy()
_ = argoutils.ValidateDestination(ctx, validatedDestination, s.db) // resolves server field if missing

revisionsMetadata, _ := s.getApplicationRevisionsMetadata(ctx, logCtx, a)
// for each resource in the application get desired and actual state,
// then stream the event
Expand All @@ -210,9 +238,10 @@ func (s *applicationEventReporter) StreamApplicationEvents(
continue
}
err := s.processResource(ctx, rs, logCtx, eventProcessingStartedAt, desiredManifests, manifestGenErr, nil, nil, &ReportedEntityParentApp{
app: a,
appTree: appTree,
revisionsMetadata: revisionsMetadata,
app: a,
appTree: appTree,
revisionsMetadata: revisionsMetadata,
validatedDestination: validatedDestination,
}, argoTrackingMetadata)
if err != nil {
s.metricsServer.IncErroredEventsCounter(metrics.MetricResourceEventType, metrics.MetricEventUnknownErrorType, a.Name)
Expand All @@ -222,13 +251,30 @@ func (s *applicationEventReporter) StreamApplicationEvents(
return nil
}

// returns appVersion from first non-ref source for multisourced apps
func (s *applicationEventReporter) resolveApplicationVersions(ctx context.Context, a *appv1.Application, logCtx *log.Entry) *apiclient.ApplicationVersions {
syncRevision := utils.GetOperationStateRevision(a)
if syncRevision == nil {
if a.Spec.HasMultipleSources() {
syncResultRevisions := utils.GetOperationSyncResultRevisions(a)
if syncResultRevisions == nil {
return nil
}

var sourcePositions []int64
for i := 0; i < len(*syncResultRevisions); i++ {
sourcePositions = append(sourcePositions, int64(i+1))
}

syncManifests, _ := s.getDesiredManifests(ctx, logCtx, a, nil, &sourcePositions, syncResultRevisions)
return syncManifests.GetApplicationVersions()
}

syncResultRevision := utils.GetOperationSyncResultRevision(a)

if syncResultRevision == nil {
return nil
}

syncManifests, _ := s.getDesiredManifests(ctx, a, syncRevision, logCtx)
syncManifests, _ := s.getDesiredManifests(ctx, logCtx, a, syncResultRevision, nil, nil)
return syncManifests.GetApplicationVersions()
}

Expand Down Expand Up @@ -279,7 +325,7 @@ func (s *applicationEventReporter) processResource(
})

// get resource desired state
desiredState := getResourceDesiredState(&rs, desiredManifests, logCtx)
desiredState, appSourceIdx := getResourceDesiredState(&rs, desiredManifests, logCtx)

actualState, err := s.getResourceActualState(ctx, logCtx, metricsEventType, rs, reportedEntityParentApp.app, originalApplication)
if err != nil {
Expand All @@ -304,18 +350,22 @@ func (s *applicationEventReporter) processResource(
actualState: actualState,
desiredState: desiredState,
manifestGenErr: manifestGenErr,
appSourceIdx: appSourceIdx,
rsAsAppInfo: &ReportedResourceAsApp{
app: originalApplication,
revisionsMetadata: originalAppRevisionMetadata,
applicationVersions: applicationVersions,
},
},
&ReportedEntityParentApp{
app: parentApplicationToReport,
appTree: reportedEntityParentApp.appTree,
revisionsMetadata: revisionMetadataToReport,
app: parentApplicationToReport,
appTree: reportedEntityParentApp.appTree,
revisionsMetadata: revisionMetadataToReport,
validatedDestination: reportedEntityParentApp.validatedDestination,
desiredManifests: reportedEntityParentApp.desiredManifests,
},
argoTrackingMetadata,
s.runtimeVersion,
)
if err != nil {
s.metricsServer.IncErroredEventsCounter(metricsEventType, metrics.MetricEventGetPayloadErrorType, reportedEntityParentApp.app.Name)
Expand Down Expand Up @@ -474,11 +524,11 @@ func applicationMetadataChanged(ae *appv1.ApplicationWatchEvent, cachedApp *appv
return !reflect.DeepEqual(newEventAppMeta, cachedAppMeta)
}

func getResourceDesiredState(rs *appv1.ResourceStatus, ds *apiclient.ManifestResponse, logger *log.Entry) *apiclient.Manifest {
func getResourceDesiredState(rs *appv1.ResourceStatus, ds *apiclient.ManifestResponse, logger *log.Entry) (manifest *apiclient.Manifest, sourceIdx int32) {
if ds == nil {
return &apiclient.Manifest{}
return &apiclient.Manifest{}, 0
}
for _, m := range ds.Manifests {
for idx, m := range ds.Manifests {
u, err := appv1.UnmarshalToUnstructured(m.CompiledManifest)
if err != nil {
logger.WithError(err).Warnf("failed to unmarshal compiled manifest")
Expand All @@ -498,11 +548,27 @@ func getResourceDesiredState(rs *appv1.ResourceStatus, ds *apiclient.ManifestRes
m.RawManifest = m.CompiledManifest
}

return m
return m, getResourceSourceIdxFromManifestResponse(idx, ds)
}
}

// no desired state for resource
// it's probably deleted from git
return &apiclient.Manifest{}
return &apiclient.Manifest{}, 0
}

func getResourceSourceIdxFromManifestResponse(rsIdx int, ds *apiclient.ManifestResponse) int32 {
if ds.SourcesManifestsStartingIdx == nil {
return -1
}

sourceIdx := int32(-1)

for currentSourceIdx, sourceStartingIdx := range ds.SourcesManifestsStartingIdx {
if int32(rsIdx) >= sourceStartingIdx {
sourceIdx = int32(currentSourceIdx)
}
}

return sourceIdx
}
Loading

0 comments on commit a26fd08

Please sign in to comment.