diff --git a/assets/swagger.json b/assets/swagger.json index c155555315d974..220fe02c5d31b4 100644 --- a/assets/swagger.json +++ b/assets/swagger.json @@ -5839,6 +5839,37 @@ } } }, + "v1alpha1ApplicationDependency": { + "type": "object", + "title": "ApplicationDependency defines", + "properties": { + "blockOnEmpty": { + "type": "boolean", + "title": "BlockOnEmpty defines whether to block sync when the list of applications determined by the selector is empty" + }, + "refreshDependencies": { + "type": "boolean", + "title": "RefreshDependencies defines whether all dependencies should be refreshed before starting a sync" + }, + "selectors": { + "type": "array", + "title": "Selectors defines conditions for matching application's dependencies", + "items": { + "$ref": "#/definitions/v1alpha1ApplicationSelector" + } + }, + "syncDelay": { + "type": "integer", + "format": "int64", + "title": "SyncDelay specifies the duration in seconds to wait before starting to sync when dependencies are defined" + }, + "timeout": { + "type": "integer", + "format": "int64", + "title": "Timeout defines the maximum duration in seconds to wait on dependencies before the sync fails" + } + } + }, "v1alpha1ApplicationDestination": { "type": "object", "title": "ApplicationDestination holds information about the application's destination", @@ -5906,6 +5937,22 @@ } } }, + "v1alpha1ApplicationSelector": { + "type": "object", + "title": "ApplicationSelector specifies which applications this Application depends on", + "properties": { + "labelSelector": { + "$ref": "#/definitions/v1LabelSelector" + }, + "namePattern": { + "type": "array", + "title": "NamePattern selects applications by matching their names", + "items": { + "type": "string" + } + } + } + }, "v1alpha1ApplicationSet": { "type": "object", "title": "ApplicationSet is a set of Application resources\n+genclient\n+genclient:noStatus\n+k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object\n+kubebuilder:resource:path=applicationsets,shortName=appset;appsets\n+kubebuilder:subresource:status", @@ -6512,6 +6559,9 @@ "description": "ApplicationSpec represents desired application state. Contains link to repository with application definition and additional parameters link definition revision.", "type": "object", "properties": { + "dependsOn": { + "$ref": "#/definitions/v1alpha1ApplicationDependency" + }, "destination": { "$ref": "#/definitions/v1alpha1ApplicationDestination" }, @@ -7518,6 +7568,10 @@ "type": "object", "title": "OperationState contains information about state of a running operation", "properties": { + "blockedOnEmpty": { + "type": "boolean", + "title": "BlockedOnEmpty is true when the application is waiting for any dependency to be created" + }, "finishedAt": { "$ref": "#/definitions/v1Time" }, @@ -7542,6 +7596,13 @@ }, "syncResult": { "$ref": "#/definitions/v1alpha1SyncOperationResult" + }, + "waitingFor": { + "type": "array", + "title": "WaitingFor specifies a list of applications that this operation is waiting for", + "items": { + "$ref": "#/definitions/v1alpha1SyncDependency" + } } } }, @@ -8847,6 +8908,20 @@ } } }, + "v1alpha1SyncDependency": { + "type": "object", + "properties": { + "applicationName": { + "type": "string" + }, + "applicationNamespace": { + "type": "string" + }, + "refreshedAt": { + "$ref": "#/definitions/v1Time" + } + } + }, "v1alpha1SyncOperation": { "description": "SyncOperation contains details about a sync operation.", "type": "object", diff --git a/cmd/argocd/commands/admin/app.go b/cmd/argocd/commands/admin/app.go index 096c92f9feb01b..f333a3e5807bc9 100644 --- a/cmd/argocd/commands/admin/app.go +++ b/cmd/argocd/commands/admin/app.go @@ -387,7 +387,7 @@ func reconcileApplications( ) appStateManager := controller.NewAppStateManager( - argoDB, appClientset, repoServerClient, namespace, kubeutil.NewKubectl(), settingsMgr, stateCache, projInformer, server, cache, time.Second, argo.NewResourceTracking(), false, 0, serverSideDiff) + argoDB, appClientset, repoServerClient, namespace, kubeutil.NewKubectl(), settingsMgr, stateCache, projInformer, appLister, server, cache, time.Second, argo.NewResourceTracking(), false, 0, serverSideDiff) appsList, err := appClientset.ArgoprojV1alpha1().Applications(namespace).List(ctx, v1.ListOptions{LabelSelector: selector}) if err != nil { diff --git a/controller/appcontroller.go b/controller/appcontroller.go index 9d89b6e6b37d65..72693ae7237217 100644 --- a/controller/appcontroller.go +++ b/controller/appcontroller.go @@ -75,6 +75,9 @@ const ( orphanedIndex = "orphaned" ) +// refreshAfterForDependencies defines the interval for refresh while waiting for dependency application +var refreshAfterForDependencies time.Duration = 2 * time.Second + type CompareWith int const ( @@ -277,7 +280,7 @@ func NewApplicationController( } } stateCache := statecache.NewLiveStateCache(db, appInformer, ctrl.settingsMgr, kubectl, ctrl.metricsServer, ctrl.handleObjectUpdated, clusterSharding, argo.NewResourceTracking()) - appStateManager := NewAppStateManager(db, applicationClientset, repoClientset, namespace, kubectl, ctrl.settingsMgr, stateCache, projInformer, ctrl.metricsServer, argoCache, ctrl.statusRefreshTimeout, argo.NewResourceTracking(), persistResourceHealth, repoErrorGracePeriod, serverSideDiff) + appStateManager := NewAppStateManager(db, applicationClientset, repoClientset, namespace, kubectl, ctrl.settingsMgr, stateCache, projInformer, appLister, ctrl.metricsServer, argoCache, ctrl.statusRefreshTimeout, argo.NewResourceTracking(), persistResourceHealth, repoErrorGracePeriod, serverSideDiff) ctrl.appInformer = appInformer ctrl.appLister = appLister ctrl.projInformer = projInformer @@ -1244,6 +1247,42 @@ func (ctrl *ApplicationController) setAppCondition(app *appv1.Application, condi } } +func (ctrl *ApplicationController) refreshDependencies(app *appv1.Application, state *appv1.OperationState) { + logCtx := log.WithField("application", app.QualifiedName()) + for i, dep := range state.WaitingFor { + + a, err := ctrl.appLister.Applications(dep.ApplicationNamespace).Get(dep.ApplicationName) + if err != nil { + logCtx.Errorf("Could not retrieve dependency %s: %v", dep.ApplicationName, err) + continue + } + + if app.Spec.GetProject() != a.Spec.GetProject() { + logCtx.Infof("Not refreshing dependency app %s because AppProject does not match: is %s, must be %s", a.QualifiedName(), a.Spec.GetProject(), app.Spec.GetProject()) + continue + } + + // Either an operation has been requested or is already in progress. + // We do not request a new one. + if a.Operation != nil || (a.Status.OperationState != nil && a.Status.OperationState.Phase == synccommon.OperationRunning) { + logCtx.Debugf("Dependency %s: Operation already in progress, not going to trigger another one", dep.QualifiedName()) + continue + } + + if dep.RefreshedAt.IsZero() { + logCtx.Infof("Requesting refresh for app %s to check dependencies", dep.QualifiedName()) + ctrl.requestAppRefresh(dep.QualifiedName(), CompareWithLatest.Pointer(), nil) + dep.RefreshedAt = &metav1.Time{Time: time.Now()} + } else { + logCtx.Debugf("Already requested a refresh for dependency %s", dep.QualifiedName()) + } + + state.WaitingFor[i] = dep + } + + ctrl.setOperationState(app, state) +} + func (ctrl *ApplicationController) processRequestedAppOperation(app *appv1.Application) { logCtx := log.WithField("application", app.QualifiedName()) var state *appv1.OperationState @@ -1300,6 +1339,9 @@ func (ctrl *ApplicationController) processRequestedAppOperation(app *appv1.Appli state.Message = err.Error() } else { ctrl.appStateManager.SyncAppState(app, state) + if state != nil && len(state.WaitingFor) > 0 { + ctrl.refreshDependencies(app, state) + } } // Check whether application is allowed to use project @@ -1327,6 +1369,24 @@ func (ctrl *ApplicationController) processRequestedAppOperation(app *appv1.Appli // SyncAppState will operate in a Terminating phase, allowing the worker to perform // cleanup (e.g. delete jobs, workflows, etc...) } + + // If the operation is in progress, and we are blocked on waiting + // for dependencies, immediately refresh the application so that + // we enter a refresh cycle. + if freshApp.Status.OperationState != nil && freshApp.Status.OperationState.Phase == synccommon.OperationRunning { + refresh := false + if freshApp.Status.OperationState.BlockedOnEmpty { + logCtx.Infof("Requesting app refresh for blocked app") + refresh = true + } else if len(freshApp.Status.OperationState.WaitingFor) > 0 { + logCtx.Infof("Requesting app refresh because waiting on dependencies") + refresh = true + } + if refresh { + ctrl.appRefreshQueue.AddAfter(fmt.Sprintf("%s/%d", freshApp.QualifiedName(), ComparisonWithNothing), refreshAfterForDependencies) + ctrl.appOperationQueue.AddAfter(freshApp.QualifiedName(), refreshAfterForDependencies) + } + } } } else if state.Phase == synccommon.OperationFailed || state.Phase == synccommon.OperationError { if !terminating && (state.RetryCount < state.Operation.Retry.Limit || state.Operation.Retry.Limit < 0) { @@ -1389,11 +1449,20 @@ func (ctrl *ApplicationController) setOperationState(app *appv1.Application, sta logCtx.Errorf("error marshaling json: %v", err) return } - if app.Status.OperationState != nil && app.Status.OperationState.FinishedAt != nil && state.FinishedAt == nil { - patchJSON, err = jsonpatch.MergeMergePatches(patchJSON, []byte(`{"status": {"operationState": {"finishedAt": null}}}`)) - if err != nil { - logCtx.Errorf("error merging operation state patch: %v", err) - return + if app.Status.OperationState != nil { + if app.Status.OperationState.FinishedAt != nil && state.FinishedAt == nil { + patchJSON, err = jsonpatch.MergeMergePatches(patchJSON, []byte(`{"status": {"operationState": {"finishedAt": null}}}`)) + if err != nil { + logCtx.Errorf("error merging operation state patch: %v", err) + return + } + } + if len(app.Status.OperationState.WaitingFor) > 0 && len(state.WaitingFor) == 0 { + patchJSON, err = jsonpatch.MergeMergePatches(patchJSON, []byte(`{"status": {"operationState": {"waitingFor": null}}}`)) + if err != nil { + logCtx.Errorf("error merging operation state patch: %v", err) + return + } } } @@ -1660,6 +1729,53 @@ func currentSourceEqualsSyncedSource(app *appv1.Application) bool { return app.Spec.Source.Equals(&app.Status.Sync.ComparedTo.Source) } +// shouldRefreshForDependency returns whether an app should be refreshed for a +// change in the state of one of its dependencies. +func (ctrl *ApplicationController) shouldRefreshForDependency(app *appv1.Application) bool { + logCtx := log.WithField("application", app.QualifiedName()) + + // No need to refresh when we're not waiting for dependencies to sync + if app.Spec.DependsOn == nil || !app.IsWaiting() { + logCtx.Debugf("not waiting for dependencies, skipping refresh") + return false + } + + // If we're waiting for any dependency to be created, we need refresh to + // see whether they're created by now. + if app.Status.OperationState.BlockedOnEmpty { + return true + } + + needRefresh := false + numWaiting := len(app.Status.OperationState.WaitingFor) + numRemoved := 0 + for _, syncDep := range app.Status.OperationState.WaitingFor { + depApp, err := ctrl.appLister.Applications(syncDep.ApplicationNamespace).Get(syncDep.ApplicationName) + if err != nil { + // The application could have been deleted meanwhile, it's not any + // of the dependencies anymore. + if apierr.IsNotFound(err) { + logCtx.Infof("Dependency application %s has been removed", syncDep.QualifiedName()) + numRemoved += 1 + continue + } else { + logCtx.Warnf("Error getting sync dependency %s: %v", syncDep.QualifiedName(), err) + return false + } + } + + if depApp.Status.Health.Status == health.HealthStatusHealthy && depApp.Status.Sync.Status == appv1.SyncStatusCodeSynced { + logCtx.Debugf("Sync dependency %s has become healthy and synced", syncDep.QualifiedName()) + needRefresh = true + } + } + + // We need to refresh when either one of our dependencies changed state to + // healthy, or when all dependencies we were waiting for have been + // removed meanwhile. + return needRefresh || (numWaiting > 0 && numRemoved == numWaiting) +} + // needRefreshAppStatus answers if application status needs to be refreshed. // Returns true if application never been compared, has changed or comparison result has expired. // Additionally, it returns whether full refresh was requested or not. @@ -1707,6 +1823,9 @@ func (ctrl *ApplicationController) needRefreshAppStatus(app *appv1.Application, } else if requested, level := ctrl.isRefreshRequested(app.QualifiedName()); requested { compareWith = level reason = "controller refresh requested" + } else if ctrl.shouldRefreshForDependency(app) { + compareWith = ComparisonWithNothing + reason = "refreshing for change in dependencies' status" } } diff --git a/controller/appcontroller_test.go b/controller/appcontroller_test.go index 33a29bc5ca3f89..cb5227b415ada3 100644 --- a/controller/appcontroller_test.go +++ b/controller/appcontroller_test.go @@ -14,6 +14,7 @@ import ( "k8s.io/client-go/rest" clustercache "github.com/argoproj/gitops-engine/pkg/cache" + "github.com/argoproj/gitops-engine/pkg/health" "github.com/argoproj/argo-cd/v2/common" statecache "github.com/argoproj/argo-cd/v2/controller/cache" @@ -1910,3 +1911,134 @@ func TestAddControllerNamespace(t *testing.T) { assert.Equal(t, test.FakeArgoCDNamespace, updatedApp.Status.ControllerNamespace) }) } + +func Test_shouldRefreshForDependencies(t *testing.T) { + t.Run("No refresh, because neither dependencies defined nor waiting", func(t *testing.T) { + app := newFakeApp() + ctrl := newFakeController(&fakeData{ + apps: []runtime.Object{app}, + }) + assert.False(t, ctrl.shouldRefreshForDependency(app)) + }) + t.Run("No refresh, because not waiting for any dependency", func(t *testing.T) { + app := newFakeApp() + app.Spec.DependsOn = &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + {NamePattern: []string{"*"}}, + }, + } + app.Status.OperationState = &v1alpha1.OperationState{ + Phase: synccommon.OperationRunning, + WaitingFor: []v1alpha1.SyncDependency{}, + } + ctrl := newFakeController(&fakeData{ + apps: []runtime.Object{app}, + }) + assert.False(t, ctrl.shouldRefreshForDependency(app)) + }) + + t.Run("No refresh, because operation is not running", func(t *testing.T) { + app := newFakeApp() + app.Spec.DependsOn = &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + {NamePattern: []string{"*"}}, + }, + } + app.Status.OperationState = &v1alpha1.OperationState{ + Phase: synccommon.OperationFailed, + BlockedOnEmpty: true, + WaitingFor: []v1alpha1.SyncDependency{}, + } + ctrl := newFakeController(&fakeData{ + apps: []runtime.Object{app}, + }) + assert.False(t, ctrl.shouldRefreshForDependency(app)) + }) + + t.Run("Refresh because blocking progress for dependencies", func(t *testing.T) { + app := newFakeApp() + app.Spec.DependsOn = &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + {NamePattern: []string{"*"}}, + }, + } + app.Status.OperationState = &v1alpha1.OperationState{ + Phase: synccommon.OperationRunning, + BlockedOnEmpty: true, + } + ctrl := newFakeController(&fakeData{ + apps: []runtime.Object{app}, + }) + assert.True(t, ctrl.shouldRefreshForDependency(app)) + }) + + t.Run("No refresh because a dependency we are waiting for is not ready", func(t *testing.T) { + app := newFakeApp() + dep := newFakeApp() + dep.Name = "dep" + dep.Status.Health.Status = health.HealthStatusProgressing + dep.Status.Sync = v1alpha1.SyncStatus{Status: v1alpha1.SyncStatusCodeSynced} + app.Spec.DependsOn = &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + {NamePattern: []string{"*"}}, + }, + } + app.Status.OperationState = &v1alpha1.OperationState{ + Phase: synccommon.OperationRunning, + WaitingFor: []v1alpha1.SyncDependency{ + {ApplicationName: "dep", ApplicationNamespace: app.Namespace}, + }, + } + ctrl := newFakeController(&fakeData{ + apps: []runtime.Object{app, dep}, + }) + assert.False(t, ctrl.shouldRefreshForDependency(app)) + }) + + t.Run("Refresh because a dependency we are waiting for became ready", func(t *testing.T) { + app := newFakeApp() + dep := newFakeApp() + dep.Name = "dep" + dep.Status.Health.Status = health.HealthStatusHealthy + dep.Status.Sync = v1alpha1.SyncStatus{Status: v1alpha1.SyncStatusCodeSynced} + app.Spec.DependsOn = &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + {NamePattern: []string{"*"}}, + }, + } + app.Status.OperationState = &v1alpha1.OperationState{ + Phase: synccommon.OperationRunning, + WaitingFor: []v1alpha1.SyncDependency{ + {ApplicationName: "dep", ApplicationNamespace: app.Namespace}, + }, + } + ctrl := newFakeController(&fakeData{ + apps: []runtime.Object{app, dep}, + }) + assert.True(t, ctrl.shouldRefreshForDependency(app)) + }) + + t.Run("Refresh because a dependency we are waiting for got deleted", func(t *testing.T) { + app := newFakeApp() + dep := newFakeApp() + dep.Name = "dep" + dep.Status.Health.Status = health.HealthStatusHealthy + dep.Status.Sync = v1alpha1.SyncStatus{Status: v1alpha1.SyncStatusCodeSynced} + app.Spec.DependsOn = &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + {NamePattern: []string{"*"}}, + }, + } + app.Status.OperationState = &v1alpha1.OperationState{ + Phase: synccommon.OperationRunning, + WaitingFor: []v1alpha1.SyncDependency{ + {ApplicationName: "dep", ApplicationNamespace: app.Namespace}, + }, + } + ctrl := newFakeController(&fakeData{ + apps: []runtime.Object{app}, + }) + assert.True(t, ctrl.shouldRefreshForDependency(app)) + }) + +} diff --git a/controller/dependencies.go b/controller/dependencies.go new file mode 100644 index 00000000000000..d3777f65c0fd94 --- /dev/null +++ b/controller/dependencies.go @@ -0,0 +1,99 @@ +package controller + +import ( + "fmt" + + log "github.com/sirupsen/logrus" + "github.com/stevenle/topsort" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/labels" + + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1" + "github.com/argoproj/argo-cd/v2/util/glob" +) + +// buildDependencyGraph recursively builds the dependency graph for application +// app. +func (mgr *appStateManager) buildDependencyGraph(app *v1alpha1.Application, depGraph *topsort.Graph) error { + if app.Spec.DependsOn == nil { + return nil + } + appName := app.GetName() + logCtx := log. + WithField("application", app.QualifiedName()) + + logCtx.Debugf("calculating dependencies") + + for _, dep := range app.Spec.DependsOn.Selectors { + // lsel := labels. + var lsel labels.Selector = labels.Everything() + var err error + if dep.LabelSelector != nil { + lsel, err = v1.LabelSelectorAsSelector(dep.LabelSelector) + if err != nil { + return fmt.Errorf("could not convert to label selector: %w", err) + } + } + apps, err := mgr.appLister.Applications(app.GetNamespace()).List(lsel) + if err != nil { + return fmt.Errorf("could not retrieve dependencies: %w", err) + } + for _, a := range apps { + // We need to filter out apps belonging to a different project + if a.Spec.Project != app.Spec.Project { + logCtx.Infof("cannot depend on application '%s', projects mismatch", a.QualifiedName()) + continue + } + + if len(dep.NamePattern) > 0 { + if !glob.MatchStringInList(dep.NamePattern, a.GetName(), false) { + continue + } + } + + // If we already have resolved this app's dependencies, no need to + // get them again and potentially inflict an endless recursion. + if depGraph.ContainsNode(a.GetName()) { + err = depGraph.AddEdge(appName, a.GetName()) + if err != nil { + return fmt.Errorf("could not calculate dependencies: %w", err) + } + continue + } + + // Store the dependency + err = depGraph.AddEdge(appName, a.GetName()) + if err != nil { + return fmt.Errorf("could not calculate dependencies: %w", err) + } + err = mgr.buildDependencyGraph(a, depGraph) + if err != nil { + return err + } + } + } + + return nil +} + +// ResolveApplicationDependencies recursively resolves all dependencies of the +// given application and returns a topologically sorted list of application +// names. +func (mgr *appStateManager) ResolveApplicationDependencies(app *v1alpha1.Application) ([]string, error) { + if app.Spec.DependsOn == nil || len(app.Spec.DependsOn.Selectors) == 0 { + return nil, nil + } + + depGraph := topsort.NewGraph() + err := mgr.buildDependencyGraph(app, depGraph) + if err != nil { + return nil, err + } + resolved, err := depGraph.TopSort(app.GetName()) + // We remove ourselves from the dependency list before returning it + if len(resolved) > 0 { + return resolved[:len(resolved)-1], err + } else { + return resolved, err + } +} diff --git a/controller/dependencies_test.go b/controller/dependencies_test.go new file mode 100644 index 00000000000000..ebdcd3f2a03a83 --- /dev/null +++ b/controller/dependencies_test.go @@ -0,0 +1,507 @@ +package controller + +import ( + "testing" + + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1" + "github.com/argoproj/argo-cd/v2/test" + "github.com/stevenle/topsort" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" +) + +// dependency is a helper function to create an ApplicationDependency struct +// with a label selector from labels. +func dependency(labels []map[string]string) *v1alpha1.ApplicationDependency { + lselector := make([]*v1.LabelSelector, len(labels)) + if len(labels) > 0 { + for i := range labels { + lselector[i] = &v1.LabelSelector{ + MatchLabels: labels[i], + } + } + } + if len(lselector) == 0 { + return &v1alpha1.ApplicationDependency{} + } + deps := make([]v1alpha1.ApplicationSelector, 0, len(labels)) + for i := range lselector { + deps = append(deps, v1alpha1.ApplicationSelector{LabelSelector: lselector[i]}) + } + return &v1alpha1.ApplicationDependency{Selectors: deps} +} + +// appWithDependency is a helper function to create an application with the given +// set of labels and a given set of dependency definitions. +func appWithDependency(name, namespace, project string, labels map[string]string, dependencies []map[string]string) *v1alpha1.Application { + app := &v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: name, + Namespace: namespace, + Labels: labels, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: project, + }, + } + if len(dependencies) > 0 { + app.Spec.DependsOn = dependency(dependencies) + } + return app +} + +func Test_buildDependencyGraph(t *testing.T) { + t.Run("No dependencies", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + }, + Spec: v1alpha1.ApplicationSpec{ + DependsOn: nil, + }, + } + data := fakeData{ + apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"foo": "bar"}, nil), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"foo": "bar"}, nil), + &app, + }, + applicationNamespaces: []string{test.FakeArgoCDNamespace}, + } + c := newFakeController(&data) + apps := topsort.NewGraph() + err := c.appStateManager.(*appStateManager).buildDependencyGraph(&app, apps) + assert.NoError(t, err) + assert.False(t, apps.ContainsNode("parent")) + assert.False(t, apps.ContainsNode("app1")) + assert.False(t, apps.ContainsNode("app2")) + }) + + t.Run("Flat dependencies", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "app1"}, {"name": "app2"}, {"name": "app3"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, nil), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app2"}, nil), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, nil), + &app, + }} + c := newFakeController(&data) + apps := topsort.NewGraph() + err := c.appStateManager.(*appStateManager).buildDependencyGraph(&app, apps) + require.NoError(t, err) + assert.True(t, apps.ContainsNode("app1")) + assert.True(t, apps.ContainsNode("app2")) + assert.True(t, apps.ContainsNode("app3")) + assert.True(t, apps.ContainsNode("parent")) + }) + + t.Run("Chained dependencies", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "app1"}, {"name": "app2"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, []map[string]string{{"name": "app2"}}), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app2"}, []map[string]string{{"name": "app3"}}), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, nil), + &app, + }} + c := newFakeController(&data) + apps := topsort.NewGraph() + err := c.appStateManager.(*appStateManager).buildDependencyGraph(&app, apps) + require.NoError(t, err) + assert.True(t, apps.ContainsNode("app1")) + assert.True(t, apps.ContainsNode("app2")) + assert.True(t, apps.ContainsNode("app3")) + assert.True(t, apps.ContainsNode("parent")) + }) + + // The circular dependency test for getDependencies() makes sure that we + // do not run into infinite recursion and all dependencies are properly + // retrieved. + t.Run("Circular dependency", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "app1"}, {"name": "parent"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, []map[string]string{{"name": "app2"}}), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app2"}, []map[string]string{{"name": "app3"}}), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, []map[string]string{{"name": "parent"}}), + &app, + }} + c := newFakeController(&data) + apps := topsort.NewGraph() + err := c.appStateManager.(*appStateManager).buildDependencyGraph(&app, apps) + require.NoError(t, err) + assert.True(t, apps.ContainsNode("app1")) + assert.True(t, apps.ContainsNode("app2")) + assert.True(t, apps.ContainsNode("app3")) + assert.True(t, apps.ContainsNode("parent")) + }) +} + +func Test_ResolveDependencies(t *testing.T) { + t.Run("No dependencies", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "foo", + Namespace: test.FakeArgoCDNamespace, + }, + Spec: v1alpha1.ApplicationSpec{ + DependsOn: nil, + }, + } + data := fakeData{ + apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"foo": "bar"}, nil), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"foo": "bar"}, nil), + &app, + }, + applicationNamespaces: []string{test.FakeArgoCDNamespace}, + } + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + assert.NoError(t, err) + assert.Len(t, apps, 0) + }) + + t.Run("Flat dependencies with label selector", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "app1"}, {"name": "app2"}, {"name": "app3"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, nil), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app2"}, nil), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, nil), + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.NoError(t, err) + require.Len(t, apps, 3) + // Flat dependencies may be in any particular order + require.Contains(t, apps, "app1") + require.Contains(t, apps, "app2") + require.Contains(t, apps, "app3") + }) + + t.Run("Flat dependencies with name selector", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + {NamePattern: []string{"*1", "*2"}}, + }, + }, + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, nil), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app2"}, nil), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, nil), + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.NoError(t, err) + require.Len(t, apps, 2) + // Flat dependencies may be in any particular order + require.Contains(t, apps, "app1") + require.Contains(t, apps, "app2") + }) + + t.Run("Flat dependencies with mixed selector", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + { + LabelSelector: &v1.LabelSelector{ + MatchExpressions: []v1.LabelSelectorRequirement{ + { + Key: "name", + Operator: v1.LabelSelectorOpExists, + }, + }, + }, + NamePattern: []string{"app2", "app3"}, + }, + }, + }, + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, nil), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app2"}, nil), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, nil), + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.NoError(t, err) + require.Len(t, apps, 2) + // Flat dependencies may be in any particular order + require.Contains(t, apps, "app2") + require.Contains(t, apps, "app3") + }) + + t.Run("Flat dependencies with two selectors", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: &v1alpha1.ApplicationDependency{ + // These selectors should select app1 by name, and app3 by label + Selectors: []v1alpha1.ApplicationSelector{ + { + LabelSelector: &v1.LabelSelector{ + MatchLabels: map[string]string{ + "name": "app3", + }, + }}, + { + NamePattern: []string{"app1"}, + }, + }, + }, + }, + } + + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", nil, nil), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", nil, nil), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, nil), + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.NoError(t, err) + require.Len(t, apps, 2) + // Flat dependencies may be in any particular order + require.Contains(t, apps, "app1") + require.Contains(t, apps, "app3") + }) + + t.Run("Chained dependencies", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "app1"}, {"name": "app2"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, []map[string]string{{"name": "app2"}}), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app2"}, []map[string]string{{"name": "app3"}}), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, nil), + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.NoError(t, err) + require.Len(t, apps, 3) + require.Equal(t, []string{"app3", "app2", "app1"}, apps) + }) + + t.Run("Circular chained dependency", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "app1"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, []map[string]string{{"name": "app3"}}), + appWithDependency("app2", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app2"}, []map[string]string{{"name": "app4"}}), + appWithDependency("app3", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app3"}, []map[string]string{{"name": "parent"}}), + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.ErrorContains(t, err, "Cycle error") + require.Len(t, apps, 0) + }) + + // App depending on itself is a circular depedency which must result in an + // error. + t.Run("Parent depends on self", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "parent"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.ErrorContains(t, err, "Cycle error") + require.Len(t, apps, 0) + }) + + t.Run("Dependency depends on self", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "parent"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.ErrorContains(t, err, "Cycle error") + require.Len(t, apps, 0) + }) + + t.Run("Ignore apps using different AppProject", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "app1"}}), + }, + } + data := fakeData{apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, []map[string]string{{"name": "app3"}}), + appWithDependency("app2", test.FakeArgoCDNamespace, "other", map[string]string{"name": "app2"}, []map[string]string{{"name": "app3"}}), + appWithDependency("app3", test.FakeArgoCDNamespace, "other", map[string]string{"name": "app3"}, []map[string]string{{"name": "parent"}}), + &app, + }} + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.NoError(t, err) + require.Len(t, apps, 1) + }) + + t.Run("Ignore apps in different namespaces", func(t *testing.T) { + app := v1alpha1.Application{ + ObjectMeta: v1.ObjectMeta{ + Name: "parent", + Namespace: test.FakeArgoCDNamespace, + Labels: map[string]string{ + "name": "parent", + }, + }, + Spec: v1alpha1.ApplicationSpec{ + Project: "default", + DependsOn: dependency([]map[string]string{{"name": "app1"}}), + }, + } + data := fakeData{ + apps: []runtime.Object{ + appWithDependency("app1", test.FakeArgoCDNamespace, "default", map[string]string{"name": "app1"}, []map[string]string{{"name": "app3"}}), + appWithDependency("app2", "default", "default", map[string]string{"name": "app2"}, []map[string]string{{"name": "app3"}}), + appWithDependency("app3", "default", "default", map[string]string{"name": "app3"}, []map[string]string{{"name": "parent"}}), + &app, + }, + applicationNamespaces: []string{"*"}, + } + c := newFakeController(&data) + apps, err := c.appStateManager.(*appStateManager).ResolveApplicationDependencies(&app) + require.NoError(t, err) + require.Len(t, apps, 1) + }) + +} diff --git a/controller/state.go b/controller/state.go index 704411558669b8..046d2791389e08 100644 --- a/controller/state.go +++ b/controller/state.go @@ -32,6 +32,7 @@ import ( "github.com/argoproj/argo-cd/v2/controller/metrics" "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1" appclientset "github.com/argoproj/argo-cd/v2/pkg/client/clientset/versioned" + applisters "github.com/argoproj/argo-cd/v2/pkg/client/listers/application/v1alpha1" "github.com/argoproj/argo-cd/v2/reposerver/apiclient" "github.com/argoproj/argo-cd/v2/util/argo" argodiff "github.com/argoproj/argo-cd/v2/util/argo/diff" @@ -106,6 +107,7 @@ type appStateManager struct { settingsMgr *settings.SettingsManager appclientset appclientset.Interface projInformer cache.SharedIndexInformer + appLister applisters.ApplicationLister kubectl kubeutil.Kubectl repoClientset apiclient.Clientset liveStateCache statecache.LiveStateCache @@ -939,6 +941,7 @@ func NewAppStateManager( settingsMgr *settings.SettingsManager, liveStateCache statecache.LiveStateCache, projInformer cache.SharedIndexInformer, + appLister applisters.ApplicationLister, metricsServer *metrics.MetricsServer, cache *appstatecache.Cache, statusRefreshTimeout time.Duration, @@ -957,6 +960,7 @@ func NewAppStateManager( namespace: namespace, settingsMgr: settingsMgr, projInformer: projInformer, + appLister: appLister, metricsServer: metricsServer, statusRefreshTimeout: statusRefreshTimeout, resourceTracking: resourceTracking, diff --git a/controller/sync.go b/controller/sync.go index 34c12bdb5da3ce..7fda95fc036b26 100644 --- a/controller/sync.go +++ b/controller/sync.go @@ -7,11 +7,13 @@ import ( "fmt" "os" "strconv" + "strings" "sync/atomic" "time" cdcommon "github.com/argoproj/argo-cd/v2/common" + "github.com/argoproj/gitops-engine/pkg/health" "github.com/argoproj/gitops-engine/pkg/sync" "github.com/argoproj/gitops-engine/pkg/sync/common" "github.com/argoproj/gitops-engine/pkg/utils/kube" @@ -78,6 +80,132 @@ func (m *appStateManager) getResourceOperations(server string) (kube.ResourceOpe return ops, cleanup, nil } +// areDependenciesReady will figure out whether all of this app's dependencies +// are in a ready state and thus whether the sync can proceed. Will return true if +// sync is allowed to proceed, or false if not. In the latter case, the state's +// phase and message will be set to a proper reason. +func (m *appStateManager) areDependenciesReady(app *v1alpha1.Application, state *v1alpha1.OperationState, syncOp v1alpha1.SyncOperation) bool { + state.BlockedOnEmpty = false + state.Message = "" + + logCtx := log.WithField("application", app.QualifiedName()) + + dependsOn := app.Spec.DependsOn + if dependsOn == nil { + logCtx.Debugf("Application has no dependencies defined") + return true + } + + // Force lets us override any constraints that would be opposed + if syncOp.SyncStrategy != nil && syncOp.SyncStrategy.Force() { + return true + } + + start := time.Now() + + // Before the sync actually starts, we need to see whether we depend on + // any other application, and if yes, check if we are ready to sync. + deps, err := m.ResolveApplicationDependencies(app) + if err != nil { + state.Phase = common.OperationFailed + state.Message = fmt.Sprintf("Could not resolve application dependencies: %v", err) + return false + } + + logCtx.Debugf("Resolved %d dependencies", len(deps)) + + // Respect any timeout setting early on + if dependsOn.Timeout != nil { + if start.After(state.StartedAt.Time.Add(*dependsOn.Timeout)) { + state.Phase = common.OperationFailed + state.Message = "Timeout waiting for dependencies to become synced and healthy" + return false + } + } + + // If BlockOnEmpty is true and we haven't found any dependencies, we + // need to wait until they are created before we proceed. + if app.IsBlockOnEmptyDependencies() && len(deps) == 0 && len(dependsOn.Selectors) > 0 { + state.Phase = common.OperationRunning + state.Message = "Waiting for any app to be created to match dependency selector" + state.BlockedOnEmpty = true + return false + } + + // If a wait period was specified and we are in autosync, we will wait the + // specified time before letting the sync proceed. + if app.IsAutomated() && dependsOn.SyncDelay != nil { + if start.Before(state.StartedAt.Time.Add(*dependsOn.SyncDelay)) { + state.Phase = common.OperationRunning + state.Message = "Delaying sync start to allow dependencies to be created" + return false + } + } + + notReadyDeps := []string{} + for _, dep := range deps { + depApp, err := m.appLister.Applications(app.Namespace).Get(dep) + if err != nil { + state.Phase = common.OperationFailed + state.Message = fmt.Sprintf("Error retrieving application dependency: %v", err) + return false + } + if depApp.Status.Sync.Status != v1alpha1.SyncStatusCodeSynced || depApp.Status.Health.Status != health.HealthStatusHealthy { + logCtx.Debugf("adding %s to list of dependencies", depApp.QualifiedName()) + notReadyDeps = append(notReadyDeps, depApp.QualifiedName()) + } + } + + // We need to remember the last refreshed property for each dependency, + // because we're going to reset the dependency list in the app's operation + // state. + waitingFor := make(map[string]*v1.Time) + for _, dep := range state.WaitingFor { + waitingFor[dep.QualifiedName()] = dep.RefreshedAt + } + + state.WaitingFor = nil + + // If there are dependencies that are not ready, we can't sync yet. + // + // However, a force sync lets us override waiting for any dependency + // and perform the sync despite dependencies not being ready. + // + // We store all non-ready dependencies in the application's status + // field. + if len(notReadyDeps) > 0 { + if !syncOp.SyncStrategy.Force() { + for _, dep := range notReadyDeps { + appName := strings.Split(dep, "/") + // This should not happen, but if it happens, log it out as + // a warning so people will be aware. + if len(appName) != 2 { + logCtx.Warnf("unexpected sync dependency: %v - should be in format namespace/name", dep) + continue + } + logCtx.Debugf("Adding unsatisfied dependency to operation state: %s", dep) + + wf, ok := waitingFor[dep] + var refreshedAt *v1.Time = nil + if ok { + refreshedAt = wf + } + state.WaitingFor = append(state.WaitingFor, v1alpha1.SyncDependency{ + ApplicationName: appName[1], + ApplicationNamespace: appName[0], + RefreshedAt: refreshedAt, + }) + } + state.Phase = common.OperationRunning + state.Message = fmt.Sprintf("Waiting for dependencies to become synced and healthy: %s", strings.Join(notReadyDeps, ", ")) + return false + } + } + + // Ready to sync this app + return true +} + func (m *appStateManager) SyncAppState(app *v1alpha1.Application, state *v1alpha1.OperationState) { // Sync requests might be requested with ambiguous revisions (e.g. master, HEAD, v1.2.3). // This can change meaning when resuming operations (e.g a hook sync). After calculating a @@ -98,6 +226,21 @@ func (m *appStateManager) SyncAppState(app *v1alpha1.Application, state *v1alpha } syncOp = *state.Operation.Sync + // If we are waiting for any dependency, it means that we started the sync + // previously but did not proceed for any reason (maybe the controller was + // restarted, or the dependencies were not ready). We already ensured that + // the sync is valid and could proceed, so there's no need to check again + // if the sync is valid before checking for readiness of dependencies. + dependenciesChecked := false + if app.IsWaiting() { + log.WithField("application", app.QualifiedName()).Debugf("Operation already in progress, checking for dependencies early on") + if !m.areDependenciesReady(app, state, syncOp) { + return + } else { + dependenciesChecked = true + } + } + // validates if it should fail the sync if it finds shared resources hasSharedResource, sharedResourceMessage := hasSharedResourceCondition(app) if syncOp.SyncOptions.HasOption("FailOnSharedResource=true") && @@ -346,6 +489,11 @@ func (m *appStateManager) SyncAppState(app *v1alpha1.Application, state *v1alpha if state.Phase == common.OperationTerminating { syncCtx.Terminate() } else { + if !dependenciesChecked { + if !m.areDependenciesReady(app, state, syncOp) { + return + } + } syncCtx.Sync() } var resState []common.ResourceSyncResult diff --git a/controller/sync_test.go b/controller/sync_test.go index 309f846ca64609..e6b9b15f772850 100644 --- a/controller/sync_test.go +++ b/controller/sync_test.go @@ -3,7 +3,9 @@ package controller import ( "context" "testing" + "time" + "github.com/argoproj/gitops-engine/pkg/health" "github.com/argoproj/gitops-engine/pkg/sync" "github.com/argoproj/gitops-engine/pkg/sync/common" "github.com/argoproj/gitops-engine/pkg/utils/kube" @@ -12,6 +14,7 @@ import ( v1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" "k8s.io/apimachinery/pkg/runtime" + "k8s.io/utils/pointer" "github.com/argoproj/argo-cd/v2/controller/testdata" "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1" @@ -386,3 +389,173 @@ func TestNormalizeTargetResources(t *testing.T) { assert.Equal(t, 2, len(containers)) }) } + +func Test_areDependenciesReady(t *testing.T) { + parent := newFakeApp() + parent.Name = "parent-app" + parent.Spec.DependsOn = &v1alpha1.ApplicationDependency{ + Selectors: []v1alpha1.ApplicationSelector{ + { + LabelSelector: &v1.LabelSelector{ + MatchLabels: map[string]string{ + "foo": "bar", + }, + }, + }, + }, + } + + dep1 := newFakeApp() + dep1.Name = "dep1" + dep1.Labels = map[string]string{"foo": "bar"} + + state := &v1alpha1.OperationState{ + Phase: common.OperationRunning, + } + + syncOp := v1alpha1.SyncOperation{SyncStrategy: &v1alpha1.SyncStrategy{}} + t.Run("Ready to sync", func(t *testing.T) { + p := parent.DeepCopy() + d := dep1.DeepCopy() + d.Status.Health = v1alpha1.HealthStatus{Status: health.HealthStatusHealthy} + d.Status.Sync = v1alpha1.SyncStatus{Status: v1alpha1.SyncStatusCodeSynced} + s := state.DeepCopy() + data := fakeData{ + apps: []runtime.Object{p, d}, + } + ctrl := newFakeController(&data) + ready := ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.True(t, ready) + assert.Equal(t, common.OperationRunning, s.Phase) + assert.Empty(t, s.Message) + assert.Empty(t, s.WaitingFor) + }) + + t.Run("Not ready to sync because dependency is out of sync", func(t *testing.T) { + p := parent.DeepCopy() + d := dep1.DeepCopy() + s := state.DeepCopy() + data := fakeData{ + apps: []runtime.Object{p, d}, + } + ctrl := newFakeController(&data) + ready := ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.False(t, ready) + assert.Equal(t, common.OperationRunning, s.Phase) + assert.Contains(t, s.Message, "Waiting for dependencies") + assert.Contains(t, s.Message, "dep1") + require.Len(t, s.WaitingFor, 1) + assert.Equal(t, "dep1", s.WaitingFor[0].ApplicationName) + assert.Equal(t, p.GetNamespace(), s.WaitingFor[0].ApplicationNamespace) + assert.Nil(t, s.WaitingFor[0].RefreshedAt) + }) + + t.Run("Preserve refresh time for dependencies", func(t *testing.T) { + p := parent.DeepCopy() + d := dep1.DeepCopy() + s := state.DeepCopy() + data := fakeData{ + apps: []runtime.Object{p, d}, + } + ctrl := newFakeController(&data) + ready := ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.False(t, ready) + require.Len(t, s.WaitingFor, 1) + s.WaitingFor[0].RefreshedAt = &v1.Time{Time: time.Now()} + ready = ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.False(t, ready) + require.Len(t, s.WaitingFor, 1) + require.NotNil(t, s.WaitingFor[0].RefreshedAt) + }) + + t.Run("Not ready to sync because blocking for dependency creation", func(t *testing.T) { + p := parent.DeepCopy() + p.Spec.DependsOn.BlockOnEmpty = pointer.Bool(true) + s := state.DeepCopy() + data := fakeData{ + apps: []runtime.Object{p}, + } + ctrl := newFakeController(&data) + ready := ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.False(t, ready) + assert.Equal(t, common.OperationRunning, s.Phase) + assert.Contains(t, s.Message, "Waiting for any app to be created") + }) + + t.Run("Sync fails due to timeout", func(t *testing.T) { + p := parent.DeepCopy() + p.Spec.DependsOn.Timeout = pointer.Duration(10 * time.Second) + s := state.DeepCopy() + started := time.Now().Add(-20 * time.Second) + s.StartedAt = v1.Time{Time: started} + data := fakeData{ + apps: []runtime.Object{p}, + } + ctrl := newFakeController(&data) + ready := ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.False(t, ready) + assert.Equal(t, common.OperationFailed, s.Phase) + assert.Contains(t, s.Message, "Timeout waiting") + }) + + t.Run("Sync proceeds because timeout not reached", func(t *testing.T) { + p := parent.DeepCopy() + p.Spec.DependsOn.Timeout = pointer.Duration(10 * time.Second) + s := state.DeepCopy() + started := time.Now() + s.StartedAt = v1.Time{Time: started} + data := fakeData{ + apps: []runtime.Object{p}, + } + ctrl := newFakeController(&data) + ready := ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.True(t, ready) + assert.Equal(t, common.OperationRunning, s.Phase) + assert.Empty(t, s.Message) + }) + + t.Run("Automatic sync start is delayed", func(t *testing.T) { + // The fake app has auto-sync enabled by default + p := parent.DeepCopy() + p.Spec.DependsOn.SyncDelay = pointer.Duration(10 * time.Second) + s := state.DeepCopy() + started := time.Now() + s.StartedAt = v1.Time{Time: started} + data := fakeData{ + apps: []runtime.Object{p}, + } + ctrl := newFakeController(&data) + + // Initially, we have a delay + ready := ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.False(t, ready) + assert.Equal(t, common.OperationRunning, s.Phase) + assert.Contains(t, s.Message, "Delaying sync start") + + // Second run without delay + s.StartedAt = v1.Time{Time: started.Add(-20 * time.Second)} + ready = ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.True(t, ready) + assert.Equal(t, common.OperationRunning, s.Phase) + assert.Empty(t, s.Message) + }) + + t.Run("Sync delay does not affect manual sync", func(t *testing.T) { + p := parent.DeepCopy() + p.Spec.DependsOn.SyncDelay = pointer.Duration(10 * time.Second) + p.Spec.SyncPolicy = nil + s := state.DeepCopy() + started := time.Now() + s.StartedAt = v1.Time{Time: started} + data := fakeData{ + apps: []runtime.Object{p}, + } + ctrl := newFakeController(&data) + + ready := ctrl.appStateManager.(*appStateManager).areDependenciesReady(p, s, syncOp) + assert.True(t, ready) + assert.Equal(t, common.OperationRunning, s.Phase) + assert.Empty(t, s.Message) + }) + +} diff --git a/docs/user-guide/app_dependencies.md b/docs/user-guide/app_dependencies.md new file mode 100644 index 00000000000000..e32bbfb608d0a2 --- /dev/null +++ b/docs/user-guide/app_dependencies.md @@ -0,0 +1,128 @@ +# Application dependencies + +Argo CD lets you natively model dependencies between Applications. This mechanism is intended to replace the prior way of modeling dependencies when using the app-of-apps pattern with sync waves on the Application custom resources. + +Using the new dependency mechanism, users are not bound to the app-of-apps pattern anymore and can define dependencies between applications using application selectors to define a directed, acyclic graph for dependency modeling. This means that an Application which defines one or more dependencies do not only depend on its direct dependencies, but to the dependencies of its dependencies as well. + +Application dependencies support both, manual and automatic sync policies. + +## Sync behaviour + +Defining dependencies for an Application will affect its sync behaviour. When a sync is started, Argo CD will evaluate the sync and health status of all other Applications in its dependency tree. Only if all of its dependent Applications have a sync status of `Synced` and a health status of `Healthy`, will the sync proceed. Until all conditions are met, the sync will stay in a progressing state and a message will indicate that the sync is waiting for dependencies. This behaviour is similar to sync waves, where a sync progresses only to the next wave when the current wave's resources are all synced and healthy. + +As a fallback mechanism, when using manual sync with the force option, Argo CD will not perform dependency resolution and the sync will proceed regardless of the states of the Application's dependencies. + +## Feature state and caveats + +This feature is in [beta state](https://github.com/argoproj/argoproj/blob/master/community/feature-status.md#beta). It will be polished across the upcoming releases. + +Bug reports and enhancement requests are most welcome. + +## Limitations and known issues + +A list of things that are good to know before getting started with Application dependencies: + +* An Application can only depend on other Applications that use the same AppProject and are in the same namespace as the dependent Application. +* A circular dependency defined anywhere in the dependency chain is considered to be an error and will prevent all applications in the circular dependency loop from syncing. +* Syncing an application that is dependent on other apps will not trigger a sync of the dependencies. However, manual sync of a dependent application whose dependencies are not satisfied will block unless force sync is used. +* Dependencies currently can only be configured in a declarative manner; the UI and CLI do not yet have first class support for configuring dependencies. + +## Configuring Application dependencies + +The dependencies of an Application are configured using the field `dependsOn` in the Application's spec. The field `dependsOn` is a complex data type, which lets you define one or more `selectors` to match a set of other Applications that this Application will be dependent upon. + +Each selector can match Applications according to one or more properties. If you specify more than one matcher in a selector, each matcher has to match in order for an Application to be considered a dependency. For example, if you specify a `labelSelector` and a `namePattern`, both will have to match in order for an Application to be selected. + +Additionally, the behaviour of dependency resolution can be configured on a per Application basis. + +## Dependency selectors + +Currently, the available properties to match against are `labels` and the `name` of the Application. Each selector can yield zero or more applications. + +### Matching on labels + +It is possible and recommended to use a standard label selector to select the dependencies for your Application: + +```yaml +spec: + dependsOn: + selectors: + - labelSelector: + matchLabels: + name: some-app +``` + +The above YAML will have the Application depend on any other Application that matches the given label selector. In this case, any application carrying a label `name` with the value of `some-app`. + +### Matching on name patterns + +Dependencies can be matched on the name of Applications by specifying the `namePattern` selector. The `namePattern` selector takes a list of Application name patterns to match against the Applications' `.metadata.name` property. Specifying multiple patterns + +```yaml +spec: + dependsOn: + selectors: + - namePattern: + - some-app + - other-app +``` + +The above YAML will have the Application depend on the Applications `some-app` and `other-app`. + +The `namePattern` matcher supports shell-style glob matching, so the following example would match all Applications whose name is starting with `stage1-`: + +```yaml +spec: + dependsOn: + selectors: + - namePattern: + - stage1-* +``` + +Regular expressions are not supported. + +## Configuring behaviour of dependency resolution + +There are several options to configure the behaviour of dependency resolution. Please note that these settings are not inherited to dependencies, but must be configured for each dependent uniquely. + +### Delayed start of automated sync + +In certain scenarios, you may want to delay the automatic sync process when there are dependencies configured between applications that are created at the same time (i.e. synced through an app of apps pattern from Git). This may allow the synchronization of all applications to succeed before the dependency tree is resolved and the sync is actually started in the right order. + +To configure the sync delay, specify the number of seconds to delay in the field `syncDelay` as follows: + +```yaml +spec: + dependsOn: + syncDelay: 5 +``` + +Please note that any delay you configure here will delay _any_ automatic sync, regardless of whether the Application has just been created or has existed for a while. + +### Block on empty dependencies + +It is possible to block syncing when dependency selectors are configured, but no dependency exists yet. This may be useful in scenarios where you want to prevent an Application from being synced unless another specific Application exists. + +To configure blocked syncs on empty dependencies, use the field `blockOnEmpty` as follows: + +```yaml +spec: + dependsOn: + blockOnEmpty: true +``` + +Note that when you set `blockOnEmpty` to true, and sync this app through the CLI, the `argocd app sync` command will only return when the application starts syncing, i.e. when it has finished waiting or times out. If you need the CLI to return immediately, use `argocd app sync --async`. + +### Configure a wait timeout + +You can configure the maximum time that Argo CD should wait for dependencies to be resolved and the sync to start. The timeout also applies to the wait spent in blocking for empty dependencies. + +To configure a timeout, set the field `timeout` to the number of seconds Argo CD should wait at most. + +```yaml +spec: + dependsOn: + timeout: 10 +``` + +When the timeout is reached, the Application's operation state will transition from `Running` into `Failed` and the sync won't proceed. diff --git a/go.mod b/go.mod index 2f3bdec276c7c4..64253d91dee0fc 100644 --- a/go.mod +++ b/go.mod @@ -73,6 +73,7 @@ require ( github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.7.0 github.com/spf13/pflag v1.0.5 + github.com/stevenle/topsort v0.2.0 github.com/stretchr/testify v1.8.4 github.com/valyala/fasttemplate v1.2.2 github.com/whilp/git-urls v1.0.0 diff --git a/go.sum b/go.sum index 495ba3ed9ba296..6d39d466725e0c 100644 --- a/go.sum +++ b/go.sum @@ -1655,6 +1655,8 @@ github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnIn github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/ssor/bom v0.0.0-20170718123548-6386211fdfcf/go.mod h1:RJID2RhlZKId02nZ62WenDCkgHFerpIOmW0iT7GKmXM= +github.com/stevenle/topsort v0.2.0 h1:LLWgtp34HPX6/RBDRS0kElVxGOTzGBLI1lSAa5Lb46k= +github.com/stevenle/topsort v0.2.0/go.mod h1:ck2WG2/ZrOr6dLApQ/5Xrqy5wv3T0qhKYWE7r9tkibc= github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= diff --git a/manifests/core-install.yaml b/manifests/core-install.yaml index 3cbaa4946e3cce..0a1c6f4e8b4b9d 100644 --- a/manifests/core-install.yaml +++ b/manifests/core-install.yaml @@ -863,6 +863,91 @@ spec: link to repository with application definition and additional parameters link definition revision. properties: + dependsOn: + description: DependsOn specifies what this application depends on + properties: + blockOnEmpty: + description: BlockOnEmpty defines whether to block sync when the + list of applications determined by the selector is empty + type: boolean + refreshDependencies: + description: RefreshDependencies defines whether all dependencies + should be refreshed before starting a sync + type: boolean + selectors: + description: Selectors defines conditions for matching application's + dependencies + items: + description: ApplicationSelector specifies which applications + this Application depends on + properties: + labelSelector: + description: LabelSelector selects applications by their + labels + properties: + matchExpressions: + description: matchExpressions is a list of label selector + requirements. The requirements are ANDed. + items: + description: A label selector requirement is a selector + that contains values, a key, and an operator that + relates the key and values. + properties: + key: + description: key is the label key that the selector + applies to. + type: string + operator: + description: operator represents a key's relationship + to a set of values. Valid operators are In, + NotIn, Exists and DoesNotExist. + type: string + values: + description: values is an array of string values. + If the operator is In or NotIn, the values array + must be non-empty. If the operator is Exists + or DoesNotExist, the values array must be empty. + This array is replaced during a strategic merge + patch. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a map of {key,value} pairs. + A single {key,value} in the matchLabels map is equivalent + to an element of matchExpressions, whose key field + is "key", the operator is "In", and the values array + contains only "value". The requirements are ANDed. + type: object + type: object + namePattern: + description: NamePattern selects applications by matching + their names + items: + type: string + type: array + type: object + type: array + syncDelay: + description: SyncDelay specifies the duration in seconds to wait + before starting to sync when dependencies are defined + format: int64 + type: integer + timeout: + description: Timeout defines the maximum duration in seconds to + wait on dependencies before the sync fails + format: int64 + type: integer + required: + - selectors + type: object destination: description: Destination is a reference to the target Kubernetes server and namespace @@ -2463,6 +2548,10 @@ spec: description: OperationState contains information about any ongoing operations, such as a sync properties: + blockedOnEmpty: + description: BlockedOnEmpty is true when the application is waiting + for any dependency to be created + type: boolean finishedAt: description: FinishedAt contains time of operation completion format: date-time @@ -4130,6 +4219,23 @@ spec: required: - revision type: object + waitingFor: + description: WaitingFor specifies a list of applications that + this operation is waiting for + items: + properties: + applicationName: + type: string + applicationNamespace: + type: string + refreshedAt: + format: date-time + type: string + required: + - applicationName + - applicationNamespace + type: object + type: array required: - operation - phase @@ -5097,6 +5203,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -5688,6 +5841,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -6283,6 +6483,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -6858,6 +7105,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -7457,6 +7751,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -8048,6 +8389,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -8643,6 +9031,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -9218,24 +9653,71 @@ spec: type: object spec: properties: - destination: + dependsOn: properties: - name: - type: string - namespace: - type: string - server: - type: string - type: object - ignoreDifferences: - items: - properties: - group: - type: string - jqPathExpressions: - items: - type: string - type: array + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object + destination: + properties: + name: + type: string + namespace: + type: string + server: + type: string + type: object + ignoreDifferences: + items: + properties: + group: + type: string + jqPathExpressions: + items: + type: string + type: array jsonPointers: items: type: string @@ -9803,6 +10285,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -10568,6 +11097,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -11328,6 +11904,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -11917,6 +12540,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -12516,6 +13186,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -13107,6 +13824,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -13702,6 +14466,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -14277,6 +15088,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -14862,6 +15720,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -15627,6 +16532,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -16387,6 +17339,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -16980,6 +17979,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -17562,6 +18608,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -18327,6 +19420,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -19087,6 +20227,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -19751,6 +20938,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: diff --git a/manifests/crds/application-crd.yaml b/manifests/crds/application-crd.yaml index aaf1347f64dfb7..ecf99f76ff79b0 100644 --- a/manifests/crds/application-crd.yaml +++ b/manifests/crds/application-crd.yaml @@ -862,6 +862,91 @@ spec: link to repository with application definition and additional parameters link definition revision. properties: + dependsOn: + description: DependsOn specifies what this application depends on + properties: + blockOnEmpty: + description: BlockOnEmpty defines whether to block sync when the + list of applications determined by the selector is empty + type: boolean + refreshDependencies: + description: RefreshDependencies defines whether all dependencies + should be refreshed before starting a sync + type: boolean + selectors: + description: Selectors defines conditions for matching application's + dependencies + items: + description: ApplicationSelector specifies which applications + this Application depends on + properties: + labelSelector: + description: LabelSelector selects applications by their + labels + properties: + matchExpressions: + description: matchExpressions is a list of label selector + requirements. The requirements are ANDed. + items: + description: A label selector requirement is a selector + that contains values, a key, and an operator that + relates the key and values. + properties: + key: + description: key is the label key that the selector + applies to. + type: string + operator: + description: operator represents a key's relationship + to a set of values. Valid operators are In, + NotIn, Exists and DoesNotExist. + type: string + values: + description: values is an array of string values. + If the operator is In or NotIn, the values array + must be non-empty. If the operator is Exists + or DoesNotExist, the values array must be empty. + This array is replaced during a strategic merge + patch. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a map of {key,value} pairs. + A single {key,value} in the matchLabels map is equivalent + to an element of matchExpressions, whose key field + is "key", the operator is "In", and the values array + contains only "value". The requirements are ANDed. + type: object + type: object + namePattern: + description: NamePattern selects applications by matching + their names + items: + type: string + type: array + type: object + type: array + syncDelay: + description: SyncDelay specifies the duration in seconds to wait + before starting to sync when dependencies are defined + format: int64 + type: integer + timeout: + description: Timeout defines the maximum duration in seconds to + wait on dependencies before the sync fails + format: int64 + type: integer + required: + - selectors + type: object destination: description: Destination is a reference to the target Kubernetes server and namespace @@ -2462,6 +2547,10 @@ spec: description: OperationState contains information about any ongoing operations, such as a sync properties: + blockedOnEmpty: + description: BlockedOnEmpty is true when the application is waiting + for any dependency to be created + type: boolean finishedAt: description: FinishedAt contains time of operation completion format: date-time @@ -4129,6 +4218,23 @@ spec: required: - revision type: object + waitingFor: + description: WaitingFor specifies a list of applications that + this operation is waiting for + items: + properties: + applicationName: + type: string + applicationNamespace: + type: string + refreshedAt: + format: date-time + type: string + required: + - applicationName + - applicationNamespace + type: object + type: array required: - operation - phase diff --git a/manifests/crds/applicationset-crd.yaml b/manifests/crds/applicationset-crd.yaml index 8d4fbb5c748fa2..dea9f60ecdc297 100644 --- a/manifests/crds/applicationset-crd.yaml +++ b/manifests/crds/applicationset-crd.yaml @@ -89,6 +89,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -680,6 +727,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -1275,6 +1369,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -1850,6 +1991,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -2449,6 +2637,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -3040,6 +3275,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -3635,6 +3917,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -4210,6 +4539,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -4795,6 +5171,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -5560,6 +5983,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -6320,6 +6790,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -6909,6 +7426,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -7508,6 +8072,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -8099,6 +8710,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -8687,13 +9345,60 @@ spec: additionalProperties: type: string type: object - name: - type: string - namespace: - type: string - type: object - spec: - properties: + name: + type: string + namespace: + type: string + type: object + spec: + properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -9269,6 +9974,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -9854,6 +10606,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -10619,6 +11418,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -11379,6 +12225,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -11972,6 +12865,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -12554,6 +13494,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -13319,6 +14306,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -14079,6 +15113,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -14743,6 +15824,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: diff --git a/manifests/ha/install.yaml b/manifests/ha/install.yaml index 322d9534a8371d..86d9ce4f3c611a 100644 --- a/manifests/ha/install.yaml +++ b/manifests/ha/install.yaml @@ -863,6 +863,91 @@ spec: link to repository with application definition and additional parameters link definition revision. properties: + dependsOn: + description: DependsOn specifies what this application depends on + properties: + blockOnEmpty: + description: BlockOnEmpty defines whether to block sync when the + list of applications determined by the selector is empty + type: boolean + refreshDependencies: + description: RefreshDependencies defines whether all dependencies + should be refreshed before starting a sync + type: boolean + selectors: + description: Selectors defines conditions for matching application's + dependencies + items: + description: ApplicationSelector specifies which applications + this Application depends on + properties: + labelSelector: + description: LabelSelector selects applications by their + labels + properties: + matchExpressions: + description: matchExpressions is a list of label selector + requirements. The requirements are ANDed. + items: + description: A label selector requirement is a selector + that contains values, a key, and an operator that + relates the key and values. + properties: + key: + description: key is the label key that the selector + applies to. + type: string + operator: + description: operator represents a key's relationship + to a set of values. Valid operators are In, + NotIn, Exists and DoesNotExist. + type: string + values: + description: values is an array of string values. + If the operator is In or NotIn, the values array + must be non-empty. If the operator is Exists + or DoesNotExist, the values array must be empty. + This array is replaced during a strategic merge + patch. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a map of {key,value} pairs. + A single {key,value} in the matchLabels map is equivalent + to an element of matchExpressions, whose key field + is "key", the operator is "In", and the values array + contains only "value". The requirements are ANDed. + type: object + type: object + namePattern: + description: NamePattern selects applications by matching + their names + items: + type: string + type: array + type: object + type: array + syncDelay: + description: SyncDelay specifies the duration in seconds to wait + before starting to sync when dependencies are defined + format: int64 + type: integer + timeout: + description: Timeout defines the maximum duration in seconds to + wait on dependencies before the sync fails + format: int64 + type: integer + required: + - selectors + type: object destination: description: Destination is a reference to the target Kubernetes server and namespace @@ -2463,6 +2548,10 @@ spec: description: OperationState contains information about any ongoing operations, such as a sync properties: + blockedOnEmpty: + description: BlockedOnEmpty is true when the application is waiting + for any dependency to be created + type: boolean finishedAt: description: FinishedAt contains time of operation completion format: date-time @@ -4130,6 +4219,23 @@ spec: required: - revision type: object + waitingFor: + description: WaitingFor specifies a list of applications that + this operation is waiting for + items: + properties: + applicationName: + type: string + applicationNamespace: + type: string + refreshedAt: + format: date-time + type: string + required: + - applicationName + - applicationNamespace + type: object + type: array required: - operation - phase @@ -5097,6 +5203,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -5688,6 +5841,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -6283,6 +6483,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -6858,6 +7105,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -7457,6 +7751,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -8048,6 +8389,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -8643,6 +9031,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -9218,24 +9653,71 @@ spec: type: object spec: properties: - destination: + dependsOn: properties: - name: - type: string - namespace: - type: string - server: - type: string - type: object - ignoreDifferences: - items: - properties: - group: - type: string - jqPathExpressions: - items: - type: string - type: array + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object + destination: + properties: + name: + type: string + namespace: + type: string + server: + type: string + type: object + ignoreDifferences: + items: + properties: + group: + type: string + jqPathExpressions: + items: + type: string + type: array jsonPointers: items: type: string @@ -9803,6 +10285,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -10568,6 +11097,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -11328,6 +11904,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -11917,6 +12540,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -12516,6 +13186,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -13107,6 +13824,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -13702,6 +14466,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -14277,6 +15088,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -14862,6 +15720,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -15627,6 +16532,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -16387,6 +17339,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -16980,6 +17979,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -17562,6 +18608,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -18327,6 +19420,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -19087,6 +20227,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -19751,6 +20938,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: diff --git a/manifests/install.yaml b/manifests/install.yaml index 8da7f4c8306b4e..f0b0b5af86262f 100644 --- a/manifests/install.yaml +++ b/manifests/install.yaml @@ -863,6 +863,91 @@ spec: link to repository with application definition and additional parameters link definition revision. properties: + dependsOn: + description: DependsOn specifies what this application depends on + properties: + blockOnEmpty: + description: BlockOnEmpty defines whether to block sync when the + list of applications determined by the selector is empty + type: boolean + refreshDependencies: + description: RefreshDependencies defines whether all dependencies + should be refreshed before starting a sync + type: boolean + selectors: + description: Selectors defines conditions for matching application's + dependencies + items: + description: ApplicationSelector specifies which applications + this Application depends on + properties: + labelSelector: + description: LabelSelector selects applications by their + labels + properties: + matchExpressions: + description: matchExpressions is a list of label selector + requirements. The requirements are ANDed. + items: + description: A label selector requirement is a selector + that contains values, a key, and an operator that + relates the key and values. + properties: + key: + description: key is the label key that the selector + applies to. + type: string + operator: + description: operator represents a key's relationship + to a set of values. Valid operators are In, + NotIn, Exists and DoesNotExist. + type: string + values: + description: values is an array of string values. + If the operator is In or NotIn, the values array + must be non-empty. If the operator is Exists + or DoesNotExist, the values array must be empty. + This array is replaced during a strategic merge + patch. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a map of {key,value} pairs. + A single {key,value} in the matchLabels map is equivalent + to an element of matchExpressions, whose key field + is "key", the operator is "In", and the values array + contains only "value". The requirements are ANDed. + type: object + type: object + namePattern: + description: NamePattern selects applications by matching + their names + items: + type: string + type: array + type: object + type: array + syncDelay: + description: SyncDelay specifies the duration in seconds to wait + before starting to sync when dependencies are defined + format: int64 + type: integer + timeout: + description: Timeout defines the maximum duration in seconds to + wait on dependencies before the sync fails + format: int64 + type: integer + required: + - selectors + type: object destination: description: Destination is a reference to the target Kubernetes server and namespace @@ -2463,6 +2548,10 @@ spec: description: OperationState contains information about any ongoing operations, such as a sync properties: + blockedOnEmpty: + description: BlockedOnEmpty is true when the application is waiting + for any dependency to be created + type: boolean finishedAt: description: FinishedAt contains time of operation completion format: date-time @@ -4130,6 +4219,23 @@ spec: required: - revision type: object + waitingFor: + description: WaitingFor specifies a list of applications that + this operation is waiting for + items: + properties: + applicationName: + type: string + applicationNamespace: + type: string + refreshedAt: + format: date-time + type: string + required: + - applicationName + - applicationNamespace + type: object + type: array required: - operation - phase @@ -5097,6 +5203,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -5688,6 +5841,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -6283,6 +6483,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -6858,6 +7105,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -7457,6 +7751,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -8048,6 +8389,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -8643,6 +9031,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -9218,24 +9653,71 @@ spec: type: object spec: properties: - destination: + dependsOn: properties: - name: - type: string - namespace: - type: string - server: - type: string - type: object - ignoreDifferences: - items: - properties: - group: - type: string - jqPathExpressions: - items: - type: string - type: array + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object + destination: + properties: + name: + type: string + namespace: + type: string + server: + type: string + type: object + ignoreDifferences: + items: + properties: + group: + type: string + jqPathExpressions: + items: + type: string + type: array jsonPointers: items: type: string @@ -9803,6 +10285,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -10568,6 +11097,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -11328,6 +11904,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -11917,6 +12540,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -12516,6 +13186,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -13107,6 +13824,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -13702,6 +14466,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -14277,6 +15088,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -14862,6 +15720,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -15627,6 +16532,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -16387,6 +17339,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -16980,6 +17979,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -17562,6 +18608,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -18327,6 +19420,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -19087,6 +20227,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: @@ -19751,6 +20938,53 @@ spec: type: object spec: properties: + dependsOn: + properties: + blockOnEmpty: + type: boolean + refreshDependencies: + type: boolean + selectors: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namePattern: + items: + type: string + type: array + type: object + type: array + syncDelay: + format: int64 + type: integer + timeout: + format: int64 + type: integer + required: + - selectors + type: object destination: properties: name: diff --git a/mkdocs.yml b/mkdocs.yml index a7e8f86e216ccb..0f366a9b01b8e0 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -176,6 +176,7 @@ nav: - user-guide/resource_tracking.md - user-guide/resource_hooks.md - user-guide/selective_sync.md + - user-guide/app_dependencies.md - user-guide/sync-waves.md - user-guide/sync_windows.md - user-guide/sync-kubectl.md diff --git a/pkg/apis/api-rules/violation_exceptions.list b/pkg/apis/api-rules/violation_exceptions.list index 2b0f2e90d00a92..66bab28a6e0a63 100644 --- a/pkg/apis/api-rules/violation_exceptions.list +++ b/pkg/apis/api-rules/violation_exceptions.list @@ -7,9 +7,11 @@ API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/ap API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,AppProjectSpec,SignatureKeys API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,AppProjectSpec,SourceNamespaces API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,AppProjectSpec,SourceRepos +API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,ApplicationDependency,Selectors API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,ApplicationMatchExpression,Values API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,ApplicationPreservedFields,Annotations API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,ApplicationPreservedFields,Labels +API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,ApplicationSelector,NamePattern API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,ApplicationSetResourceIgnoreDifferences,JQPathExpressions API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,ApplicationSetResourceIgnoreDifferences,JSONPointers API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,ApplicationSetRolloutStep,MatchExpressions @@ -52,6 +54,7 @@ API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/ap API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,MergeGenerator,MergeKeys API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,NestedMergeGenerator,MergeKeys API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,Operation,Info +API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,OperationState,WaitingFor API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,OptionalArray,Array API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,OrphanedResourcesMonitorSettings,Ignore API rule violation: list_type_missing,github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1,OverrideIgnoreDiff,JQPathExpressions diff --git a/pkg/apis/application/v1alpha1/generated.pb.go b/pkg/apis/application/v1alpha1/generated.pb.go index f6a253d23ed7d4..94096c154ce5ca 100644 --- a/pkg/apis/application/v1alpha1/generated.pb.go +++ b/pkg/apis/application/v1alpha1/generated.pb.go @@ -23,6 +23,7 @@ import ( math_bits "math/bits" reflect "reflect" strings "strings" + time "time" intstr "k8s.io/apimachinery/pkg/util/intstr" k8s_io_apimachinery_pkg_watch "k8s.io/apimachinery/pkg/watch" @@ -32,6 +33,7 @@ import ( var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +var _ = time.Kitchen // This is a compile-time assertion to ensure that this generated file // is compatible with the proto package it is being compiled against. @@ -235,10 +237,38 @@ func (m *ApplicationCondition) XXX_DiscardUnknown() { var xxx_messageInfo_ApplicationCondition proto.InternalMessageInfo +func (m *ApplicationDependency) Reset() { *m = ApplicationDependency{} } +func (*ApplicationDependency) ProtoMessage() {} +func (*ApplicationDependency) Descriptor() ([]byte, []int) { + return fileDescriptor_030104ce3b95bcac, []int{7} +} +func (m *ApplicationDependency) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ApplicationDependency) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *ApplicationDependency) XXX_Merge(src proto.Message) { + xxx_messageInfo_ApplicationDependency.Merge(m, src) +} +func (m *ApplicationDependency) XXX_Size() int { + return m.Size() +} +func (m *ApplicationDependency) XXX_DiscardUnknown() { + xxx_messageInfo_ApplicationDependency.DiscardUnknown(m) +} + +var xxx_messageInfo_ApplicationDependency proto.InternalMessageInfo + func (m *ApplicationDestination) Reset() { *m = ApplicationDestination{} } func (*ApplicationDestination) ProtoMessage() {} func (*ApplicationDestination) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{7} + return fileDescriptor_030104ce3b95bcac, []int{8} } func (m *ApplicationDestination) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -266,7 +296,7 @@ var xxx_messageInfo_ApplicationDestination proto.InternalMessageInfo func (m *ApplicationList) Reset() { *m = ApplicationList{} } func (*ApplicationList) ProtoMessage() {} func (*ApplicationList) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{8} + return fileDescriptor_030104ce3b95bcac, []int{9} } func (m *ApplicationList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -294,7 +324,7 @@ var xxx_messageInfo_ApplicationList proto.InternalMessageInfo func (m *ApplicationMatchExpression) Reset() { *m = ApplicationMatchExpression{} } func (*ApplicationMatchExpression) ProtoMessage() {} func (*ApplicationMatchExpression) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{9} + return fileDescriptor_030104ce3b95bcac, []int{10} } func (m *ApplicationMatchExpression) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -322,7 +352,7 @@ var xxx_messageInfo_ApplicationMatchExpression proto.InternalMessageInfo func (m *ApplicationPreservedFields) Reset() { *m = ApplicationPreservedFields{} } func (*ApplicationPreservedFields) ProtoMessage() {} func (*ApplicationPreservedFields) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{10} + return fileDescriptor_030104ce3b95bcac, []int{11} } func (m *ApplicationPreservedFields) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -347,10 +377,38 @@ func (m *ApplicationPreservedFields) XXX_DiscardUnknown() { var xxx_messageInfo_ApplicationPreservedFields proto.InternalMessageInfo +func (m *ApplicationSelector) Reset() { *m = ApplicationSelector{} } +func (*ApplicationSelector) ProtoMessage() {} +func (*ApplicationSelector) Descriptor() ([]byte, []int) { + return fileDescriptor_030104ce3b95bcac, []int{12} +} +func (m *ApplicationSelector) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ApplicationSelector) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *ApplicationSelector) XXX_Merge(src proto.Message) { + xxx_messageInfo_ApplicationSelector.Merge(m, src) +} +func (m *ApplicationSelector) XXX_Size() int { + return m.Size() +} +func (m *ApplicationSelector) XXX_DiscardUnknown() { + xxx_messageInfo_ApplicationSelector.DiscardUnknown(m) +} + +var xxx_messageInfo_ApplicationSelector proto.InternalMessageInfo + func (m *ApplicationSet) Reset() { *m = ApplicationSet{} } func (*ApplicationSet) ProtoMessage() {} func (*ApplicationSet) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{11} + return fileDescriptor_030104ce3b95bcac, []int{13} } func (m *ApplicationSet) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -378,7 +436,7 @@ var xxx_messageInfo_ApplicationSet proto.InternalMessageInfo func (m *ApplicationSetApplicationStatus) Reset() { *m = ApplicationSetApplicationStatus{} } func (*ApplicationSetApplicationStatus) ProtoMessage() {} func (*ApplicationSetApplicationStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{12} + return fileDescriptor_030104ce3b95bcac, []int{14} } func (m *ApplicationSetApplicationStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -406,7 +464,7 @@ var xxx_messageInfo_ApplicationSetApplicationStatus proto.InternalMessageInfo func (m *ApplicationSetCondition) Reset() { *m = ApplicationSetCondition{} } func (*ApplicationSetCondition) ProtoMessage() {} func (*ApplicationSetCondition) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{13} + return fileDescriptor_030104ce3b95bcac, []int{15} } func (m *ApplicationSetCondition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -434,7 +492,7 @@ var xxx_messageInfo_ApplicationSetCondition proto.InternalMessageInfo func (m *ApplicationSetGenerator) Reset() { *m = ApplicationSetGenerator{} } func (*ApplicationSetGenerator) ProtoMessage() {} func (*ApplicationSetGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{14} + return fileDescriptor_030104ce3b95bcac, []int{16} } func (m *ApplicationSetGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -462,7 +520,7 @@ var xxx_messageInfo_ApplicationSetGenerator proto.InternalMessageInfo func (m *ApplicationSetList) Reset() { *m = ApplicationSetList{} } func (*ApplicationSetList) ProtoMessage() {} func (*ApplicationSetList) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{15} + return fileDescriptor_030104ce3b95bcac, []int{17} } func (m *ApplicationSetList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -490,7 +548,7 @@ var xxx_messageInfo_ApplicationSetList proto.InternalMessageInfo func (m *ApplicationSetNestedGenerator) Reset() { *m = ApplicationSetNestedGenerator{} } func (*ApplicationSetNestedGenerator) ProtoMessage() {} func (*ApplicationSetNestedGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{16} + return fileDescriptor_030104ce3b95bcac, []int{18} } func (m *ApplicationSetNestedGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -520,7 +578,7 @@ func (m *ApplicationSetResourceIgnoreDifferences) Reset() { } func (*ApplicationSetResourceIgnoreDifferences) ProtoMessage() {} func (*ApplicationSetResourceIgnoreDifferences) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{17} + return fileDescriptor_030104ce3b95bcac, []int{19} } func (m *ApplicationSetResourceIgnoreDifferences) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -548,7 +606,7 @@ var xxx_messageInfo_ApplicationSetResourceIgnoreDifferences proto.InternalMessag func (m *ApplicationSetRolloutStep) Reset() { *m = ApplicationSetRolloutStep{} } func (*ApplicationSetRolloutStep) ProtoMessage() {} func (*ApplicationSetRolloutStep) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{18} + return fileDescriptor_030104ce3b95bcac, []int{20} } func (m *ApplicationSetRolloutStep) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -576,7 +634,7 @@ var xxx_messageInfo_ApplicationSetRolloutStep proto.InternalMessageInfo func (m *ApplicationSetRolloutStrategy) Reset() { *m = ApplicationSetRolloutStrategy{} } func (*ApplicationSetRolloutStrategy) ProtoMessage() {} func (*ApplicationSetRolloutStrategy) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{19} + return fileDescriptor_030104ce3b95bcac, []int{21} } func (m *ApplicationSetRolloutStrategy) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -604,7 +662,7 @@ var xxx_messageInfo_ApplicationSetRolloutStrategy proto.InternalMessageInfo func (m *ApplicationSetSpec) Reset() { *m = ApplicationSetSpec{} } func (*ApplicationSetSpec) ProtoMessage() {} func (*ApplicationSetSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{20} + return fileDescriptor_030104ce3b95bcac, []int{22} } func (m *ApplicationSetSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -632,7 +690,7 @@ var xxx_messageInfo_ApplicationSetSpec proto.InternalMessageInfo func (m *ApplicationSetStatus) Reset() { *m = ApplicationSetStatus{} } func (*ApplicationSetStatus) ProtoMessage() {} func (*ApplicationSetStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{21} + return fileDescriptor_030104ce3b95bcac, []int{23} } func (m *ApplicationSetStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -660,7 +718,7 @@ var xxx_messageInfo_ApplicationSetStatus proto.InternalMessageInfo func (m *ApplicationSetStrategy) Reset() { *m = ApplicationSetStrategy{} } func (*ApplicationSetStrategy) ProtoMessage() {} func (*ApplicationSetStrategy) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{22} + return fileDescriptor_030104ce3b95bcac, []int{24} } func (m *ApplicationSetStrategy) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -688,7 +746,7 @@ var xxx_messageInfo_ApplicationSetStrategy proto.InternalMessageInfo func (m *ApplicationSetSyncPolicy) Reset() { *m = ApplicationSetSyncPolicy{} } func (*ApplicationSetSyncPolicy) ProtoMessage() {} func (*ApplicationSetSyncPolicy) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{23} + return fileDescriptor_030104ce3b95bcac, []int{25} } func (m *ApplicationSetSyncPolicy) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -716,7 +774,7 @@ var xxx_messageInfo_ApplicationSetSyncPolicy proto.InternalMessageInfo func (m *ApplicationSetTemplate) Reset() { *m = ApplicationSetTemplate{} } func (*ApplicationSetTemplate) ProtoMessage() {} func (*ApplicationSetTemplate) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{24} + return fileDescriptor_030104ce3b95bcac, []int{26} } func (m *ApplicationSetTemplate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -744,7 +802,7 @@ var xxx_messageInfo_ApplicationSetTemplate proto.InternalMessageInfo func (m *ApplicationSetTemplateMeta) Reset() { *m = ApplicationSetTemplateMeta{} } func (*ApplicationSetTemplateMeta) ProtoMessage() {} func (*ApplicationSetTemplateMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{25} + return fileDescriptor_030104ce3b95bcac, []int{27} } func (m *ApplicationSetTemplateMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -772,7 +830,7 @@ var xxx_messageInfo_ApplicationSetTemplateMeta proto.InternalMessageInfo func (m *ApplicationSetTerminalGenerator) Reset() { *m = ApplicationSetTerminalGenerator{} } func (*ApplicationSetTerminalGenerator) ProtoMessage() {} func (*ApplicationSetTerminalGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{26} + return fileDescriptor_030104ce3b95bcac, []int{28} } func (m *ApplicationSetTerminalGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -800,7 +858,7 @@ var xxx_messageInfo_ApplicationSetTerminalGenerator proto.InternalMessageInfo func (m *ApplicationSource) Reset() { *m = ApplicationSource{} } func (*ApplicationSource) ProtoMessage() {} func (*ApplicationSource) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{27} + return fileDescriptor_030104ce3b95bcac, []int{29} } func (m *ApplicationSource) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -828,7 +886,7 @@ var xxx_messageInfo_ApplicationSource proto.InternalMessageInfo func (m *ApplicationSourceDirectory) Reset() { *m = ApplicationSourceDirectory{} } func (*ApplicationSourceDirectory) ProtoMessage() {} func (*ApplicationSourceDirectory) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{28} + return fileDescriptor_030104ce3b95bcac, []int{30} } func (m *ApplicationSourceDirectory) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -856,7 +914,7 @@ var xxx_messageInfo_ApplicationSourceDirectory proto.InternalMessageInfo func (m *ApplicationSourceHelm) Reset() { *m = ApplicationSourceHelm{} } func (*ApplicationSourceHelm) ProtoMessage() {} func (*ApplicationSourceHelm) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{29} + return fileDescriptor_030104ce3b95bcac, []int{31} } func (m *ApplicationSourceHelm) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -884,7 +942,7 @@ var xxx_messageInfo_ApplicationSourceHelm proto.InternalMessageInfo func (m *ApplicationSourceJsonnet) Reset() { *m = ApplicationSourceJsonnet{} } func (*ApplicationSourceJsonnet) ProtoMessage() {} func (*ApplicationSourceJsonnet) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{30} + return fileDescriptor_030104ce3b95bcac, []int{32} } func (m *ApplicationSourceJsonnet) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -912,7 +970,7 @@ var xxx_messageInfo_ApplicationSourceJsonnet proto.InternalMessageInfo func (m *ApplicationSourceKustomize) Reset() { *m = ApplicationSourceKustomize{} } func (*ApplicationSourceKustomize) ProtoMessage() {} func (*ApplicationSourceKustomize) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{31} + return fileDescriptor_030104ce3b95bcac, []int{33} } func (m *ApplicationSourceKustomize) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -940,7 +998,7 @@ var xxx_messageInfo_ApplicationSourceKustomize proto.InternalMessageInfo func (m *ApplicationSourcePlugin) Reset() { *m = ApplicationSourcePlugin{} } func (*ApplicationSourcePlugin) ProtoMessage() {} func (*ApplicationSourcePlugin) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{32} + return fileDescriptor_030104ce3b95bcac, []int{34} } func (m *ApplicationSourcePlugin) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -968,7 +1026,7 @@ var xxx_messageInfo_ApplicationSourcePlugin proto.InternalMessageInfo func (m *ApplicationSourcePluginParameter) Reset() { *m = ApplicationSourcePluginParameter{} } func (*ApplicationSourcePluginParameter) ProtoMessage() {} func (*ApplicationSourcePluginParameter) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{33} + return fileDescriptor_030104ce3b95bcac, []int{35} } func (m *ApplicationSourcePluginParameter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -996,7 +1054,7 @@ var xxx_messageInfo_ApplicationSourcePluginParameter proto.InternalMessageInfo func (m *ApplicationSpec) Reset() { *m = ApplicationSpec{} } func (*ApplicationSpec) ProtoMessage() {} func (*ApplicationSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{34} + return fileDescriptor_030104ce3b95bcac, []int{36} } func (m *ApplicationSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1024,7 +1082,7 @@ var xxx_messageInfo_ApplicationSpec proto.InternalMessageInfo func (m *ApplicationStatus) Reset() { *m = ApplicationStatus{} } func (*ApplicationStatus) ProtoMessage() {} func (*ApplicationStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{35} + return fileDescriptor_030104ce3b95bcac, []int{37} } func (m *ApplicationStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1052,7 +1110,7 @@ var xxx_messageInfo_ApplicationStatus proto.InternalMessageInfo func (m *ApplicationSummary) Reset() { *m = ApplicationSummary{} } func (*ApplicationSummary) ProtoMessage() {} func (*ApplicationSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{36} + return fileDescriptor_030104ce3b95bcac, []int{38} } func (m *ApplicationSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1080,7 +1138,7 @@ var xxx_messageInfo_ApplicationSummary proto.InternalMessageInfo func (m *ApplicationTree) Reset() { *m = ApplicationTree{} } func (*ApplicationTree) ProtoMessage() {} func (*ApplicationTree) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{37} + return fileDescriptor_030104ce3b95bcac, []int{39} } func (m *ApplicationTree) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1108,7 +1166,7 @@ var xxx_messageInfo_ApplicationTree proto.InternalMessageInfo func (m *ApplicationWatchEvent) Reset() { *m = ApplicationWatchEvent{} } func (*ApplicationWatchEvent) ProtoMessage() {} func (*ApplicationWatchEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{38} + return fileDescriptor_030104ce3b95bcac, []int{40} } func (m *ApplicationWatchEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1136,7 +1194,7 @@ var xxx_messageInfo_ApplicationWatchEvent proto.InternalMessageInfo func (m *Backoff) Reset() { *m = Backoff{} } func (*Backoff) ProtoMessage() {} func (*Backoff) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{39} + return fileDescriptor_030104ce3b95bcac, []int{41} } func (m *Backoff) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1164,7 +1222,7 @@ var xxx_messageInfo_Backoff proto.InternalMessageInfo func (m *BasicAuthBitbucketServer) Reset() { *m = BasicAuthBitbucketServer{} } func (*BasicAuthBitbucketServer) ProtoMessage() {} func (*BasicAuthBitbucketServer) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{40} + return fileDescriptor_030104ce3b95bcac, []int{42} } func (m *BasicAuthBitbucketServer) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1192,7 +1250,7 @@ var xxx_messageInfo_BasicAuthBitbucketServer proto.InternalMessageInfo func (m *BearerTokenBitbucketCloud) Reset() { *m = BearerTokenBitbucketCloud{} } func (*BearerTokenBitbucketCloud) ProtoMessage() {} func (*BearerTokenBitbucketCloud) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{41} + return fileDescriptor_030104ce3b95bcac, []int{43} } func (m *BearerTokenBitbucketCloud) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1220,7 +1278,7 @@ var xxx_messageInfo_BearerTokenBitbucketCloud proto.InternalMessageInfo func (m *ChartDetails) Reset() { *m = ChartDetails{} } func (*ChartDetails) ProtoMessage() {} func (*ChartDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{42} + return fileDescriptor_030104ce3b95bcac, []int{44} } func (m *ChartDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1248,7 +1306,7 @@ var xxx_messageInfo_ChartDetails proto.InternalMessageInfo func (m *Cluster) Reset() { *m = Cluster{} } func (*Cluster) ProtoMessage() {} func (*Cluster) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{43} + return fileDescriptor_030104ce3b95bcac, []int{45} } func (m *Cluster) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1276,7 +1334,7 @@ var xxx_messageInfo_Cluster proto.InternalMessageInfo func (m *ClusterCacheInfo) Reset() { *m = ClusterCacheInfo{} } func (*ClusterCacheInfo) ProtoMessage() {} func (*ClusterCacheInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{44} + return fileDescriptor_030104ce3b95bcac, []int{46} } func (m *ClusterCacheInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1304,7 +1362,7 @@ var xxx_messageInfo_ClusterCacheInfo proto.InternalMessageInfo func (m *ClusterConfig) Reset() { *m = ClusterConfig{} } func (*ClusterConfig) ProtoMessage() {} func (*ClusterConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{45} + return fileDescriptor_030104ce3b95bcac, []int{47} } func (m *ClusterConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1332,7 +1390,7 @@ var xxx_messageInfo_ClusterConfig proto.InternalMessageInfo func (m *ClusterGenerator) Reset() { *m = ClusterGenerator{} } func (*ClusterGenerator) ProtoMessage() {} func (*ClusterGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{46} + return fileDescriptor_030104ce3b95bcac, []int{48} } func (m *ClusterGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1360,7 +1418,7 @@ var xxx_messageInfo_ClusterGenerator proto.InternalMessageInfo func (m *ClusterInfo) Reset() { *m = ClusterInfo{} } func (*ClusterInfo) ProtoMessage() {} func (*ClusterInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{47} + return fileDescriptor_030104ce3b95bcac, []int{49} } func (m *ClusterInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1388,7 +1446,7 @@ var xxx_messageInfo_ClusterInfo proto.InternalMessageInfo func (m *ClusterList) Reset() { *m = ClusterList{} } func (*ClusterList) ProtoMessage() {} func (*ClusterList) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{48} + return fileDescriptor_030104ce3b95bcac, []int{50} } func (m *ClusterList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1416,7 +1474,7 @@ var xxx_messageInfo_ClusterList proto.InternalMessageInfo func (m *Command) Reset() { *m = Command{} } func (*Command) ProtoMessage() {} func (*Command) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{49} + return fileDescriptor_030104ce3b95bcac, []int{51} } func (m *Command) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1444,7 +1502,7 @@ var xxx_messageInfo_Command proto.InternalMessageInfo func (m *ComparedTo) Reset() { *m = ComparedTo{} } func (*ComparedTo) ProtoMessage() {} func (*ComparedTo) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{50} + return fileDescriptor_030104ce3b95bcac, []int{52} } func (m *ComparedTo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1472,7 +1530,7 @@ var xxx_messageInfo_ComparedTo proto.InternalMessageInfo func (m *ComponentParameter) Reset() { *m = ComponentParameter{} } func (*ComponentParameter) ProtoMessage() {} func (*ComponentParameter) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{51} + return fileDescriptor_030104ce3b95bcac, []int{53} } func (m *ComponentParameter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1500,7 +1558,7 @@ var xxx_messageInfo_ComponentParameter proto.InternalMessageInfo func (m *ConfigManagementPlugin) Reset() { *m = ConfigManagementPlugin{} } func (*ConfigManagementPlugin) ProtoMessage() {} func (*ConfigManagementPlugin) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{52} + return fileDescriptor_030104ce3b95bcac, []int{54} } func (m *ConfigManagementPlugin) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1528,7 +1586,7 @@ var xxx_messageInfo_ConfigManagementPlugin proto.InternalMessageInfo func (m *ConnectionState) Reset() { *m = ConnectionState{} } func (*ConnectionState) ProtoMessage() {} func (*ConnectionState) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{53} + return fileDescriptor_030104ce3b95bcac, []int{55} } func (m *ConnectionState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1556,7 +1614,7 @@ var xxx_messageInfo_ConnectionState proto.InternalMessageInfo func (m *DuckTypeGenerator) Reset() { *m = DuckTypeGenerator{} } func (*DuckTypeGenerator) ProtoMessage() {} func (*DuckTypeGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{54} + return fileDescriptor_030104ce3b95bcac, []int{56} } func (m *DuckTypeGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1584,7 +1642,7 @@ var xxx_messageInfo_DuckTypeGenerator proto.InternalMessageInfo func (m *EnvEntry) Reset() { *m = EnvEntry{} } func (*EnvEntry) ProtoMessage() {} func (*EnvEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{55} + return fileDescriptor_030104ce3b95bcac, []int{57} } func (m *EnvEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1612,7 +1670,7 @@ var xxx_messageInfo_EnvEntry proto.InternalMessageInfo func (m *ExecProviderConfig) Reset() { *m = ExecProviderConfig{} } func (*ExecProviderConfig) ProtoMessage() {} func (*ExecProviderConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{56} + return fileDescriptor_030104ce3b95bcac, []int{58} } func (m *ExecProviderConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1640,7 +1698,7 @@ var xxx_messageInfo_ExecProviderConfig proto.InternalMessageInfo func (m *GitDirectoryGeneratorItem) Reset() { *m = GitDirectoryGeneratorItem{} } func (*GitDirectoryGeneratorItem) ProtoMessage() {} func (*GitDirectoryGeneratorItem) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{57} + return fileDescriptor_030104ce3b95bcac, []int{59} } func (m *GitDirectoryGeneratorItem) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1668,7 +1726,7 @@ var xxx_messageInfo_GitDirectoryGeneratorItem proto.InternalMessageInfo func (m *GitFileGeneratorItem) Reset() { *m = GitFileGeneratorItem{} } func (*GitFileGeneratorItem) ProtoMessage() {} func (*GitFileGeneratorItem) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{58} + return fileDescriptor_030104ce3b95bcac, []int{60} } func (m *GitFileGeneratorItem) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1696,7 +1754,7 @@ var xxx_messageInfo_GitFileGeneratorItem proto.InternalMessageInfo func (m *GitGenerator) Reset() { *m = GitGenerator{} } func (*GitGenerator) ProtoMessage() {} func (*GitGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{59} + return fileDescriptor_030104ce3b95bcac, []int{61} } func (m *GitGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1724,7 +1782,7 @@ var xxx_messageInfo_GitGenerator proto.InternalMessageInfo func (m *GnuPGPublicKey) Reset() { *m = GnuPGPublicKey{} } func (*GnuPGPublicKey) ProtoMessage() {} func (*GnuPGPublicKey) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{60} + return fileDescriptor_030104ce3b95bcac, []int{62} } func (m *GnuPGPublicKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1752,7 +1810,7 @@ var xxx_messageInfo_GnuPGPublicKey proto.InternalMessageInfo func (m *GnuPGPublicKeyList) Reset() { *m = GnuPGPublicKeyList{} } func (*GnuPGPublicKeyList) ProtoMessage() {} func (*GnuPGPublicKeyList) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{61} + return fileDescriptor_030104ce3b95bcac, []int{63} } func (m *GnuPGPublicKeyList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1780,7 +1838,7 @@ var xxx_messageInfo_GnuPGPublicKeyList proto.InternalMessageInfo func (m *HealthStatus) Reset() { *m = HealthStatus{} } func (*HealthStatus) ProtoMessage() {} func (*HealthStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{62} + return fileDescriptor_030104ce3b95bcac, []int{64} } func (m *HealthStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1808,7 +1866,7 @@ var xxx_messageInfo_HealthStatus proto.InternalMessageInfo func (m *HelmFileParameter) Reset() { *m = HelmFileParameter{} } func (*HelmFileParameter) ProtoMessage() {} func (*HelmFileParameter) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{63} + return fileDescriptor_030104ce3b95bcac, []int{65} } func (m *HelmFileParameter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1836,7 +1894,7 @@ var xxx_messageInfo_HelmFileParameter proto.InternalMessageInfo func (m *HelmOptions) Reset() { *m = HelmOptions{} } func (*HelmOptions) ProtoMessage() {} func (*HelmOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{64} + return fileDescriptor_030104ce3b95bcac, []int{66} } func (m *HelmOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1864,7 +1922,7 @@ var xxx_messageInfo_HelmOptions proto.InternalMessageInfo func (m *HelmParameter) Reset() { *m = HelmParameter{} } func (*HelmParameter) ProtoMessage() {} func (*HelmParameter) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{65} + return fileDescriptor_030104ce3b95bcac, []int{67} } func (m *HelmParameter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1892,7 +1950,7 @@ var xxx_messageInfo_HelmParameter proto.InternalMessageInfo func (m *HostInfo) Reset() { *m = HostInfo{} } func (*HostInfo) ProtoMessage() {} func (*HostInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{66} + return fileDescriptor_030104ce3b95bcac, []int{68} } func (m *HostInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1920,7 +1978,7 @@ var xxx_messageInfo_HostInfo proto.InternalMessageInfo func (m *HostResourceInfo) Reset() { *m = HostResourceInfo{} } func (*HostResourceInfo) ProtoMessage() {} func (*HostResourceInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{67} + return fileDescriptor_030104ce3b95bcac, []int{69} } func (m *HostResourceInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1948,7 +2006,7 @@ var xxx_messageInfo_HostResourceInfo proto.InternalMessageInfo func (m *Info) Reset() { *m = Info{} } func (*Info) ProtoMessage() {} func (*Info) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{68} + return fileDescriptor_030104ce3b95bcac, []int{70} } func (m *Info) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1976,7 +2034,7 @@ var xxx_messageInfo_Info proto.InternalMessageInfo func (m *InfoItem) Reset() { *m = InfoItem{} } func (*InfoItem) ProtoMessage() {} func (*InfoItem) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{69} + return fileDescriptor_030104ce3b95bcac, []int{71} } func (m *InfoItem) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2004,7 +2062,7 @@ var xxx_messageInfo_InfoItem proto.InternalMessageInfo func (m *JWTToken) Reset() { *m = JWTToken{} } func (*JWTToken) ProtoMessage() {} func (*JWTToken) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{70} + return fileDescriptor_030104ce3b95bcac, []int{72} } func (m *JWTToken) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2032,7 +2090,7 @@ var xxx_messageInfo_JWTToken proto.InternalMessageInfo func (m *JWTTokens) Reset() { *m = JWTTokens{} } func (*JWTTokens) ProtoMessage() {} func (*JWTTokens) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{71} + return fileDescriptor_030104ce3b95bcac, []int{73} } func (m *JWTTokens) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2060,7 +2118,7 @@ var xxx_messageInfo_JWTTokens proto.InternalMessageInfo func (m *JsonnetVar) Reset() { *m = JsonnetVar{} } func (*JsonnetVar) ProtoMessage() {} func (*JsonnetVar) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{72} + return fileDescriptor_030104ce3b95bcac, []int{74} } func (m *JsonnetVar) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2088,7 +2146,7 @@ var xxx_messageInfo_JsonnetVar proto.InternalMessageInfo func (m *KnownTypeField) Reset() { *m = KnownTypeField{} } func (*KnownTypeField) ProtoMessage() {} func (*KnownTypeField) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{73} + return fileDescriptor_030104ce3b95bcac, []int{75} } func (m *KnownTypeField) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2116,7 +2174,7 @@ var xxx_messageInfo_KnownTypeField proto.InternalMessageInfo func (m *KustomizeGvk) Reset() { *m = KustomizeGvk{} } func (*KustomizeGvk) ProtoMessage() {} func (*KustomizeGvk) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{74} + return fileDescriptor_030104ce3b95bcac, []int{76} } func (m *KustomizeGvk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2144,7 +2202,7 @@ var xxx_messageInfo_KustomizeGvk proto.InternalMessageInfo func (m *KustomizeOptions) Reset() { *m = KustomizeOptions{} } func (*KustomizeOptions) ProtoMessage() {} func (*KustomizeOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{75} + return fileDescriptor_030104ce3b95bcac, []int{77} } func (m *KustomizeOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2172,7 +2230,7 @@ var xxx_messageInfo_KustomizeOptions proto.InternalMessageInfo func (m *KustomizePatch) Reset() { *m = KustomizePatch{} } func (*KustomizePatch) ProtoMessage() {} func (*KustomizePatch) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{76} + return fileDescriptor_030104ce3b95bcac, []int{78} } func (m *KustomizePatch) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2200,7 +2258,7 @@ var xxx_messageInfo_KustomizePatch proto.InternalMessageInfo func (m *KustomizeReplica) Reset() { *m = KustomizeReplica{} } func (*KustomizeReplica) ProtoMessage() {} func (*KustomizeReplica) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{77} + return fileDescriptor_030104ce3b95bcac, []int{79} } func (m *KustomizeReplica) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2228,7 +2286,7 @@ var xxx_messageInfo_KustomizeReplica proto.InternalMessageInfo func (m *KustomizeResId) Reset() { *m = KustomizeResId{} } func (*KustomizeResId) ProtoMessage() {} func (*KustomizeResId) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{78} + return fileDescriptor_030104ce3b95bcac, []int{80} } func (m *KustomizeResId) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2256,7 +2314,7 @@ var xxx_messageInfo_KustomizeResId proto.InternalMessageInfo func (m *KustomizeSelector) Reset() { *m = KustomizeSelector{} } func (*KustomizeSelector) ProtoMessage() {} func (*KustomizeSelector) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{79} + return fileDescriptor_030104ce3b95bcac, []int{81} } func (m *KustomizeSelector) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2284,7 +2342,7 @@ var xxx_messageInfo_KustomizeSelector proto.InternalMessageInfo func (m *ListGenerator) Reset() { *m = ListGenerator{} } func (*ListGenerator) ProtoMessage() {} func (*ListGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{80} + return fileDescriptor_030104ce3b95bcac, []int{82} } func (m *ListGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2312,7 +2370,7 @@ var xxx_messageInfo_ListGenerator proto.InternalMessageInfo func (m *ManagedNamespaceMetadata) Reset() { *m = ManagedNamespaceMetadata{} } func (*ManagedNamespaceMetadata) ProtoMessage() {} func (*ManagedNamespaceMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{81} + return fileDescriptor_030104ce3b95bcac, []int{83} } func (m *ManagedNamespaceMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2340,7 +2398,7 @@ var xxx_messageInfo_ManagedNamespaceMetadata proto.InternalMessageInfo func (m *MatrixGenerator) Reset() { *m = MatrixGenerator{} } func (*MatrixGenerator) ProtoMessage() {} func (*MatrixGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{82} + return fileDescriptor_030104ce3b95bcac, []int{84} } func (m *MatrixGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2368,7 +2426,7 @@ var xxx_messageInfo_MatrixGenerator proto.InternalMessageInfo func (m *MergeGenerator) Reset() { *m = MergeGenerator{} } func (*MergeGenerator) ProtoMessage() {} func (*MergeGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{83} + return fileDescriptor_030104ce3b95bcac, []int{85} } func (m *MergeGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2396,7 +2454,7 @@ var xxx_messageInfo_MergeGenerator proto.InternalMessageInfo func (m *NestedMatrixGenerator) Reset() { *m = NestedMatrixGenerator{} } func (*NestedMatrixGenerator) ProtoMessage() {} func (*NestedMatrixGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{84} + return fileDescriptor_030104ce3b95bcac, []int{86} } func (m *NestedMatrixGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2424,7 +2482,7 @@ var xxx_messageInfo_NestedMatrixGenerator proto.InternalMessageInfo func (m *NestedMergeGenerator) Reset() { *m = NestedMergeGenerator{} } func (*NestedMergeGenerator) ProtoMessage() {} func (*NestedMergeGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{85} + return fileDescriptor_030104ce3b95bcac, []int{87} } func (m *NestedMergeGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2452,7 +2510,7 @@ var xxx_messageInfo_NestedMergeGenerator proto.InternalMessageInfo func (m *Operation) Reset() { *m = Operation{} } func (*Operation) ProtoMessage() {} func (*Operation) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{86} + return fileDescriptor_030104ce3b95bcac, []int{88} } func (m *Operation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2480,7 +2538,7 @@ var xxx_messageInfo_Operation proto.InternalMessageInfo func (m *OperationInitiator) Reset() { *m = OperationInitiator{} } func (*OperationInitiator) ProtoMessage() {} func (*OperationInitiator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{87} + return fileDescriptor_030104ce3b95bcac, []int{89} } func (m *OperationInitiator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2508,7 +2566,7 @@ var xxx_messageInfo_OperationInitiator proto.InternalMessageInfo func (m *OperationState) Reset() { *m = OperationState{} } func (*OperationState) ProtoMessage() {} func (*OperationState) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{88} + return fileDescriptor_030104ce3b95bcac, []int{90} } func (m *OperationState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2536,7 +2594,7 @@ var xxx_messageInfo_OperationState proto.InternalMessageInfo func (m *OptionalArray) Reset() { *m = OptionalArray{} } func (*OptionalArray) ProtoMessage() {} func (*OptionalArray) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{89} + return fileDescriptor_030104ce3b95bcac, []int{91} } func (m *OptionalArray) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2564,7 +2622,7 @@ var xxx_messageInfo_OptionalArray proto.InternalMessageInfo func (m *OptionalMap) Reset() { *m = OptionalMap{} } func (*OptionalMap) ProtoMessage() {} func (*OptionalMap) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{90} + return fileDescriptor_030104ce3b95bcac, []int{92} } func (m *OptionalMap) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2592,7 +2650,7 @@ var xxx_messageInfo_OptionalMap proto.InternalMessageInfo func (m *OrphanedResourceKey) Reset() { *m = OrphanedResourceKey{} } func (*OrphanedResourceKey) ProtoMessage() {} func (*OrphanedResourceKey) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{91} + return fileDescriptor_030104ce3b95bcac, []int{93} } func (m *OrphanedResourceKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2620,7 +2678,7 @@ var xxx_messageInfo_OrphanedResourceKey proto.InternalMessageInfo func (m *OrphanedResourcesMonitorSettings) Reset() { *m = OrphanedResourcesMonitorSettings{} } func (*OrphanedResourcesMonitorSettings) ProtoMessage() {} func (*OrphanedResourcesMonitorSettings) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{92} + return fileDescriptor_030104ce3b95bcac, []int{94} } func (m *OrphanedResourcesMonitorSettings) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2648,7 +2706,7 @@ var xxx_messageInfo_OrphanedResourcesMonitorSettings proto.InternalMessageInfo func (m *OverrideIgnoreDiff) Reset() { *m = OverrideIgnoreDiff{} } func (*OverrideIgnoreDiff) ProtoMessage() {} func (*OverrideIgnoreDiff) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{93} + return fileDescriptor_030104ce3b95bcac, []int{95} } func (m *OverrideIgnoreDiff) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2676,7 +2734,7 @@ var xxx_messageInfo_OverrideIgnoreDiff proto.InternalMessageInfo func (m *PluginConfigMapRef) Reset() { *m = PluginConfigMapRef{} } func (*PluginConfigMapRef) ProtoMessage() {} func (*PluginConfigMapRef) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{94} + return fileDescriptor_030104ce3b95bcac, []int{96} } func (m *PluginConfigMapRef) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2704,7 +2762,7 @@ var xxx_messageInfo_PluginConfigMapRef proto.InternalMessageInfo func (m *PluginGenerator) Reset() { *m = PluginGenerator{} } func (*PluginGenerator) ProtoMessage() {} func (*PluginGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{95} + return fileDescriptor_030104ce3b95bcac, []int{97} } func (m *PluginGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2732,7 +2790,7 @@ var xxx_messageInfo_PluginGenerator proto.InternalMessageInfo func (m *PluginInput) Reset() { *m = PluginInput{} } func (*PluginInput) ProtoMessage() {} func (*PluginInput) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{96} + return fileDescriptor_030104ce3b95bcac, []int{98} } func (m *PluginInput) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2760,7 +2818,7 @@ var xxx_messageInfo_PluginInput proto.InternalMessageInfo func (m *ProjectRole) Reset() { *m = ProjectRole{} } func (*ProjectRole) ProtoMessage() {} func (*ProjectRole) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{97} + return fileDescriptor_030104ce3b95bcac, []int{99} } func (m *ProjectRole) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2788,7 +2846,7 @@ var xxx_messageInfo_ProjectRole proto.InternalMessageInfo func (m *PullRequestGenerator) Reset() { *m = PullRequestGenerator{} } func (*PullRequestGenerator) ProtoMessage() {} func (*PullRequestGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{98} + return fileDescriptor_030104ce3b95bcac, []int{100} } func (m *PullRequestGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2816,7 +2874,7 @@ var xxx_messageInfo_PullRequestGenerator proto.InternalMessageInfo func (m *PullRequestGeneratorAzureDevOps) Reset() { *m = PullRequestGeneratorAzureDevOps{} } func (*PullRequestGeneratorAzureDevOps) ProtoMessage() {} func (*PullRequestGeneratorAzureDevOps) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{99} + return fileDescriptor_030104ce3b95bcac, []int{101} } func (m *PullRequestGeneratorAzureDevOps) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2844,7 +2902,7 @@ var xxx_messageInfo_PullRequestGeneratorAzureDevOps proto.InternalMessageInfo func (m *PullRequestGeneratorBitbucket) Reset() { *m = PullRequestGeneratorBitbucket{} } func (*PullRequestGeneratorBitbucket) ProtoMessage() {} func (*PullRequestGeneratorBitbucket) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{100} + return fileDescriptor_030104ce3b95bcac, []int{102} } func (m *PullRequestGeneratorBitbucket) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2872,7 +2930,7 @@ var xxx_messageInfo_PullRequestGeneratorBitbucket proto.InternalMessageInfo func (m *PullRequestGeneratorBitbucketServer) Reset() { *m = PullRequestGeneratorBitbucketServer{} } func (*PullRequestGeneratorBitbucketServer) ProtoMessage() {} func (*PullRequestGeneratorBitbucketServer) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{101} + return fileDescriptor_030104ce3b95bcac, []int{103} } func (m *PullRequestGeneratorBitbucketServer) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2900,7 +2958,7 @@ var xxx_messageInfo_PullRequestGeneratorBitbucketServer proto.InternalMessageInf func (m *PullRequestGeneratorFilter) Reset() { *m = PullRequestGeneratorFilter{} } func (*PullRequestGeneratorFilter) ProtoMessage() {} func (*PullRequestGeneratorFilter) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{102} + return fileDescriptor_030104ce3b95bcac, []int{104} } func (m *PullRequestGeneratorFilter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2928,7 +2986,7 @@ var xxx_messageInfo_PullRequestGeneratorFilter proto.InternalMessageInfo func (m *PullRequestGeneratorGitLab) Reset() { *m = PullRequestGeneratorGitLab{} } func (*PullRequestGeneratorGitLab) ProtoMessage() {} func (*PullRequestGeneratorGitLab) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{103} + return fileDescriptor_030104ce3b95bcac, []int{105} } func (m *PullRequestGeneratorGitLab) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2956,7 +3014,7 @@ var xxx_messageInfo_PullRequestGeneratorGitLab proto.InternalMessageInfo func (m *PullRequestGeneratorGitea) Reset() { *m = PullRequestGeneratorGitea{} } func (*PullRequestGeneratorGitea) ProtoMessage() {} func (*PullRequestGeneratorGitea) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{104} + return fileDescriptor_030104ce3b95bcac, []int{106} } func (m *PullRequestGeneratorGitea) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2984,7 +3042,7 @@ var xxx_messageInfo_PullRequestGeneratorGitea proto.InternalMessageInfo func (m *PullRequestGeneratorGithub) Reset() { *m = PullRequestGeneratorGithub{} } func (*PullRequestGeneratorGithub) ProtoMessage() {} func (*PullRequestGeneratorGithub) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{105} + return fileDescriptor_030104ce3b95bcac, []int{107} } func (m *PullRequestGeneratorGithub) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3012,7 +3070,7 @@ var xxx_messageInfo_PullRequestGeneratorGithub proto.InternalMessageInfo func (m *RefTarget) Reset() { *m = RefTarget{} } func (*RefTarget) ProtoMessage() {} func (*RefTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{106} + return fileDescriptor_030104ce3b95bcac, []int{108} } func (m *RefTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3040,7 +3098,7 @@ var xxx_messageInfo_RefTarget proto.InternalMessageInfo func (m *RepoCreds) Reset() { *m = RepoCreds{} } func (*RepoCreds) ProtoMessage() {} func (*RepoCreds) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{107} + return fileDescriptor_030104ce3b95bcac, []int{109} } func (m *RepoCreds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3068,7 +3126,7 @@ var xxx_messageInfo_RepoCreds proto.InternalMessageInfo func (m *RepoCredsList) Reset() { *m = RepoCredsList{} } func (*RepoCredsList) ProtoMessage() {} func (*RepoCredsList) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{108} + return fileDescriptor_030104ce3b95bcac, []int{110} } func (m *RepoCredsList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3096,7 +3154,7 @@ var xxx_messageInfo_RepoCredsList proto.InternalMessageInfo func (m *Repository) Reset() { *m = Repository{} } func (*Repository) ProtoMessage() {} func (*Repository) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{109} + return fileDescriptor_030104ce3b95bcac, []int{111} } func (m *Repository) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3124,7 +3182,7 @@ var xxx_messageInfo_Repository proto.InternalMessageInfo func (m *RepositoryCertificate) Reset() { *m = RepositoryCertificate{} } func (*RepositoryCertificate) ProtoMessage() {} func (*RepositoryCertificate) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{110} + return fileDescriptor_030104ce3b95bcac, []int{112} } func (m *RepositoryCertificate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3152,7 +3210,7 @@ var xxx_messageInfo_RepositoryCertificate proto.InternalMessageInfo func (m *RepositoryCertificateList) Reset() { *m = RepositoryCertificateList{} } func (*RepositoryCertificateList) ProtoMessage() {} func (*RepositoryCertificateList) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{111} + return fileDescriptor_030104ce3b95bcac, []int{113} } func (m *RepositoryCertificateList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3180,7 +3238,7 @@ var xxx_messageInfo_RepositoryCertificateList proto.InternalMessageInfo func (m *RepositoryList) Reset() { *m = RepositoryList{} } func (*RepositoryList) ProtoMessage() {} func (*RepositoryList) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{112} + return fileDescriptor_030104ce3b95bcac, []int{114} } func (m *RepositoryList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3208,7 +3266,7 @@ var xxx_messageInfo_RepositoryList proto.InternalMessageInfo func (m *ResourceAction) Reset() { *m = ResourceAction{} } func (*ResourceAction) ProtoMessage() {} func (*ResourceAction) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{113} + return fileDescriptor_030104ce3b95bcac, []int{115} } func (m *ResourceAction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3236,7 +3294,7 @@ var xxx_messageInfo_ResourceAction proto.InternalMessageInfo func (m *ResourceActionDefinition) Reset() { *m = ResourceActionDefinition{} } func (*ResourceActionDefinition) ProtoMessage() {} func (*ResourceActionDefinition) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{114} + return fileDescriptor_030104ce3b95bcac, []int{116} } func (m *ResourceActionDefinition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3264,7 +3322,7 @@ var xxx_messageInfo_ResourceActionDefinition proto.InternalMessageInfo func (m *ResourceActionParam) Reset() { *m = ResourceActionParam{} } func (*ResourceActionParam) ProtoMessage() {} func (*ResourceActionParam) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{115} + return fileDescriptor_030104ce3b95bcac, []int{117} } func (m *ResourceActionParam) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3292,7 +3350,7 @@ var xxx_messageInfo_ResourceActionParam proto.InternalMessageInfo func (m *ResourceActions) Reset() { *m = ResourceActions{} } func (*ResourceActions) ProtoMessage() {} func (*ResourceActions) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{116} + return fileDescriptor_030104ce3b95bcac, []int{118} } func (m *ResourceActions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3320,7 +3378,7 @@ var xxx_messageInfo_ResourceActions proto.InternalMessageInfo func (m *ResourceDiff) Reset() { *m = ResourceDiff{} } func (*ResourceDiff) ProtoMessage() {} func (*ResourceDiff) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{117} + return fileDescriptor_030104ce3b95bcac, []int{119} } func (m *ResourceDiff) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3348,7 +3406,7 @@ var xxx_messageInfo_ResourceDiff proto.InternalMessageInfo func (m *ResourceIgnoreDifferences) Reset() { *m = ResourceIgnoreDifferences{} } func (*ResourceIgnoreDifferences) ProtoMessage() {} func (*ResourceIgnoreDifferences) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{118} + return fileDescriptor_030104ce3b95bcac, []int{120} } func (m *ResourceIgnoreDifferences) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3376,7 +3434,7 @@ var xxx_messageInfo_ResourceIgnoreDifferences proto.InternalMessageInfo func (m *ResourceNetworkingInfo) Reset() { *m = ResourceNetworkingInfo{} } func (*ResourceNetworkingInfo) ProtoMessage() {} func (*ResourceNetworkingInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{119} + return fileDescriptor_030104ce3b95bcac, []int{121} } func (m *ResourceNetworkingInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3404,7 +3462,7 @@ var xxx_messageInfo_ResourceNetworkingInfo proto.InternalMessageInfo func (m *ResourceNode) Reset() { *m = ResourceNode{} } func (*ResourceNode) ProtoMessage() {} func (*ResourceNode) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{120} + return fileDescriptor_030104ce3b95bcac, []int{122} } func (m *ResourceNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3432,7 +3490,7 @@ var xxx_messageInfo_ResourceNode proto.InternalMessageInfo func (m *ResourceOverride) Reset() { *m = ResourceOverride{} } func (*ResourceOverride) ProtoMessage() {} func (*ResourceOverride) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{121} + return fileDescriptor_030104ce3b95bcac, []int{123} } func (m *ResourceOverride) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3460,7 +3518,7 @@ var xxx_messageInfo_ResourceOverride proto.InternalMessageInfo func (m *ResourceRef) Reset() { *m = ResourceRef{} } func (*ResourceRef) ProtoMessage() {} func (*ResourceRef) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{122} + return fileDescriptor_030104ce3b95bcac, []int{124} } func (m *ResourceRef) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3488,7 +3546,7 @@ var xxx_messageInfo_ResourceRef proto.InternalMessageInfo func (m *ResourceResult) Reset() { *m = ResourceResult{} } func (*ResourceResult) ProtoMessage() {} func (*ResourceResult) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{123} + return fileDescriptor_030104ce3b95bcac, []int{125} } func (m *ResourceResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3516,7 +3574,7 @@ var xxx_messageInfo_ResourceResult proto.InternalMessageInfo func (m *ResourceStatus) Reset() { *m = ResourceStatus{} } func (*ResourceStatus) ProtoMessage() {} func (*ResourceStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{124} + return fileDescriptor_030104ce3b95bcac, []int{126} } func (m *ResourceStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3544,7 +3602,7 @@ var xxx_messageInfo_ResourceStatus proto.InternalMessageInfo func (m *RetryStrategy) Reset() { *m = RetryStrategy{} } func (*RetryStrategy) ProtoMessage() {} func (*RetryStrategy) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{125} + return fileDescriptor_030104ce3b95bcac, []int{127} } func (m *RetryStrategy) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3572,7 +3630,7 @@ var xxx_messageInfo_RetryStrategy proto.InternalMessageInfo func (m *RevisionHistory) Reset() { *m = RevisionHistory{} } func (*RevisionHistory) ProtoMessage() {} func (*RevisionHistory) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{126} + return fileDescriptor_030104ce3b95bcac, []int{128} } func (m *RevisionHistory) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3600,7 +3658,7 @@ var xxx_messageInfo_RevisionHistory proto.InternalMessageInfo func (m *RevisionMetadata) Reset() { *m = RevisionMetadata{} } func (*RevisionMetadata) ProtoMessage() {} func (*RevisionMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{127} + return fileDescriptor_030104ce3b95bcac, []int{129} } func (m *RevisionMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3628,7 +3686,7 @@ var xxx_messageInfo_RevisionMetadata proto.InternalMessageInfo func (m *SCMProviderGenerator) Reset() { *m = SCMProviderGenerator{} } func (*SCMProviderGenerator) ProtoMessage() {} func (*SCMProviderGenerator) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{128} + return fileDescriptor_030104ce3b95bcac, []int{130} } func (m *SCMProviderGenerator) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3656,7 +3714,7 @@ var xxx_messageInfo_SCMProviderGenerator proto.InternalMessageInfo func (m *SCMProviderGeneratorAWSCodeCommit) Reset() { *m = SCMProviderGeneratorAWSCodeCommit{} } func (*SCMProviderGeneratorAWSCodeCommit) ProtoMessage() {} func (*SCMProviderGeneratorAWSCodeCommit) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{129} + return fileDescriptor_030104ce3b95bcac, []int{131} } func (m *SCMProviderGeneratorAWSCodeCommit) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3684,7 +3742,7 @@ var xxx_messageInfo_SCMProviderGeneratorAWSCodeCommit proto.InternalMessageInfo func (m *SCMProviderGeneratorAzureDevOps) Reset() { *m = SCMProviderGeneratorAzureDevOps{} } func (*SCMProviderGeneratorAzureDevOps) ProtoMessage() {} func (*SCMProviderGeneratorAzureDevOps) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{130} + return fileDescriptor_030104ce3b95bcac, []int{132} } func (m *SCMProviderGeneratorAzureDevOps) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3712,7 +3770,7 @@ var xxx_messageInfo_SCMProviderGeneratorAzureDevOps proto.InternalMessageInfo func (m *SCMProviderGeneratorBitbucket) Reset() { *m = SCMProviderGeneratorBitbucket{} } func (*SCMProviderGeneratorBitbucket) ProtoMessage() {} func (*SCMProviderGeneratorBitbucket) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{131} + return fileDescriptor_030104ce3b95bcac, []int{133} } func (m *SCMProviderGeneratorBitbucket) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3740,7 +3798,7 @@ var xxx_messageInfo_SCMProviderGeneratorBitbucket proto.InternalMessageInfo func (m *SCMProviderGeneratorBitbucketServer) Reset() { *m = SCMProviderGeneratorBitbucketServer{} } func (*SCMProviderGeneratorBitbucketServer) ProtoMessage() {} func (*SCMProviderGeneratorBitbucketServer) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{132} + return fileDescriptor_030104ce3b95bcac, []int{134} } func (m *SCMProviderGeneratorBitbucketServer) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3768,7 +3826,7 @@ var xxx_messageInfo_SCMProviderGeneratorBitbucketServer proto.InternalMessageInf func (m *SCMProviderGeneratorFilter) Reset() { *m = SCMProviderGeneratorFilter{} } func (*SCMProviderGeneratorFilter) ProtoMessage() {} func (*SCMProviderGeneratorFilter) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{133} + return fileDescriptor_030104ce3b95bcac, []int{135} } func (m *SCMProviderGeneratorFilter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3796,7 +3854,7 @@ var xxx_messageInfo_SCMProviderGeneratorFilter proto.InternalMessageInfo func (m *SCMProviderGeneratorGitea) Reset() { *m = SCMProviderGeneratorGitea{} } func (*SCMProviderGeneratorGitea) ProtoMessage() {} func (*SCMProviderGeneratorGitea) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{134} + return fileDescriptor_030104ce3b95bcac, []int{136} } func (m *SCMProviderGeneratorGitea) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3824,7 +3882,7 @@ var xxx_messageInfo_SCMProviderGeneratorGitea proto.InternalMessageInfo func (m *SCMProviderGeneratorGithub) Reset() { *m = SCMProviderGeneratorGithub{} } func (*SCMProviderGeneratorGithub) ProtoMessage() {} func (*SCMProviderGeneratorGithub) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{135} + return fileDescriptor_030104ce3b95bcac, []int{137} } func (m *SCMProviderGeneratorGithub) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3852,7 +3910,7 @@ var xxx_messageInfo_SCMProviderGeneratorGithub proto.InternalMessageInfo func (m *SCMProviderGeneratorGitlab) Reset() { *m = SCMProviderGeneratorGitlab{} } func (*SCMProviderGeneratorGitlab) ProtoMessage() {} func (*SCMProviderGeneratorGitlab) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{136} + return fileDescriptor_030104ce3b95bcac, []int{138} } func (m *SCMProviderGeneratorGitlab) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3880,7 +3938,7 @@ var xxx_messageInfo_SCMProviderGeneratorGitlab proto.InternalMessageInfo func (m *SecretRef) Reset() { *m = SecretRef{} } func (*SecretRef) ProtoMessage() {} func (*SecretRef) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{137} + return fileDescriptor_030104ce3b95bcac, []int{139} } func (m *SecretRef) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3908,7 +3966,7 @@ var xxx_messageInfo_SecretRef proto.InternalMessageInfo func (m *SignatureKey) Reset() { *m = SignatureKey{} } func (*SignatureKey) ProtoMessage() {} func (*SignatureKey) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{138} + return fileDescriptor_030104ce3b95bcac, []int{140} } func (m *SignatureKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3933,10 +3991,38 @@ func (m *SignatureKey) XXX_DiscardUnknown() { var xxx_messageInfo_SignatureKey proto.InternalMessageInfo +func (m *SyncDependency) Reset() { *m = SyncDependency{} } +func (*SyncDependency) ProtoMessage() {} +func (*SyncDependency) Descriptor() ([]byte, []int) { + return fileDescriptor_030104ce3b95bcac, []int{141} +} +func (m *SyncDependency) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SyncDependency) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *SyncDependency) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncDependency.Merge(m, src) +} +func (m *SyncDependency) XXX_Size() int { + return m.Size() +} +func (m *SyncDependency) XXX_DiscardUnknown() { + xxx_messageInfo_SyncDependency.DiscardUnknown(m) +} + +var xxx_messageInfo_SyncDependency proto.InternalMessageInfo + func (m *SyncOperation) Reset() { *m = SyncOperation{} } func (*SyncOperation) ProtoMessage() {} func (*SyncOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{139} + return fileDescriptor_030104ce3b95bcac, []int{142} } func (m *SyncOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3964,7 +4050,7 @@ var xxx_messageInfo_SyncOperation proto.InternalMessageInfo func (m *SyncOperationResource) Reset() { *m = SyncOperationResource{} } func (*SyncOperationResource) ProtoMessage() {} func (*SyncOperationResource) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{140} + return fileDescriptor_030104ce3b95bcac, []int{143} } func (m *SyncOperationResource) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3992,7 +4078,7 @@ var xxx_messageInfo_SyncOperationResource proto.InternalMessageInfo func (m *SyncOperationResult) Reset() { *m = SyncOperationResult{} } func (*SyncOperationResult) ProtoMessage() {} func (*SyncOperationResult) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{141} + return fileDescriptor_030104ce3b95bcac, []int{144} } func (m *SyncOperationResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4020,7 +4106,7 @@ var xxx_messageInfo_SyncOperationResult proto.InternalMessageInfo func (m *SyncPolicy) Reset() { *m = SyncPolicy{} } func (*SyncPolicy) ProtoMessage() {} func (*SyncPolicy) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{142} + return fileDescriptor_030104ce3b95bcac, []int{145} } func (m *SyncPolicy) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4048,7 +4134,7 @@ var xxx_messageInfo_SyncPolicy proto.InternalMessageInfo func (m *SyncPolicyAutomated) Reset() { *m = SyncPolicyAutomated{} } func (*SyncPolicyAutomated) ProtoMessage() {} func (*SyncPolicyAutomated) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{143} + return fileDescriptor_030104ce3b95bcac, []int{146} } func (m *SyncPolicyAutomated) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4076,7 +4162,7 @@ var xxx_messageInfo_SyncPolicyAutomated proto.InternalMessageInfo func (m *SyncStatus) Reset() { *m = SyncStatus{} } func (*SyncStatus) ProtoMessage() {} func (*SyncStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{144} + return fileDescriptor_030104ce3b95bcac, []int{147} } func (m *SyncStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4104,7 +4190,7 @@ var xxx_messageInfo_SyncStatus proto.InternalMessageInfo func (m *SyncStrategy) Reset() { *m = SyncStrategy{} } func (*SyncStrategy) ProtoMessage() {} func (*SyncStrategy) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{145} + return fileDescriptor_030104ce3b95bcac, []int{148} } func (m *SyncStrategy) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4132,7 +4218,7 @@ var xxx_messageInfo_SyncStrategy proto.InternalMessageInfo func (m *SyncStrategyApply) Reset() { *m = SyncStrategyApply{} } func (*SyncStrategyApply) ProtoMessage() {} func (*SyncStrategyApply) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{146} + return fileDescriptor_030104ce3b95bcac, []int{149} } func (m *SyncStrategyApply) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4160,7 +4246,7 @@ var xxx_messageInfo_SyncStrategyApply proto.InternalMessageInfo func (m *SyncStrategyHook) Reset() { *m = SyncStrategyHook{} } func (*SyncStrategyHook) ProtoMessage() {} func (*SyncStrategyHook) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{147} + return fileDescriptor_030104ce3b95bcac, []int{150} } func (m *SyncStrategyHook) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4188,7 +4274,7 @@ var xxx_messageInfo_SyncStrategyHook proto.InternalMessageInfo func (m *SyncWindow) Reset() { *m = SyncWindow{} } func (*SyncWindow) ProtoMessage() {} func (*SyncWindow) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{148} + return fileDescriptor_030104ce3b95bcac, []int{151} } func (m *SyncWindow) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4216,7 +4302,7 @@ var xxx_messageInfo_SyncWindow proto.InternalMessageInfo func (m *TLSClientConfig) Reset() { *m = TLSClientConfig{} } func (*TLSClientConfig) ProtoMessage() {} func (*TLSClientConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{149} + return fileDescriptor_030104ce3b95bcac, []int{152} } func (m *TLSClientConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4244,7 +4330,7 @@ var xxx_messageInfo_TLSClientConfig proto.InternalMessageInfo func (m *TagFilter) Reset() { *m = TagFilter{} } func (*TagFilter) ProtoMessage() {} func (*TagFilter) Descriptor() ([]byte, []int) { - return fileDescriptor_030104ce3b95bcac, []int{150} + return fileDescriptor_030104ce3b95bcac, []int{153} } func (m *TagFilter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4278,10 +4364,12 @@ func init() { proto.RegisterMapType((map[string]JWTTokens)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.AppProjectStatus.JwtTokensByRoleEntry") proto.RegisterType((*Application)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.Application") proto.RegisterType((*ApplicationCondition)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationCondition") + proto.RegisterType((*ApplicationDependency)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationDependency") proto.RegisterType((*ApplicationDestination)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationDestination") proto.RegisterType((*ApplicationList)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationList") proto.RegisterType((*ApplicationMatchExpression)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationMatchExpression") proto.RegisterType((*ApplicationPreservedFields)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationPreservedFields") + proto.RegisterType((*ApplicationSelector)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationSelector") proto.RegisterType((*ApplicationSet)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationSet") proto.RegisterType((*ApplicationSetApplicationStatus)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationSetApplicationStatus") proto.RegisterType((*ApplicationSetCondition)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.ApplicationSetCondition") @@ -4429,6 +4517,7 @@ func init() { proto.RegisterType((*SCMProviderGeneratorGitlab)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.SCMProviderGeneratorGitlab") proto.RegisterType((*SecretRef)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.SecretRef") proto.RegisterType((*SignatureKey)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.SignatureKey") + proto.RegisterType((*SyncDependency)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.SyncDependency") proto.RegisterType((*SyncOperation)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.SyncOperation") proto.RegisterType((*SyncOperationResource)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.SyncOperationResource") proto.RegisterType((*SyncOperationResult)(nil), "github.com.argoproj.argo_cd.v2.pkg.apis.application.v1alpha1.SyncOperationResult") @@ -4448,697 +4537,717 @@ func init() { } var fileDescriptor_030104ce3b95bcac = []byte{ - // 11030 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x7d, 0x6d, 0x70, 0x1c, 0xc9, - 0x75, 0x98, 0x66, 0x17, 0x0b, 0xec, 0x3e, 0x7c, 0x90, 0x6c, 0x92, 0x77, 0x20, 0x75, 0x77, 0xa0, - 0xe7, 0xe2, 0xd3, 0x39, 0xba, 0x03, 0x7c, 0xf4, 0x9d, 0x7c, 0xf1, 0xd9, 0x92, 0xb1, 0x00, 0x09, - 0x82, 0x04, 0x08, 0x5c, 0x03, 0x24, 0xa5, 0x93, 0x4f, 0xa7, 0xc1, 0x6e, 0x63, 0x31, 0xc4, 0xec, - 0xcc, 0xdc, 0xcc, 0x2c, 0x08, 0x9c, 0x25, 0x59, 0xb2, 0x64, 0x5b, 0x89, 0x3e, 0x4e, 0x91, 0x92, - 0xf2, 0x39, 0xb1, 0x14, 0xd9, 0x72, 0x52, 0x71, 0x25, 0xaa, 0x38, 0xc9, 0x8f, 0x38, 0x71, 0x52, - 0x2e, 0xdb, 0xa9, 0x94, 0x52, 0x4a, 0xca, 0x2e, 0x97, 0xcb, 0x72, 0x12, 0x1b, 0x91, 0x98, 0x4a, - 0x25, 0x95, 0xaa, 0xb8, 0xca, 0x89, 0x7f, 0x24, 0x4c, 0x7e, 0xa4, 0xfa, 0xbb, 0x67, 0x76, 0x16, - 0x58, 0x00, 0x03, 0x92, 0x52, 0xee, 0xdf, 0x6e, 0xbf, 0x37, 0xef, 0xf5, 0xf4, 0x74, 0xbf, 0xf7, - 0xfa, 0xf5, 0x7b, 0xaf, 0x61, 0xa1, 0xe5, 0x26, 0x1b, 0x9d, 0xb5, 0xc9, 0x46, 0xd0, 0x9e, 0x72, - 0xa2, 0x56, 0x10, 0x46, 0xc1, 0x6d, 0xf6, 0xe3, 0xd9, 0x46, 0x73, 0x6a, 0xeb, 0xe2, 0x54, 0xb8, - 0xd9, 0x9a, 0x72, 0x42, 0x37, 0x9e, 0x72, 0xc2, 0xd0, 0x73, 0x1b, 0x4e, 0xe2, 0x06, 0xfe, 0xd4, - 0xd6, 0x73, 0x8e, 0x17, 0x6e, 0x38, 0xcf, 0x4d, 0xb5, 0x88, 0x4f, 0x22, 0x27, 0x21, 0xcd, 0xc9, - 0x30, 0x0a, 0x92, 0x00, 0xfd, 0xa8, 0xa6, 0x36, 0x29, 0xa9, 0xb1, 0x1f, 0xaf, 0x35, 0x9a, 0x93, - 0x5b, 0x17, 0x27, 0xc3, 0xcd, 0xd6, 0x24, 0xa5, 0x36, 0x69, 0x50, 0x9b, 0x94, 0xd4, 0xce, 0x3f, - 0x6b, 0xf4, 0xa5, 0x15, 0xb4, 0x82, 0x29, 0x46, 0x74, 0xad, 0xb3, 0xce, 0xfe, 0xb1, 0x3f, 0xec, - 0x17, 0x67, 0x76, 0xde, 0xde, 0x7c, 0x31, 0x9e, 0x74, 0x03, 0xda, 0xbd, 0xa9, 0x46, 0x10, 0x91, - 0xa9, 0xad, 0xae, 0x0e, 0x9d, 0xbf, 0xa2, 0x71, 0xc8, 0x76, 0x42, 0xfc, 0xd8, 0x0d, 0xfc, 0xf8, - 0x59, 0xda, 0x05, 0x12, 0x6d, 0x91, 0xc8, 0x7c, 0x3d, 0x03, 0x21, 0x8f, 0xd2, 0xf3, 0x9a, 0x52, - 0xdb, 0x69, 0x6c, 0xb8, 0x3e, 0x89, 0x76, 0xf4, 0xe3, 0x6d, 0x92, 0x38, 0x79, 0x4f, 0x4d, 0xf5, - 0x7a, 0x2a, 0xea, 0xf8, 0x89, 0xdb, 0x26, 0x5d, 0x0f, 0xbc, 0x67, 0xbf, 0x07, 0xe2, 0xc6, 0x06, - 0x69, 0x3b, 0x5d, 0xcf, 0xfd, 0x50, 0xaf, 0xe7, 0x3a, 0x89, 0xeb, 0x4d, 0xb9, 0x7e, 0x12, 0x27, - 0x51, 0xf6, 0x21, 0xfb, 0x17, 0x2d, 0x18, 0x9d, 0xbe, 0xb5, 0x32, 0xdd, 0x49, 0x36, 0x66, 0x02, - 0x7f, 0xdd, 0x6d, 0xa1, 0x17, 0x60, 0xb8, 0xe1, 0x75, 0xe2, 0x84, 0x44, 0xd7, 0x9d, 0x36, 0x19, - 0xb7, 0x2e, 0x58, 0x4f, 0xd7, 0xea, 0xa7, 0xbf, 0xb1, 0x3b, 0xf1, 0x8e, 0xbb, 0xbb, 0x13, 0xc3, - 0x33, 0x1a, 0x84, 0x4d, 0x3c, 0xf4, 0x03, 0x30, 0x14, 0x05, 0x1e, 0x99, 0xc6, 0xd7, 0xc7, 0x4b, - 0xec, 0x91, 0x13, 0xe2, 0x91, 0x21, 0xcc, 0x9b, 0xb1, 0x84, 0x53, 0xd4, 0x30, 0x0a, 0xd6, 0x5d, - 0x8f, 0x8c, 0x97, 0xd3, 0xa8, 0xcb, 0xbc, 0x19, 0x4b, 0xb8, 0xfd, 0x87, 0x25, 0x80, 0xe9, 0x30, - 0x5c, 0x8e, 0x82, 0xdb, 0xa4, 0x91, 0xa0, 0x0f, 0x43, 0x95, 0x0e, 0x73, 0xd3, 0x49, 0x1c, 0xd6, - 0xb1, 0xe1, 0x8b, 0x3f, 0x38, 0xc9, 0xdf, 0x7a, 0xd2, 0x7c, 0x6b, 0x3d, 0xc9, 0x28, 0xf6, 0xe4, - 0xd6, 0x73, 0x93, 0x4b, 0x6b, 0xf4, 0xf9, 0x45, 0x92, 0x38, 0x75, 0x24, 0x98, 0x81, 0x6e, 0xc3, - 0x8a, 0x2a, 0xf2, 0x61, 0x20, 0x0e, 0x49, 0x83, 0xbd, 0xc3, 0xf0, 0xc5, 0x85, 0xc9, 0xa3, 0xcc, - 0xe6, 0x49, 0xdd, 0xf3, 0x95, 0x90, 0x34, 0xea, 0x23, 0x82, 0xf3, 0x00, 0xfd, 0x87, 0x19, 0x1f, - 0xb4, 0x05, 0x83, 0x71, 0xe2, 0x24, 0x9d, 0x98, 0x0d, 0xc5, 0xf0, 0xc5, 0xeb, 0x85, 0x71, 0x64, - 0x54, 0xeb, 0x63, 0x82, 0xe7, 0x20, 0xff, 0x8f, 0x05, 0x37, 0xfb, 0x4f, 0x2c, 0x18, 0xd3, 0xc8, - 0x0b, 0x6e, 0x9c, 0xa0, 0x9f, 0xe8, 0x1a, 0xdc, 0xc9, 0xfe, 0x06, 0x97, 0x3e, 0xcd, 0x86, 0xf6, - 0xa4, 0x60, 0x56, 0x95, 0x2d, 0xc6, 0xc0, 0xb6, 0xa1, 0xe2, 0x26, 0xa4, 0x1d, 0x8f, 0x97, 0x2e, - 0x94, 0x9f, 0x1e, 0xbe, 0x78, 0xa5, 0xa8, 0xf7, 0xac, 0x8f, 0x0a, 0xa6, 0x95, 0x79, 0x4a, 0x1e, - 0x73, 0x2e, 0xf6, 0xaf, 0x8e, 0x98, 0xef, 0x47, 0x07, 0x1c, 0x3d, 0x07, 0xc3, 0x71, 0xd0, 0x89, - 0x1a, 0x04, 0x93, 0x30, 0x88, 0xc7, 0xad, 0x0b, 0x65, 0x3a, 0xf5, 0xe8, 0xa4, 0x5e, 0xd1, 0xcd, - 0xd8, 0xc4, 0x41, 0x9f, 0xb7, 0x60, 0xa4, 0x49, 0xe2, 0xc4, 0xf5, 0x19, 0x7f, 0xd9, 0xf9, 0xd5, - 0x23, 0x77, 0x5e, 0x36, 0xce, 0x6a, 0xe2, 0xf5, 0x33, 0xe2, 0x45, 0x46, 0x8c, 0xc6, 0x18, 0xa7, - 0xf8, 0xd3, 0xc5, 0xd9, 0x24, 0x71, 0x23, 0x72, 0x43, 0xfa, 0x5f, 0x2c, 0x1f, 0xb5, 0x38, 0x67, - 0x35, 0x08, 0x9b, 0x78, 0xc8, 0x87, 0x0a, 0x5d, 0x7c, 0xf1, 0xf8, 0x00, 0xeb, 0xff, 0xfc, 0xd1, - 0xfa, 0x2f, 0x06, 0x95, 0xae, 0x6b, 0x3d, 0xfa, 0xf4, 0x5f, 0x8c, 0x39, 0x1b, 0xf4, 0x39, 0x0b, - 0xc6, 0x85, 0x70, 0xc0, 0x84, 0x0f, 0xe8, 0xad, 0x0d, 0x37, 0x21, 0x9e, 0x1b, 0x27, 0xe3, 0x15, - 0xd6, 0x87, 0xa9, 0xfe, 0xe6, 0xd6, 0x5c, 0x14, 0x74, 0xc2, 0x6b, 0xae, 0xdf, 0xac, 0x5f, 0x10, - 0x9c, 0xc6, 0x67, 0x7a, 0x10, 0xc6, 0x3d, 0x59, 0xa2, 0x2f, 0x59, 0x70, 0xde, 0x77, 0xda, 0x24, - 0x0e, 0x1d, 0xfa, 0x69, 0x39, 0xb8, 0xee, 0x39, 0x8d, 0x4d, 0xd6, 0xa3, 0xc1, 0xc3, 0xf5, 0xc8, - 0x16, 0x3d, 0x3a, 0x7f, 0xbd, 0x27, 0x69, 0xbc, 0x07, 0x5b, 0xf4, 0x35, 0x0b, 0x4e, 0x05, 0x51, - 0xb8, 0xe1, 0xf8, 0xa4, 0x29, 0xa1, 0xf1, 0xf8, 0x10, 0x5b, 0x7a, 0x1f, 0x3a, 0xda, 0x27, 0x5a, - 0xca, 0x92, 0x5d, 0x0c, 0x7c, 0x37, 0x09, 0xa2, 0x15, 0x92, 0x24, 0xae, 0xdf, 0x8a, 0xeb, 0x67, - 0xef, 0xee, 0x4e, 0x9c, 0xea, 0xc2, 0xc2, 0xdd, 0xfd, 0x41, 0x3f, 0x09, 0xc3, 0xf1, 0x8e, 0xdf, - 0xb8, 0xe5, 0xfa, 0xcd, 0xe0, 0x4e, 0x3c, 0x5e, 0x2d, 0x62, 0xf9, 0xae, 0x28, 0x82, 0x62, 0x01, - 0x6a, 0x06, 0xd8, 0xe4, 0x96, 0xff, 0xe1, 0xf4, 0x54, 0xaa, 0x15, 0xfd, 0xe1, 0xf4, 0x64, 0xda, - 0x83, 0x2d, 0xfa, 0x39, 0x0b, 0x46, 0x63, 0xb7, 0xe5, 0x3b, 0x49, 0x27, 0x22, 0xd7, 0xc8, 0x4e, - 0x3c, 0x0e, 0xac, 0x23, 0x57, 0x8f, 0x38, 0x2a, 0x06, 0xc9, 0xfa, 0x59, 0xd1, 0xc7, 0x51, 0xb3, - 0x35, 0xc6, 0x69, 0xbe, 0x79, 0x0b, 0x4d, 0x4f, 0xeb, 0xe1, 0x62, 0x17, 0x9a, 0x9e, 0xd4, 0x3d, - 0x59, 0xa2, 0x1f, 0x87, 0x93, 0xbc, 0x49, 0x8d, 0x6c, 0x3c, 0x3e, 0xc2, 0x04, 0xed, 0x99, 0xbb, - 0xbb, 0x13, 0x27, 0x57, 0x32, 0x30, 0xdc, 0x85, 0x8d, 0x5e, 0x87, 0x89, 0x90, 0x44, 0x6d, 0x37, - 0x59, 0xf2, 0xbd, 0x1d, 0x29, 0xbe, 0x1b, 0x41, 0x48, 0x9a, 0xa2, 0x3b, 0xf1, 0xf8, 0xe8, 0x05, - 0xeb, 0xe9, 0x6a, 0xfd, 0x5d, 0xa2, 0x9b, 0x13, 0xcb, 0x7b, 0xa3, 0xe3, 0xfd, 0xe8, 0xd9, 0xff, - 0xba, 0x04, 0x27, 0xb3, 0x8a, 0x13, 0xfd, 0x1d, 0x0b, 0x4e, 0xdc, 0xbe, 0x93, 0xac, 0x06, 0x9b, - 0xc4, 0x8f, 0xeb, 0x3b, 0x54, 0xbc, 0x31, 0x95, 0x31, 0x7c, 0xb1, 0x51, 0xac, 0x8a, 0x9e, 0xbc, - 0x9a, 0xe6, 0x72, 0xc9, 0x4f, 0xa2, 0x9d, 0xfa, 0xa3, 0xe2, 0xed, 0x4e, 0x5c, 0xbd, 0xb5, 0x6a, - 0x42, 0x71, 0xb6, 0x53, 0xe7, 0x3f, 0x63, 0xc1, 0x99, 0x3c, 0x12, 0xe8, 0x24, 0x94, 0x37, 0xc9, - 0x0e, 0x37, 0xe0, 0x30, 0xfd, 0x89, 0x5e, 0x85, 0xca, 0x96, 0xe3, 0x75, 0x88, 0xb0, 0x6e, 0xe6, - 0x8e, 0xf6, 0x22, 0xaa, 0x67, 0x98, 0x53, 0xfd, 0x91, 0xd2, 0x8b, 0x96, 0xfd, 0xbb, 0x65, 0x18, - 0x36, 0xf4, 0xdb, 0x7d, 0xb0, 0xd8, 0x82, 0x94, 0xc5, 0xb6, 0x58, 0x98, 0x6a, 0xee, 0x69, 0xb2, - 0xdd, 0xc9, 0x98, 0x6c, 0x4b, 0xc5, 0xb1, 0xdc, 0xd3, 0x66, 0x43, 0x09, 0xd4, 0x82, 0x90, 0x5a, - 0xef, 0x54, 0xf5, 0x0f, 0x14, 0xf1, 0x09, 0x97, 0x24, 0xb9, 0xfa, 0xe8, 0xdd, 0xdd, 0x89, 0x9a, - 0xfa, 0x8b, 0x35, 0x23, 0xfb, 0x5b, 0x16, 0x9c, 0x31, 0xfa, 0x38, 0x13, 0xf8, 0x4d, 0x97, 0x7d, - 0xda, 0x0b, 0x30, 0x90, 0xec, 0x84, 0x72, 0x87, 0xa0, 0x46, 0x6a, 0x75, 0x27, 0x24, 0x98, 0x41, - 0xa8, 0xa1, 0xdf, 0x26, 0x71, 0xec, 0xb4, 0x48, 0x76, 0x4f, 0xb0, 0xc8, 0x9b, 0xb1, 0x84, 0xa3, - 0x08, 0x90, 0xe7, 0xc4, 0xc9, 0x6a, 0xe4, 0xf8, 0x31, 0x23, 0xbf, 0xea, 0xb6, 0x89, 0x18, 0xe0, - 0xbf, 0xd8, 0xdf, 0x8c, 0xa1, 0x4f, 0xd4, 0x1f, 0xb9, 0xbb, 0x3b, 0x81, 0x16, 0xba, 0x28, 0xe1, - 0x1c, 0xea, 0xf6, 0x97, 0x2c, 0x78, 0x24, 0xdf, 0x16, 0x43, 0x4f, 0xc1, 0x20, 0xdf, 0x1e, 0x8a, - 0xb7, 0xd3, 0x9f, 0x84, 0xb5, 0x62, 0x01, 0x45, 0x53, 0x50, 0x53, 0x7a, 0x42, 0xbc, 0xe3, 0x29, - 0x81, 0x5a, 0xd3, 0xca, 0x45, 0xe3, 0xd0, 0x41, 0xa3, 0x7f, 0x84, 0xe5, 0xa6, 0x06, 0x8d, 0xed, - 0xa7, 0x18, 0xc4, 0xfe, 0x8f, 0x16, 0x9c, 0x30, 0x7a, 0x75, 0x1f, 0x4c, 0x73, 0x3f, 0x6d, 0x9a, - 0xcf, 0x17, 0x36, 0x9f, 0x7b, 0xd8, 0xe6, 0x9f, 0xb3, 0xe0, 0xbc, 0x81, 0xb5, 0xe8, 0x24, 0x8d, - 0x8d, 0x4b, 0xdb, 0x61, 0x44, 0x62, 0xba, 0xf5, 0x46, 0x8f, 0x1b, 0x72, 0xab, 0x3e, 0x2c, 0x28, - 0x94, 0xaf, 0x91, 0x1d, 0x2e, 0xc4, 0x9e, 0x81, 0x2a, 0x9f, 0x9c, 0x41, 0x24, 0x46, 0x5c, 0xbd, - 0xdb, 0x92, 0x68, 0xc7, 0x0a, 0x03, 0xd9, 0x30, 0xc8, 0x84, 0x13, 0x5d, 0xac, 0x54, 0x0d, 0x01, - 0xfd, 0x88, 0x37, 0x59, 0x0b, 0x16, 0x10, 0x3b, 0x4e, 0x75, 0x67, 0x39, 0x22, 0xec, 0xe3, 0x36, - 0x2f, 0xbb, 0xc4, 0x6b, 0xc6, 0x74, 0xdb, 0xe0, 0xf8, 0x7e, 0x90, 0x88, 0x1d, 0x80, 0xb1, 0x6d, - 0x98, 0xd6, 0xcd, 0xd8, 0xc4, 0xa1, 0x4c, 0x3d, 0x67, 0x8d, 0x78, 0x7c, 0x44, 0x05, 0xd3, 0x05, - 0xd6, 0x82, 0x05, 0xc4, 0xbe, 0x5b, 0x62, 0x1b, 0x14, 0xb5, 0xf4, 0xc9, 0xfd, 0xd8, 0xdd, 0x46, - 0x29, 0x59, 0xb9, 0x5c, 0x9c, 0xe0, 0x22, 0xbd, 0x77, 0xb8, 0x6f, 0x64, 0xc4, 0x25, 0x2e, 0x94, - 0xeb, 0xde, 0xbb, 0xdc, 0xdf, 0x2a, 0xc1, 0x44, 0xfa, 0x81, 0x2e, 0x69, 0x4b, 0xb7, 0x54, 0x06, - 0xa3, 0xac, 0xbf, 0xc3, 0xc0, 0xc7, 0x26, 0x5e, 0x0f, 0x81, 0x55, 0x3a, 0x4e, 0x81, 0x65, 0xca, - 0xd3, 0xf2, 0x3e, 0xf2, 0xf4, 0x29, 0x35, 0xea, 0x03, 0x19, 0x01, 0x96, 0xd6, 0x29, 0x17, 0x60, - 0x20, 0x4e, 0x48, 0x38, 0x5e, 0x49, 0xcb, 0xa3, 0x95, 0x84, 0x84, 0x98, 0x41, 0xec, 0xff, 0x56, - 0x82, 0x47, 0xd3, 0x63, 0xa8, 0x55, 0xc0, 0xfb, 0x52, 0x2a, 0xe0, 0xdd, 0xa6, 0x0a, 0xb8, 0xb7, - 0x3b, 0xf1, 0xce, 0x1e, 0x8f, 0x7d, 0xd7, 0x68, 0x08, 0x34, 0x97, 0x19, 0xc5, 0xa9, 0xf4, 0x28, - 0xde, 0xdb, 0x9d, 0x78, 0xbc, 0xc7, 0x3b, 0x66, 0x86, 0xf9, 0x29, 0x18, 0x8c, 0x88, 0x13, 0x07, - 0xbe, 0x18, 0x68, 0xf5, 0x39, 0x30, 0x6b, 0xc5, 0x02, 0x6a, 0xff, 0x7e, 0x2d, 0x3b, 0xd8, 0x73, - 0xdc, 0x61, 0x17, 0x44, 0xc8, 0x85, 0x01, 0x66, 0xd6, 0x73, 0xd1, 0x70, 0xed, 0x68, 0xcb, 0x88, - 0xaa, 0x01, 0x45, 0xba, 0x5e, 0xa5, 0x5f, 0x8d, 0x36, 0x61, 0xc6, 0x02, 0x6d, 0x43, 0xb5, 0x21, - 0xad, 0xed, 0x52, 0x11, 0x7e, 0x29, 0x61, 0x6b, 0x6b, 0x8e, 0x23, 0x54, 0x5e, 0x2b, 0x13, 0x5d, - 0x71, 0x43, 0x04, 0xca, 0x2d, 0x37, 0x11, 0x9f, 0xf5, 0x88, 0xfb, 0xa9, 0x39, 0xd7, 0x78, 0xc5, - 0x21, 0xaa, 0x44, 0xe6, 0xdc, 0x04, 0x53, 0xfa, 0xe8, 0x67, 0x2c, 0x18, 0x8e, 0x1b, 0xed, 0xe5, - 0x28, 0xd8, 0x72, 0x9b, 0x24, 0x12, 0xd6, 0xd4, 0x11, 0x45, 0xd3, 0xca, 0xcc, 0xa2, 0x24, 0xa8, - 0xf9, 0xf2, 0xfd, 0xad, 0x86, 0x60, 0x93, 0x2f, 0xdd, 0x65, 0x3c, 0x2a, 0xde, 0x7d, 0x96, 0x34, - 0x5c, 0xaa, 0xff, 0xe4, 0xa6, 0x8a, 0xcd, 0x94, 0x23, 0x5b, 0x97, 0xb3, 0x9d, 0xc6, 0x26, 0x5d, - 0x6f, 0xba, 0x43, 0xef, 0xbc, 0xbb, 0x3b, 0xf1, 0xe8, 0x4c, 0x3e, 0x4f, 0xdc, 0xab, 0x33, 0x6c, - 0xc0, 0xc2, 0x8e, 0xe7, 0x61, 0xf2, 0x7a, 0x87, 0x30, 0x97, 0x49, 0x01, 0x03, 0xb6, 0xac, 0x09, - 0x66, 0x06, 0xcc, 0x80, 0x60, 0x93, 0x2f, 0x7a, 0x1d, 0x06, 0xdb, 0x4e, 0x12, 0xb9, 0xdb, 0xc2, - 0x4f, 0x72, 0x44, 0x7b, 0x7f, 0x91, 0xd1, 0xd2, 0xcc, 0x99, 0xa6, 0xe6, 0x8d, 0x58, 0x30, 0x42, - 0x6d, 0xa8, 0xb4, 0x49, 0xd4, 0x22, 0xe3, 0xd5, 0x22, 0x7c, 0xc2, 0x8b, 0x94, 0x94, 0x66, 0x58, - 0xa3, 0xd6, 0x11, 0x6b, 0xc3, 0x9c, 0x0b, 0x7a, 0x15, 0xaa, 0x31, 0xf1, 0x48, 0x83, 0xda, 0x37, - 0x35, 0xc6, 0xf1, 0x87, 0xfa, 0xb4, 0xf5, 0xa8, 0x61, 0xb1, 0x22, 0x1e, 0xe5, 0x0b, 0x4c, 0xfe, - 0xc3, 0x8a, 0x24, 0x1d, 0xc0, 0xd0, 0xeb, 0xb4, 0x5c, 0x7f, 0x1c, 0x8a, 0x18, 0xc0, 0x65, 0x46, - 0x2b, 0x33, 0x80, 0xbc, 0x11, 0x0b, 0x46, 0xf6, 0x7f, 0xb6, 0x00, 0xa5, 0x85, 0xda, 0x7d, 0x30, - 0x6a, 0x5f, 0x4f, 0x1b, 0xb5, 0x0b, 0x45, 0x5a, 0x1d, 0x3d, 0xec, 0xda, 0xdf, 0xa8, 0x41, 0x46, - 0x1d, 0x5c, 0x27, 0x71, 0x42, 0x9a, 0x6f, 0x8b, 0xf0, 0xb7, 0x45, 0xf8, 0xdb, 0x22, 0x5c, 0x89, - 0xf0, 0xb5, 0x8c, 0x08, 0x7f, 0xaf, 0xb1, 0xea, 0xf5, 0x01, 0xec, 0x6b, 0xea, 0x84, 0xd6, 0xec, - 0x81, 0x81, 0x40, 0x25, 0xc1, 0xd5, 0x95, 0xa5, 0xeb, 0xb9, 0x32, 0xfb, 0xb5, 0xb4, 0xcc, 0x3e, - 0x2a, 0x8b, 0xff, 0x1f, 0xa4, 0xf4, 0xbf, 0xb2, 0xe0, 0x5d, 0x69, 0xe9, 0x25, 0x67, 0xce, 0x7c, - 0xcb, 0x0f, 0x22, 0x32, 0xeb, 0xae, 0xaf, 0x93, 0x88, 0xf8, 0x0d, 0x12, 0x2b, 0x2f, 0x86, 0xd5, - 0xcb, 0x8b, 0x81, 0x9e, 0x87, 0x91, 0xdb, 0x71, 0xe0, 0x2f, 0x07, 0xae, 0x2f, 0x44, 0x10, 0xdd, - 0x08, 0x9f, 0xbc, 0xbb, 0x3b, 0x31, 0x42, 0x47, 0x54, 0xb6, 0xe3, 0x14, 0x16, 0x9a, 0x81, 0x53, - 0xb7, 0x5f, 0x5f, 0x76, 0x12, 0xc3, 0x1d, 0x20, 0x37, 0xee, 0xec, 0xc0, 0xe2, 0xea, 0xcb, 0x19, - 0x20, 0xee, 0xc6, 0xb7, 0xff, 0x66, 0x09, 0xce, 0x65, 0x5e, 0x24, 0xf0, 0xbc, 0xa0, 0x93, 0xd0, - 0x4d, 0x0d, 0xfa, 0x8a, 0x05, 0x27, 0xdb, 0x69, 0x8f, 0x43, 0x2c, 0x1c, 0xbb, 0xef, 0x2f, 0x4c, - 0x47, 0x64, 0x5c, 0x1a, 0xf5, 0x71, 0x31, 0x42, 0x27, 0x33, 0x80, 0x18, 0x77, 0xf5, 0x05, 0xbd, - 0x0a, 0xb5, 0xb6, 0xb3, 0x7d, 0x23, 0x6c, 0x3a, 0x89, 0xdc, 0x4f, 0xf6, 0x76, 0x03, 0x74, 0x12, - 0xd7, 0x9b, 0xe4, 0x47, 0xfb, 0x93, 0xf3, 0x7e, 0xb2, 0x14, 0xad, 0x24, 0x91, 0xeb, 0xb7, 0xb8, - 0x3b, 0x6f, 0x51, 0x92, 0xc1, 0x9a, 0xa2, 0xfd, 0x65, 0x2b, 0xab, 0xa4, 0xd4, 0xe8, 0x44, 0x4e, - 0x42, 0x5a, 0x3b, 0xe8, 0x23, 0x50, 0xa1, 0x1b, 0x3f, 0x39, 0x2a, 0xb7, 0x8a, 0xd4, 0x9c, 0xc6, - 0x97, 0xd0, 0x4a, 0x94, 0xfe, 0x8b, 0x31, 0x67, 0x6a, 0x7f, 0xa5, 0x96, 0x35, 0x16, 0xd8, 0xe1, - 0xed, 0x45, 0x80, 0x56, 0xb0, 0x4a, 0xda, 0xa1, 0x47, 0x87, 0xc5, 0x62, 0x27, 0x00, 0xca, 0xd7, - 0x31, 0xa7, 0x20, 0xd8, 0xc0, 0x42, 0x7f, 0xd9, 0x02, 0x68, 0xc9, 0x39, 0x2f, 0x0d, 0x81, 0x1b, - 0x45, 0xbe, 0x8e, 0x5e, 0x51, 0xba, 0x2f, 0x8a, 0x21, 0x36, 0x98, 0xa3, 0x9f, 0xb6, 0xa0, 0x9a, - 0xc8, 0xee, 0x73, 0xd5, 0xb8, 0x5a, 0x64, 0x4f, 0xe4, 0x4b, 0x6b, 0x9b, 0x48, 0x0d, 0x89, 0xe2, - 0x8b, 0x7e, 0xd6, 0x02, 0x88, 0x77, 0xfc, 0xc6, 0x72, 0xe0, 0xb9, 0x8d, 0x1d, 0xa1, 0x31, 0x6f, - 0x16, 0xea, 0x8f, 0x51, 0xd4, 0xeb, 0x63, 0x74, 0x34, 0xf4, 0x7f, 0x6c, 0x70, 0x46, 0x1f, 0x83, - 0x6a, 0x2c, 0xa6, 0x9b, 0xd0, 0x91, 0xab, 0xc5, 0x7a, 0x85, 0x38, 0x6d, 0x21, 0x5e, 0xc5, 0x3f, - 0xac, 0x78, 0xa2, 0x9f, 0xb7, 0xe0, 0x44, 0x98, 0xf6, 0xf3, 0x09, 0x75, 0x58, 0x9c, 0x0c, 0xc8, - 0xf8, 0x11, 0xeb, 0xa7, 0xef, 0xee, 0x4e, 0x9c, 0xc8, 0x34, 0xe2, 0x6c, 0x2f, 0xa8, 0x04, 0xd4, - 0x33, 0x78, 0x29, 0xe4, 0x3e, 0xc7, 0x21, 0x2d, 0x01, 0xe7, 0xb2, 0x40, 0xdc, 0x8d, 0x8f, 0x96, - 0xe1, 0x0c, 0xed, 0xdd, 0x0e, 0x37, 0x3f, 0xa5, 0x7a, 0x89, 0x99, 0x32, 0xac, 0xd6, 0x1f, 0x13, - 0x33, 0x84, 0x79, 0xf5, 0xb3, 0x38, 0x38, 0xf7, 0x49, 0xf4, 0xbb, 0x16, 0x3c, 0xe6, 0x32, 0x35, - 0x60, 0x3a, 0xcc, 0xb5, 0x46, 0x10, 0x27, 0xb1, 0xa4, 0x50, 0x59, 0xd1, 0x4b, 0xfd, 0xd4, 0xff, - 0x82, 0x78, 0x83, 0xc7, 0xe6, 0xf7, 0xe8, 0x12, 0xde, 0xb3, 0xc3, 0xe8, 0x87, 0x61, 0x54, 0xae, - 0x8b, 0x65, 0x2a, 0x82, 0x99, 0xa2, 0xad, 0xd5, 0x4f, 0xdd, 0xdd, 0x9d, 0x18, 0x5d, 0x35, 0x01, - 0x38, 0x8d, 0x67, 0x7f, 0xb3, 0x94, 0x3a, 0x0f, 0x51, 0x4e, 0x48, 0x26, 0x6e, 0x1a, 0xd2, 0xff, - 0x23, 0xa5, 0x67, 0xa1, 0xe2, 0x46, 0x79, 0x97, 0xb4, 0xb8, 0x51, 0x4d, 0x31, 0x36, 0x98, 0x53, - 0xa3, 0xf4, 0x94, 0x93, 0x75, 0x75, 0x0a, 0x09, 0xf8, 0x6a, 0x91, 0x5d, 0xea, 0x3e, 0xbd, 0x3a, - 0x27, 0xba, 0x76, 0xaa, 0x0b, 0x84, 0xbb, 0xbb, 0x64, 0x7f, 0x33, 0x7d, 0x06, 0x63, 0x2c, 0xde, - 0x3e, 0xce, 0x97, 0x3e, 0x6f, 0xc1, 0x70, 0x14, 0x78, 0x9e, 0xeb, 0xb7, 0xa8, 0xa0, 0x11, 0xda, - 0xf2, 0x83, 0xc7, 0xa2, 0xb0, 0x84, 0x44, 0x61, 0xa6, 0x2d, 0xd6, 0x3c, 0xb1, 0xd9, 0x01, 0xfb, - 0x4f, 0x2c, 0x18, 0xef, 0x25, 0x10, 0x11, 0x81, 0x77, 0xca, 0xd5, 0xae, 0xa2, 0x2b, 0x96, 0xfc, - 0x59, 0xe2, 0x11, 0xe5, 0x78, 0xae, 0xd6, 0x9f, 0x14, 0xaf, 0xf9, 0xce, 0xe5, 0xde, 0xa8, 0x78, - 0x2f, 0x3a, 0xe8, 0x15, 0x38, 0x69, 0xbc, 0x57, 0xac, 0x06, 0xa6, 0x56, 0x9f, 0xa4, 0x16, 0xc8, - 0x74, 0x06, 0x76, 0x6f, 0x77, 0xe2, 0x91, 0x6c, 0x9b, 0x90, 0xd8, 0x5d, 0x74, 0xec, 0x5f, 0x29, - 0x65, 0xbf, 0x96, 0x52, 0xb6, 0x6f, 0x59, 0x5d, 0xdb, 0xf9, 0xf7, 0x1f, 0x87, 0x82, 0x63, 0x1b, - 0x7f, 0x15, 0xc0, 0xd1, 0x1b, 0xe7, 0x01, 0x9e, 0x10, 0xdb, 0xff, 0x66, 0x00, 0xf6, 0xe8, 0x59, - 0x1f, 0xd6, 0xf3, 0x81, 0x8f, 0x15, 0x3f, 0x6b, 0xa9, 0x23, 0xa7, 0x32, 0x5b, 0xe4, 0xcd, 0xe3, - 0x1a, 0x7b, 0xbe, 0x81, 0x89, 0x79, 0x94, 0x82, 0x72, 0x63, 0xa7, 0x0f, 0xb7, 0xd0, 0x57, 0xad, - 0xf4, 0xa1, 0x19, 0x0f, 0x3b, 0x73, 0x8f, 0xad, 0x4f, 0xc6, 0x49, 0x1c, 0xef, 0x98, 0x3e, 0xbf, - 0xe9, 0x75, 0x46, 0x37, 0x09, 0xb0, 0xee, 0xfa, 0x8e, 0xe7, 0xbe, 0x41, 0xb7, 0x27, 0x15, 0xa6, - 0x61, 0x99, 0xc9, 0x72, 0x59, 0xb5, 0x62, 0x03, 0xe3, 0xfc, 0x5f, 0x82, 0x61, 0xe3, 0xcd, 0x73, - 0x82, 0x2b, 0xce, 0x98, 0xc1, 0x15, 0x35, 0x23, 0x26, 0xe2, 0xfc, 0x7b, 0xe1, 0x64, 0xb6, 0x83, - 0x07, 0x79, 0xde, 0xfe, 0x5f, 0x43, 0xd9, 0x53, 0xac, 0x55, 0x12, 0xb5, 0x69, 0xd7, 0xde, 0xf6, - 0x2c, 0xbd, 0xed, 0x59, 0x7a, 0xdb, 0xb3, 0x64, 0x1e, 0x0e, 0x08, 0xaf, 0xc9, 0xd0, 0x7d, 0xf2, - 0x9a, 0xa4, 0xfc, 0x40, 0xd5, 0xc2, 0xfd, 0x40, 0xf6, 0xdd, 0x0a, 0xa4, 0xec, 0x28, 0x3e, 0xde, - 0x3f, 0x00, 0x43, 0x11, 0x09, 0x83, 0x1b, 0x78, 0x41, 0xe8, 0x10, 0x1d, 0x6b, 0xcf, 0x9b, 0xb1, - 0x84, 0x53, 0x5d, 0x13, 0x3a, 0xc9, 0x86, 0x50, 0x22, 0x4a, 0xd7, 0x2c, 0x3b, 0xc9, 0x06, 0x66, - 0x10, 0xf4, 0x5e, 0x18, 0x4b, 0x9c, 0xa8, 0x45, 0xed, 0xed, 0x2d, 0xf6, 0x59, 0xc5, 0x59, 0xe7, - 0x23, 0x02, 0x77, 0x6c, 0x35, 0x05, 0xc5, 0x19, 0x6c, 0xf4, 0x3a, 0x0c, 0x6c, 0x10, 0xaf, 0x2d, - 0x86, 0x7c, 0xa5, 0x38, 0x19, 0xcf, 0xde, 0xf5, 0x0a, 0xf1, 0xda, 0x5c, 0x02, 0xd1, 0x5f, 0x98, - 0xb1, 0xa2, 0xf3, 0xad, 0xb6, 0xd9, 0x89, 0x93, 0xa0, 0xed, 0xbe, 0x21, 0x5d, 0x7c, 0xef, 0x2f, - 0x98, 0xf1, 0x35, 0x49, 0x9f, 0xfb, 0x52, 0xd4, 0x5f, 0xac, 0x39, 0xb3, 0x7e, 0x34, 0xdd, 0x88, - 0x7d, 0xaa, 0x1d, 0xe1, 0xa9, 0x2b, 0xba, 0x1f, 0xb3, 0x92, 0x3e, 0xef, 0x87, 0xfa, 0x8b, 0x35, - 0x67, 0xb4, 0xa3, 0xe6, 0xfd, 0x30, 0xeb, 0xc3, 0x8d, 0x82, 0xfb, 0xc0, 0xe7, 0x7c, 0xee, 0xfc, - 0x7f, 0x12, 0x2a, 0x8d, 0x0d, 0x27, 0x4a, 0xc6, 0x47, 0xd8, 0xa4, 0x51, 0x3e, 0x9d, 0x19, 0xda, - 0x88, 0x39, 0x0c, 0x3d, 0x0e, 0xe5, 0x88, 0xac, 0xb3, 0xb8, 0x4d, 0x23, 0xa2, 0x07, 0x93, 0x75, - 0x4c, 0xdb, 0xed, 0x5f, 0x2a, 0xa5, 0xcd, 0xa5, 0xf4, 0x7b, 0xf3, 0xd9, 0xde, 0xe8, 0x44, 0xb1, - 0xf4, 0xfb, 0x18, 0xb3, 0x9d, 0x35, 0x63, 0x09, 0x47, 0x9f, 0xb0, 0x60, 0xe8, 0x76, 0x1c, 0xf8, - 0x3e, 0x49, 0x84, 0x6a, 0xba, 0x59, 0xf0, 0x50, 0x5c, 0xe5, 0xd4, 0x75, 0x1f, 0x44, 0x03, 0x96, - 0x7c, 0x69, 0x77, 0xc9, 0x76, 0xc3, 0xeb, 0x34, 0xbb, 0x82, 0x34, 0x2e, 0xf1, 0x66, 0x2c, 0xe1, - 0x14, 0xd5, 0xf5, 0x39, 0xea, 0x40, 0x1a, 0x75, 0xde, 0x17, 0xa8, 0x02, 0x6e, 0xff, 0xf5, 0x41, - 0x38, 0x9b, 0xbb, 0x38, 0xa8, 0x21, 0xc3, 0x4c, 0x85, 0xcb, 0xae, 0x47, 0x64, 0x78, 0x12, 0x33, - 0x64, 0x6e, 0xaa, 0x56, 0x6c, 0x60, 0xa0, 0x9f, 0x02, 0x08, 0x9d, 0xc8, 0x69, 0x13, 0xe5, 0x97, - 0x3d, 0xb2, 0xbd, 0x40, 0xfb, 0xb1, 0x2c, 0x69, 0xea, 0xbd, 0xa9, 0x6a, 0x8a, 0xb1, 0xc1, 0x12, - 0xbd, 0x00, 0xc3, 0x11, 0xf1, 0x88, 0x13, 0xb3, 0xb0, 0xdf, 0x6c, 0x0e, 0x03, 0xd6, 0x20, 0x6c, - 0xe2, 0xa1, 0xa7, 0x54, 0x24, 0x57, 0x26, 0xa2, 0x25, 0x1d, 0xcd, 0x85, 0xde, 0xb4, 0x60, 0x6c, - 0xdd, 0xf5, 0x88, 0xe6, 0x2e, 0x32, 0x0e, 0x96, 0x8e, 0xfe, 0x92, 0x97, 0x4d, 0xba, 0x5a, 0x42, - 0xa6, 0x9a, 0x63, 0x9c, 0x61, 0x4f, 0x3f, 0xf3, 0x16, 0x89, 0x98, 0x68, 0x1d, 0x4c, 0x7f, 0xe6, - 0x9b, 0xbc, 0x19, 0x4b, 0x38, 0x9a, 0x86, 0x13, 0xa1, 0x13, 0xc7, 0x33, 0x11, 0x69, 0x12, 0x3f, - 0x71, 0x1d, 0x8f, 0xe7, 0x03, 0x54, 0x75, 0x3c, 0xf0, 0x72, 0x1a, 0x8c, 0xb3, 0xf8, 0xe8, 0x03, - 0xf0, 0x28, 0x77, 0x7c, 0x2c, 0xba, 0x71, 0xec, 0xfa, 0x2d, 0x3d, 0x0d, 0x84, 0xff, 0x67, 0x42, - 0x90, 0x7a, 0x74, 0x3e, 0x1f, 0x0d, 0xf7, 0x7a, 0x1e, 0x3d, 0x03, 0xd5, 0x78, 0xd3, 0x0d, 0x67, - 0xa2, 0x66, 0xcc, 0x0e, 0x3d, 0xaa, 0xda, 0xdb, 0xb8, 0x22, 0xda, 0xb1, 0xc2, 0x40, 0x0d, 0x18, - 0xe1, 0x9f, 0x84, 0x87, 0xa2, 0x09, 0xf9, 0xf8, 0x6c, 0x4f, 0xf5, 0x28, 0xd2, 0xdb, 0x26, 0xb1, - 0x73, 0xe7, 0x92, 0x3c, 0x82, 0xe1, 0x27, 0x06, 0x37, 0x0d, 0x32, 0x38, 0x45, 0xd4, 0xfe, 0x85, - 0x52, 0x7a, 0xc7, 0x6d, 0x2e, 0x52, 0x14, 0xd3, 0xa5, 0x98, 0xdc, 0x74, 0x22, 0xe9, 0x8d, 0x39, - 0x62, 0xda, 0x82, 0xa0, 0x7b, 0xd3, 0x89, 0xcc, 0x45, 0xcd, 0x18, 0x60, 0xc9, 0x09, 0xdd, 0x86, - 0x81, 0xc4, 0x73, 0x0a, 0xca, 0x73, 0x32, 0x38, 0x6a, 0x07, 0xc8, 0xc2, 0x74, 0x8c, 0x19, 0x0f, - 0xf4, 0x18, 0xb5, 0xfa, 0xd7, 0xe4, 0x11, 0x89, 0x30, 0xd4, 0xd7, 0x62, 0xcc, 0x5a, 0xed, 0x7b, - 0x90, 0x23, 0x57, 0x95, 0x22, 0x43, 0x17, 0x01, 0xe8, 0x06, 0x72, 0x39, 0x22, 0xeb, 0xee, 0xb6, - 0x30, 0x24, 0xd4, 0xda, 0xbd, 0xae, 0x20, 0xd8, 0xc0, 0x92, 0xcf, 0xac, 0x74, 0xd6, 0xe9, 0x33, - 0xa5, 0xee, 0x67, 0x38, 0x04, 0x1b, 0x58, 0xe8, 0x79, 0x18, 0x74, 0xdb, 0x4e, 0x4b, 0x85, 0x60, - 0x3e, 0x46, 0x17, 0xed, 0x3c, 0x6b, 0xb9, 0xb7, 0x3b, 0x31, 0xa6, 0x3a, 0xc4, 0x9a, 0xb0, 0xc0, - 0x45, 0xbf, 0x62, 0xc1, 0x48, 0x23, 0x68, 0xb7, 0x03, 0x9f, 0x6f, 0xbb, 0xc4, 0x1e, 0xf2, 0xf6, - 0x71, 0xa9, 0xf9, 0xc9, 0x19, 0x83, 0x19, 0xdf, 0x44, 0xaa, 0x84, 0x2c, 0x13, 0x84, 0x53, 0xbd, - 0x32, 0xd7, 0x76, 0x65, 0x9f, 0xb5, 0xfd, 0xeb, 0x16, 0x9c, 0xe2, 0xcf, 0x1a, 0xbb, 0x41, 0x91, - 0x7b, 0x14, 0x1c, 0xf3, 0x6b, 0x75, 0x6d, 0x90, 0x95, 0x97, 0xae, 0x0b, 0x8e, 0xbb, 0x3b, 0x89, - 0xe6, 0xe0, 0xd4, 0x7a, 0x10, 0x35, 0x88, 0x39, 0x10, 0x42, 0x30, 0x29, 0x42, 0x97, 0xb3, 0x08, - 0xb8, 0xfb, 0x19, 0x74, 0x13, 0x1e, 0x31, 0x1a, 0xcd, 0x71, 0xe0, 0xb2, 0xe9, 0x09, 0x41, 0xed, - 0x91, 0xcb, 0xb9, 0x58, 0xb8, 0xc7, 0xd3, 0x69, 0x87, 0x49, 0xad, 0x0f, 0x87, 0xc9, 0x6b, 0x70, - 0xae, 0xd1, 0x3d, 0x32, 0x5b, 0x71, 0x67, 0x2d, 0xe6, 0x92, 0xaa, 0x5a, 0xff, 0x3e, 0x41, 0xe0, - 0xdc, 0x4c, 0x2f, 0x44, 0xdc, 0x9b, 0x06, 0xfa, 0x08, 0x54, 0x23, 0xc2, 0xbe, 0x4a, 0x2c, 0x12, - 0x71, 0x8e, 0xb8, 0x4b, 0xd6, 0x16, 0x28, 0x27, 0xab, 0x65, 0xaf, 0x68, 0x88, 0xb1, 0xe2, 0x88, - 0xee, 0xc0, 0x50, 0xe8, 0x24, 0x8d, 0x0d, 0x91, 0x7e, 0x73, 0xe4, 0xf8, 0x17, 0xc5, 0x9c, 0xf9, - 0xc0, 0x8d, 0x84, 0x5d, 0xce, 0x04, 0x4b, 0x6e, 0xd4, 0x1a, 0x69, 0x04, 0xed, 0x30, 0xf0, 0x89, - 0x9f, 0xc4, 0xe3, 0xa3, 0xda, 0x1a, 0x99, 0x51, 0xad, 0xd8, 0xc0, 0x40, 0xcb, 0x70, 0x86, 0xf9, - 0x8c, 0x6e, 0xb9, 0xc9, 0x46, 0xd0, 0x49, 0xe4, 0x16, 0x68, 0x7c, 0x2c, 0x7d, 0x54, 0xb1, 0x90, - 0x83, 0x83, 0x73, 0x9f, 0x3c, 0xff, 0x3e, 0x38, 0xd5, 0xb5, 0x94, 0x0f, 0xe4, 0xae, 0x99, 0x85, - 0x47, 0xf2, 0x17, 0xcd, 0x81, 0x9c, 0x36, 0xff, 0x38, 0x13, 0x36, 0x6b, 0x18, 0xd2, 0x7d, 0x38, - 0x00, 0x1d, 0x28, 0x13, 0x7f, 0x4b, 0xe8, 0x90, 0xcb, 0x47, 0xfb, 0x76, 0x97, 0xfc, 0x2d, 0xbe, - 0xe6, 0x99, 0x97, 0xe3, 0x92, 0xbf, 0x85, 0x29, 0x6d, 0xf4, 0x45, 0x2b, 0x65, 0x08, 0x72, 0xb7, - 0xe1, 0x87, 0x8e, 0x65, 0xe7, 0xd0, 0xb7, 0x6d, 0x68, 0xff, 0xdb, 0x12, 0x5c, 0xd8, 0x8f, 0x48, - 0x1f, 0xc3, 0xf7, 0x24, 0x0c, 0xc6, 0xec, 0x20, 0x5c, 0x08, 0xe5, 0x61, 0x3a, 0x57, 0xf9, 0xd1, - 0xf8, 0x6b, 0x58, 0x80, 0x90, 0x07, 0xe5, 0xb6, 0x13, 0x0a, 0x6f, 0xd2, 0xfc, 0x51, 0x13, 0x69, - 0xe8, 0x7f, 0xc7, 0x5b, 0x74, 0x42, 0xee, 0xa3, 0x30, 0x1a, 0x30, 0x65, 0x83, 0x12, 0xa8, 0x38, - 0x51, 0xe4, 0xc8, 0x53, 0xd7, 0x6b, 0xc5, 0xf0, 0x9b, 0xa6, 0x24, 0xf9, 0xa1, 0x55, 0xaa, 0x09, - 0x73, 0x66, 0xf6, 0x67, 0x87, 0x52, 0xc9, 0x24, 0xec, 0x28, 0x3d, 0x86, 0x41, 0xe1, 0x44, 0xb2, - 0x8a, 0xce, 0x5f, 0xe2, 0xd9, 0x80, 0x6c, 0x9f, 0x28, 0x72, 0xaa, 0x05, 0x2b, 0xf4, 0x19, 0x8b, - 0x65, 0x2e, 0xcb, 0x04, 0x1b, 0xb1, 0x3b, 0x3b, 0x9e, 0x44, 0x6a, 0x33, 0x1f, 0x5a, 0x36, 0x62, - 0x93, 0xbb, 0xa8, 0x40, 0xc0, 0xac, 0xd2, 0xee, 0x0a, 0x04, 0xcc, 0xca, 0x94, 0x70, 0xb4, 0x9d, - 0x73, 0x64, 0x5e, 0x40, 0xf6, 0x6b, 0x1f, 0x87, 0xe4, 0x5f, 0xb5, 0xe0, 0x94, 0x9b, 0x3d, 0xfb, - 0x14, 0x7b, 0x99, 0x23, 0x06, 0x65, 0xf4, 0x3e, 0x5a, 0x55, 0xea, 0xbc, 0x0b, 0x84, 0xbb, 0x3b, - 0x83, 0x9a, 0x30, 0xe0, 0xfa, 0xeb, 0x81, 0x30, 0x62, 0xea, 0x47, 0xeb, 0xd4, 0xbc, 0xbf, 0x1e, - 0xe8, 0xd5, 0x4c, 0xff, 0x61, 0x46, 0x1d, 0x2d, 0xc0, 0x99, 0x48, 0x78, 0x9b, 0xae, 0xb8, 0x71, - 0x12, 0x44, 0x3b, 0x0b, 0x6e, 0xdb, 0x4d, 0x98, 0x01, 0x52, 0xae, 0x8f, 0x53, 0xfd, 0x80, 0x73, - 0xe0, 0x38, 0xf7, 0x29, 0xf4, 0x06, 0x0c, 0xc9, 0x54, 0xeb, 0x6a, 0x11, 0xfb, 0xc2, 0xee, 0xf9, - 0xaf, 0x26, 0xd3, 0x8a, 0xc8, 0xaa, 0x96, 0x0c, 0xed, 0x37, 0x87, 0xa1, 0xfb, 0x58, 0x14, 0x7d, - 0x14, 0x6a, 0x91, 0x4a, 0xff, 0xb6, 0x8a, 0x50, 0xd7, 0xf2, 0xfb, 0x8a, 0x23, 0x59, 0x65, 0x0a, - 0xe9, 0x44, 0x6f, 0xcd, 0x91, 0x6e, 0x58, 0x62, 0x7d, 0x7a, 0x5a, 0xc0, 0xdc, 0x16, 0x5c, 0xf5, - 0xc9, 0xd8, 0x8e, 0xdf, 0xc0, 0x8c, 0x07, 0x8a, 0x60, 0x70, 0x83, 0x38, 0x5e, 0xb2, 0x51, 0x8c, - 0x13, 0xff, 0x0a, 0xa3, 0x95, 0x4d, 0x02, 0xe2, 0xad, 0x58, 0x70, 0x42, 0xdb, 0x30, 0xb4, 0xc1, - 0x27, 0x80, 0xd8, 0x43, 0x2c, 0x1e, 0x75, 0x70, 0x53, 0xb3, 0x4a, 0x7f, 0x6e, 0xd1, 0x80, 0x25, - 0x3b, 0x16, 0x6f, 0x63, 0x44, 0x04, 0xf0, 0xa5, 0x5b, 0x5c, 0xfe, 0x53, 0xff, 0xe1, 0x00, 0x1f, - 0x86, 0x91, 0x88, 0x34, 0x02, 0xbf, 0xe1, 0x7a, 0xa4, 0x39, 0x2d, 0x1d, 0xf4, 0x07, 0xc9, 0x9a, - 0x61, 0xfb, 0x70, 0x6c, 0xd0, 0xc0, 0x29, 0x8a, 0xe8, 0xd3, 0x16, 0x8c, 0xa9, 0x9c, 0x51, 0xfa, - 0x41, 0x88, 0x70, 0x08, 0x2f, 0x14, 0x94, 0xa1, 0xca, 0x68, 0xd6, 0xd1, 0xdd, 0xdd, 0x89, 0xb1, - 0x74, 0x1b, 0xce, 0xf0, 0x45, 0xaf, 0x00, 0x04, 0x6b, 0x3c, 0xa8, 0x66, 0x3a, 0x11, 0xde, 0xe1, - 0x83, 0xbc, 0xea, 0x18, 0x4f, 0x9f, 0x93, 0x14, 0xb0, 0x41, 0x0d, 0x5d, 0x03, 0xe0, 0xcb, 0x66, - 0x75, 0x27, 0x94, 0x1b, 0x0d, 0x99, 0xf6, 0x04, 0x2b, 0x0a, 0x72, 0x6f, 0x77, 0xa2, 0xdb, 0x5b, - 0xc7, 0x02, 0x17, 0x8c, 0xc7, 0xd1, 0x4f, 0xc2, 0x50, 0xdc, 0x69, 0xb7, 0x1d, 0xe5, 0x3b, 0x2e, - 0x30, 0x21, 0x8f, 0xd3, 0x35, 0x44, 0x11, 0x6f, 0xc0, 0x92, 0x23, 0xba, 0x4d, 0x85, 0x6a, 0x2c, - 0xdc, 0x88, 0x6c, 0x15, 0x71, 0x9b, 0x60, 0x98, 0xbd, 0xd3, 0x7b, 0xa4, 0xe1, 0x8d, 0x73, 0x70, - 0xee, 0xed, 0x4e, 0x3c, 0x92, 0x6e, 0x5f, 0x08, 0x44, 0x8a, 0x5c, 0x2e, 0x4d, 0x74, 0x55, 0x56, - 0x5e, 0xa1, 0xaf, 0x2d, 0x0b, 0x02, 0x3c, 0xad, 0x2b, 0xaf, 0xb0, 0xe6, 0xde, 0x63, 0x66, 0x3e, - 0x8c, 0x16, 0xe1, 0x74, 0x23, 0xf0, 0x93, 0x28, 0xf0, 0x3c, 0x5e, 0x79, 0x88, 0xef, 0xf9, 0xb8, - 0x6f, 0xf9, 0x9d, 0xa2, 0xdb, 0xa7, 0x67, 0xba, 0x51, 0x70, 0xde, 0x73, 0xb6, 0x9f, 0x8e, 0x36, - 0x14, 0x83, 0xf3, 0x3c, 0x8c, 0x90, 0xed, 0x84, 0x44, 0xbe, 0xe3, 0xdd, 0xc0, 0x0b, 0xd2, 0xab, - 0xca, 0xd6, 0xc0, 0x25, 0xa3, 0x1d, 0xa7, 0xb0, 0x90, 0xad, 0x1c, 0x1d, 0x46, 0xda, 0x27, 0x77, - 0x74, 0x48, 0xb7, 0x86, 0xfd, 0xbf, 0x4b, 0x29, 0x83, 0x6c, 0x35, 0x22, 0x04, 0x05, 0x50, 0xf1, - 0x83, 0xa6, 0x92, 0xfd, 0x57, 0x8b, 0x91, 0xfd, 0xd7, 0x83, 0xa6, 0x51, 0x9e, 0x85, 0xfe, 0x8b, - 0x31, 0xe7, 0xc3, 0xea, 0x57, 0xc8, 0x42, 0x1f, 0x0c, 0x20, 0x36, 0x1a, 0x45, 0x72, 0x56, 0xf5, - 0x2b, 0x96, 0x4c, 0x46, 0x38, 0xcd, 0x17, 0x6d, 0x42, 0x65, 0x23, 0x88, 0x13, 0xb9, 0xfd, 0x38, - 0xe2, 0x4e, 0xe7, 0x4a, 0x10, 0x27, 0xcc, 0x8a, 0x50, 0xaf, 0x4d, 0x5b, 0x62, 0xcc, 0x79, 0xd8, - 0xff, 0xc5, 0x4a, 0xf9, 0xd0, 0x6f, 0xb1, 0xc8, 0xdb, 0x2d, 0xe2, 0xd3, 0x65, 0x6d, 0x86, 0x1a, - 0xfd, 0x70, 0x26, 0x8f, 0xf1, 0x5d, 0xbd, 0x0a, 0x6b, 0xdd, 0xa1, 0x14, 0x26, 0x19, 0x09, 0x23, - 0x2a, 0xe9, 0xe3, 0x56, 0x3a, 0xa3, 0xb4, 0x54, 0xc4, 0x06, 0xc3, 0xcc, 0xaa, 0xde, 0x37, 0x39, - 0xd5, 0xfe, 0xa2, 0x05, 0x43, 0x75, 0xa7, 0xb1, 0x19, 0xac, 0xaf, 0xa3, 0x67, 0xa0, 0xda, 0xec, - 0x44, 0x66, 0x72, 0xab, 0x72, 0x1c, 0xcc, 0x8a, 0x76, 0xac, 0x30, 0xe8, 0x1c, 0x5e, 0x77, 0x1a, - 0x32, 0xb7, 0xba, 0xcc, 0xe7, 0xf0, 0x65, 0xd6, 0x82, 0x05, 0x04, 0xbd, 0x00, 0xc3, 0x6d, 0x67, - 0x5b, 0x3e, 0x9c, 0x75, 0xe0, 0x2f, 0x6a, 0x10, 0x36, 0xf1, 0xec, 0x7f, 0x69, 0xc1, 0x78, 0xdd, - 0x89, 0xdd, 0xc6, 0x74, 0x27, 0xd9, 0xa8, 0xbb, 0xc9, 0x5a, 0xa7, 0xb1, 0x49, 0x12, 0x9e, 0x50, - 0x4f, 0x7b, 0xd9, 0x89, 0xe9, 0x52, 0x52, 0xfb, 0x3a, 0xd5, 0xcb, 0x1b, 0xa2, 0x1d, 0x2b, 0x0c, - 0xf4, 0x06, 0x0c, 0x87, 0x4e, 0x1c, 0xdf, 0x09, 0xa2, 0x26, 0x26, 0xeb, 0xc5, 0x94, 0xb3, 0x58, - 0x21, 0x8d, 0x88, 0x24, 0x98, 0xac, 0x8b, 0x43, 0x66, 0x4d, 0x1f, 0x9b, 0xcc, 0xec, 0xcf, 0x5b, - 0x70, 0xae, 0x4e, 0x9c, 0x88, 0x44, 0xac, 0xfa, 0x85, 0x7a, 0x91, 0x19, 0x2f, 0xe8, 0x34, 0xd1, - 0xeb, 0x50, 0x4d, 0x68, 0x33, 0xed, 0x96, 0x55, 0x6c, 0xb7, 0xd8, 0x19, 0xf1, 0xaa, 0x20, 0x8e, - 0x15, 0x1b, 0xfb, 0x6f, 0x58, 0x30, 0xc2, 0x8e, 0xdb, 0x66, 0x49, 0xe2, 0xb8, 0x5e, 0x57, 0x91, - 0x28, 0xab, 0xcf, 0x22, 0x51, 0x17, 0x60, 0x60, 0x23, 0x68, 0x93, 0xec, 0x51, 0xf1, 0x95, 0x80, - 0x6e, 0xab, 0x29, 0x04, 0x3d, 0x47, 0x3f, 0xbc, 0xeb, 0x27, 0x0e, 0x5d, 0x02, 0xd2, 0x9d, 0x7b, - 0x82, 0x7f, 0x74, 0xd5, 0x8c, 0x4d, 0x1c, 0xfb, 0xb7, 0x6a, 0x30, 0x24, 0xe2, 0x09, 0xfa, 0x2e, - 0xaa, 0x20, 0xf7, 0xf7, 0xa5, 0x9e, 0xfb, 0xfb, 0x18, 0x06, 0x1b, 0xac, 0x5a, 0x9d, 0x30, 0x23, - 0xaf, 0x15, 0x12, 0x80, 0xc2, 0x0b, 0xe0, 0xe9, 0x6e, 0xf1, 0xff, 0x58, 0xb0, 0x42, 0x5f, 0xb0, - 0xe0, 0x44, 0x23, 0xf0, 0x7d, 0xd2, 0xd0, 0x36, 0xce, 0x40, 0x11, 0x71, 0x06, 0x33, 0x69, 0xa2, - 0xfa, 0xac, 0x27, 0x03, 0xc0, 0x59, 0xf6, 0xe8, 0x25, 0x18, 0xe5, 0x63, 0x76, 0x33, 0xe5, 0x83, - 0xd6, 0xb5, 0x83, 0x4c, 0x20, 0x4e, 0xe3, 0xa2, 0x49, 0xee, 0xcb, 0x17, 0x55, 0x7a, 0x06, 0xb5, - 0xab, 0xce, 0xa8, 0xcf, 0x63, 0x60, 0xa0, 0x08, 0x50, 0x44, 0xd6, 0x23, 0x12, 0x6f, 0x88, 0x78, - 0x0b, 0x66, 0x5f, 0x0d, 0x1d, 0x2e, 0x01, 0x1b, 0x77, 0x51, 0xc2, 0x39, 0xd4, 0xd1, 0xa6, 0xd8, - 0x60, 0x56, 0x8b, 0x90, 0xa1, 0xe2, 0x33, 0xf7, 0xdc, 0x67, 0x4e, 0x40, 0x25, 0xde, 0x70, 0xa2, - 0x26, 0xb3, 0xeb, 0xca, 0x3c, 0xe9, 0x67, 0x85, 0x36, 0x60, 0xde, 0x8e, 0x66, 0xe1, 0x64, 0xa6, - 0xf2, 0x51, 0x2c, 0x7c, 0xc5, 0x2a, 0xc1, 0x23, 0x53, 0x33, 0x29, 0xc6, 0x5d, 0x4f, 0x98, 0xce, - 0x87, 0xe1, 0x7d, 0x9c, 0x0f, 0x3b, 0x2a, 0xaa, 0x8f, 0x7b, 0x71, 0x5f, 0x2e, 0x64, 0x00, 0xfa, - 0x0a, 0xe1, 0xfb, 0x5c, 0x26, 0x84, 0x6f, 0x94, 0x75, 0xe0, 0x66, 0x31, 0x1d, 0x38, 0x78, 0xbc, - 0xde, 0x83, 0x8c, 0xbf, 0xfb, 0x73, 0x0b, 0xe4, 0x77, 0x9d, 0x71, 0x1a, 0x1b, 0x84, 0x4e, 0x19, - 0xf4, 0x5e, 0x18, 0x53, 0x5b, 0xe8, 0x99, 0xa0, 0xe3, 0xf3, 0xd0, 0xbb, 0xb2, 0x3e, 0x14, 0xc6, - 0x29, 0x28, 0xce, 0x60, 0xa3, 0x29, 0xa8, 0xd1, 0x71, 0xe2, 0x8f, 0x72, 0x5d, 0xab, 0xb6, 0xe9, - 0xd3, 0xcb, 0xf3, 0xe2, 0x29, 0x8d, 0x83, 0x02, 0x38, 0xe5, 0x39, 0x71, 0xc2, 0x7a, 0x40, 0x77, - 0xd4, 0x87, 0x2c, 0x7f, 0xc0, 0xb2, 0x08, 0x16, 0xb2, 0x84, 0x70, 0x37, 0x6d, 0xfb, 0x5b, 0x03, - 0x30, 0x9a, 0x92, 0x8c, 0x07, 0x54, 0xd2, 0xcf, 0x40, 0x55, 0xea, 0xcd, 0x6c, 0xa1, 0x16, 0xa5, - 0x5c, 0x15, 0x06, 0x55, 0x5a, 0x6b, 0x5a, 0xab, 0x66, 0x8d, 0x0a, 0x43, 0xe1, 0x62, 0x13, 0x8f, - 0x09, 0xe5, 0xc4, 0x8b, 0x67, 0x3c, 0x97, 0xf8, 0x09, 0xef, 0x66, 0x31, 0x42, 0x79, 0x75, 0x61, - 0xc5, 0x24, 0xaa, 0x85, 0x72, 0x06, 0x80, 0xb3, 0xec, 0xd1, 0xa7, 0x2c, 0x18, 0x75, 0xee, 0xc4, - 0xba, 0xa4, 0xaa, 0x08, 0xd6, 0x3b, 0xa2, 0x92, 0x4a, 0x55, 0x69, 0xe5, 0x2e, 0xdf, 0x54, 0x13, - 0x4e, 0x33, 0x45, 0x6f, 0x59, 0x80, 0xc8, 0x36, 0x69, 0xc8, 0x70, 0x42, 0xd1, 0x97, 0xc1, 0x22, - 0x76, 0x9a, 0x97, 0xba, 0xe8, 0x72, 0xa9, 0xde, 0xdd, 0x8e, 0x73, 0xfa, 0x60, 0xff, 0xb3, 0xb2, - 0x5a, 0x50, 0x3a, 0x82, 0xd5, 0x31, 0x22, 0xe9, 0xac, 0xc3, 0x47, 0xd2, 0xe9, 0x88, 0x84, 0xee, - 0xac, 0xca, 0x54, 0x12, 0x56, 0xe9, 0x01, 0x25, 0x61, 0xfd, 0xb4, 0x95, 0x2a, 0x49, 0x34, 0x7c, - 0xf1, 0x95, 0x62, 0xa3, 0x67, 0x27, 0x79, 0xb4, 0x44, 0x46, 0xba, 0xa7, 0x83, 0x64, 0xa8, 0x34, - 0x35, 0xd0, 0x0e, 0x24, 0x0d, 0xff, 0x7d, 0x19, 0x86, 0x0d, 0x4d, 0x9a, 0x6b, 0x16, 0x59, 0x0f, - 0x99, 0x59, 0x54, 0x3a, 0x80, 0x59, 0xf4, 0x53, 0x50, 0x6b, 0x48, 0x29, 0x5f, 0x4c, 0x51, 0xde, - 0xac, 0xee, 0xd0, 0x82, 0x5e, 0x35, 0x61, 0xcd, 0x13, 0xcd, 0xa5, 0x52, 0x77, 0x84, 0x86, 0x18, - 0x60, 0x1a, 0x22, 0x2f, 0xb7, 0x46, 0x68, 0x8a, 0xee, 0x67, 0x58, 0xe5, 0xaa, 0xd0, 0x15, 0xef, - 0x25, 0x63, 0xdc, 0x79, 0xe5, 0xaa, 0xe5, 0x79, 0xd9, 0x8c, 0x4d, 0x1c, 0xfb, 0x5b, 0x96, 0xfa, - 0xb8, 0xf7, 0xa1, 0x46, 0xc3, 0xed, 0x74, 0x8d, 0x86, 0x4b, 0x85, 0x0c, 0x73, 0x8f, 0xe2, 0x0c, - 0xd7, 0x61, 0x68, 0x26, 0x68, 0xb7, 0x1d, 0xbf, 0x89, 0xbe, 0x1f, 0x86, 0x1a, 0xfc, 0xa7, 0x70, - 0xec, 0xb0, 0xe3, 0x41, 0x01, 0xc5, 0x12, 0x86, 0x1e, 0x83, 0x01, 0x27, 0x6a, 0x49, 0x67, 0x0e, - 0x0b, 0xae, 0x99, 0x8e, 0x5a, 0x31, 0x66, 0xad, 0xf6, 0x3f, 0x1a, 0x00, 0x76, 0xa6, 0xed, 0x44, - 0xa4, 0xb9, 0x1a, 0xb0, 0xa2, 0x80, 0xc7, 0x7a, 0xa8, 0xa6, 0x37, 0x4b, 0x0f, 0xf3, 0xc1, 0x9a, - 0x71, 0xb8, 0x52, 0xbe, 0xcf, 0x87, 0x2b, 0x3d, 0xce, 0xcb, 0x06, 0x1e, 0xa2, 0xf3, 0x32, 0xfb, - 0xb3, 0x16, 0x20, 0x15, 0x08, 0xa1, 0x0f, 0xb4, 0xa7, 0xa0, 0xa6, 0x42, 0x22, 0x84, 0x61, 0xa5, - 0x45, 0x84, 0x04, 0x60, 0x8d, 0xd3, 0xc7, 0x0e, 0xf9, 0x49, 0x29, 0xbf, 0xcb, 0xe9, 0xb8, 0x5c, - 0x26, 0xf5, 0x85, 0x38, 0xb7, 0x7f, 0xbb, 0x04, 0x8f, 0x70, 0x95, 0xbc, 0xe8, 0xf8, 0x4e, 0x8b, - 0xb4, 0x69, 0xaf, 0xfa, 0x0d, 0x51, 0x68, 0xd0, 0xad, 0x99, 0x2b, 0xe3, 0x6c, 0x8f, 0xba, 0x76, - 0xf9, 0x9a, 0xe3, 0xab, 0x6c, 0xde, 0x77, 0x13, 0xcc, 0x88, 0xa3, 0x18, 0xaa, 0xb2, 0x62, 0xbd, - 0x90, 0xc5, 0x05, 0x31, 0x52, 0x62, 0x49, 0xe8, 0x4d, 0x82, 0x15, 0x23, 0x6a, 0xb8, 0x7a, 0x41, - 0x63, 0x13, 0x93, 0x30, 0x60, 0x72, 0xd7, 0x08, 0x73, 0x5c, 0x10, 0xed, 0x58, 0x61, 0xd8, 0xbf, - 0x6d, 0x41, 0x56, 0x23, 0x19, 0xd5, 0xd7, 0xac, 0x3d, 0xab, 0xaf, 0x1d, 0xa0, 0xfc, 0xd9, 0x4f, - 0xc0, 0xb0, 0x93, 0x50, 0x23, 0x82, 0x6f, 0xbb, 0xcb, 0x87, 0x3b, 0xd6, 0x58, 0x0c, 0x9a, 0xee, - 0xba, 0xcb, 0xb6, 0xdb, 0x26, 0x39, 0xfb, 0x7f, 0x0c, 0xc0, 0xa9, 0xae, 0x6c, 0x10, 0xf4, 0x22, - 0x8c, 0x34, 0xc4, 0xf4, 0x08, 0xa5, 0x43, 0xab, 0x66, 0x86, 0xc5, 0x69, 0x18, 0x4e, 0x61, 0xf6, - 0x31, 0x41, 0xe7, 0xe1, 0x74, 0x44, 0x37, 0xfa, 0x1d, 0x32, 0xbd, 0x9e, 0x90, 0x68, 0x85, 0x34, - 0x02, 0xbf, 0xc9, 0x6b, 0x04, 0x96, 0xeb, 0x8f, 0xde, 0xdd, 0x9d, 0x38, 0x8d, 0xbb, 0xc1, 0x38, - 0xef, 0x19, 0x14, 0xc2, 0xa8, 0x67, 0xda, 0x80, 0x62, 0x03, 0x70, 0x28, 0xf3, 0x51, 0xd9, 0x08, - 0xa9, 0x66, 0x9c, 0x66, 0x90, 0x36, 0x24, 0x2b, 0x0f, 0xc8, 0x90, 0xfc, 0xa4, 0x36, 0x24, 0xf9, - 0xf9, 0xfb, 0x07, 0x0b, 0xce, 0x06, 0x3a, 0x6e, 0x4b, 0xf2, 0x65, 0xa8, 0xca, 0xd8, 0xa4, 0xbe, - 0x62, 0x7a, 0x4c, 0x3a, 0x3d, 0x24, 0xda, 0xbd, 0x12, 0xe4, 0x6c, 0x42, 0xe8, 0x3a, 0xd3, 0x1a, - 0x3f, 0xb5, 0xce, 0x0e, 0xa6, 0xf5, 0xd1, 0x36, 0x8f, 0xcb, 0xe2, 0xba, 0xed, 0x03, 0x45, 0x6f, - 0xa2, 0x74, 0xa8, 0x96, 0x4a, 0x92, 0x50, 0xe1, 0x5a, 0x17, 0x01, 0xb4, 0xa1, 0x26, 0x42, 0xe0, - 0xd5, 0xb1, 0xaf, 0xb6, 0xe7, 0xb0, 0x81, 0x45, 0xf7, 0xd4, 0xae, 0x1f, 0x27, 0x8e, 0xe7, 0x5d, - 0x71, 0xfd, 0x44, 0x38, 0x07, 0x95, 0x12, 0x9f, 0xd7, 0x20, 0x6c, 0xe2, 0x9d, 0x7f, 0x8f, 0xf1, - 0x5d, 0x0e, 0xf2, 0x3d, 0x37, 0xe0, 0xdc, 0x9c, 0x9b, 0xa8, 0xc4, 0x0d, 0x35, 0x8f, 0xa8, 0x1d, - 0xa6, 0x12, 0x91, 0xac, 0x9e, 0x89, 0x48, 0x46, 0xe2, 0x44, 0x29, 0x9d, 0xe7, 0x91, 0x4d, 0x9c, - 0xb0, 0x5f, 0x84, 0x33, 0x73, 0x6e, 0x72, 0xd9, 0xf5, 0xc8, 0x01, 0x99, 0xd8, 0xbf, 0x39, 0x08, - 0x23, 0x66, 0xea, 0xdf, 0x41, 0x72, 0xa9, 0x3e, 0x4f, 0x4d, 0x2d, 0xf1, 0x76, 0xae, 0x3a, 0x34, - 0xbb, 0x75, 0xe4, 0x3c, 0xc4, 0xfc, 0x11, 0x33, 0xac, 0x2d, 0xcd, 0x13, 0x9b, 0x1d, 0x40, 0x77, - 0xa0, 0xb2, 0xce, 0x02, 0xfb, 0xcb, 0x45, 0x44, 0x16, 0xe4, 0x8d, 0xa8, 0x5e, 0x66, 0x3c, 0x35, - 0x80, 0xf3, 0xa3, 0x1a, 0x32, 0x4a, 0x67, 0x8b, 0x19, 0xc1, 0xa8, 0x22, 0x4f, 0x4c, 0x61, 0xf4, - 0x12, 0xf5, 0x95, 0x43, 0x88, 0xfa, 0x94, 0xe0, 0x1d, 0x7c, 0x40, 0x82, 0x97, 0x25, 0x69, 0x24, - 0x1b, 0xcc, 0x7e, 0x13, 0xd1, 0xf3, 0x43, 0x6c, 0x10, 0x8c, 0x24, 0x8d, 0x14, 0x18, 0x67, 0xf1, - 0xd1, 0xc7, 0x94, 0xe8, 0xae, 0x16, 0xe1, 0x57, 0x35, 0x67, 0xf4, 0x71, 0x4b, 0xed, 0xcf, 0x96, - 0x60, 0x6c, 0xce, 0xef, 0x2c, 0xcf, 0x2d, 0x77, 0xd6, 0x3c, 0xb7, 0x71, 0x8d, 0xec, 0x50, 0xd1, - 0xbc, 0x49, 0x76, 0xe6, 0x67, 0xc5, 0x0a, 0x52, 0x73, 0xe6, 0x1a, 0x6d, 0xc4, 0x1c, 0x46, 0x85, - 0xd1, 0xba, 0xeb, 0xb7, 0x48, 0x14, 0x46, 0xae, 0x70, 0x79, 0x1a, 0xc2, 0xe8, 0xb2, 0x06, 0x61, - 0x13, 0x8f, 0xd2, 0x0e, 0xee, 0xf8, 0x24, 0xca, 0x1a, 0xb2, 0x4b, 0xb4, 0x11, 0x73, 0x18, 0x45, - 0x4a, 0xa2, 0x4e, 0x9c, 0x88, 0xc9, 0xa8, 0x90, 0x56, 0x69, 0x23, 0xe6, 0x30, 0xba, 0xd2, 0xe3, - 0xce, 0x1a, 0x0b, 0xdc, 0xc8, 0x84, 0xea, 0xaf, 0xf0, 0x66, 0x2c, 0xe1, 0x14, 0x75, 0x93, 0xec, - 0xcc, 0xd2, 0x5d, 0x6f, 0x26, 0x63, 0xe7, 0x1a, 0x6f, 0xc6, 0x12, 0xce, 0x8a, 0x1b, 0xa6, 0x87, - 0xe3, 0xbb, 0xae, 0xb8, 0x61, 0xba, 0xfb, 0x3d, 0xf6, 0xcf, 0xbf, 0x6c, 0xc1, 0x88, 0x19, 0x6e, - 0x85, 0x5a, 0x19, 0x1b, 0x77, 0xa9, 0xab, 0x36, 0xee, 0x8f, 0xe5, 0x5d, 0x2c, 0xd6, 0x72, 0x93, - 0x20, 0x8c, 0x9f, 0x25, 0x7e, 0xcb, 0xf5, 0x09, 0x3b, 0x45, 0xe7, 0x61, 0x5a, 0xa9, 0x58, 0xae, - 0x99, 0xa0, 0x49, 0x0e, 0x61, 0x24, 0xdb, 0xb7, 0xe0, 0x54, 0x57, 0x9a, 0x56, 0x1f, 0xa6, 0xc5, - 0xbe, 0x49, 0xb2, 0x36, 0x86, 0x61, 0x4a, 0x58, 0x16, 0xd8, 0x99, 0x81, 0x53, 0x7c, 0x21, 0x51, - 0x4e, 0x2b, 0x8d, 0x0d, 0xd2, 0x56, 0xa9, 0x77, 0xcc, 0xbf, 0x7e, 0x33, 0x0b, 0xc4, 0xdd, 0xf8, - 0xf6, 0xe7, 0x2c, 0x18, 0x4d, 0x65, 0xce, 0x15, 0x64, 0x04, 0xb1, 0x95, 0x16, 0xb0, 0xe8, 0x3f, - 0x16, 0x02, 0x5d, 0x66, 0xca, 0x54, 0xaf, 0x34, 0x0d, 0xc2, 0x26, 0x9e, 0xfd, 0xc5, 0x12, 0x54, - 0x65, 0x04, 0x45, 0x1f, 0x5d, 0xf9, 0x8c, 0x05, 0xa3, 0xea, 0x4c, 0x83, 0x39, 0xcb, 0x4a, 0x45, - 0xa4, 0x39, 0xd0, 0x1e, 0xa8, 0xed, 0xb6, 0xbf, 0x1e, 0x68, 0x8b, 0x1c, 0x9b, 0xcc, 0x70, 0x9a, - 0x37, 0xba, 0x09, 0x10, 0xef, 0xc4, 0x09, 0x69, 0x1b, 0x6e, 0x3b, 0xdb, 0x58, 0x71, 0x93, 0x8d, - 0x20, 0x22, 0x74, 0x7d, 0x5d, 0x0f, 0x9a, 0x64, 0x45, 0x61, 0x6a, 0x13, 0x4a, 0xb7, 0x61, 0x83, - 0x92, 0xfd, 0x0f, 0x4a, 0x70, 0x32, 0xdb, 0x25, 0xf4, 0x41, 0x18, 0x91, 0xdc, 0x8d, 0x3b, 0xd2, - 0x64, 0xd8, 0xc8, 0x08, 0x36, 0x60, 0xf7, 0x76, 0x27, 0x26, 0xba, 0x2f, 0xa9, 0x9b, 0x34, 0x51, - 0x70, 0x8a, 0x18, 0x3f, 0x58, 0x12, 0x27, 0xa0, 0xf5, 0x9d, 0xe9, 0x30, 0x14, 0xa7, 0x43, 0xc6, - 0xc1, 0x92, 0x09, 0xc5, 0x19, 0x6c, 0xb4, 0x0c, 0x67, 0x8c, 0x96, 0xeb, 0xc4, 0x6d, 0x6d, 0xac, - 0x05, 0x91, 0xdc, 0x59, 0x3d, 0xa6, 0x03, 0xbb, 0xba, 0x71, 0x70, 0xee, 0x93, 0x54, 0xdb, 0x37, - 0x9c, 0xd0, 0x69, 0xb8, 0xc9, 0x8e, 0xf0, 0x43, 0x2a, 0xd9, 0x34, 0x23, 0xda, 0xb1, 0xc2, 0xb0, - 0x17, 0x61, 0xa0, 0xcf, 0x19, 0xd4, 0x97, 0x45, 0xff, 0x32, 0x54, 0x29, 0x39, 0x69, 0xde, 0x15, - 0x41, 0x32, 0x80, 0xaa, 0xbc, 0xbb, 0x04, 0xd9, 0x50, 0x76, 0x1d, 0x79, 0x76, 0xa7, 0x5e, 0x6b, - 0x3e, 0x8e, 0x3b, 0x6c, 0x93, 0x4c, 0x81, 0xe8, 0x49, 0x28, 0x93, 0xed, 0x30, 0x7b, 0x48, 0x77, - 0x69, 0x3b, 0x74, 0x23, 0x12, 0x53, 0x24, 0xb2, 0x1d, 0xa2, 0xf3, 0x50, 0x72, 0x9b, 0x42, 0x49, - 0x81, 0xc0, 0x29, 0xcd, 0xcf, 0xe2, 0x92, 0xdb, 0xb4, 0xb7, 0xa1, 0xa6, 0x2e, 0x4b, 0x41, 0x9b, - 0x52, 0x76, 0x5b, 0x45, 0x84, 0x3c, 0x49, 0xba, 0x3d, 0xa4, 0x76, 0x07, 0x40, 0xa7, 0x10, 0x16, - 0x25, 0x5f, 0x2e, 0xc0, 0x40, 0x23, 0x10, 0xe9, 0xcd, 0x55, 0x4d, 0x86, 0x09, 0x6d, 0x06, 0xb1, - 0x6f, 0xc1, 0xd8, 0x35, 0x3f, 0xb8, 0xc3, 0x2a, 0xbd, 0xb3, 0xc2, 0x66, 0x94, 0xf0, 0x3a, 0xfd, - 0x91, 0x35, 0x11, 0x18, 0x14, 0x73, 0x98, 0xaa, 0xf8, 0x54, 0xea, 0x55, 0xf1, 0xc9, 0xfe, 0xb8, - 0x05, 0x23, 0x2a, 0x17, 0x69, 0x6e, 0x6b, 0x93, 0xd2, 0x6d, 0x45, 0x41, 0x27, 0xcc, 0xd2, 0x65, - 0xd7, 0x19, 0x61, 0x0e, 0x33, 0x93, 0xf4, 0x4a, 0xfb, 0x24, 0xe9, 0x5d, 0x80, 0x81, 0x4d, 0xd7, - 0x6f, 0x66, 0xef, 0xe7, 0xb8, 0xe6, 0xfa, 0x4d, 0xcc, 0x20, 0xb4, 0x0b, 0x27, 0x55, 0x17, 0xa4, - 0x42, 0x78, 0x11, 0x46, 0xd6, 0x3a, 0xae, 0xd7, 0x94, 0x15, 0xdb, 0x32, 0x9e, 0x92, 0xba, 0x01, - 0xc3, 0x29, 0x4c, 0xba, 0xaf, 0x5b, 0x73, 0x7d, 0x27, 0xda, 0x59, 0xd6, 0x1a, 0x48, 0x09, 0xa5, - 0xba, 0x82, 0x60, 0x03, 0xcb, 0x7e, 0xb3, 0x0c, 0x63, 0xe9, 0x8c, 0xac, 0x3e, 0xb6, 0x57, 0x4f, - 0x42, 0x85, 0x25, 0x69, 0x65, 0x3f, 0x2d, 0x2f, 0x72, 0xc6, 0x61, 0x28, 0x86, 0x41, 0x5e, 0xde, - 0xa1, 0x98, 0xbb, 0x6d, 0x54, 0x27, 0x95, 0x7f, 0x85, 0xc5, 0x93, 0x89, 0x8a, 0x12, 0x82, 0x15, - 0xfa, 0x94, 0x05, 0x43, 0x41, 0x68, 0x56, 0x0a, 0xfa, 0x40, 0x91, 0xd9, 0x6a, 0x22, 0x59, 0x46, - 0x58, 0xc4, 0xea, 0xd3, 0xcb, 0xcf, 0x21, 0x59, 0x9f, 0xff, 0x11, 0x18, 0x31, 0x31, 0xf7, 0x33, - 0x8a, 0xab, 0xa6, 0x51, 0xfc, 0x19, 0x73, 0x52, 0x88, 0x7c, 0xbc, 0x3e, 0x96, 0xdb, 0x0d, 0xa8, - 0x34, 0x54, 0x00, 0xc0, 0xa1, 0xea, 0x7c, 0xaa, 0x7a, 0x0b, 0xec, 0x10, 0x88, 0x53, 0xb3, 0xbf, - 0x65, 0x19, 0xf3, 0x03, 0x93, 0x78, 0xbe, 0x89, 0x22, 0x28, 0xb7, 0xb6, 0x36, 0x85, 0x29, 0x7a, - 0xb5, 0xa0, 0xe1, 0x9d, 0xdb, 0xda, 0xd4, 0x73, 0xdc, 0x6c, 0xc5, 0x94, 0x59, 0x1f, 0x4e, 0xc0, - 0x54, 0xda, 0x66, 0x79, 0xff, 0xb4, 0x4d, 0xfb, 0xad, 0x12, 0x9c, 0xea, 0x9a, 0x54, 0xe8, 0x0d, - 0xa8, 0x44, 0xf4, 0x2d, 0xc5, 0xeb, 0x2d, 0x14, 0x96, 0x68, 0x19, 0xcf, 0x37, 0xb5, 0xde, 0x4d, - 0xb7, 0x63, 0xce, 0x12, 0x5d, 0x05, 0xa4, 0xc3, 0x54, 0x94, 0x07, 0x92, 0xbf, 0xf2, 0x79, 0xf1, - 0x28, 0x9a, 0xee, 0xc2, 0xc0, 0x39, 0x4f, 0xa1, 0x97, 0xb2, 0x8e, 0xcc, 0x72, 0xfa, 0xdc, 0x72, - 0x2f, 0x9f, 0xa4, 0xfd, 0xcf, 0x4b, 0x30, 0x9a, 0x2a, 0xdc, 0x84, 0x3c, 0xa8, 0x12, 0x8f, 0x39, - 0xf5, 0xa5, 0xb2, 0x39, 0x6a, 0x1d, 0x64, 0xa5, 0x20, 0x2f, 0x09, 0xba, 0x58, 0x71, 0x78, 0x38, - 0x0e, 0xd7, 0x5f, 0x84, 0x11, 0xd9, 0xa1, 0x0f, 0x38, 0x6d, 0x4f, 0x0c, 0xa0, 0x9a, 0xa3, 0x97, - 0x0c, 0x18, 0x4e, 0x61, 0xda, 0xbf, 0x53, 0x86, 0x71, 0x7e, 0x0a, 0xd2, 0x54, 0x33, 0x6f, 0x51, - 0xee, 0xb7, 0xfe, 0x8a, 0x2e, 0xaf, 0xc6, 0x07, 0x72, 0xed, 0xa8, 0xd7, 0x0e, 0xe4, 0x33, 0xea, - 0x2b, 0x32, 0xeb, 0x2b, 0x99, 0xc8, 0x2c, 0x6e, 0x76, 0xb7, 0x8e, 0xa9, 0x47, 0xdf, 0x5d, 0xa1, - 0x5a, 0x7f, 0xb7, 0x04, 0x27, 0x32, 0x77, 0x3a, 0xa0, 0x37, 0xd3, 0x65, 0x80, 0xad, 0x22, 0x7c, - 0xe5, 0x7b, 0x96, 0xf9, 0x3f, 0x58, 0x31, 0xe0, 0x07, 0xb4, 0x54, 0xec, 0x3f, 0x28, 0xc1, 0x58, - 0xfa, 0x32, 0x8a, 0x87, 0x70, 0xa4, 0xde, 0x0d, 0x35, 0x56, 0x6f, 0x9d, 0x5d, 0xb2, 0xc9, 0x5d, - 0xf2, 0xbc, 0xb4, 0xb5, 0x6c, 0xc4, 0x1a, 0xfe, 0x50, 0xd4, 0x58, 0xb6, 0xff, 0x9e, 0x05, 0x67, - 0xf9, 0x5b, 0x66, 0xe7, 0xe1, 0x5f, 0xcd, 0x1b, 0xdd, 0x57, 0x8b, 0xed, 0x60, 0xa6, 0x2c, 0xe0, - 0x7e, 0xe3, 0xcb, 0x2e, 0xf7, 0x13, 0xbd, 0x4d, 0x4f, 0x85, 0x87, 0xb0, 0xb3, 0x07, 0x9a, 0x0c, - 0xf6, 0x1f, 0x94, 0x41, 0xdf, 0x67, 0x88, 0x5c, 0x91, 0xe3, 0x58, 0x48, 0x79, 0xc4, 0x95, 0x1d, - 0xbf, 0xa1, 0x6f, 0x4e, 0xac, 0x66, 0x52, 0x1c, 0x7f, 0xce, 0x82, 0x61, 0xd7, 0x77, 0x13, 0xd7, - 0x61, 0xdb, 0xe8, 0x62, 0xee, 0x5a, 0x53, 0xec, 0xe6, 0x39, 0xe5, 0x20, 0x32, 0xcf, 0x71, 0x14, - 0x33, 0x6c, 0x72, 0x46, 0x1f, 0x16, 0xc1, 0xd3, 0xe5, 0xc2, 0xb2, 0x73, 0xab, 0x99, 0x88, 0xe9, - 0x90, 0x1a, 0x5e, 0x49, 0x54, 0x50, 0x52, 0x3b, 0xa6, 0xa4, 0x54, 0xa5, 0x5d, 0x7d, 0xb3, 0x34, - 0x6d, 0xc6, 0x9c, 0x91, 0x1d, 0x03, 0xea, 0x1e, 0x8b, 0x03, 0x06, 0xa6, 0x4e, 0x41, 0xcd, 0xe9, - 0x24, 0x41, 0x9b, 0x0e, 0x93, 0x38, 0x6a, 0xd2, 0xa1, 0xb7, 0x12, 0x80, 0x35, 0x8e, 0xfd, 0x66, - 0x05, 0x32, 0x49, 0x87, 0x68, 0xdb, 0xbc, 0x8b, 0xd3, 0x2a, 0xf6, 0x2e, 0x4e, 0xd5, 0x99, 0xbc, - 0xfb, 0x38, 0x51, 0x0b, 0x2a, 0xe1, 0x86, 0x13, 0x4b, 0xb3, 0xfa, 0x65, 0xb5, 0x8f, 0xa3, 0x8d, - 0xf7, 0x76, 0x27, 0x7e, 0xbc, 0x3f, 0xaf, 0x2b, 0x9d, 0xab, 0x53, 0xbc, 0x7c, 0x89, 0x66, 0xcd, - 0x68, 0x60, 0x4e, 0xff, 0x20, 0xb7, 0xcd, 0x7d, 0x42, 0x14, 0x96, 0xc7, 0x24, 0xee, 0x78, 0x89, - 0x98, 0x0d, 0x2f, 0x17, 0xb8, 0xca, 0x38, 0x61, 0x9d, 0x2e, 0xcf, 0xff, 0x63, 0x83, 0x29, 0xfa, - 0x20, 0xd4, 0xe2, 0xc4, 0x89, 0x92, 0x43, 0x26, 0xb8, 0xaa, 0x41, 0x5f, 0x91, 0x44, 0xb0, 0xa6, - 0x87, 0x5e, 0x61, 0xd5, 0x62, 0xdd, 0x78, 0xe3, 0x90, 0x39, 0x0f, 0xb2, 0xb2, 0xac, 0xa0, 0x80, - 0x0d, 0x6a, 0xe8, 0x22, 0x00, 0x9b, 0xdb, 0x3c, 0xd0, 0xaf, 0xca, 0xbc, 0x4c, 0x4a, 0x14, 0x62, - 0x05, 0xc1, 0x06, 0x96, 0xfd, 0x83, 0x90, 0xae, 0xf7, 0x80, 0x26, 0x64, 0x79, 0x09, 0xee, 0x85, - 0x66, 0xb9, 0x0b, 0xa9, 0x4a, 0x10, 0xbf, 0x6e, 0x81, 0x59, 0x94, 0x02, 0xbd, 0xce, 0xab, 0x5f, - 0x58, 0x45, 0x9c, 0x1c, 0x1a, 0x74, 0x27, 0x17, 0x9d, 0x30, 0x73, 0x84, 0x2d, 0x4b, 0x60, 0x9c, - 0x7f, 0x0f, 0x54, 0x25, 0xf4, 0x40, 0x46, 0xdd, 0xc7, 0xe0, 0x74, 0xf6, 0xa6, 0x72, 0x71, 0xea, - 0xb4, 0xbf, 0xeb, 0x47, 0xfa, 0x73, 0x4a, 0xbd, 0xfc, 0x39, 0x7d, 0xdc, 0xc8, 0xfa, 0x1b, 0x16, - 0x5c, 0xd8, 0xef, 0x42, 0x75, 0xf4, 0x18, 0x0c, 0xdc, 0x71, 0x22, 0x59, 0xc6, 0x9b, 0x09, 0xca, - 0x5b, 0x4e, 0xe4, 0x63, 0xd6, 0x8a, 0x76, 0x60, 0x90, 0x47, 0x83, 0x09, 0x6b, 0xfd, 0xe5, 0x62, - 0xaf, 0x77, 0xbf, 0x46, 0x8c, 0xed, 0x02, 0x8f, 0x44, 0xc3, 0x82, 0xa1, 0xfd, 0x6d, 0x0b, 0xd0, - 0xd2, 0x16, 0x89, 0x22, 0xb7, 0x69, 0xc4, 0xaf, 0xb1, 0x0b, 0x5a, 0x8c, 0x8b, 0x58, 0xcc, 0x14, - 0xd7, 0xcc, 0x05, 0x2d, 0xc6, 0xbf, 0xfc, 0x0b, 0x5a, 0x4a, 0x07, 0xbb, 0xa0, 0x05, 0x2d, 0xc1, - 0xd9, 0x36, 0xdf, 0x6e, 0xf0, 0x4b, 0x0f, 0xf8, 0xde, 0x43, 0x25, 0x94, 0x9d, 0xbb, 0xbb, 0x3b, - 0x71, 0x76, 0x31, 0x0f, 0x01, 0xe7, 0x3f, 0x67, 0xbf, 0x07, 0x10, 0x0f, 0x5b, 0x9b, 0xc9, 0x8b, - 0x41, 0xea, 0xe9, 0x7e, 0xb1, 0xbf, 0x5c, 0x81, 0x13, 0x99, 0x22, 0xaf, 0x74, 0xab, 0xd7, 0x1d, - 0xf4, 0x74, 0x64, 0xfd, 0xdd, 0xdd, 0xbd, 0xbe, 0xc2, 0xa8, 0x7c, 0xa8, 0xb8, 0x7e, 0xd8, 0x49, - 0x8a, 0xc9, 0x21, 0xe5, 0x9d, 0x98, 0xa7, 0x04, 0x0d, 0x77, 0x31, 0xfd, 0x8b, 0x39, 0x9b, 0x22, - 0x83, 0xb2, 0x52, 0xc6, 0xf8, 0xc0, 0x03, 0x72, 0x07, 0x7c, 0x42, 0x87, 0x48, 0x55, 0x8a, 0x70, - 0x2c, 0x66, 0x26, 0xcb, 0x71, 0x1f, 0xb5, 0xff, 0x5a, 0x09, 0x86, 0x8d, 0x8f, 0x86, 0x7e, 0x29, - 0x5d, 0xb2, 0xc9, 0x2a, 0xee, 0x95, 0x18, 0xfd, 0x49, 0x5d, 0x94, 0x89, 0xbf, 0xd2, 0x53, 0xdd, - 0xd5, 0x9a, 0xee, 0xed, 0x4e, 0x9c, 0xcc, 0xd4, 0x63, 0x4a, 0x55, 0x70, 0x3a, 0xff, 0x51, 0x38, - 0x91, 0x21, 0x93, 0xf3, 0xca, 0xab, 0xe9, 0x8b, 0xe8, 0x8f, 0xe8, 0x96, 0x32, 0x87, 0xec, 0xeb, - 0x74, 0xc8, 0x44, 0x1a, 0x5d, 0xe0, 0x91, 0x3e, 0x7c, 0xb0, 0x99, 0x6c, 0xd9, 0x52, 0x9f, 0xd9, - 0xb2, 0x4f, 0x43, 0x35, 0x0c, 0x3c, 0xb7, 0xe1, 0xaa, 0xba, 0x86, 0x2c, 0x3f, 0x77, 0x59, 0xb4, - 0x61, 0x05, 0x45, 0x77, 0xa0, 0xa6, 0xee, 0xec, 0x17, 0xfe, 0xed, 0xa2, 0x0e, 0x7d, 0x94, 0xd1, - 0xa2, 0xef, 0xe2, 0xd7, 0xbc, 0x90, 0x0d, 0x83, 0x4c, 0x09, 0xca, 0xd0, 0x7f, 0xe6, 0x7b, 0x67, - 0xda, 0x31, 0xc6, 0x02, 0x62, 0x7f, 0xad, 0x06, 0x67, 0xf2, 0x2a, 0x6d, 0xa3, 0x8f, 0xc0, 0x20, - 0xef, 0x63, 0x31, 0x97, 0x39, 0xe4, 0xf1, 0x98, 0x63, 0x04, 0x45, 0xb7, 0xd8, 0x6f, 0x2c, 0x78, - 0x0a, 0xee, 0x9e, 0xb3, 0x26, 0x66, 0xc8, 0xf1, 0x70, 0x5f, 0x70, 0x34, 0xf7, 0x05, 0x87, 0x73, - 0xf7, 0x9c, 0x35, 0xb4, 0x0d, 0x95, 0x96, 0x9b, 0x10, 0x47, 0x38, 0x11, 0x6e, 0x1d, 0x0b, 0x73, - 0xe2, 0x70, 0x2b, 0x8d, 0xfd, 0xc4, 0x9c, 0x21, 0xfa, 0xaa, 0x05, 0x27, 0xd6, 0xd2, 0xa9, 0xf1, - 0x42, 0x78, 0x3a, 0xc7, 0x50, 0x4d, 0x3d, 0xcd, 0x88, 0xdf, 0x50, 0x94, 0x69, 0xc4, 0xd9, 0xee, - 0xa0, 0x4f, 0x5a, 0x30, 0xb4, 0xee, 0x7a, 0x46, 0x61, 0xdd, 0x63, 0xf8, 0x38, 0x97, 0x19, 0x03, - 0xbd, 0xe3, 0xe0, 0xff, 0x63, 0x2c, 0x39, 0xf7, 0xd2, 0x54, 0x83, 0x47, 0xd5, 0x54, 0x43, 0x0f, - 0x48, 0x53, 0x7d, 0xda, 0x82, 0x9a, 0x1a, 0x69, 0x91, 0xee, 0xfc, 0xc1, 0x63, 0xfc, 0xe4, 0xdc, - 0x73, 0xa2, 0xfe, 0x62, 0xcd, 0x1c, 0x7d, 0xc1, 0x82, 0x61, 0xe7, 0x8d, 0x4e, 0x44, 0x9a, 0x64, - 0x2b, 0x08, 0x63, 0x71, 0xbd, 0xe1, 0xab, 0xc5, 0x77, 0x66, 0x9a, 0x32, 0x99, 0x25, 0x5b, 0x4b, - 0x61, 0x2c, 0xd2, 0x92, 0x74, 0x03, 0x36, 0xbb, 0x60, 0xef, 0x96, 0x60, 0x62, 0x1f, 0x0a, 0xe8, - 0x45, 0x18, 0x09, 0xa2, 0x96, 0xe3, 0xbb, 0x6f, 0x98, 0xb5, 0x2e, 0x94, 0x95, 0xb5, 0x64, 0xc0, - 0x70, 0x0a, 0xd3, 0x4c, 0xc8, 0x2e, 0xed, 0x93, 0x90, 0x7d, 0x01, 0x06, 0x22, 0x12, 0x06, 0xd9, - 0xcd, 0x02, 0x4b, 0x09, 0x60, 0x10, 0xf4, 0x38, 0x94, 0x9d, 0xd0, 0x15, 0x81, 0x68, 0x6a, 0x0f, - 0x34, 0xbd, 0x3c, 0x8f, 0x69, 0x7b, 0xaa, 0x3e, 0x44, 0xe5, 0xbe, 0xd4, 0x87, 0xa0, 0x6a, 0x40, - 0x9c, 0x5d, 0x0c, 0x6a, 0x35, 0x90, 0x3e, 0x53, 0xb0, 0xdf, 0x2a, 0xc3, 0xe3, 0x7b, 0xce, 0x17, - 0x1d, 0x87, 0x67, 0xed, 0x11, 0x87, 0x27, 0x87, 0xa7, 0xb4, 0xdf, 0xf0, 0x94, 0x7b, 0x0c, 0xcf, - 0x27, 0xe9, 0x32, 0x90, 0x35, 0x42, 0x8a, 0xb9, 0xa0, 0xae, 0x57, 0xc9, 0x11, 0xb1, 0x02, 0x24, - 0x14, 0x6b, 0xbe, 0x74, 0x0f, 0x90, 0x4a, 0x46, 0xae, 0x14, 0xa1, 0x06, 0x7a, 0xd6, 0x0c, 0xe1, - 0x73, 0xbf, 0x57, 0x86, 0xb3, 0xfd, 0xf3, 0x25, 0x78, 0xb2, 0x0f, 0xe9, 0x6d, 0xce, 0x62, 0xab, - 0xcf, 0x59, 0xfc, 0xdd, 0xfd, 0x99, 0xec, 0xbf, 0x66, 0xc1, 0xf9, 0xde, 0xca, 0x03, 0x3d, 0x07, - 0xc3, 0x6b, 0x91, 0xe3, 0x37, 0x36, 0xd8, 0xa5, 0x9b, 0x72, 0x50, 0xd8, 0x58, 0xeb, 0x66, 0x6c, - 0xe2, 0xd0, 0xed, 0x2d, 0x8f, 0x49, 0x30, 0x30, 0x64, 0xf2, 0x28, 0xdd, 0xde, 0xae, 0x66, 0x81, - 0xb8, 0x1b, 0xdf, 0xfe, 0xb3, 0x52, 0x7e, 0xb7, 0xb8, 0x91, 0x71, 0x90, 0xef, 0x24, 0xbe, 0x42, - 0xa9, 0x0f, 0x59, 0x52, 0xbe, 0xdf, 0xb2, 0x64, 0xa0, 0x97, 0x2c, 0x41, 0xb3, 0x70, 0xd2, 0xb8, - 0x94, 0x85, 0x27, 0x04, 0xf3, 0x80, 0x5b, 0x55, 0x25, 0x63, 0x39, 0x03, 0xc7, 0x5d, 0x4f, 0xa0, - 0x67, 0xa0, 0xea, 0xfa, 0x31, 0x69, 0x74, 0x22, 0x1e, 0xe8, 0x6d, 0x24, 0x61, 0xcd, 0x8b, 0x76, - 0xac, 0x30, 0xec, 0x5f, 0x2e, 0xc1, 0xb9, 0x9e, 0x76, 0xd6, 0x7d, 0x92, 0x5d, 0xe6, 0xe7, 0x18, - 0xb8, 0x3f, 0x9f, 0xc3, 0x1c, 0xa4, 0xca, 0xbe, 0x83, 0xf4, 0x87, 0xbd, 0x27, 0x26, 0xb5, 0xb9, - 0xbf, 0x67, 0x47, 0xe9, 0x25, 0x18, 0x75, 0xc2, 0x90, 0xe3, 0xb1, 0x78, 0xcd, 0x4c, 0x95, 0x9c, - 0x69, 0x13, 0x88, 0xd3, 0xb8, 0x7d, 0x69, 0xcf, 0x3f, 0xb6, 0xa0, 0x86, 0xc9, 0x3a, 0x97, 0x0e, - 0xe8, 0xb6, 0x18, 0x22, 0xab, 0x88, 0x7a, 0x9a, 0x74, 0x60, 0x63, 0x97, 0xd5, 0x99, 0xcc, 0x1b, - 0xec, 0xee, 0xcb, 0x7b, 0x4a, 0x07, 0xba, 0xbc, 0x47, 0x5d, 0xdf, 0x52, 0xee, 0x7d, 0x7d, 0x8b, - 0xfd, 0xf5, 0x21, 0xfa, 0x7a, 0x61, 0x30, 0x13, 0x91, 0x66, 0x4c, 0xbf, 0x6f, 0x27, 0xf2, 0xc4, - 0x24, 0x51, 0xdf, 0xf7, 0x06, 0x5e, 0xc0, 0xb4, 0x3d, 0x75, 0x14, 0x53, 0x3a, 0x50, 0x8d, 0x90, - 0xf2, 0xbe, 0x35, 0x42, 0x5e, 0x82, 0xd1, 0x38, 0xde, 0x58, 0x8e, 0xdc, 0x2d, 0x27, 0x21, 0xd7, - 0xc8, 0x8e, 0xb0, 0xb2, 0x74, 0x5e, 0xff, 0xca, 0x15, 0x0d, 0xc4, 0x69, 0x5c, 0x34, 0x07, 0xa7, - 0x74, 0xa5, 0x0e, 0x12, 0x25, 0x2c, 0xba, 0x9f, 0xcf, 0x04, 0x95, 0xc4, 0xab, 0x6b, 0x7b, 0x08, - 0x04, 0xdc, 0xfd, 0x0c, 0x95, 0x6f, 0xa9, 0x46, 0xda, 0x91, 0xc1, 0xb4, 0x7c, 0x4b, 0xd1, 0xa1, - 0x7d, 0xe9, 0x7a, 0x02, 0x2d, 0xc2, 0x69, 0x3e, 0x31, 0xa6, 0xc3, 0xd0, 0x78, 0xa3, 0xa1, 0x74, - 0x1d, 0xc3, 0xb9, 0x6e, 0x14, 0x9c, 0xf7, 0x1c, 0x7a, 0x01, 0x86, 0x55, 0xf3, 0xfc, 0xac, 0x38, - 0x45, 0x50, 0x5e, 0x0c, 0x45, 0x66, 0xbe, 0x89, 0x4d, 0x3c, 0xf4, 0x01, 0x78, 0x54, 0xff, 0xe5, - 0x29, 0x60, 0xfc, 0x68, 0x6d, 0x56, 0x14, 0x41, 0x52, 0x97, 0x85, 0xcc, 0xe5, 0xa2, 0x35, 0x71, - 0xaf, 0xe7, 0xd1, 0x1a, 0x9c, 0x57, 0xa0, 0x4b, 0x7e, 0xc2, 0xf2, 0x39, 0x62, 0x52, 0x77, 0x62, - 0x72, 0x23, 0xf2, 0xc4, 0x6d, 0xab, 0xea, 0x1e, 0xc7, 0x39, 0x37, 0xb9, 0x92, 0x87, 0x89, 0x17, - 0xf0, 0x1e, 0x54, 0xd0, 0x14, 0xd4, 0x88, 0xef, 0xac, 0x79, 0x64, 0x69, 0x66, 0x9e, 0x15, 0x53, - 0x32, 0x4e, 0xf2, 0x2e, 0x49, 0x00, 0xd6, 0x38, 0x2a, 0xc2, 0x74, 0xa4, 0xe7, 0x9d, 0xa2, 0xcb, - 0x70, 0xa6, 0xd5, 0x08, 0xa9, 0xed, 0xe1, 0x36, 0xc8, 0x74, 0x83, 0x05, 0xd4, 0xd1, 0x0f, 0xc3, - 0x0b, 0x4c, 0xaa, 0xf0, 0xe9, 0xb9, 0x99, 0xe5, 0x2e, 0x1c, 0x9c, 0xfb, 0x24, 0x0b, 0xbc, 0x8c, - 0x82, 0xed, 0x9d, 0xf1, 0xd3, 0x99, 0xc0, 0x4b, 0xda, 0x88, 0x39, 0x0c, 0x5d, 0x05, 0xc4, 0x62, - 0xf1, 0xaf, 0x24, 0x49, 0xa8, 0x8c, 0x9d, 0xf1, 0x33, 0xec, 0x95, 0x54, 0x18, 0xd9, 0xe5, 0x2e, - 0x0c, 0x9c, 0xf3, 0x94, 0xfd, 0x1f, 0x2c, 0x18, 0x55, 0xeb, 0xf5, 0x3e, 0x64, 0xa3, 0x78, 0xe9, - 0x6c, 0x94, 0xb9, 0xa3, 0x4b, 0x3c, 0xd6, 0xf3, 0x1e, 0x21, 0xcd, 0x3f, 0x33, 0x0c, 0xa0, 0xa5, - 0xa2, 0x52, 0x48, 0x56, 0x4f, 0x85, 0xf4, 0xd0, 0x4a, 0xa4, 0xbc, 0xca, 0x29, 0x95, 0x07, 0x5b, - 0x39, 0x65, 0x05, 0xce, 0x4a, 0x73, 0x81, 0x9f, 0x15, 0x5d, 0x09, 0x62, 0x25, 0xe0, 0xaa, 0xf5, - 0xc7, 0x05, 0xa1, 0xb3, 0xf3, 0x79, 0x48, 0x38, 0xff, 0xd9, 0x94, 0x95, 0x32, 0xb4, 0x9f, 0x95, - 0xa2, 0xd7, 0xf4, 0xc2, 0xba, 0xbc, 0x15, 0x24, 0xb3, 0xa6, 0x17, 0x2e, 0xaf, 0x60, 0x8d, 0x93, - 0x2f, 0xd8, 0x6b, 0x05, 0x09, 0x76, 0x38, 0xb0, 0x60, 0x97, 0x22, 0x66, 0xb8, 0xa7, 0x88, 0x91, - 0x3e, 0xe9, 0x91, 0x9e, 0x3e, 0xe9, 0xf7, 0xc2, 0x98, 0xeb, 0x6f, 0x90, 0xc8, 0x4d, 0x48, 0x93, - 0xad, 0x05, 0x26, 0x7e, 0xaa, 0x5a, 0xad, 0xcf, 0xa7, 0xa0, 0x38, 0x83, 0x9d, 0x96, 0x8b, 0x63, - 0x7d, 0xc8, 0xc5, 0x1e, 0xda, 0xe8, 0x44, 0x31, 0xda, 0xe8, 0xe4, 0xd1, 0xb5, 0xd1, 0xa9, 0x63, - 0xd5, 0x46, 0xa8, 0x10, 0x6d, 0xd4, 0x97, 0xa0, 0x37, 0xb6, 0x7f, 0x67, 0xf6, 0xd9, 0xfe, 0xf5, - 0x52, 0x45, 0x67, 0x0f, 0xad, 0x8a, 0xf2, 0xb5, 0xcc, 0x23, 0x87, 0xd2, 0x32, 0x9f, 0x2e, 0xc1, - 0x59, 0x2d, 0x87, 0xe9, 0xec, 0x77, 0xd7, 0xa9, 0x24, 0x62, 0x17, 0x4b, 0xf1, 0x73, 0x1b, 0x23, - 0x39, 0x4a, 0xe7, 0x59, 0x29, 0x08, 0x36, 0xb0, 0x58, 0x8e, 0x11, 0x89, 0x58, 0x19, 0xdd, 0xac, - 0x90, 0x9e, 0x11, 0xed, 0x58, 0x61, 0xd0, 0xf9, 0x45, 0x7f, 0x8b, 0xbc, 0xcd, 0x6c, 0xb1, 0xb8, - 0x19, 0x0d, 0xc2, 0x26, 0x1e, 0x7a, 0x9a, 0x33, 0x61, 0x02, 0x82, 0x0a, 0xea, 0x11, 0x71, 0xd3, - 0xac, 0x94, 0x09, 0x0a, 0x2a, 0xbb, 0xc3, 0x92, 0xc9, 0x2a, 0xdd, 0xdd, 0x61, 0x21, 0x50, 0x0a, - 0xc3, 0xfe, 0x9f, 0x16, 0x9c, 0xcb, 0x1d, 0x8a, 0xfb, 0xa0, 0x7c, 0xb7, 0xd3, 0xca, 0x77, 0xa5, - 0xa8, 0xed, 0x86, 0xf1, 0x16, 0x3d, 0x14, 0xf1, 0xbf, 0xb3, 0x60, 0x4c, 0xe3, 0xdf, 0x87, 0x57, - 0x75, 0xd3, 0xaf, 0x5a, 0xdc, 0xce, 0xaa, 0xd6, 0xf5, 0x6e, 0xbf, 0x53, 0x02, 0x55, 0xc0, 0x71, - 0xba, 0x21, 0xcb, 0xe3, 0xee, 0x73, 0x92, 0xb8, 0x03, 0x83, 0xec, 0x20, 0x34, 0x2e, 0x26, 0xc8, - 0x23, 0xcd, 0x9f, 0x1d, 0xaa, 0xea, 0x43, 0x66, 0xf6, 0x37, 0xc6, 0x82, 0x21, 0x2b, 0xf2, 0xec, - 0xc6, 0x54, 0x9a, 0x37, 0x45, 0x5a, 0x96, 0x2e, 0xf2, 0x2c, 0xda, 0xb1, 0xc2, 0xa0, 0xea, 0xc1, - 0x6d, 0x04, 0xfe, 0x8c, 0xe7, 0xc4, 0xf2, 0x36, 0x45, 0xa5, 0x1e, 0xe6, 0x25, 0x00, 0x6b, 0x1c, - 0x76, 0x46, 0xea, 0xc6, 0xa1, 0xe7, 0xec, 0x18, 0xfb, 0x67, 0xa3, 0x3e, 0x81, 0x02, 0x61, 0x13, - 0xcf, 0x6e, 0xc3, 0x78, 0xfa, 0x25, 0x66, 0xc9, 0x3a, 0x0b, 0x50, 0xec, 0x6b, 0x38, 0xa7, 0xa0, - 0xe6, 0xb0, 0xa7, 0x16, 0x3a, 0x4e, 0xf6, 0x12, 0xf4, 0x69, 0x09, 0xc0, 0x1a, 0xc7, 0xfe, 0x55, - 0x0b, 0x4e, 0xe7, 0x0c, 0x5a, 0x81, 0x69, 0x6f, 0x89, 0x96, 0x36, 0x79, 0x8a, 0xfd, 0x07, 0x60, - 0xa8, 0x49, 0xd6, 0x1d, 0x19, 0x02, 0x67, 0xc8, 0xf6, 0x59, 0xde, 0x8c, 0x25, 0xdc, 0xfe, 0xef, - 0x16, 0x9c, 0x48, 0xf7, 0x35, 0x66, 0xa9, 0x24, 0x7c, 0x98, 0xdc, 0xb8, 0x11, 0x6c, 0x91, 0x68, - 0x87, 0xbe, 0xb9, 0x95, 0x49, 0x25, 0xe9, 0xc2, 0xc0, 0x39, 0x4f, 0xb1, 0xf2, 0xad, 0x4d, 0x35, - 0xda, 0x72, 0x46, 0xde, 0x2c, 0x72, 0x46, 0xea, 0x8f, 0x69, 0x1e, 0x97, 0x2b, 0x96, 0xd8, 0xe4, - 0x6f, 0x7f, 0x7b, 0x00, 0x54, 0x5e, 0x2c, 0x8b, 0x3f, 0x2a, 0x28, 0x7a, 0xeb, 0xa0, 0x19, 0x44, - 0x6a, 0x32, 0x0c, 0xec, 0x15, 0x10, 0xc0, 0xbd, 0x24, 0xa6, 0xeb, 0x52, 0xbd, 0xe1, 0xaa, 0x06, - 0x61, 0x13, 0x8f, 0xf6, 0xc4, 0x73, 0xb7, 0x08, 0x7f, 0x68, 0x30, 0xdd, 0x93, 0x05, 0x09, 0xc0, - 0x1a, 0x87, 0xf6, 0xa4, 0xe9, 0xae, 0xaf, 0x8b, 0x2d, 0xbf, 0xea, 0x09, 0x1d, 0x1d, 0xcc, 0x20, - 0xbc, 0x22, 0x77, 0xb0, 0x29, 0xac, 0x60, 0xa3, 0x22, 0x77, 0xb0, 0x89, 0x19, 0x84, 0xda, 0x6d, - 0x7e, 0x10, 0xb5, 0xd9, 0x25, 0xf5, 0x4d, 0xc5, 0x45, 0x58, 0xbf, 0xca, 0x6e, 0xbb, 0xde, 0x8d, - 0x82, 0xf3, 0x9e, 0xa3, 0x33, 0x30, 0x8c, 0x48, 0xd3, 0x6d, 0x24, 0x26, 0x35, 0x48, 0xcf, 0xc0, - 0xe5, 0x2e, 0x0c, 0x9c, 0xf3, 0x14, 0x9a, 0x86, 0x13, 0x32, 0xaf, 0x59, 0x56, 0xad, 0x19, 0x4e, - 0x57, 0xc9, 0xc0, 0x69, 0x30, 0xce, 0xe2, 0x53, 0xa9, 0xd6, 0x16, 0x05, 0xab, 0x98, 0xb1, 0x6c, - 0x48, 0x35, 0x59, 0xc8, 0x0a, 0x2b, 0x0c, 0xfb, 0x13, 0x65, 0xaa, 0x85, 0x7b, 0x14, 0x6a, 0xbb, - 0x6f, 0xd1, 0x82, 0xe9, 0x19, 0x39, 0xd0, 0xc7, 0x8c, 0x7c, 0x1e, 0x46, 0x6e, 0xc7, 0x81, 0xaf, - 0x22, 0xf1, 0x2a, 0x3d, 0x23, 0xf1, 0x0c, 0xac, 0xfc, 0x48, 0xbc, 0xc1, 0xa2, 0x22, 0xf1, 0x86, - 0x0e, 0x19, 0x89, 0xf7, 0xcd, 0x0a, 0xa8, 0xab, 0x41, 0xae, 0x93, 0xe4, 0x4e, 0x10, 0x6d, 0xba, - 0x7e, 0x8b, 0xe5, 0x83, 0x7f, 0xd5, 0x82, 0x11, 0xbe, 0x5e, 0x16, 0xcc, 0x4c, 0xaa, 0xf5, 0x82, - 0xee, 0x9c, 0x48, 0x31, 0x9b, 0x5c, 0x35, 0x18, 0x65, 0x2e, 0xf3, 0x34, 0x41, 0x38, 0xd5, 0x23, - 0xf4, 0x51, 0x00, 0xe9, 0x1f, 0x5d, 0x97, 0x22, 0x73, 0xbe, 0x98, 0xfe, 0x61, 0xb2, 0xae, 0x6d, - 0xe0, 0x55, 0xc5, 0x04, 0x1b, 0x0c, 0xd1, 0xa7, 0x75, 0x96, 0x19, 0x0f, 0xd9, 0xff, 0xf0, 0xb1, - 0x8c, 0x4d, 0x3f, 0x39, 0x66, 0x18, 0x86, 0x5c, 0xbf, 0x45, 0xe7, 0x89, 0x88, 0x58, 0x7a, 0x57, - 0x5e, 0x2d, 0x85, 0x85, 0xc0, 0x69, 0xd6, 0x1d, 0xcf, 0xf1, 0x1b, 0x24, 0x9a, 0xe7, 0xe8, 0xe6, - 0x15, 0xd6, 0xac, 0x01, 0x4b, 0x42, 0x5d, 0x97, 0xaa, 0x54, 0xfa, 0xb9, 0x54, 0xe5, 0xfc, 0xfb, - 0xe0, 0x54, 0xd7, 0xc7, 0x3c, 0x50, 0x4a, 0xd9, 0xe1, 0xb3, 0xd1, 0xec, 0x7f, 0x31, 0xa8, 0x95, - 0xd6, 0xf5, 0xa0, 0xc9, 0xaf, 0xf6, 0x88, 0xf4, 0x17, 0x15, 0x36, 0x6e, 0x81, 0x53, 0xc4, 0xb8, - 0x06, 0x5b, 0x35, 0x62, 0x93, 0x25, 0x9d, 0xa3, 0xa1, 0x13, 0x11, 0xff, 0xb8, 0xe7, 0xe8, 0xb2, - 0x62, 0x82, 0x0d, 0x86, 0x68, 0x23, 0x95, 0x53, 0x72, 0xf9, 0xe8, 0x39, 0x25, 0xac, 0xca, 0x54, - 0x5e, 0x35, 0xfe, 0x2f, 0x58, 0x30, 0xe6, 0xa7, 0x66, 0x6e, 0x31, 0x61, 0xa4, 0xf9, 0xab, 0x82, - 0xdf, 0x2c, 0x95, 0x6e, 0xc3, 0x19, 0xfe, 0x79, 0x2a, 0xad, 0x72, 0x40, 0x95, 0xa6, 0xef, 0x08, - 0x1a, 0xec, 0x75, 0x47, 0x10, 0xf2, 0xd5, 0x25, 0x69, 0x43, 0x85, 0x5f, 0x92, 0x06, 0x39, 0x17, - 0xa4, 0xdd, 0x82, 0x5a, 0x23, 0x22, 0x4e, 0x72, 0xc8, 0xfb, 0xb2, 0xd8, 0x01, 0xfd, 0x8c, 0x24, - 0x80, 0x35, 0x2d, 0xfb, 0xff, 0x0c, 0xc0, 0x49, 0x39, 0x22, 0x32, 0x04, 0x9d, 0xea, 0x47, 0xce, - 0x57, 0x1b, 0xb7, 0x4a, 0x3f, 0x5e, 0x91, 0x00, 0xac, 0x71, 0xa8, 0x3d, 0xd6, 0x89, 0xc9, 0x52, - 0x48, 0xfc, 0x05, 0x77, 0x2d, 0x16, 0xe7, 0x9c, 0x6a, 0xa1, 0xdc, 0xd0, 0x20, 0x6c, 0xe2, 0x51, - 0x63, 0x9c, 0xdb, 0xc5, 0x71, 0x36, 0x7d, 0x45, 0xd8, 0xdb, 0x58, 0xc2, 0xd1, 0x2f, 0xe4, 0x56, - 0x8e, 0x2d, 0x26, 0x71, 0xab, 0x2b, 0xf2, 0xfe, 0x80, 0x57, 0x2c, 0xfe, 0x6d, 0x0b, 0xce, 0xf2, - 0x56, 0x39, 0x92, 0x37, 0xc2, 0xa6, 0x93, 0x90, 0xb8, 0x98, 0x4a, 0xee, 0x39, 0xfd, 0xd3, 0x4e, - 0xde, 0x3c, 0xb6, 0x38, 0xbf, 0x37, 0xe8, 0x4d, 0x0b, 0x4e, 0x6c, 0xa6, 0x6a, 0x7e, 0x48, 0xd5, - 0x71, 0xd4, 0x74, 0xfc, 0x14, 0x51, 0xbd, 0xd4, 0xd2, 0xed, 0x31, 0xce, 0x72, 0xb7, 0xff, 0xcc, - 0x02, 0x53, 0x8c, 0xde, 0xff, 0x52, 0x21, 0x07, 0x37, 0x05, 0xa5, 0x75, 0x59, 0xe9, 0x69, 0x5d, - 0x3e, 0x0e, 0xe5, 0x8e, 0xdb, 0x14, 0xfb, 0x0b, 0x7d, 0xfa, 0x3a, 0x3f, 0x8b, 0x69, 0xbb, 0xfd, - 0x4f, 0x2b, 0xda, 0x6f, 0x21, 0xf2, 0xa2, 0xbe, 0x27, 0x5e, 0x7b, 0x5d, 0x15, 0x1b, 0xe3, 0x6f, - 0x7e, 0xbd, 0xab, 0xd8, 0xd8, 0x8f, 0x1e, 0x3c, 0xed, 0x8d, 0x0f, 0x50, 0xaf, 0x5a, 0x63, 0x43, - 0xfb, 0xe4, 0xbc, 0xdd, 0x86, 0x2a, 0xdd, 0x82, 0x31, 0x07, 0x64, 0x35, 0xd5, 0xa9, 0xea, 0x15, - 0xd1, 0x7e, 0x6f, 0x77, 0xe2, 0x47, 0x0e, 0xde, 0x2d, 0xf9, 0x34, 0x56, 0xf4, 0x51, 0x0c, 0x35, - 0xfa, 0x9b, 0xa5, 0xe7, 0x89, 0xcd, 0xdd, 0x0d, 0x25, 0x33, 0x25, 0xa0, 0x90, 0xdc, 0x3f, 0xcd, - 0x07, 0xf9, 0x50, 0x63, 0xb7, 0xd1, 0x32, 0xa6, 0x7c, 0x0f, 0xb8, 0xac, 0x92, 0xe4, 0x24, 0xe0, - 0xde, 0xee, 0xc4, 0x4b, 0x07, 0x67, 0xaa, 0x1e, 0xc7, 0x9a, 0x85, 0xfd, 0xc5, 0x01, 0x3d, 0x77, - 0x45, 0x8d, 0xb9, 0xef, 0x89, 0xb9, 0xfb, 0x62, 0x66, 0xee, 0x5e, 0xe8, 0x9a, 0xbb, 0x63, 0xfa, - 0xd6, 0xd4, 0xd4, 0x6c, 0xbc, 0xdf, 0x86, 0xc0, 0xfe, 0xfe, 0x06, 0x66, 0x01, 0xbd, 0xde, 0x71, - 0x23, 0x12, 0x2f, 0x47, 0x1d, 0xdf, 0xf5, 0x5b, 0x6c, 0x3a, 0x56, 0x4d, 0x0b, 0x28, 0x05, 0xc6, - 0x59, 0x7c, 0xba, 0xa9, 0xa7, 0xdf, 0xfc, 0x96, 0xb3, 0xc5, 0x67, 0x95, 0x51, 0x76, 0x6b, 0x45, - 0xb4, 0x63, 0x85, 0x61, 0x7f, 0x9d, 0x9d, 0x65, 0x1b, 0x79, 0xc1, 0x74, 0x4e, 0x78, 0xec, 0xfa, - 0x5f, 0x5e, 0xb3, 0x4b, 0xcd, 0x09, 0x7e, 0xe7, 0x2f, 0x87, 0xa1, 0x3b, 0x30, 0xb4, 0xc6, 0xef, - 0xbf, 0x2b, 0xa6, 0x3e, 0xb9, 0xb8, 0x4c, 0x8f, 0xdd, 0x72, 0x22, 0x6f, 0xd6, 0xbb, 0xa7, 0x7f, - 0x62, 0xc9, 0xcd, 0xfe, 0xfd, 0x0a, 0x9c, 0xc8, 0x5c, 0x10, 0x9b, 0xaa, 0x96, 0x5a, 0xda, 0xb7, - 0x5a, 0xea, 0x87, 0x00, 0x9a, 0x24, 0xf4, 0x82, 0x1d, 0x66, 0x8e, 0x0d, 0x1c, 0xd8, 0x1c, 0x53, - 0x16, 0xfc, 0xac, 0xa2, 0x82, 0x0d, 0x8a, 0xa2, 0x50, 0x19, 0x2f, 0xbe, 0x9a, 0x29, 0x54, 0x66, - 0xdc, 0x62, 0x30, 0x78, 0x7f, 0x6f, 0x31, 0x70, 0xe1, 0x04, 0xef, 0xa2, 0xca, 0xbe, 0x3d, 0x44, - 0x92, 0x2d, 0xcb, 0x5f, 0x98, 0x4d, 0x93, 0xc1, 0x59, 0xba, 0x0f, 0xf2, 0xfe, 0x67, 0xf4, 0x6e, - 0xa8, 0xc9, 0xef, 0x1c, 0x8f, 0xd7, 0x74, 0x05, 0x03, 0x39, 0x0d, 0xd8, 0xbd, 0xcc, 0xe2, 0x67, - 0x57, 0x21, 0x01, 0x78, 0x50, 0x85, 0x04, 0xec, 0xcf, 0x97, 0xa8, 0x1d, 0xcf, 0xfb, 0xa5, 0x6a, - 0xe2, 0x3c, 0x05, 0x83, 0x4e, 0x27, 0xd9, 0x08, 0xba, 0x6e, 0xf3, 0x9b, 0x66, 0xad, 0x58, 0x40, - 0xd1, 0x02, 0x0c, 0x34, 0x75, 0x9d, 0x93, 0x83, 0x7c, 0x4f, 0xed, 0x12, 0x75, 0x12, 0x82, 0x19, - 0x15, 0xf4, 0x18, 0x0c, 0x24, 0x4e, 0x4b, 0xa6, 0x5c, 0xb1, 0x34, 0xdb, 0x55, 0xa7, 0x15, 0x63, - 0xd6, 0x6a, 0xaa, 0xef, 0x81, 0x7d, 0xd4, 0xf7, 0x4b, 0x30, 0x1a, 0xbb, 0x2d, 0xdf, 0x49, 0x3a, - 0x11, 0x31, 0x8e, 0xf9, 0x74, 0xe4, 0x86, 0x09, 0xc4, 0x69, 0x5c, 0xfb, 0x37, 0x47, 0xe0, 0xcc, - 0xca, 0xcc, 0xa2, 0xac, 0xde, 0x7d, 0x6c, 0x59, 0x53, 0x79, 0x3c, 0xee, 0x5f, 0xd6, 0x54, 0x0f, - 0xee, 0x9e, 0x91, 0x35, 0xe5, 0x19, 0x59, 0x53, 0xe9, 0x14, 0x96, 0x72, 0x11, 0x29, 0x2c, 0x79, - 0x3d, 0xe8, 0x27, 0x85, 0xe5, 0xd8, 0xd2, 0xa8, 0xf6, 0xec, 0xd0, 0x81, 0xd2, 0xa8, 0x54, 0x8e, - 0x59, 0x21, 0xc9, 0x05, 0x3d, 0x3e, 0x55, 0x6e, 0x8e, 0x99, 0xca, 0xef, 0xe1, 0x89, 0x33, 0x42, - 0xd4, 0xbf, 0x5a, 0x7c, 0x07, 0xfa, 0xc8, 0xef, 0x11, 0xb9, 0x3b, 0x66, 0x4e, 0xd9, 0x50, 0x11, - 0x39, 0x65, 0x79, 0xdd, 0xd9, 0x37, 0xa7, 0xec, 0x25, 0x18, 0x6d, 0x78, 0x81, 0x4f, 0x96, 0xa3, - 0x20, 0x09, 0x1a, 0x81, 0x27, 0xcc, 0x7a, 0x25, 0x12, 0x66, 0x4c, 0x20, 0x4e, 0xe3, 0xf6, 0x4a, - 0x48, 0xab, 0x1d, 0x35, 0x21, 0x0d, 0x1e, 0x50, 0x42, 0xda, 0xcf, 0xea, 0xd4, 0xe9, 0x61, 0xf6, - 0x45, 0x3e, 0x54, 0xfc, 0x17, 0xe9, 0x27, 0x7f, 0x1a, 0xbd, 0xc5, 0xaf, 0xd3, 0xa3, 0x86, 0xf1, - 0x4c, 0xd0, 0xa6, 0x86, 0xdf, 0x08, 0x1b, 0x92, 0xd7, 0x8e, 0x61, 0xc2, 0xde, 0x5a, 0xd1, 0x6c, - 0xd4, 0x15, 0x7b, 0xba, 0x09, 0xa7, 0x3b, 0x72, 0x94, 0xd4, 0xee, 0x2f, 0x97, 0xe0, 0xfb, 0xf6, - 0xed, 0x02, 0xba, 0x03, 0x90, 0x38, 0x2d, 0x31, 0x51, 0xc5, 0x81, 0xc9, 0x11, 0xc3, 0x2b, 0x57, - 0x25, 0x3d, 0x5e, 0x93, 0x44, 0xfd, 0x65, 0x47, 0x11, 0xf2, 0x37, 0x8b, 0xaa, 0x0c, 0xbc, 0xae, - 0xd2, 0x8d, 0x38, 0xf0, 0x08, 0x66, 0x10, 0xaa, 0xfe, 0x23, 0xd2, 0xd2, 0xf7, 0x3f, 0xab, 0xcf, - 0x87, 0x59, 0x2b, 0x16, 0x50, 0xf4, 0x02, 0x0c, 0x3b, 0x9e, 0xc7, 0xf3, 0x63, 0x48, 0x2c, 0xee, - 0xd3, 0xd1, 0x35, 0xe4, 0x34, 0x08, 0x9b, 0x78, 0xf6, 0x9f, 0x96, 0x60, 0x62, 0x1f, 0x99, 0xd2, - 0x95, 0xf1, 0x57, 0xe9, 0x3b, 0xe3, 0x4f, 0xe4, 0x28, 0x0c, 0xf6, 0xc8, 0x51, 0x78, 0x01, 0x86, - 0x13, 0xe2, 0xb4, 0x45, 0x40, 0x96, 0xf0, 0x04, 0xe8, 0x13, 0x60, 0x0d, 0xc2, 0x26, 0x1e, 0x95, - 0x62, 0x63, 0x4e, 0xa3, 0x41, 0xe2, 0x58, 0x26, 0x21, 0x08, 0x6f, 0x6a, 0x61, 0x19, 0x0e, 0xcc, - 0x49, 0x3d, 0x9d, 0x62, 0x81, 0x33, 0x2c, 0xb3, 0x03, 0x5e, 0xeb, 0x73, 0xc0, 0xbf, 0x56, 0x82, - 0xc7, 0xf7, 0xd4, 0x6e, 0x7d, 0xe7, 0x87, 0x74, 0x62, 0x12, 0x65, 0x27, 0xce, 0x8d, 0x98, 0x44, - 0x98, 0x41, 0xf8, 0x28, 0x85, 0xa1, 0x71, 0xbf, 0x76, 0xd1, 0xc9, 0x4b, 0x7c, 0x94, 0x52, 0x2c, - 0x70, 0x86, 0xe5, 0x61, 0xa7, 0xe5, 0xdf, 0x2f, 0xc1, 0x93, 0x7d, 0xd8, 0x00, 0x05, 0x26, 0x79, - 0xa5, 0x53, 0xed, 0xca, 0x0f, 0x28, 0x23, 0xf2, 0x90, 0xc3, 0xf5, 0xf5, 0x12, 0x9c, 0xef, 0xad, - 0x8a, 0xd1, 0x8f, 0xc1, 0x89, 0x48, 0x45, 0x61, 0x99, 0x59, 0x7a, 0xa7, 0xb9, 0x27, 0x21, 0x05, - 0xc2, 0x59, 0x5c, 0x34, 0x09, 0x10, 0x3a, 0xc9, 0x46, 0x7c, 0x69, 0xdb, 0x8d, 0x13, 0x51, 0x85, - 0x66, 0x8c, 0x9f, 0x5d, 0xc9, 0x56, 0x6c, 0x60, 0x50, 0x76, 0xec, 0xdf, 0x6c, 0x70, 0x3d, 0x48, - 0xf8, 0x43, 0x7c, 0x1b, 0x71, 0x5a, 0xde, 0xd9, 0x61, 0x80, 0x70, 0x16, 0x97, 0xb2, 0x63, 0xa7, - 0xa3, 0xbc, 0xa3, 0x7c, 0x7f, 0xc1, 0xd8, 0x2d, 0xa8, 0x56, 0x6c, 0x60, 0x64, 0xf3, 0x0f, 0x2b, - 0xfb, 0xe7, 0x1f, 0xda, 0xff, 0xa4, 0x04, 0xe7, 0x7a, 0x9a, 0x72, 0xfd, 0x2d, 0xc0, 0x87, 0x2f, - 0x67, 0xf0, 0x70, 0x73, 0xe7, 0x80, 0xb9, 0x6d, 0x7f, 0xdc, 0x63, 0xa6, 0x89, 0xdc, 0xb6, 0xc3, - 0x27, 0x87, 0x3f, 0x7c, 0xe3, 0xd9, 0x95, 0xce, 0x36, 0x70, 0x80, 0x74, 0xb6, 0xcc, 0xc7, 0xa8, - 0xf4, 0xb9, 0x90, 0xff, 0xbc, 0xdc, 0x73, 0x78, 0xe9, 0xd6, 0xaf, 0x2f, 0x3f, 0xed, 0x2c, 0x9c, - 0x74, 0x7d, 0x76, 0x7f, 0xd3, 0x4a, 0x67, 0x4d, 0x14, 0x26, 0x29, 0xa5, 0x6f, 0x4f, 0x9f, 0xcf, - 0xc0, 0x71, 0xd7, 0x13, 0x0f, 0x61, 0x7a, 0xe1, 0xe1, 0x86, 0xf4, 0x60, 0x09, 0xae, 0x68, 0x09, - 0xce, 0xca, 0xa1, 0xd8, 0x70, 0x22, 0xd2, 0x14, 0x6a, 0x24, 0x16, 0x09, 0x15, 0xe7, 0x78, 0x52, - 0x46, 0x0e, 0x02, 0xce, 0x7f, 0x8e, 0x5d, 0x99, 0x13, 0x84, 0x6e, 0x43, 0x6c, 0x72, 0xf4, 0x95, - 0x39, 0xb4, 0x11, 0x73, 0x98, 0xfd, 0x21, 0xa8, 0xa9, 0xf7, 0xe7, 0x61, 0xdd, 0x6a, 0xd2, 0x75, - 0x85, 0x75, 0xab, 0x19, 0x67, 0x60, 0xd1, 0xaf, 0x45, 0x4d, 0xe2, 0xcc, 0xea, 0xb9, 0x46, 0x76, - 0x98, 0x7d, 0x6c, 0xff, 0x10, 0x8c, 0x28, 0x3f, 0x4b, 0xbf, 0x17, 0x09, 0xd9, 0x5f, 0x1c, 0x84, - 0xd1, 0x54, 0x71, 0xc0, 0x94, 0x83, 0xd5, 0xda, 0xd7, 0xc1, 0xca, 0xc2, 0xf4, 0x3b, 0xbe, 0xbc, - 0x65, 0xcc, 0x08, 0xd3, 0xef, 0xf8, 0x04, 0x73, 0x18, 0x35, 0x6f, 0x9b, 0xd1, 0x0e, 0xee, 0xf8, - 0x22, 0x9c, 0x56, 0x99, 0xb7, 0xb3, 0xac, 0x15, 0x0b, 0x28, 0xfa, 0xb8, 0x05, 0x23, 0x31, 0xf3, - 0xde, 0x73, 0xf7, 0xb4, 0x98, 0x74, 0x57, 0x8f, 0x5e, 0xfb, 0x50, 0x15, 0xc2, 0x64, 0x11, 0x32, - 0x66, 0x0b, 0x4e, 0x71, 0x44, 0x9f, 0xb2, 0xa0, 0xa6, 0x2e, 0x43, 0x11, 0x57, 0x01, 0xae, 0x14, - 0x5b, 0x7b, 0x91, 0xfb, 0x35, 0xd5, 0x41, 0x88, 0x2a, 0x82, 0x87, 0x35, 0x63, 0x14, 0x2b, 0xdf, - 0xf1, 0xd0, 0xf1, 0xf8, 0x8e, 0x21, 0xc7, 0x6f, 0xfc, 0x6e, 0xa8, 0xb5, 0x1d, 0xdf, 0x5d, 0x27, - 0x71, 0xc2, 0xdd, 0xb9, 0xb2, 0x24, 0xac, 0x6c, 0xc4, 0x1a, 0x4e, 0x15, 0x72, 0xcc, 0x5e, 0x2c, - 0x31, 0xfc, 0xaf, 0x4c, 0x21, 0xaf, 0xe8, 0x66, 0x6c, 0xe2, 0x98, 0xce, 0x62, 0x78, 0xa0, 0xce, - 0xe2, 0xe1, 0xbd, 0x9d, 0xc5, 0xf6, 0x3f, 0xb4, 0xe0, 0x6c, 0xee, 0x57, 0x7b, 0x78, 0x03, 0x1f, - 0xed, 0x2f, 0x55, 0xe0, 0x74, 0x4e, 0x95, 0x4f, 0xb4, 0x63, 0xce, 0x67, 0xab, 0x88, 0x18, 0x82, - 0xf4, 0x91, 0xb8, 0x1c, 0xc6, 0x9c, 0x49, 0x7c, 0xb0, 0xa3, 0x1a, 0x7d, 0x5c, 0x52, 0xbe, 0xbf, - 0xc7, 0x25, 0xc6, 0xb4, 0x1c, 0x78, 0xa0, 0xd3, 0xb2, 0xb2, 0xcf, 0x19, 0xc6, 0xaf, 0x59, 0x30, - 0xde, 0xee, 0x51, 0x5a, 0x5e, 0x38, 0x1e, 0x6f, 0x1e, 0x4f, 0xe1, 0xfa, 0xfa, 0x63, 0x77, 0x77, - 0x27, 0x7a, 0x56, 0xf4, 0xc7, 0x3d, 0x7b, 0x65, 0x7f, 0xbb, 0x0c, 0xac, 0xc4, 0x2c, 0xab, 0xe4, - 0xb6, 0x83, 0x3e, 0x66, 0x16, 0x0b, 0xb6, 0x8a, 0x2a, 0x6c, 0xcb, 0x89, 0xab, 0x62, 0xc3, 0x7c, - 0x04, 0xf3, 0x6a, 0x0f, 0x67, 0x85, 0x56, 0xa9, 0x0f, 0xa1, 0xe5, 0xc9, 0xaa, 0xcc, 0xe5, 0xe2, - 0xab, 0x32, 0xd7, 0xb2, 0x15, 0x99, 0xf7, 0xfe, 0xc4, 0x03, 0x0f, 0xe5, 0x27, 0xfe, 0x45, 0x8b, - 0x0b, 0x9e, 0xcc, 0x57, 0xd0, 0x96, 0x81, 0xb5, 0x87, 0x65, 0xf0, 0x0c, 0x54, 0x63, 0xe2, 0xad, - 0x5f, 0x21, 0x8e, 0x27, 0x2c, 0x08, 0x7d, 0x7e, 0x2d, 0xda, 0xb1, 0xc2, 0x60, 0xd7, 0xb6, 0x7a, - 0x5e, 0x70, 0xe7, 0x52, 0x3b, 0x4c, 0x76, 0x84, 0x2d, 0xa1, 0xaf, 0x6d, 0x55, 0x10, 0x6c, 0x60, - 0xd9, 0x7f, 0xab, 0xc4, 0x67, 0xa0, 0x08, 0x82, 0x78, 0x31, 0x73, 0xd1, 0x5e, 0xff, 0xf1, 0x03, - 0x1f, 0x01, 0x68, 0xa8, 0x2b, 0xea, 0xc5, 0x99, 0xd0, 0x95, 0x23, 0xdf, 0x9f, 0x2d, 0xe8, 0xe9, - 0xd7, 0xd0, 0x6d, 0xd8, 0xe0, 0x97, 0x92, 0xa5, 0xe5, 0x7d, 0x65, 0x69, 0x4a, 0xac, 0x0c, 0xec, - 0xa3, 0xed, 0xfe, 0xd4, 0x82, 0x94, 0x45, 0x84, 0x42, 0xa8, 0xd0, 0xee, 0xee, 0x14, 0x73, 0xfb, - 0xbe, 0x49, 0x9a, 0x8a, 0x46, 0x31, 0xed, 0xd9, 0x4f, 0xcc, 0x19, 0x21, 0x4f, 0xc4, 0x4a, 0xf0, - 0x51, 0xbd, 0x5e, 0x1c, 0xc3, 0x2b, 0x41, 0xb0, 0xc9, 0x0f, 0x36, 0x75, 0xdc, 0x85, 0xfd, 0x22, - 0x9c, 0xea, 0xea, 0x14, 0xbb, 0x53, 0x2b, 0xa0, 0xda, 0x27, 0x33, 0x5d, 0x59, 0x02, 0x27, 0xe6, - 0x30, 0xfb, 0xeb, 0x16, 0x9c, 0xcc, 0x92, 0x47, 0x6f, 0x59, 0x70, 0x2a, 0xce, 0xd2, 0x3b, 0xae, - 0xb1, 0x53, 0xf1, 0x8e, 0x5d, 0x20, 0xdc, 0xdd, 0x09, 0xfb, 0xff, 0x8a, 0xc9, 0x7f, 0xcb, 0xf5, - 0x9b, 0xc1, 0x1d, 0x65, 0x98, 0x58, 0x3d, 0x0d, 0x13, 0xba, 0x1e, 0x1b, 0x1b, 0xa4, 0xd9, 0xf1, - 0xba, 0x32, 0x47, 0x57, 0x44, 0x3b, 0x56, 0x18, 0x2c, 0x51, 0xae, 0x23, 0xca, 0xb6, 0x67, 0x26, - 0xe5, 0xac, 0x68, 0xc7, 0x0a, 0x03, 0x3d, 0x0f, 0x23, 0xc6, 0x4b, 0xca, 0x79, 0xc9, 0x0c, 0x72, - 0x43, 0x65, 0xc6, 0x38, 0x85, 0x85, 0x26, 0x01, 0x94, 0x91, 0x23, 0x55, 0x24, 0x73, 0x14, 0x29, - 0x49, 0x14, 0x63, 0x03, 0x83, 0xa5, 0xa5, 0x7a, 0x9d, 0x98, 0xf9, 0xf8, 0x07, 0x75, 0x29, 0xd1, - 0x19, 0xd1, 0x86, 0x15, 0x94, 0x4a, 0x93, 0xb6, 0xe3, 0x77, 0x1c, 0x8f, 0x8e, 0x90, 0xd8, 0xfa, - 0xa9, 0x65, 0xb8, 0xa8, 0x20, 0xd8, 0xc0, 0xa2, 0x6f, 0x9c, 0xb8, 0x6d, 0xf2, 0x4a, 0xe0, 0xcb, - 0x38, 0x35, 0x7d, 0xec, 0x23, 0xda, 0xb1, 0xc2, 0xb0, 0xff, 0xab, 0x05, 0x27, 0x74, 0x92, 0x3b, - 0xbf, 0x3d, 0xdb, 0xdc, 0xa9, 0x5a, 0xfb, 0xee, 0x54, 0xd3, 0xd9, 0xbf, 0xa5, 0xbe, 0xb2, 0x7f, - 0xcd, 0xc4, 0xdc, 0xf2, 0x9e, 0x89, 0xb9, 0xdf, 0xaf, 0x6f, 0x66, 0xe5, 0x19, 0xbc, 0xc3, 0x79, - 0xb7, 0xb2, 0x22, 0x1b, 0x06, 0x1b, 0x8e, 0xaa, 0xf0, 0x32, 0xc2, 0xf7, 0x0e, 0x33, 0xd3, 0x0c, - 0x49, 0x40, 0xec, 0x25, 0xa8, 0xa9, 0xd3, 0x0f, 0xb9, 0x51, 0xb5, 0xf2, 0x37, 0xaa, 0x7d, 0x25, - 0x08, 0xd6, 0xd7, 0xbe, 0xf1, 0x9d, 0x27, 0xde, 0xf1, 0x7b, 0xdf, 0x79, 0xe2, 0x1d, 0x7f, 0xf4, - 0x9d, 0x27, 0xde, 0xf1, 0xf1, 0xbb, 0x4f, 0x58, 0xdf, 0xb8, 0xfb, 0x84, 0xf5, 0x7b, 0x77, 0x9f, - 0xb0, 0xfe, 0xe8, 0xee, 0x13, 0xd6, 0xb7, 0xef, 0x3e, 0x61, 0x7d, 0xe1, 0x3f, 0x3d, 0xf1, 0x8e, - 0x57, 0x72, 0x03, 0x15, 0xe9, 0x8f, 0x67, 0x1b, 0xcd, 0xa9, 0xad, 0x8b, 0x2c, 0x56, 0x8e, 0x2e, - 0xaf, 0x29, 0x63, 0x4e, 0x4d, 0xc9, 0xe5, 0xf5, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x87, 0xd4, - 0x96, 0xc0, 0xad, 0xe1, 0x00, 0x00, + // 11354 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0xbd, 0x7d, 0x70, 0x24, 0xc7, + 0x75, 0x18, 0xae, 0xd9, 0xc5, 0x02, 0xbb, 0x0f, 0x5f, 0x77, 0x7d, 0x77, 0x24, 0x78, 0x22, 0x89, + 0xf3, 0xf0, 0x67, 0x8a, 0xfa, 0x89, 0x04, 0xcc, 0x13, 0x49, 0x33, 0xa2, 0x4c, 0x19, 0x0b, 0xdc, + 0xe1, 0x70, 0x07, 0x1c, 0xc0, 0x06, 0xee, 0x4e, 0xa2, 0x4c, 0x51, 0x83, 0xdd, 0xc6, 0x62, 0x0e, + 0xb3, 0x33, 0xc3, 0x99, 0x59, 0xdc, 0x81, 0x96, 0x64, 0xea, 0xc3, 0xb6, 0x12, 0x7d, 0x50, 0x91, + 0x92, 0x32, 0x9d, 0x58, 0x8a, 0x6c, 0x39, 0xa9, 0xb8, 0x12, 0x55, 0x9c, 0xa4, 0x2a, 0x71, 0xe2, + 0xa4, 0x5c, 0xb6, 0x53, 0x29, 0xa5, 0x94, 0x94, 0x5d, 0x2a, 0x97, 0xe5, 0x24, 0x36, 0x22, 0x5d, + 0x2a, 0x95, 0x54, 0xaa, 0xe2, 0x2a, 0x27, 0xfe, 0x23, 0xb9, 0xe4, 0x8f, 0x54, 0x7f, 0xf7, 0xcc, + 0xce, 0x02, 0x8b, 0xc3, 0xe0, 0xee, 0xa4, 0xf0, 0xbf, 0xdd, 0x7e, 0x6f, 0xde, 0x7b, 0xd3, 0xd3, + 0xfd, 0xfa, 0xf5, 0xeb, 0xf7, 0x5e, 0xc3, 0x62, 0xcb, 0x4d, 0x36, 0x3b, 0xeb, 0x53, 0x8d, 0xa0, + 0x3d, 0xed, 0x44, 0xad, 0x20, 0x8c, 0x82, 0xeb, 0xec, 0xc7, 0x53, 0x8d, 0xe6, 0xf4, 0xf6, 0xd9, + 0xe9, 0x70, 0xab, 0x35, 0xed, 0x84, 0x6e, 0x3c, 0xed, 0x84, 0xa1, 0xe7, 0x36, 0x9c, 0xc4, 0x0d, + 0xfc, 0xe9, 0xed, 0xa7, 0x1d, 0x2f, 0xdc, 0x74, 0x9e, 0x9e, 0x6e, 0x11, 0x9f, 0x44, 0x4e, 0x42, + 0x9a, 0x53, 0x61, 0x14, 0x24, 0x01, 0x7a, 0xbf, 0xa6, 0x36, 0x25, 0xa9, 0xb1, 0x1f, 0xaf, 0x36, + 0x9a, 0x53, 0xdb, 0x67, 0xa7, 0xc2, 0xad, 0xd6, 0x14, 0xa5, 0x36, 0x65, 0x50, 0x9b, 0x92, 0xd4, + 0x4e, 0x3f, 0x65, 0xc8, 0xd2, 0x0a, 0x5a, 0xc1, 0x34, 0x23, 0xba, 0xde, 0xd9, 0x60, 0xff, 0xd8, + 0x1f, 0xf6, 0x8b, 0x33, 0x3b, 0x6d, 0x6f, 0x3d, 0x1f, 0x4f, 0xb9, 0x01, 0x15, 0x6f, 0xba, 0x11, + 0x44, 0x64, 0x7a, 0xbb, 0x4b, 0xa0, 0xd3, 0x17, 0x34, 0x0e, 0xb9, 0x99, 0x10, 0x3f, 0x76, 0x03, + 0x3f, 0x7e, 0x8a, 0x8a, 0x40, 0xa2, 0x6d, 0x12, 0x99, 0xaf, 0x67, 0x20, 0xe4, 0x51, 0x7a, 0x46, + 0x53, 0x6a, 0x3b, 0x8d, 0x4d, 0xd7, 0x27, 0xd1, 0x8e, 0x7e, 0xbc, 0x4d, 0x12, 0x27, 0xef, 0xa9, + 0xe9, 0x5e, 0x4f, 0x45, 0x1d, 0x3f, 0x71, 0xdb, 0xa4, 0xeb, 0x81, 0xe7, 0xf6, 0x7b, 0x20, 0x6e, + 0x6c, 0x92, 0xb6, 0xd3, 0xf5, 0xdc, 0x7b, 0x7b, 0x3d, 0xd7, 0x49, 0x5c, 0x6f, 0xda, 0xf5, 0x93, + 0x38, 0x89, 0xb2, 0x0f, 0xd9, 0xbf, 0x64, 0xc1, 0xe8, 0xcc, 0xb5, 0xd5, 0x99, 0x4e, 0xb2, 0x39, + 0x1b, 0xf8, 0x1b, 0x6e, 0x0b, 0x3d, 0x0b, 0xc3, 0x0d, 0xaf, 0x13, 0x27, 0x24, 0xba, 0xec, 0xb4, + 0xc9, 0x84, 0x75, 0xc6, 0x7a, 0xa2, 0x56, 0x3f, 0xf1, 0xad, 0xdd, 0xc9, 0x77, 0xdc, 0xda, 0x9d, + 0x1c, 0x9e, 0xd5, 0x20, 0x6c, 0xe2, 0xa1, 0x77, 0xc3, 0x50, 0x14, 0x78, 0x64, 0x06, 0x5f, 0x9e, + 0x28, 0xb1, 0x47, 0xc6, 0xc5, 0x23, 0x43, 0x98, 0x37, 0x63, 0x09, 0xa7, 0xa8, 0x61, 0x14, 0x6c, + 0xb8, 0x1e, 0x99, 0x28, 0xa7, 0x51, 0x57, 0x78, 0x33, 0x96, 0x70, 0xfb, 0x0f, 0x4b, 0x00, 0x33, + 0x61, 0xb8, 0x12, 0x05, 0xd7, 0x49, 0x23, 0x41, 0x1f, 0x85, 0x2a, 0xed, 0xe6, 0xa6, 0x93, 0x38, + 0x4c, 0xb0, 0xe1, 0xb3, 0x3f, 0x36, 0xc5, 0xdf, 0x7a, 0xca, 0x7c, 0x6b, 0x3d, 0xc8, 0x28, 0xf6, + 0xd4, 0xf6, 0xd3, 0x53, 0xcb, 0xeb, 0xf4, 0xf9, 0x25, 0x92, 0x38, 0x75, 0x24, 0x98, 0x81, 0x6e, + 0xc3, 0x8a, 0x2a, 0xf2, 0x61, 0x20, 0x0e, 0x49, 0x83, 0xbd, 0xc3, 0xf0, 0xd9, 0xc5, 0xa9, 0xc3, + 0x8c, 0xe6, 0x29, 0x2d, 0xf9, 0x6a, 0x48, 0x1a, 0xf5, 0x11, 0xc1, 0x79, 0x80, 0xfe, 0xc3, 0x8c, + 0x0f, 0xda, 0x86, 0xc1, 0x38, 0x71, 0x92, 0x4e, 0xcc, 0xba, 0x62, 0xf8, 0xec, 0xe5, 0xc2, 0x38, + 0x32, 0xaa, 0xf5, 0x31, 0xc1, 0x73, 0x90, 0xff, 0xc7, 0x82, 0x9b, 0xfd, 0x27, 0x16, 0x8c, 0x69, + 0xe4, 0x45, 0x37, 0x4e, 0xd0, 0x4f, 0x75, 0x75, 0xee, 0x54, 0x7f, 0x9d, 0x4b, 0x9f, 0x66, 0x5d, + 0x7b, 0x4c, 0x30, 0xab, 0xca, 0x16, 0xa3, 0x63, 0xdb, 0x50, 0x71, 0x13, 0xd2, 0x8e, 0x27, 0x4a, + 0x67, 0xca, 0x4f, 0x0c, 0x9f, 0xbd, 0x50, 0xd4, 0x7b, 0xd6, 0x47, 0x05, 0xd3, 0xca, 0x02, 0x25, + 0x8f, 0x39, 0x17, 0xfb, 0xd7, 0x46, 0xcc, 0xf7, 0xa3, 0x1d, 0x8e, 0x9e, 0x86, 0xe1, 0x38, 0xe8, + 0x44, 0x0d, 0x82, 0x49, 0x18, 0xc4, 0x13, 0xd6, 0x99, 0x32, 0x1d, 0x7a, 0x74, 0x50, 0xaf, 0xea, + 0x66, 0x6c, 0xe2, 0xa0, 0x2f, 0x5a, 0x30, 0xd2, 0x24, 0x71, 0xe2, 0xfa, 0x8c, 0xbf, 0x14, 0x7e, + 0xed, 0xd0, 0xc2, 0xcb, 0xc6, 0x39, 0x4d, 0xbc, 0x7e, 0x52, 0xbc, 0xc8, 0x88, 0xd1, 0x18, 0xe3, + 0x14, 0x7f, 0x3a, 0x39, 0x9b, 0x24, 0x6e, 0x44, 0x6e, 0x48, 0xff, 0x8b, 0xe9, 0xa3, 0x26, 0xe7, + 0x9c, 0x06, 0x61, 0x13, 0x0f, 0xf9, 0x50, 0xa1, 0x93, 0x2f, 0x9e, 0x18, 0x60, 0xf2, 0x2f, 0x1c, + 0x4e, 0x7e, 0xd1, 0xa9, 0x74, 0x5e, 0xeb, 0xde, 0xa7, 0xff, 0x62, 0xcc, 0xd9, 0xa0, 0x2f, 0x58, + 0x30, 0x21, 0x94, 0x03, 0x26, 0xbc, 0x43, 0xaf, 0x6d, 0xba, 0x09, 0xf1, 0xdc, 0x38, 0x99, 0xa8, + 0x30, 0x19, 0xa6, 0xfb, 0x1b, 0x5b, 0xf3, 0x51, 0xd0, 0x09, 0x2f, 0xb9, 0x7e, 0xb3, 0x7e, 0x46, + 0x70, 0x9a, 0x98, 0xed, 0x41, 0x18, 0xf7, 0x64, 0x89, 0xbe, 0x62, 0xc1, 0x69, 0xdf, 0x69, 0x93, + 0x38, 0x74, 0xe8, 0xa7, 0xe5, 0xe0, 0xba, 0xe7, 0x34, 0xb6, 0x98, 0x44, 0x83, 0x77, 0x26, 0x91, + 0x2d, 0x24, 0x3a, 0x7d, 0xb9, 0x27, 0x69, 0xbc, 0x07, 0x5b, 0xf4, 0x0d, 0x0b, 0x8e, 0x07, 0x51, + 0xb8, 0xe9, 0xf8, 0xa4, 0x29, 0xa1, 0xf1, 0xc4, 0x10, 0x9b, 0x7a, 0x1f, 0x39, 0xdc, 0x27, 0x5a, + 0xce, 0x92, 0x5d, 0x0a, 0x7c, 0x37, 0x09, 0xa2, 0x55, 0x92, 0x24, 0xae, 0xdf, 0x8a, 0xeb, 0xa7, + 0x6e, 0xed, 0x4e, 0x1e, 0xef, 0xc2, 0xc2, 0xdd, 0xf2, 0xa0, 0x9f, 0x86, 0xe1, 0x78, 0xc7, 0x6f, + 0x5c, 0x73, 0xfd, 0x66, 0x70, 0x23, 0x9e, 0xa8, 0x16, 0x31, 0x7d, 0x57, 0x15, 0x41, 0x31, 0x01, + 0x35, 0x03, 0x6c, 0x72, 0xcb, 0xff, 0x70, 0x7a, 0x28, 0xd5, 0x8a, 0xfe, 0x70, 0x7a, 0x30, 0xed, + 0xc1, 0x16, 0xfd, 0xbc, 0x05, 0xa3, 0xb1, 0xdb, 0xf2, 0x9d, 0xa4, 0x13, 0x91, 0x4b, 0x64, 0x27, + 0x9e, 0x00, 0x26, 0xc8, 0xc5, 0x43, 0xf6, 0x8a, 0x41, 0xb2, 0x7e, 0x4a, 0xc8, 0x38, 0x6a, 0xb6, + 0xc6, 0x38, 0xcd, 0x37, 0x6f, 0xa2, 0xe9, 0x61, 0x3d, 0x5c, 0xec, 0x44, 0xd3, 0x83, 0xba, 0x27, + 0x4b, 0xf4, 0x93, 0x70, 0x8c, 0x37, 0xa9, 0x9e, 0x8d, 0x27, 0x46, 0x98, 0xa2, 0x3d, 0x79, 0x6b, + 0x77, 0xf2, 0xd8, 0x6a, 0x06, 0x86, 0xbb, 0xb0, 0xd1, 0x6b, 0x30, 0x19, 0x92, 0xa8, 0xed, 0x26, + 0xcb, 0xbe, 0xb7, 0x23, 0xd5, 0x77, 0x23, 0x08, 0x49, 0x53, 0x88, 0x13, 0x4f, 0x8c, 0x9e, 0xb1, + 0x9e, 0xa8, 0xd6, 0xdf, 0x25, 0xc4, 0x9c, 0x5c, 0xd9, 0x1b, 0x1d, 0xef, 0x47, 0xcf, 0xfe, 0x57, + 0x25, 0x38, 0x96, 0x5d, 0x38, 0xd1, 0xdf, 0xb2, 0x60, 0xfc, 0xfa, 0x8d, 0x64, 0x2d, 0xd8, 0x22, + 0x7e, 0x5c, 0xdf, 0xa1, 0xea, 0x8d, 0x2d, 0x19, 0xc3, 0x67, 0x1b, 0xc5, 0x2e, 0xd1, 0x53, 0x17, + 0xd3, 0x5c, 0xce, 0xf9, 0x49, 0xb4, 0x53, 0x7f, 0x50, 0xbc, 0xdd, 0xf8, 0xc5, 0x6b, 0x6b, 0x26, + 0x14, 0x67, 0x85, 0x3a, 0xfd, 0x39, 0x0b, 0x4e, 0xe6, 0x91, 0x40, 0xc7, 0xa0, 0xbc, 0x45, 0x76, + 0xb8, 0x01, 0x87, 0xe9, 0x4f, 0xf4, 0x0a, 0x54, 0xb6, 0x1d, 0xaf, 0x43, 0x84, 0x75, 0x33, 0x7f, + 0xb8, 0x17, 0x51, 0x92, 0x61, 0x4e, 0xf5, 0x7d, 0xa5, 0xe7, 0x2d, 0xfb, 0xf7, 0xca, 0x30, 0x6c, + 0xac, 0x6f, 0x77, 0xc1, 0x62, 0x0b, 0x52, 0x16, 0xdb, 0x52, 0x61, 0x4b, 0x73, 0x4f, 0x93, 0xed, + 0x46, 0xc6, 0x64, 0x5b, 0x2e, 0x8e, 0xe5, 0x9e, 0x36, 0x1b, 0x4a, 0xa0, 0x16, 0x84, 0xd4, 0x7a, + 0xa7, 0x4b, 0xff, 0x40, 0x11, 0x9f, 0x70, 0x59, 0x92, 0xab, 0x8f, 0xde, 0xda, 0x9d, 0xac, 0xa9, + 0xbf, 0x58, 0x33, 0xb2, 0xbf, 0x6b, 0xc1, 0x49, 0x43, 0xc6, 0xd9, 0xc0, 0x6f, 0xba, 0xec, 0xd3, + 0x9e, 0x81, 0x81, 0x64, 0x27, 0x94, 0x3b, 0x04, 0xd5, 0x53, 0x6b, 0x3b, 0x21, 0xc1, 0x0c, 0x42, + 0x0d, 0xfd, 0x36, 0x89, 0x63, 0xa7, 0x45, 0xb2, 0x7b, 0x82, 0x25, 0xde, 0x8c, 0x25, 0x1c, 0x45, + 0x80, 0x3c, 0x27, 0x4e, 0xd6, 0x22, 0xc7, 0x8f, 0x19, 0xf9, 0x35, 0xb7, 0x4d, 0x44, 0x07, 0xff, + 0xff, 0xfd, 0x8d, 0x18, 0xfa, 0x44, 0xfd, 0x81, 0x5b, 0xbb, 0x93, 0x68, 0xb1, 0x8b, 0x12, 0xce, + 0xa1, 0x6e, 0x7f, 0xa5, 0x0c, 0xa7, 0x52, 0xb6, 0x58, 0x48, 0xfc, 0x26, 0xf1, 0x1b, 0x3b, 0xe8, + 0x53, 0x16, 0xd4, 0x62, 0xe2, 0x91, 0x46, 0x12, 0x44, 0xb1, 0x98, 0xf6, 0x2f, 0x15, 0xf7, 0x99, + 0x05, 0xe5, 0xfa, 0x71, 0xd1, 0x1d, 0x35, 0xd9, 0x12, 0x63, 0xcd, 0x16, 0x3d, 0x03, 0x23, 0xeb, + 0x5e, 0xd0, 0xd8, 0x5a, 0xf6, 0xcf, 0xb5, 0xc3, 0x64, 0x87, 0x75, 0x61, 0xb5, 0x7e, 0x8c, 0x5a, + 0x88, 0x75, 0xa3, 0x1d, 0xa7, 0xb0, 0xd0, 0xfb, 0xa1, 0x46, 0x17, 0xd0, 0x39, 0xe2, 0x39, 0x3b, + 0xac, 0xff, 0xca, 0xf5, 0x47, 0x19, 0x0b, 0xd9, 0x78, 0x7b, 0x77, 0x72, 0x94, 0x6e, 0x23, 0xa7, + 0xe6, 0x3a, 0xf2, 0x63, 0xab, 0x07, 0xd0, 0x73, 0x30, 0x44, 0x61, 0x41, 0x27, 0x61, 0x03, 0xac, + 0x5c, 0x7f, 0x98, 0x7e, 0xad, 0x35, 0xde, 0xd4, 0xfd, 0xa4, 0x44, 0x46, 0x0b, 0x70, 0x22, 0x22, + 0x1b, 0x11, 0x89, 0x37, 0x55, 0x2f, 0xba, 0x24, 0x9e, 0xa8, 0x30, 0x91, 0x1f, 0xbc, 0xb5, 0x3b, + 0x79, 0x02, 0x77, 0x83, 0x71, 0xde, 0x33, 0xf6, 0x57, 0x2c, 0x78, 0x20, 0xdf, 0x42, 0x46, 0x8f, + 0xc3, 0x20, 0xdf, 0xb4, 0x8b, 0x31, 0xa7, 0x27, 0x0a, 0x6b, 0xc5, 0x02, 0x8a, 0xa6, 0xa1, 0xa6, + 0x56, 0x6f, 0x31, 0xf2, 0x54, 0x57, 0xeb, 0x25, 0x5f, 0xe3, 0xd0, 0xa1, 0x4c, 0xff, 0x08, 0x7b, + 0x5a, 0x0d, 0x65, 0xb6, 0xcb, 0x65, 0x10, 0xfb, 0x3f, 0x58, 0x30, 0x6e, 0x48, 0x75, 0x17, 0x36, + 0x4c, 0x7e, 0x7a, 0xc3, 0xb4, 0x50, 0xd8, 0xf0, 0xeb, 0xb1, 0x63, 0xfa, 0x82, 0x05, 0xa7, 0x0d, + 0xac, 0x25, 0x27, 0x69, 0x6c, 0x9e, 0xbb, 0x19, 0x46, 0x24, 0x8e, 0x69, 0xdf, 0x3f, 0x62, 0xac, + 0x26, 0xf5, 0x61, 0x41, 0xa1, 0x7c, 0x89, 0xec, 0xf0, 0xa5, 0xe5, 0x49, 0xa8, 0x72, 0x95, 0x11, + 0x44, 0xa2, 0xc7, 0xd5, 0xbb, 0x2d, 0x8b, 0x76, 0xac, 0x30, 0x90, 0x0d, 0x83, 0x6c, 0xc9, 0xa0, + 0x2a, 0x94, 0x1a, 0x07, 0x40, 0x3f, 0xe2, 0x55, 0xd6, 0x82, 0x05, 0xc4, 0x8e, 0x53, 0xe2, 0xac, + 0x44, 0x84, 0x7d, 0xdc, 0xe6, 0x79, 0x97, 0x78, 0xcd, 0x98, 0x6e, 0xe6, 0x1c, 0xdf, 0x0f, 0x12, + 0xb1, 0x2f, 0x33, 0x36, 0x73, 0x33, 0xba, 0x19, 0x9b, 0x38, 0x94, 0xa9, 0xe7, 0xac, 0x13, 0x8f, + 0xf7, 0xa8, 0x60, 0xba, 0xc8, 0x5a, 0xb0, 0x80, 0xd8, 0xff, 0xc8, 0x82, 0x13, 0x39, 0x33, 0x15, + 0x79, 0x30, 0xca, 0x30, 0x64, 0x83, 0xf8, 0xde, 0xef, 0xed, 0xf3, 0x7b, 0x9b, 0x8f, 0xd6, 0x8f, + 0x53, 0xab, 0x2e, 0xd5, 0x84, 0xd3, 0xc4, 0xe9, 0xcb, 0xd1, 0x41, 0xb7, 0xe2, 0x24, 0x09, 0x89, + 0x7c, 0x21, 0x2e, 0x7b, 0xb9, 0xcb, 0xba, 0x19, 0x9b, 0x38, 0xf6, 0xad, 0x12, 0xdb, 0xef, 0x6a, + 0xc1, 0xef, 0x86, 0xb3, 0x24, 0x4a, 0x2d, 0xbd, 0x2b, 0x05, 0x2a, 0xc8, 0xde, 0x0e, 0x93, 0xd7, + 0x33, 0xab, 0x2f, 0x2e, 0x94, 0xeb, 0xde, 0x4e, 0x93, 0xdf, 0x2e, 0xc1, 0x64, 0xfa, 0x81, 0xae, + 0xc5, 0x9b, 0xee, 0xd0, 0x0d, 0x46, 0x59, 0xf7, 0x99, 0x81, 0x8f, 0x4d, 0xbc, 0x1e, 0xeb, 0x5f, + 0xe9, 0x28, 0xd7, 0x3f, 0x73, 0x79, 0x2e, 0xef, 0xb3, 0x3c, 0x3f, 0xae, 0x7a, 0x7d, 0x20, 0xa3, + 0x79, 0xd3, 0x26, 0xca, 0x19, 0x18, 0x88, 0x13, 0x12, 0x32, 0xc5, 0x6f, 0x28, 0xd2, 0xd5, 0x84, + 0x84, 0x98, 0x41, 0xec, 0xff, 0x5a, 0x82, 0x07, 0xd3, 0x7d, 0xa8, 0x2d, 0x8a, 0x0f, 0xa4, 0x2c, + 0x8a, 0xf7, 0x98, 0x16, 0xc5, 0xed, 0xdd, 0xc9, 0x77, 0xf6, 0x78, 0xec, 0x07, 0xc6, 0xe0, 0x40, + 0xf3, 0x99, 0x5e, 0x9c, 0x4e, 0xf7, 0xe2, 0xed, 0xdd, 0xc9, 0x47, 0x7a, 0xbc, 0x63, 0xa6, 0x9b, + 0x1f, 0x87, 0xc1, 0x88, 0x38, 0x71, 0xe0, 0x8b, 0x8e, 0x56, 0x9f, 0x03, 0xb3, 0x56, 0x2c, 0xa0, + 0xf6, 0x77, 0x6a, 0xd9, 0xce, 0x9e, 0xe7, 0xfe, 0xdf, 0x20, 0x42, 0x2e, 0x0c, 0xb0, 0x5d, 0x22, + 0x57, 0x0d, 0x97, 0x0e, 0x37, 0x8d, 0xe8, 0xfa, 0xa5, 0x48, 0xd7, 0xab, 0xf4, 0xab, 0xd1, 0x26, + 0xcc, 0x58, 0xa0, 0x9b, 0x50, 0x6d, 0xc8, 0xcd, 0x5b, 0xa9, 0x08, 0x37, 0xa7, 0xd8, 0xba, 0x69, + 0x8e, 0x23, 0x74, 0xa1, 0x51, 0x3b, 0x3e, 0xc5, 0x0d, 0x11, 0x28, 0xb7, 0xdc, 0x44, 0x7c, 0xd6, + 0x43, 0x6e, 0xcf, 0xe7, 0x5d, 0xe3, 0x15, 0x87, 0xe8, 0xea, 0x37, 0xef, 0x26, 0x98, 0xd2, 0x47, + 0x3f, 0x6b, 0xc1, 0x70, 0xdc, 0x68, 0xaf, 0x44, 0xc1, 0xb6, 0xdb, 0x24, 0x91, 0x30, 0xce, 0x0f, + 0xa9, 0x9a, 0x56, 0x67, 0x97, 0x24, 0x41, 0xcd, 0x97, 0xbb, 0x4b, 0x34, 0x04, 0x9b, 0x7c, 0xe9, + 0xa6, 0xf5, 0x41, 0xf1, 0xee, 0x73, 0xa4, 0xe1, 0xd2, 0x85, 0x5b, 0xee, 0xd1, 0xd9, 0x48, 0x39, + 0xf4, 0x66, 0x65, 0xae, 0xd3, 0xd8, 0xa2, 0xf3, 0x4d, 0x0b, 0xf4, 0xce, 0x5b, 0xbb, 0x93, 0x0f, + 0xce, 0xe6, 0xf3, 0xc4, 0xbd, 0x84, 0x61, 0x1d, 0x16, 0x76, 0x3c, 0x0f, 0x93, 0xd7, 0x3a, 0x84, + 0x79, 0xe0, 0x0a, 0xe8, 0xb0, 0x15, 0x4d, 0x30, 0xd3, 0x61, 0x06, 0x04, 0x9b, 0x7c, 0xd1, 0x6b, + 0x30, 0xd8, 0x76, 0x92, 0xc8, 0xbd, 0x29, 0xdc, 0x6e, 0x87, 0xdc, 0x3e, 0x2e, 0x31, 0x5a, 0x9a, + 0x39, 0x33, 0x31, 0x78, 0x23, 0x16, 0x8c, 0x50, 0x1b, 0x2a, 0x6d, 0x12, 0xb5, 0xc8, 0x44, 0xb5, + 0x88, 0x23, 0x86, 0x25, 0x4a, 0x4a, 0x33, 0xac, 0x51, 0xb3, 0x8e, 0xb5, 0x61, 0xce, 0x05, 0xbd, + 0x02, 0x55, 0xb9, 0xa5, 0x98, 0xa8, 0xdd, 0xb9, 0xd1, 0xc2, 0x26, 0x98, 0xb2, 0x57, 0x14, 0x49, + 0xda, 0x81, 0xa1, 0xd7, 0x69, 0xb9, 0xfe, 0x04, 0x14, 0xd1, 0x81, 0x2b, 0x8c, 0x56, 0xa6, 0x03, + 0x79, 0x23, 0x16, 0x8c, 0xec, 0xff, 0x64, 0x01, 0x4a, 0x2b, 0xb5, 0xbb, 0x60, 0x8d, 0xbf, 0x96, + 0xb6, 0xc6, 0x17, 0x8b, 0xb4, 0x3a, 0x7a, 0x18, 0xe4, 0xbf, 0x59, 0x83, 0xcc, 0x72, 0x70, 0x99, + 0xc4, 0x09, 0x69, 0xbe, 0xad, 0xc2, 0xdf, 0x56, 0xe1, 0x6f, 0xab, 0x70, 0xa5, 0xc2, 0xd7, 0x33, + 0x2a, 0xfc, 0x45, 0x63, 0xd6, 0xeb, 0xf3, 0xfc, 0x57, 0xd5, 0x81, 0xbf, 0x29, 0x81, 0x81, 0x40, + 0x35, 0xc1, 0xc5, 0xd5, 0xe5, 0xcb, 0xb9, 0x3a, 0xfb, 0xd5, 0xb4, 0xce, 0x3e, 0x2c, 0x8b, 0xff, + 0x17, 0xb4, 0xf4, 0xbf, 0xb4, 0xe0, 0x5d, 0x69, 0xed, 0x25, 0x47, 0xce, 0x42, 0xcb, 0x0f, 0x22, + 0x32, 0xe7, 0x6e, 0x6c, 0x90, 0x88, 0xf8, 0x0d, 0x12, 0x2b, 0xf7, 0x8b, 0xd5, 0xcb, 0xfd, 0x82, + 0x9e, 0x81, 0x91, 0xeb, 0x71, 0xe0, 0xaf, 0x04, 0xae, 0x2f, 0x54, 0x10, 0xdd, 0x12, 0x33, 0x5f, + 0x18, 0xed, 0x51, 0xd9, 0x8e, 0x53, 0x58, 0x68, 0x16, 0x8e, 0x5f, 0x7f, 0x6d, 0xc5, 0x49, 0x0c, + 0x3f, 0x86, 0xf4, 0x38, 0xb0, 0xf3, 0xaf, 0x8b, 0x2f, 0x65, 0x80, 0xb8, 0x1b, 0xdf, 0xfe, 0xeb, + 0x25, 0x78, 0x28, 0xf3, 0x22, 0x81, 0xe7, 0x05, 0x9d, 0x84, 0x6e, 0x6a, 0xd0, 0xd7, 0x2c, 0x38, + 0xd6, 0x4e, 0xbb, 0x4a, 0xa4, 0xc3, 0xf0, 0x83, 0x85, 0xad, 0x11, 0x19, 0x5f, 0x4c, 0x7d, 0x42, + 0xf4, 0xd0, 0xb1, 0x0c, 0x20, 0xc6, 0x5d, 0xb2, 0xa0, 0x57, 0xa0, 0xd6, 0x76, 0x6e, 0x5e, 0x09, + 0x9b, 0x4e, 0x22, 0xf7, 0x93, 0xbd, 0xdd, 0x00, 0x9d, 0xc4, 0xf5, 0xa6, 0x78, 0xa4, 0xc8, 0xd4, + 0x82, 0x9f, 0x2c, 0x47, 0xab, 0x49, 0xe4, 0xfa, 0x2d, 0xee, 0x1d, 0x5e, 0x92, 0x64, 0xb0, 0xa6, + 0x68, 0x7f, 0xd5, 0xca, 0x2e, 0x52, 0xaa, 0x77, 0x22, 0x27, 0x21, 0xad, 0x1d, 0xf4, 0x31, 0xa8, + 0xd0, 0x8d, 0x9f, 0xec, 0x95, 0x6b, 0x45, 0xae, 0x9c, 0xc6, 0x97, 0xd0, 0x8b, 0x28, 0xfd, 0x17, + 0x63, 0xce, 0xd4, 0xfe, 0x5a, 0x2d, 0x6b, 0x2c, 0xb0, 0x58, 0x80, 0xb3, 0x00, 0xad, 0x60, 0x8d, + 0xb4, 0x43, 0x8f, 0x76, 0x8b, 0xc5, 0xdc, 0x94, 0xca, 0xd7, 0x31, 0xaf, 0x20, 0xd8, 0xc0, 0x42, + 0x7f, 0xd1, 0x02, 0x68, 0xc9, 0x31, 0x2f, 0x0d, 0x81, 0x2b, 0x45, 0xbe, 0x8e, 0x9e, 0x51, 0x5a, + 0x16, 0xc5, 0x10, 0x1b, 0xcc, 0xd1, 0xa7, 0x2c, 0xa8, 0x26, 0x52, 0x7c, 0xbe, 0x34, 0xae, 0x15, + 0x29, 0x89, 0x7c, 0x69, 0x6d, 0x13, 0xa9, 0x2e, 0x51, 0x7c, 0xd1, 0xcf, 0x59, 0x00, 0xf1, 0x8e, + 0xdf, 0x58, 0x09, 0x3c, 0xb7, 0xb1, 0x23, 0x56, 0xcc, 0xab, 0x85, 0xfa, 0x63, 0x14, 0xf5, 0xfa, + 0x18, 0xed, 0x0d, 0xfd, 0x1f, 0x1b, 0x9c, 0xd1, 0x27, 0xa0, 0x1a, 0x8b, 0xe1, 0x26, 0xd6, 0xc8, + 0xb5, 0x62, 0xbd, 0x42, 0x9c, 0xb6, 0x50, 0xaf, 0xe2, 0x1f, 0x56, 0x3c, 0xd1, 0x2f, 0x58, 0x30, + 0x1e, 0xa6, 0x1d, 0x94, 0x62, 0x39, 0x2c, 0x4e, 0x07, 0x64, 0x1c, 0xa0, 0xf5, 0x13, 0xb7, 0x76, + 0x27, 0xc7, 0x33, 0x8d, 0x38, 0x2b, 0x05, 0xd5, 0x80, 0x7a, 0x04, 0x2f, 0x87, 0xdc, 0x59, 0x3a, + 0xa4, 0x35, 0xe0, 0x7c, 0x16, 0x88, 0xbb, 0xf1, 0xd1, 0x0a, 0x9c, 0xa4, 0xd2, 0xed, 0x70, 0xf3, + 0x53, 0x9d, 0x55, 0xb0, 0xc5, 0xb0, 0x5a, 0x7f, 0x58, 0x8c, 0x10, 0x76, 0x48, 0x94, 0xc5, 0xc1, + 0xb9, 0x4f, 0xa2, 0xdf, 0xb3, 0xe0, 0x61, 0x97, 0x2d, 0x03, 0xa6, 0xa7, 0x5f, 0xaf, 0x08, 0xe2, + 0x60, 0x9f, 0x14, 0xaa, 0x2b, 0x7a, 0x2d, 0x3f, 0xf5, 0xff, 0x4f, 0xbc, 0xc1, 0xc3, 0x0b, 0x7b, + 0x88, 0x84, 0xf7, 0x14, 0x18, 0xfd, 0x38, 0x8c, 0xca, 0x79, 0xb1, 0x42, 0x55, 0x30, 0x5b, 0x68, + 0x6b, 0xdc, 0xd7, 0xbb, 0x66, 0x02, 0x70, 0x1a, 0xcf, 0xfe, 0x76, 0x29, 0x75, 0xbc, 0xa6, 0x9c, + 0x90, 0x4c, 0xdd, 0x34, 0xa4, 0xff, 0x47, 0x6a, 0xcf, 0x42, 0xd5, 0x8d, 0xf2, 0x2e, 0x69, 0x75, + 0xa3, 0x9a, 0x62, 0x6c, 0x30, 0xa7, 0x46, 0xe9, 0x71, 0x27, 0xeb, 0xea, 0x14, 0x1a, 0xf0, 0x95, + 0x22, 0x45, 0xea, 0x3e, 0x0c, 0x7d, 0x48, 0x88, 0x76, 0xbc, 0x0b, 0x84, 0xbb, 0x45, 0xb2, 0xbf, + 0x9d, 0x3e, 0x3c, 0x32, 0x26, 0x6f, 0x1f, 0xc7, 0x95, 0x5f, 0xb4, 0x60, 0x38, 0x0a, 0x3c, 0xcf, + 0xf5, 0x5b, 0x54, 0xd1, 0x88, 0xd5, 0xf2, 0xc3, 0x47, 0xb2, 0x60, 0x09, 0x8d, 0xc2, 0x4c, 0x5b, + 0xac, 0x79, 0x62, 0x53, 0x00, 0xfb, 0x4f, 0x2c, 0x98, 0xe8, 0xa5, 0x10, 0x11, 0x81, 0x77, 0xca, + 0xd9, 0xae, 0x82, 0x75, 0x96, 0xfd, 0x39, 0xe2, 0x11, 0xe5, 0x78, 0xae, 0xd6, 0x1f, 0x13, 0xaf, + 0xf9, 0xce, 0x95, 0xde, 0xa8, 0x78, 0x2f, 0x3a, 0xe8, 0x65, 0x38, 0x66, 0xbc, 0x57, 0xac, 0x3a, + 0xa6, 0x56, 0x9f, 0xa2, 0x16, 0xc8, 0x4c, 0x06, 0x76, 0x7b, 0x77, 0xf2, 0x81, 0x6c, 0x9b, 0xd0, + 0xd8, 0x5d, 0x74, 0xec, 0x5f, 0x2d, 0x65, 0xbf, 0x96, 0x5a, 0x6c, 0xdf, 0xb2, 0xba, 0xb6, 0xf3, + 0x1f, 0x3c, 0x8a, 0x05, 0x8e, 0x6d, 0xfc, 0x55, 0x3c, 0x50, 0x6f, 0x9c, 0x7b, 0x18, 0x70, 0x60, + 0xff, 0xeb, 0x01, 0xd8, 0x43, 0xb2, 0x3e, 0xac, 0xe7, 0x03, 0x9f, 0x87, 0x7e, 0xde, 0x52, 0x67, + 0x65, 0x65, 0x36, 0xc9, 0x9b, 0x47, 0xd5, 0xf7, 0x7c, 0x03, 0x13, 0xf3, 0xa0, 0x17, 0xe5, 0xc6, + 0x4e, 0x9f, 0xca, 0xa1, 0xaf, 0x5b, 0xe9, 0xd3, 0x3e, 0x1e, 0xc5, 0xe8, 0x1e, 0x99, 0x4c, 0xc6, + 0x11, 0x22, 0x17, 0x4c, 0x9f, 0xdf, 0xf4, 0x3a, 0x5c, 0x9c, 0x02, 0xd8, 0x70, 0x7d, 0xc7, 0x73, + 0x5f, 0xa7, 0xdb, 0x93, 0x0a, 0x5b, 0x61, 0x99, 0xc9, 0x72, 0x5e, 0xb5, 0x62, 0x03, 0xe3, 0xf4, + 0x5f, 0x80, 0x61, 0xe3, 0xcd, 0x73, 0x62, 0x75, 0x4e, 0x9a, 0xb1, 0x3a, 0x35, 0x23, 0xc4, 0xe6, + 0xf4, 0x8b, 0x70, 0x2c, 0x2b, 0xe0, 0x41, 0x9e, 0xb7, 0xff, 0xe7, 0x50, 0xf6, 0x14, 0x6b, 0x8d, + 0x44, 0x6d, 0x2a, 0xda, 0xdb, 0x9e, 0xa5, 0xb7, 0x3d, 0x4b, 0x6f, 0x7b, 0x96, 0xcc, 0xc3, 0x01, + 0xe1, 0x35, 0x19, 0xba, 0x4b, 0x5e, 0x93, 0x94, 0x1f, 0xa8, 0x5a, 0xb8, 0x1f, 0xc8, 0xbe, 0x55, + 0x81, 0x94, 0x1d, 0xc5, 0xfb, 0xfb, 0xdd, 0x30, 0x14, 0x91, 0x30, 0xb8, 0x82, 0x17, 0xc5, 0x1a, + 0xa2, 0x53, 0x37, 0x78, 0x33, 0x96, 0x70, 0xba, 0xd6, 0x84, 0x4e, 0xb2, 0x29, 0x16, 0x11, 0xb5, + 0xd6, 0xac, 0x38, 0xc9, 0x26, 0x66, 0x10, 0xf4, 0x22, 0x8c, 0x25, 0x4e, 0xd4, 0xa2, 0xf6, 0xf6, + 0x36, 0xfb, 0xac, 0xe2, 0xac, 0xf3, 0x01, 0x81, 0x3b, 0xb6, 0x96, 0x82, 0xe2, 0x0c, 0x36, 0x7a, + 0x0d, 0x06, 0x36, 0x89, 0xd7, 0x16, 0x5d, 0xbe, 0x5a, 0x9c, 0x8e, 0x67, 0xef, 0x7a, 0x81, 0x78, + 0x6d, 0xae, 0x81, 0xe8, 0x2f, 0xcc, 0x58, 0xd1, 0xf1, 0x56, 0xdb, 0xea, 0xc4, 0x49, 0xd0, 0x76, + 0x5f, 0x97, 0x2e, 0xbe, 0x0f, 0x16, 0xcc, 0xf8, 0x92, 0xa4, 0xcf, 0x7d, 0x29, 0xea, 0x2f, 0xd6, + 0x9c, 0x99, 0x1c, 0x4d, 0x37, 0x62, 0x9f, 0x6a, 0x47, 0x78, 0xea, 0x8a, 0x96, 0x63, 0x4e, 0xd2, + 0xe7, 0x72, 0xa8, 0xbf, 0x58, 0x73, 0x46, 0x3b, 0x6a, 0xdc, 0x0f, 0x33, 0x19, 0xae, 0x14, 0x2c, + 0x03, 0x1f, 0xf3, 0xb9, 0xe3, 0xff, 0x31, 0xa8, 0x34, 0x36, 0x9d, 0x28, 0x99, 0x18, 0x61, 0x83, + 0x46, 0xf9, 0x74, 0x66, 0x69, 0x23, 0xe6, 0x30, 0xf4, 0x08, 0x94, 0x23, 0xb2, 0xc1, 0xc2, 0x80, + 0x8d, 0x50, 0x24, 0x4c, 0x36, 0x30, 0x6d, 0xb7, 0x7f, 0xb9, 0x94, 0x36, 0x97, 0xd2, 0xef, 0xcd, + 0x47, 0x7b, 0xa3, 0x13, 0xc5, 0xd2, 0xef, 0x63, 0x8c, 0x76, 0xd6, 0x8c, 0x25, 0x1c, 0x7d, 0xd2, + 0x82, 0xa1, 0xeb, 0x71, 0xe0, 0xfb, 0x24, 0x11, 0x4b, 0xd3, 0xd5, 0x82, 0xbb, 0xe2, 0x22, 0xa7, + 0xae, 0x65, 0x10, 0x0d, 0x58, 0xf2, 0xa5, 0xe2, 0x92, 0x9b, 0x0d, 0xaf, 0xd3, 0xec, 0x0a, 0xd2, + 0x38, 0xc7, 0x9b, 0xb1, 0x84, 0x53, 0x54, 0xd7, 0xe7, 0xa8, 0x03, 0x69, 0xd4, 0x05, 0x5f, 0xa0, + 0x0a, 0xb8, 0xfd, 0x57, 0x07, 0x53, 0xa1, 0x8f, 0x7a, 0x72, 0x50, 0x43, 0x86, 0x99, 0x0a, 0xe7, + 0x5d, 0x8f, 0xc8, 0xb8, 0x2a, 0x66, 0xc8, 0x5c, 0x55, 0xad, 0xd8, 0xc0, 0x40, 0x3f, 0x03, 0x10, + 0x3a, 0x91, 0xd3, 0x26, 0xca, 0x2f, 0x7b, 0x68, 0x7b, 0x81, 0xca, 0xb1, 0x22, 0x69, 0xea, 0xbd, + 0xa9, 0x6a, 0x8a, 0xb1, 0xc1, 0x12, 0x3d, 0x0b, 0xc3, 0x11, 0xf1, 0x88, 0x13, 0xb3, 0x28, 0xf2, + 0x6c, 0x4a, 0x0c, 0xd6, 0x20, 0x6c, 0xe2, 0xa1, 0xc7, 0x55, 0x08, 0x5a, 0x26, 0xa2, 0x25, 0x1d, + 0x86, 0x86, 0xde, 0xb4, 0x60, 0x6c, 0xc3, 0xf5, 0x88, 0xe6, 0x2e, 0x12, 0x58, 0x96, 0x0f, 0xff, + 0x92, 0xe7, 0x4d, 0xba, 0x5a, 0x43, 0xa6, 0x9a, 0x63, 0x9c, 0x61, 0x4f, 0x3f, 0xf3, 0x36, 0x89, + 0x98, 0x6a, 0x1d, 0x4c, 0x7f, 0xe6, 0xab, 0xbc, 0x19, 0x4b, 0x38, 0x9a, 0x81, 0xf1, 0xd0, 0x89, + 0xe3, 0xd9, 0x88, 0x34, 0x89, 0x9f, 0xb8, 0x8e, 0xc7, 0xd3, 0x4b, 0xaa, 0x3a, 0xbc, 0x7c, 0x25, + 0x0d, 0xc6, 0x59, 0x7c, 0xf4, 0x21, 0x78, 0x90, 0x3b, 0x3e, 0x96, 0xdc, 0x38, 0x76, 0xfd, 0x96, + 0x1e, 0x06, 0xc2, 0xff, 0x33, 0x29, 0x48, 0x3d, 0xb8, 0x90, 0x8f, 0x86, 0x7b, 0x3d, 0x8f, 0x9e, + 0x84, 0x6a, 0xbc, 0xe5, 0x86, 0xb3, 0x51, 0x33, 0x66, 0x87, 0x1e, 0x55, 0xed, 0x6d, 0x5c, 0x15, + 0xed, 0x58, 0x61, 0xa0, 0x06, 0x8c, 0xf0, 0x4f, 0xc2, 0x43, 0xd1, 0x84, 0x7e, 0x7c, 0xaa, 0xe7, + 0xf2, 0x28, 0xb2, 0x25, 0xa7, 0xb0, 0x73, 0xe3, 0x9c, 0x3c, 0x82, 0xe1, 0x27, 0x06, 0x57, 0x0d, + 0x32, 0x38, 0x45, 0xd4, 0xfe, 0xc5, 0x52, 0x7a, 0xc7, 0x6d, 0x4e, 0x52, 0x14, 0xd3, 0xa9, 0x98, + 0x5c, 0x75, 0x54, 0x48, 0xf0, 0x21, 0xb3, 0x60, 0x04, 0xdd, 0xab, 0x4e, 0x64, 0x4e, 0x6a, 0xc6, + 0x00, 0x4b, 0x4e, 0xe8, 0x3a, 0x0c, 0x24, 0x9e, 0x53, 0x50, 0xda, 0x9c, 0xc1, 0x51, 0x3b, 0x40, + 0x16, 0x67, 0x62, 0xcc, 0x78, 0xa0, 0x87, 0xa9, 0xd5, 0xbf, 0x2e, 0x8f, 0x48, 0x84, 0xa1, 0xbe, + 0x1e, 0x63, 0xd6, 0x6a, 0xdf, 0x86, 0x1c, 0xbd, 0xaa, 0x16, 0x32, 0x74, 0x16, 0x80, 0x05, 0x24, + 0x46, 0x64, 0xc3, 0xbd, 0x29, 0x0c, 0x09, 0x35, 0x77, 0x2f, 0x2b, 0x08, 0x36, 0xb0, 0xe4, 0x33, + 0xab, 0x9d, 0x0d, 0xfa, 0x4c, 0xa9, 0xfb, 0x19, 0x0e, 0xc1, 0x06, 0x16, 0x7a, 0x06, 0x06, 0xdd, + 0xb6, 0xd3, 0x52, 0xb1, 0xa3, 0x0f, 0xd3, 0x49, 0xbb, 0xc0, 0x5a, 0x6e, 0xef, 0x4e, 0x8e, 0x29, + 0x81, 0x58, 0x13, 0x16, 0xb8, 0xe8, 0x57, 0x2d, 0x18, 0x69, 0x04, 0xed, 0x76, 0xe0, 0xf3, 0x6d, + 0x97, 0xd8, 0x43, 0x5e, 0x3f, 0xaa, 0x65, 0x7e, 0x6a, 0xd6, 0x60, 0xc6, 0x37, 0x91, 0x2a, 0xbf, + 0xcf, 0x04, 0xe1, 0x94, 0x54, 0xe6, 0xdc, 0xae, 0xec, 0x33, 0xb7, 0x7f, 0xc3, 0x82, 0xe3, 0xfc, + 0x59, 0x63, 0x37, 0x28, 0x52, 0xd9, 0x82, 0x23, 0x7e, 0xad, 0xae, 0x0d, 0xb2, 0xf2, 0xd2, 0x75, + 0xc1, 0x71, 0xb7, 0x90, 0x68, 0x1e, 0x8e, 0x6f, 0x04, 0x51, 0x83, 0x98, 0x1d, 0x21, 0x14, 0x93, + 0x22, 0x74, 0x3e, 0x8b, 0x80, 0xbb, 0x9f, 0x41, 0x57, 0xe1, 0x01, 0xa3, 0xd1, 0xec, 0x07, 0xae, + 0x9b, 0x1e, 0x15, 0xd4, 0x1e, 0x38, 0x9f, 0x8b, 0x85, 0x7b, 0x3c, 0x9d, 0x76, 0x98, 0xd4, 0xfa, + 0x70, 0x98, 0xbc, 0x0a, 0x0f, 0x35, 0xba, 0x7b, 0x66, 0x3b, 0xee, 0xac, 0xc7, 0x5c, 0x53, 0x55, + 0xeb, 0x3f, 0x22, 0x08, 0x3c, 0x34, 0xdb, 0x0b, 0x11, 0xf7, 0xa6, 0x81, 0x3e, 0x06, 0xd5, 0x88, + 0xb0, 0xaf, 0x12, 0x8b, 0xbc, 0xae, 0x43, 0xee, 0x92, 0xb5, 0x05, 0xca, 0xc9, 0x6a, 0xdd, 0x2b, + 0x1a, 0x62, 0xac, 0x38, 0xa2, 0x1b, 0x30, 0x14, 0x3a, 0x49, 0x63, 0x53, 0x64, 0x73, 0x1d, 0x3a, + 0xfe, 0x45, 0x31, 0x67, 0x3e, 0x70, 0x23, 0xff, 0x9b, 0x33, 0xc1, 0x92, 0x1b, 0xb5, 0x46, 0x1a, + 0x41, 0x3b, 0x0c, 0x7c, 0xe2, 0x27, 0xf1, 0xc4, 0xa8, 0xb6, 0x46, 0x66, 0x55, 0x2b, 0x36, 0x30, + 0xd0, 0x0a, 0x9c, 0x64, 0x3e, 0xa3, 0x6b, 0x6e, 0xb2, 0x19, 0x74, 0x12, 0x15, 0xae, 0x3d, 0x96, + 0x3e, 0xaa, 0x58, 0xcc, 0xc1, 0xc1, 0xb9, 0x4f, 0x9e, 0xfe, 0x00, 0x1c, 0xef, 0x9a, 0xca, 0x07, + 0x72, 0xd7, 0xcc, 0xc1, 0x03, 0xf9, 0x93, 0xe6, 0x40, 0x4e, 0x9b, 0x7f, 0x98, 0x09, 0x9b, 0x35, + 0x0c, 0xe9, 0x3e, 0x1c, 0x80, 0x0e, 0x94, 0x89, 0xbf, 0x2d, 0xd6, 0x90, 0xf3, 0x87, 0xfb, 0x76, + 0xe7, 0xfc, 0x6d, 0x3e, 0xe7, 0x99, 0x97, 0xe3, 0x9c, 0xbf, 0x8d, 0x29, 0x6d, 0xf4, 0x65, 0x2b, + 0x65, 0x08, 0x72, 0xb7, 0xe1, 0x47, 0x8e, 0x64, 0xe7, 0xd0, 0xb7, 0x6d, 0x68, 0xff, 0x9b, 0x12, + 0x9c, 0xd9, 0x8f, 0x48, 0x1f, 0xdd, 0xf7, 0x18, 0x0c, 0xc6, 0xec, 0x20, 0x5c, 0x28, 0xe5, 0x61, + 0x3a, 0x56, 0xf9, 0xd1, 0xf8, 0xab, 0x58, 0x80, 0x90, 0x07, 0xe5, 0xb6, 0x13, 0x0a, 0x6f, 0xd2, + 0xc2, 0x61, 0xf3, 0xb2, 0xe8, 0x7f, 0xc7, 0x5b, 0x72, 0x42, 0xee, 0xa3, 0x30, 0x1a, 0x30, 0x65, + 0x83, 0x12, 0xa8, 0x38, 0x51, 0xe4, 0xc8, 0x53, 0xd7, 0x4b, 0xc5, 0xf0, 0x9b, 0xa1, 0x24, 0xf9, + 0xa1, 0x55, 0xaa, 0x09, 0x73, 0x66, 0xf6, 0x9b, 0xd5, 0x54, 0x16, 0x0c, 0x3b, 0x4a, 0x8f, 0x61, + 0x50, 0x38, 0x91, 0xac, 0xa2, 0xd3, 0xe1, 0x78, 0x72, 0x29, 0xdb, 0x27, 0x8a, 0x14, 0x7d, 0xc1, + 0x0a, 0x7d, 0xce, 0x62, 0x89, 0xf0, 0x32, 0x33, 0x48, 0xec, 0xce, 0x8e, 0x26, 0x2f, 0xdf, 0x4c, + 0xaf, 0x97, 0x8d, 0xd8, 0xe4, 0x2e, 0x0a, 0x5a, 0x30, 0xab, 0xb4, 0xbb, 0xa0, 0x05, 0xb3, 0x32, + 0x25, 0x1c, 0xdd, 0xcc, 0x39, 0x32, 0x2f, 0x20, 0x99, 0xba, 0x8f, 0x43, 0xf2, 0xaf, 0x5b, 0x70, + 0xdc, 0xcd, 0x9e, 0x7d, 0x8a, 0xbd, 0xcc, 0x21, 0x83, 0x32, 0x7a, 0x1f, 0xad, 0xaa, 0xe5, 0xbc, + 0x0b, 0x84, 0xbb, 0x85, 0x41, 0x4d, 0x18, 0x70, 0xfd, 0x8d, 0x40, 0x18, 0x31, 0xf5, 0xc3, 0x09, + 0xb5, 0xe0, 0x6f, 0x04, 0x7a, 0x36, 0xd3, 0x7f, 0x98, 0x51, 0x47, 0x8b, 0x70, 0x32, 0x12, 0xde, + 0xa6, 0x0b, 0x6e, 0x9c, 0x04, 0xd1, 0xce, 0xa2, 0xdb, 0x76, 0x13, 0x66, 0x80, 0x94, 0xeb, 0x13, + 0x74, 0x7d, 0xc0, 0x39, 0x70, 0x9c, 0xfb, 0x14, 0x7a, 0x1d, 0x86, 0x64, 0xe6, 0x7e, 0xb5, 0x88, + 0x7d, 0x61, 0xf7, 0xf8, 0x57, 0x83, 0x69, 0x55, 0x24, 0xe9, 0x4b, 0x86, 0xe8, 0x0d, 0x0b, 0x6a, + 0x4d, 0x96, 0x3b, 0x17, 0x2f, 0xfb, 0x22, 0x6e, 0x6c, 0xb5, 0xc0, 0x39, 0x20, 0xf3, 0x21, 0x85, + 0xaf, 0x48, 0x72, 0xc2, 0x9a, 0xa9, 0xfd, 0xe6, 0x30, 0x74, 0x9f, 0xcc, 0xa2, 0x8f, 0x43, 0x2d, + 0x52, 0x05, 0x0d, 0xac, 0x22, 0x2c, 0x06, 0x39, 0xc4, 0xc4, 0xa9, 0xb0, 0xb2, 0xc6, 0x74, 0xe9, + 0x02, 0xcd, 0x91, 0xee, 0x99, 0x62, 0x7d, 0x80, 0x5b, 0xc0, 0xf4, 0x12, 0x5c, 0xf5, 0xe1, 0xdc, + 0x8e, 0xdf, 0xc0, 0x8c, 0x07, 0x8a, 0x60, 0x70, 0x93, 0x38, 0x5e, 0xb2, 0x59, 0xcc, 0x39, 0xc2, + 0x05, 0x46, 0x2b, 0x9b, 0x87, 0xc4, 0x5b, 0xb1, 0xe0, 0x84, 0x6e, 0xc2, 0xd0, 0x26, 0x1f, 0x83, + 0x62, 0x1b, 0xb3, 0x74, 0xd8, 0xce, 0x4d, 0x0d, 0x6c, 0x3d, 0xe2, 0x44, 0x03, 0x96, 0xec, 0x58, + 0xc8, 0x8f, 0x11, 0x94, 0xc0, 0xb5, 0x47, 0x71, 0x29, 0x58, 0xfd, 0x47, 0x24, 0x7c, 0x14, 0x46, + 0x22, 0xd2, 0x08, 0xfc, 0x86, 0xeb, 0x91, 0xe6, 0x8c, 0x3c, 0x23, 0x38, 0x48, 0xe2, 0x0e, 0x73, + 0x05, 0x60, 0x83, 0x06, 0x4e, 0x51, 0x44, 0x9f, 0xb5, 0x60, 0x4c, 0x65, 0x41, 0xd3, 0x0f, 0x42, + 0x84, 0x4f, 0x7a, 0xb1, 0xa0, 0x9c, 0x6b, 0x46, 0xb3, 0x8e, 0x6e, 0xed, 0x4e, 0x8e, 0xa5, 0xdb, + 0x70, 0x86, 0x2f, 0x7a, 0x19, 0x20, 0x58, 0xe7, 0x71, 0x3d, 0x33, 0x89, 0x70, 0x50, 0x1f, 0xe4, + 0x55, 0xc7, 0x78, 0x06, 0x9f, 0xa4, 0x80, 0x0d, 0x6a, 0xe8, 0x12, 0x00, 0x9f, 0x36, 0x6b, 0x3b, + 0xa1, 0xdc, 0xeb, 0xc8, 0xcc, 0x2b, 0x58, 0x55, 0x90, 0xdb, 0xbb, 0x93, 0xdd, 0x0e, 0x43, 0x16, + 0x3b, 0x61, 0x3c, 0x8e, 0x7e, 0x1a, 0x86, 0xe2, 0x4e, 0xbb, 0xed, 0x28, 0xf7, 0x75, 0x81, 0x39, + 0x81, 0x9c, 0xae, 0xa1, 0x0d, 0x79, 0x03, 0x96, 0x1c, 0xd1, 0x75, 0xaa, 0xd7, 0x63, 0xe1, 0xc9, + 0x64, 0xb3, 0x88, 0x9b, 0x25, 0xc3, 0xec, 0x9d, 0x9e, 0x93, 0xb6, 0x3f, 0xce, 0xc1, 0xb9, 0xbd, + 0x3b, 0xf9, 0x40, 0xba, 0x7d, 0x31, 0x10, 0x59, 0x7a, 0xb9, 0x34, 0xd1, 0x45, 0x59, 0x4b, 0x88, + 0xbe, 0xb6, 0x2c, 0x71, 0xf1, 0x84, 0xae, 0x25, 0xc4, 0x9a, 0x7b, 0xf7, 0x99, 0xf9, 0x30, 0x5a, + 0x82, 0x13, 0x8d, 0xc0, 0x4f, 0xa2, 0xc0, 0xf3, 0x78, 0x2d, 0x2d, 0xbe, 0xed, 0xe4, 0xee, 0xed, + 0x77, 0x0a, 0xb1, 0x4f, 0xcc, 0x76, 0xa3, 0xe0, 0xbc, 0xe7, 0x6c, 0x3f, 0x1d, 0xf0, 0x28, 0x3a, + 0xe7, 0x19, 0x18, 0x21, 0x37, 0x13, 0x12, 0xf9, 0x8e, 0x77, 0x05, 0x2f, 0x4a, 0xc7, 0x2e, 0x9b, + 0x03, 0xe7, 0x8c, 0x76, 0x9c, 0xc2, 0x42, 0xb6, 0xf2, 0xb5, 0x18, 0x29, 0xb3, 0xdc, 0xd7, 0x22, + 0x3d, 0x2b, 0xf6, 0xff, 0x2a, 0xa5, 0x6c, 0xc2, 0xb5, 0x88, 0x10, 0x14, 0x40, 0xc5, 0x0f, 0x9a, + 0x4a, 0xf7, 0x5f, 0x2c, 0x46, 0xf7, 0x5f, 0x0e, 0x9a, 0x46, 0xc1, 0x21, 0xfa, 0x2f, 0xc6, 0x9c, + 0x0f, 0xab, 0xc8, 0x22, 0x4b, 0xd7, 0x30, 0x80, 0xd8, 0xeb, 0x14, 0xc9, 0x59, 0x55, 0x64, 0x59, + 0x36, 0x19, 0xe1, 0x34, 0x5f, 0xb4, 0x05, 0x95, 0xcd, 0x20, 0x4e, 0xe4, 0x0e, 0xe8, 0x90, 0x9b, + 0xad, 0x0b, 0x41, 0x9c, 0x30, 0x43, 0x46, 0xbd, 0x36, 0x6d, 0x89, 0x31, 0xe7, 0x61, 0xff, 0x67, + 0x2b, 0xe5, 0xc6, 0xbf, 0xc6, 0x82, 0x7f, 0xb7, 0x89, 0x4f, 0xa7, 0xb5, 0x19, 0xed, 0xf4, 0xe3, + 0x99, 0x54, 0xca, 0x77, 0xf5, 0x2a, 0x15, 0x77, 0x83, 0x52, 0x98, 0x62, 0x24, 0x8c, 0xc0, 0xa8, + 0x37, 0xac, 0x74, 0x52, 0x6b, 0xa9, 0x88, 0x3d, 0x8e, 0x99, 0x91, 0xbe, 0x6f, 0x7e, 0xac, 0xfd, + 0x65, 0x0b, 0x86, 0xea, 0x4e, 0x63, 0x2b, 0xd8, 0xd8, 0x40, 0x4f, 0x42, 0xb5, 0x29, 0x2a, 0x10, + 0x88, 0xf7, 0x53, 0xbe, 0x0b, 0x55, 0x99, 0x40, 0x61, 0xd0, 0x31, 0xbc, 0xe1, 0x34, 0x64, 0x5e, + 0x7a, 0x99, 0x8f, 0xe1, 0xf3, 0xac, 0x05, 0x0b, 0x08, 0x7a, 0x16, 0x86, 0xdb, 0xce, 0x4d, 0xf9, + 0x70, 0xf6, 0x0c, 0x61, 0x49, 0x83, 0xb0, 0x89, 0x67, 0xff, 0x0b, 0x0b, 0x26, 0xea, 0x4e, 0xec, + 0x36, 0x66, 0x3a, 0xc9, 0x66, 0xdd, 0x4d, 0xd6, 0x3b, 0x8d, 0x2d, 0x92, 0xf0, 0x62, 0x04, 0x54, + 0xca, 0x4e, 0x4c, 0xa7, 0x92, 0xda, 0x5a, 0x2a, 0x29, 0xaf, 0x88, 0x76, 0xac, 0x30, 0xd0, 0xeb, + 0x30, 0x1c, 0x3a, 0x71, 0x7c, 0x23, 0x88, 0x9a, 0x98, 0x6c, 0x14, 0x53, 0xa0, 0x65, 0x95, 0x34, + 0x22, 0x92, 0x60, 0xb2, 0x21, 0xce, 0xb9, 0x35, 0x7d, 0x6c, 0x32, 0xb3, 0xbf, 0x68, 0xc1, 0x43, + 0x75, 0xe2, 0x44, 0x24, 0x62, 0xf5, 0x5c, 0xd4, 0x8b, 0xcc, 0x7a, 0x41, 0xa7, 0x89, 0x5e, 0x83, + 0x6a, 0x42, 0x9b, 0xa9, 0x58, 0x56, 0xb1, 0x62, 0xb1, 0x63, 0xea, 0x35, 0x41, 0x1c, 0x2b, 0x36, + 0xf6, 0x5f, 0xb3, 0x60, 0x84, 0x9d, 0xf8, 0xcd, 0x91, 0xc4, 0x71, 0xbd, 0xae, 0xb2, 0x67, 0x56, + 0x9f, 0x65, 0xcf, 0xce, 0xc0, 0xc0, 0x66, 0xd0, 0x26, 0xd9, 0xd3, 0xea, 0x0b, 0x01, 0xdd, 0xd9, + 0x53, 0x08, 0x7a, 0x9a, 0x7e, 0x78, 0xd7, 0x4f, 0x1c, 0x3a, 0x05, 0xa4, 0x47, 0x79, 0x9c, 0x7f, + 0x74, 0xd5, 0x8c, 0x4d, 0x1c, 0xfb, 0xb7, 0x6b, 0x30, 0x24, 0x42, 0x1a, 0xfa, 0x2e, 0x48, 0x21, + 0x5d, 0x0c, 0xa5, 0x9e, 0x2e, 0x86, 0x18, 0x06, 0x1b, 0xac, 0xfe, 0xa2, 0x30, 0x23, 0x2f, 0x15, + 0x12, 0x03, 0xc3, 0x4b, 0x3a, 0x6a, 0xb1, 0xf8, 0x7f, 0x2c, 0x58, 0xa1, 0x2f, 0x59, 0x30, 0xde, + 0x08, 0x7c, 0x9f, 0x34, 0xb4, 0x8d, 0x33, 0x50, 0x44, 0xa8, 0xc3, 0x6c, 0x9a, 0xa8, 0x3e, 0x6e, + 0xca, 0x00, 0x70, 0x96, 0x3d, 0x7a, 0x01, 0x46, 0x79, 0x9f, 0x5d, 0x4d, 0xb9, 0xc1, 0x75, 0x35, + 0x2c, 0x13, 0x88, 0xd3, 0xb8, 0x68, 0x8a, 0x1f, 0x27, 0x88, 0xba, 0x53, 0x83, 0xda, 0x5b, 0x68, + 0x54, 0x9c, 0x32, 0x30, 0x50, 0x04, 0x48, 0x54, 0x20, 0x11, 0x21, 0x1f, 0xcc, 0xbe, 0x1a, 0xba, + 0xb3, 0x1c, 0x70, 0xdc, 0x45, 0x09, 0xe7, 0x50, 0x47, 0x5b, 0x62, 0x8f, 0x5b, 0x2d, 0x42, 0x87, + 0x8a, 0xcf, 0xdc, 0x73, 0xab, 0x3b, 0x09, 0x95, 0x78, 0xd3, 0x89, 0x9a, 0xcc, 0xae, 0x2b, 0xf3, + 0xbc, 0xa3, 0x55, 0xda, 0x80, 0x79, 0x3b, 0x9a, 0x83, 0x63, 0x99, 0x5a, 0x5e, 0xb1, 0x70, 0x57, + 0xab, 0x1c, 0x93, 0x4c, 0x15, 0xb0, 0x18, 0x77, 0x3d, 0x61, 0xfa, 0x3f, 0x86, 0xf7, 0xf1, 0x7f, + 0xec, 0xa8, 0xc0, 0xc2, 0x91, 0x22, 0xaa, 0xea, 0x08, 0xe1, 0xfa, 0x8a, 0x22, 0xfc, 0x42, 0x26, + 0x8a, 0x70, 0x94, 0x09, 0x70, 0xb5, 0x18, 0x01, 0x0e, 0x1e, 0x32, 0x78, 0x2f, 0x43, 0x00, 0xff, + 0xdc, 0x02, 0xf9, 0x5d, 0x67, 0x9d, 0xc6, 0x26, 0xa1, 0x43, 0x06, 0xbd, 0x08, 0x63, 0x6a, 0x0b, + 0x3d, 0x1b, 0x74, 0x7c, 0x1e, 0xfd, 0x57, 0xd6, 0xe7, 0xd2, 0x38, 0x05, 0xc5, 0x19, 0x6c, 0x34, + 0x0d, 0x35, 0xda, 0x4f, 0xfc, 0x51, 0xbe, 0xd6, 0xaa, 0x6d, 0xfa, 0xcc, 0xca, 0x82, 0x78, 0x4a, + 0xe3, 0xa0, 0x00, 0x8e, 0x7b, 0x4e, 0x9c, 0x30, 0x09, 0xe8, 0x8e, 0xfa, 0x0e, 0x2b, 0x30, 0xb0, + 0x44, 0x86, 0xc5, 0x2c, 0x21, 0xdc, 0x4d, 0xdb, 0xfe, 0xee, 0x00, 0x8c, 0xa6, 0x34, 0xe3, 0x01, + 0x17, 0xe9, 0x27, 0xa1, 0x2a, 0xd7, 0xcd, 0x6c, 0x91, 0x1b, 0xb5, 0xb8, 0x2a, 0x0c, 0xba, 0x68, + 0xad, 0xeb, 0x55, 0x35, 0x6b, 0x54, 0x18, 0x0b, 0x2e, 0x36, 0xf1, 0x98, 0x52, 0x4e, 0xbc, 0x78, + 0xd6, 0x73, 0x89, 0x9f, 0x70, 0x31, 0x8b, 0x51, 0xca, 0x6b, 0x8b, 0xab, 0x26, 0x51, 0xad, 0x94, + 0x33, 0x00, 0x9c, 0x65, 0x8f, 0x3e, 0x63, 0xc1, 0xa8, 0x73, 0x23, 0xd6, 0x45, 0x82, 0x45, 0xbc, + 0xe0, 0x21, 0x17, 0xa9, 0x54, 0xdd, 0x61, 0xee, 0x75, 0x4e, 0x35, 0xe1, 0x34, 0x53, 0xf4, 0x96, + 0x05, 0x88, 0xdc, 0x24, 0x0d, 0x19, 0xd1, 0x28, 0x64, 0x19, 0x2c, 0x62, 0xa7, 0x79, 0xae, 0x8b, + 0x2e, 0xd7, 0xea, 0xdd, 0xed, 0x38, 0x47, 0x06, 0xfb, 0x9f, 0x96, 0xd5, 0x84, 0xd2, 0x41, 0xb4, + 0x8e, 0x11, 0xcc, 0x77, 0x88, 0x7a, 0x41, 0x3a, 0x28, 0xa2, 0x3b, 0xb1, 0x33, 0x95, 0x07, 0x56, + 0xba, 0x47, 0x79, 0x60, 0x9f, 0xb2, 0x52, 0xe5, 0x9c, 0x86, 0xcf, 0xbe, 0x5c, 0x6c, 0x00, 0xef, + 0x14, 0x0f, 0xd8, 0xc8, 0x68, 0xf7, 0x74, 0x9c, 0x0e, 0xd5, 0xa6, 0x06, 0xda, 0x81, 0xb4, 0xe1, + 0xbf, 0x2b, 0xc3, 0xb0, 0xb1, 0x92, 0xe6, 0x9a, 0x45, 0xd6, 0x7d, 0x66, 0x16, 0x95, 0x0e, 0x60, + 0x16, 0xfd, 0x0c, 0xd4, 0x1a, 0x52, 0xcb, 0x17, 0x53, 0x66, 0x3a, 0xbb, 0x76, 0x68, 0x45, 0xaf, + 0x9a, 0xb0, 0xe6, 0x89, 0xe6, 0x53, 0xd9, 0x43, 0x62, 0x85, 0xe0, 0xf5, 0xe5, 0xf2, 0xd2, 0x7b, + 0xc4, 0x4a, 0xd1, 0xfd, 0x0c, 0xab, 0xfa, 0x15, 0xba, 0xe2, 0xbd, 0x64, 0x98, 0x3d, 0xaf, 0xfa, + 0xb5, 0xb2, 0x20, 0x9b, 0xb1, 0x89, 0x63, 0x7f, 0xd7, 0x52, 0x1f, 0xf7, 0x2e, 0x94, 0x89, 0xb8, + 0x9e, 0x2e, 0x13, 0x71, 0xae, 0x90, 0x6e, 0xee, 0x51, 0x1f, 0xe2, 0x32, 0x0c, 0xcd, 0x06, 0xed, + 0xb6, 0xe3, 0x37, 0xd1, 0x8f, 0xc2, 0x50, 0x83, 0xff, 0x14, 0x8e, 0x1d, 0x76, 0x42, 0x29, 0xa0, + 0x58, 0xc2, 0xd0, 0xc3, 0x30, 0xe0, 0x44, 0x2d, 0xe9, 0xcc, 0x61, 0xf1, 0x3d, 0x33, 0x51, 0x2b, + 0xc6, 0xac, 0xd5, 0xfe, 0x07, 0x03, 0xc0, 0x8e, 0xd5, 0x9d, 0x88, 0x34, 0xd7, 0x02, 0x56, 0xe6, + 0xf2, 0x48, 0xcf, 0xf5, 0xf4, 0x66, 0xe9, 0x7e, 0x3e, 0xdb, 0x33, 0xce, 0x77, 0xca, 0x77, 0xfb, + 0x7c, 0x27, 0xff, 0xc8, 0x6e, 0xe0, 0x3e, 0x3a, 0xb2, 0xb3, 0x3f, 0x6f, 0x01, 0x52, 0xb1, 0x18, + 0xfa, 0x4c, 0x7d, 0x1a, 0x6a, 0x2a, 0x2a, 0x43, 0x18, 0x56, 0x5a, 0x45, 0x48, 0x00, 0xd6, 0x38, + 0x7d, 0xec, 0x90, 0x1f, 0x93, 0xfa, 0xbb, 0x9c, 0x0e, 0x0d, 0x66, 0x5a, 0x5f, 0xa8, 0x73, 0xfb, + 0x77, 0x4a, 0xf0, 0x00, 0x5f, 0x92, 0x97, 0x1c, 0xdf, 0x69, 0x91, 0x36, 0x95, 0xaa, 0xdf, 0x28, + 0x89, 0x06, 0xdd, 0x9a, 0xb9, 0x32, 0xd4, 0xf7, 0xb0, 0x73, 0x97, 0xcf, 0x39, 0x3e, 0xcb, 0x16, + 0x7c, 0x37, 0xc1, 0x8c, 0x38, 0x8a, 0xa1, 0x2a, 0xef, 0x60, 0x10, 0xba, 0xb8, 0x20, 0x46, 0x4a, + 0x2d, 0x89, 0x75, 0x93, 0x60, 0xc5, 0x88, 0x1a, 0xae, 0x5e, 0xd0, 0xd8, 0xc2, 0x24, 0x0c, 0x98, + 0xde, 0x35, 0x22, 0x2d, 0x17, 0x45, 0x3b, 0x56, 0x18, 0xf6, 0xef, 0x58, 0x90, 0x5d, 0x91, 0x8c, + 0x02, 0x70, 0xd6, 0x9e, 0x05, 0xe0, 0x0e, 0x50, 0x81, 0xed, 0xa7, 0x60, 0xd8, 0x49, 0xa8, 0x11, + 0xc1, 0xb7, 0xdd, 0xe5, 0x3b, 0x3b, 0xd6, 0x58, 0x0a, 0x9a, 0xee, 0x86, 0xcb, 0xb6, 0xdb, 0x26, + 0x39, 0xfb, 0xbf, 0x0f, 0xc0, 0xf1, 0xae, 0x84, 0x14, 0xf4, 0x3c, 0x8c, 0x34, 0xc4, 0xf0, 0x08, + 0xa5, 0x43, 0xab, 0x66, 0x46, 0xe6, 0x69, 0x18, 0x4e, 0x61, 0xf6, 0x31, 0x40, 0x59, 0x0d, 0xd4, + 0xd7, 0x3a, 0xa4, 0x43, 0x66, 0x36, 0x12, 0x12, 0xad, 0x92, 0x46, 0xe0, 0x37, 0x63, 0x51, 0x83, + 0x55, 0xd4, 0x40, 0xed, 0x02, 0xe3, 0xbc, 0x67, 0x50, 0x98, 0x2d, 0x37, 0x39, 0x70, 0xe7, 0xe6, + 0xa3, 0xb2, 0x11, 0xf6, 0x2c, 0x39, 0x99, 0x32, 0x24, 0x2b, 0xf7, 0xc8, 0x90, 0xfc, 0xb4, 0x36, + 0x24, 0x79, 0x08, 0xc0, 0x87, 0x0b, 0x4e, 0x48, 0x3a, 0x6a, 0x4b, 0xf2, 0x25, 0xa8, 0xca, 0xf0, + 0xa8, 0xbe, 0xc2, 0x8a, 0x4c, 0x3a, 0x3d, 0x34, 0xda, 0xed, 0x12, 0xe4, 0x6c, 0x42, 0xe8, 0x3c, + 0xd3, 0x2b, 0x7e, 0x6a, 0x9e, 0x1d, 0x6c, 0xd5, 0x47, 0x37, 0x79, 0x68, 0x18, 0x5f, 0xdb, 0x3e, + 0x54, 0xf4, 0x26, 0x4a, 0x47, 0x8b, 0xa9, 0x3c, 0x0d, 0x15, 0x31, 0x76, 0x16, 0x40, 0x1b, 0x6a, + 0x22, 0x0a, 0x5f, 0x1d, 0xfb, 0x6a, 0x7b, 0x0e, 0x1b, 0x58, 0x74, 0x4f, 0xed, 0xfa, 0x71, 0xe2, + 0x78, 0xde, 0x05, 0xd7, 0x4f, 0x84, 0x73, 0x50, 0x2d, 0xe2, 0x0b, 0x1a, 0x84, 0x4d, 0xbc, 0xd3, + 0xcf, 0x19, 0xdf, 0xe5, 0x20, 0xdf, 0x73, 0x13, 0x1e, 0x9a, 0x77, 0x13, 0x95, 0x3b, 0xa2, 0xc6, + 0x11, 0xb5, 0xc3, 0x54, 0x2e, 0x94, 0xd5, 0x33, 0x17, 0xca, 0xc8, 0xdd, 0x28, 0xa5, 0x53, 0x4d, + 0xb2, 0xb9, 0x1b, 0xf6, 0xf3, 0x70, 0x72, 0xde, 0x4d, 0xce, 0xbb, 0x1e, 0x39, 0x20, 0x13, 0xfb, + 0xb7, 0x06, 0x61, 0xc4, 0xcc, 0x3e, 0x3c, 0x48, 0x3a, 0xd7, 0x17, 0xa9, 0xa9, 0x25, 0xde, 0xce, + 0x55, 0x87, 0x66, 0xd7, 0x0e, 0x9d, 0x0a, 0x99, 0xdf, 0x63, 0x86, 0xb5, 0xa5, 0x79, 0x62, 0x53, + 0x00, 0x74, 0x03, 0x2a, 0x1b, 0x2c, 0xb7, 0xa0, 0x5c, 0x44, 0x64, 0x41, 0x5e, 0x8f, 0xea, 0x69, + 0xc6, 0xb3, 0x13, 0x38, 0x3f, 0xba, 0x42, 0x46, 0xe9, 0x84, 0x35, 0x23, 0x1e, 0x56, 0xa4, 0xaa, + 0x29, 0x8c, 0x5e, 0xaa, 0xbe, 0x72, 0x07, 0xaa, 0x3e, 0xa5, 0x78, 0x07, 0xef, 0x91, 0xe2, 0x65, + 0x79, 0x22, 0xc9, 0x26, 0xb3, 0xdf, 0x44, 0x00, 0xff, 0x10, 0xeb, 0x04, 0x23, 0x4f, 0x24, 0x05, + 0xc6, 0x59, 0x7c, 0xf4, 0x09, 0xa5, 0xba, 0xab, 0x45, 0xf8, 0x55, 0xcd, 0x11, 0x7d, 0xd4, 0x5a, + 0xfb, 0xf3, 0x25, 0x18, 0x9b, 0xf7, 0x3b, 0x2b, 0xf3, 0x2b, 0x9d, 0x75, 0xcf, 0x6d, 0x5c, 0x22, + 0x3b, 0x54, 0x35, 0x6f, 0x91, 0x9d, 0x85, 0x39, 0x31, 0x83, 0xd4, 0x98, 0xb9, 0x44, 0x1b, 0x31, + 0x87, 0x51, 0x65, 0xb4, 0xe1, 0xfa, 0x2d, 0x12, 0x85, 0x91, 0x2b, 0x5c, 0x9e, 0x86, 0x32, 0x3a, + 0xaf, 0x41, 0xd8, 0xc4, 0xa3, 0xb4, 0x83, 0x1b, 0x3e, 0x89, 0xb2, 0x86, 0xec, 0x32, 0x6d, 0xc4, + 0x1c, 0x46, 0x91, 0x92, 0xa8, 0x13, 0x27, 0x62, 0x30, 0x2a, 0xa4, 0x35, 0xda, 0x88, 0x39, 0x8c, + 0xce, 0xf4, 0xb8, 0xb3, 0xce, 0x02, 0x37, 0x32, 0xd9, 0x02, 0xab, 0xbc, 0x19, 0x4b, 0x38, 0x45, + 0xdd, 0x22, 0x3b, 0x73, 0x74, 0xd7, 0x9b, 0x49, 0x1a, 0xba, 0xc4, 0x9b, 0xb1, 0x84, 0xb3, 0xfa, + 0x8a, 0xe9, 0xee, 0xf8, 0x81, 0xab, 0xaf, 0x98, 0x16, 0xbf, 0xc7, 0xfe, 0xf9, 0x57, 0x2c, 0x18, + 0x31, 0xc3, 0xad, 0x50, 0x2b, 0x63, 0xe3, 0x2e, 0x77, 0x95, 0xe7, 0xfd, 0x89, 0xbc, 0xab, 0xf2, + 0x5a, 0x6e, 0x12, 0x84, 0xf1, 0x53, 0xc4, 0x6f, 0xb9, 0x3e, 0x61, 0xa7, 0xe8, 0x3c, 0x4c, 0x2b, + 0x15, 0xcb, 0x35, 0x1b, 0x34, 0xc9, 0x1d, 0x18, 0xc9, 0xf6, 0x35, 0x38, 0xde, 0x95, 0x29, 0xd6, + 0x87, 0x69, 0xb1, 0x6f, 0x9e, 0xae, 0x8d, 0x61, 0x98, 0x12, 0x96, 0x35, 0x7e, 0x66, 0xe1, 0x38, + 0x9f, 0x48, 0x94, 0xd3, 0x6a, 0x63, 0x93, 0xb4, 0x55, 0xf6, 0x1f, 0xf3, 0xaf, 0x5f, 0xcd, 0x02, + 0x71, 0x37, 0xbe, 0xfd, 0x05, 0x0b, 0x46, 0x53, 0xc9, 0x7b, 0x05, 0x19, 0x41, 0x6c, 0xa6, 0x05, + 0x2c, 0xfa, 0x8f, 0x45, 0x61, 0x97, 0xd9, 0x62, 0xaa, 0x67, 0x9a, 0x06, 0x61, 0x13, 0xcf, 0xfe, + 0x72, 0x09, 0xaa, 0x32, 0x82, 0xa2, 0x0f, 0x51, 0x3e, 0x67, 0xc1, 0xa8, 0x3a, 0xd3, 0x60, 0xce, + 0xb2, 0x52, 0x11, 0x99, 0x16, 0x54, 0x02, 0xb5, 0xdd, 0xf6, 0x37, 0x02, 0x6d, 0x91, 0x63, 0x93, + 0x19, 0x4e, 0xf3, 0x46, 0x57, 0x01, 0xe2, 0x9d, 0x38, 0x21, 0x6d, 0xc3, 0x6d, 0x67, 0x1b, 0x33, + 0x6e, 0xaa, 0x11, 0x44, 0x84, 0xce, 0xaf, 0xcb, 0x41, 0x93, 0xac, 0x2a, 0x4c, 0x6d, 0x42, 0xe9, + 0x36, 0x6c, 0x50, 0xb2, 0xff, 0x5e, 0x09, 0x8e, 0x65, 0x45, 0x42, 0x1f, 0x86, 0x11, 0xc9, 0xdd, + 0xb8, 0xf5, 0x4f, 0x86, 0x8d, 0x8c, 0x60, 0x03, 0x76, 0x7b, 0x77, 0x72, 0xb2, 0xfb, 0xda, 0xc5, + 0x29, 0x13, 0x05, 0xa7, 0x88, 0xf1, 0x83, 0x25, 0x71, 0x02, 0x5a, 0xdf, 0x99, 0x09, 0x43, 0x71, + 0x3a, 0x64, 0x1c, 0x2c, 0x99, 0x50, 0x9c, 0xc1, 0x46, 0x2b, 0x70, 0xd2, 0x68, 0xb9, 0x4c, 0xdc, + 0xd6, 0xe6, 0x7a, 0x10, 0xc9, 0x9d, 0xd5, 0xc3, 0x3a, 0xb0, 0xab, 0x1b, 0x07, 0xe7, 0x3e, 0x49, + 0x57, 0xfb, 0x86, 0x13, 0x3a, 0x0d, 0x37, 0xd9, 0x11, 0x7e, 0x48, 0xa5, 0x9b, 0x66, 0x45, 0x3b, + 0x56, 0x18, 0xf6, 0x12, 0x0c, 0xf4, 0x39, 0x82, 0xfa, 0xb2, 0xe8, 0x5f, 0x82, 0x2a, 0x25, 0x27, + 0xcd, 0xbb, 0x22, 0x48, 0x06, 0x50, 0x95, 0xb7, 0xf1, 0x20, 0x1b, 0xca, 0xae, 0x23, 0xcf, 0xee, + 0xd4, 0x6b, 0x2d, 0xc4, 0x71, 0x87, 0x6d, 0x92, 0x29, 0x10, 0x3d, 0x06, 0x65, 0x72, 0x33, 0xcc, + 0x1e, 0xd2, 0x9d, 0xbb, 0x19, 0xba, 0x11, 0x89, 0x29, 0x12, 0xb9, 0x19, 0xa2, 0xd3, 0x50, 0x72, + 0x9b, 0x62, 0x91, 0x02, 0x81, 0x53, 0x5a, 0x98, 0xc3, 0x25, 0xb7, 0x69, 0xdf, 0x84, 0x9a, 0xba, + 0xfe, 0x07, 0x6d, 0x49, 0xdd, 0x6d, 0x15, 0x11, 0xf2, 0x24, 0xe9, 0xf6, 0xd0, 0xda, 0x1d, 0x00, + 0x9d, 0xc5, 0x58, 0x94, 0x7e, 0x39, 0x03, 0x03, 0x8d, 0x40, 0x64, 0x58, 0x57, 0x35, 0x19, 0xa6, + 0xb4, 0x19, 0xc4, 0xbe, 0x06, 0x63, 0x97, 0xfc, 0xe0, 0x06, 0x2b, 0x36, 0xcf, 0x6a, 0xab, 0x51, + 0xc2, 0x1b, 0xf4, 0x47, 0xd6, 0x44, 0x60, 0x50, 0xcc, 0x61, 0xaa, 0xe8, 0x54, 0xa9, 0x57, 0xd1, + 0x29, 0xfb, 0x0d, 0x0b, 0x46, 0x54, 0x3a, 0xd4, 0xfc, 0xf6, 0x16, 0xa5, 0xdb, 0x8a, 0x82, 0x4e, + 0x98, 0xa5, 0xcb, 0x2e, 0xe8, 0xc2, 0x1c, 0x66, 0xe6, 0x09, 0x96, 0xf6, 0xc9, 0x13, 0x3c, 0x03, + 0x03, 0x5b, 0xae, 0xdf, 0xcc, 0xde, 0x6d, 0x72, 0xc9, 0xf5, 0x9b, 0x98, 0x41, 0xa8, 0x08, 0xc7, + 0x94, 0x08, 0x72, 0x41, 0x78, 0x1e, 0x46, 0xd6, 0x3b, 0xae, 0xd7, 0x94, 0x45, 0xe3, 0x32, 0x9e, + 0x92, 0xba, 0x01, 0xc3, 0x29, 0x4c, 0xba, 0xaf, 0x5b, 0x77, 0x7d, 0x27, 0xda, 0x59, 0xd1, 0x2b, + 0x90, 0x52, 0x4a, 0x75, 0x05, 0xc1, 0x06, 0x96, 0xfd, 0x66, 0x19, 0xc6, 0xd2, 0x49, 0x61, 0x7d, + 0x6c, 0xaf, 0x1e, 0x83, 0x0a, 0xcb, 0x13, 0xcb, 0x7e, 0x5a, 0x5e, 0x67, 0x8d, 0xc3, 0x50, 0x0c, + 0x83, 0xbc, 0xc2, 0x44, 0x31, 0xb7, 0x35, 0x29, 0x21, 0x95, 0x7f, 0x85, 0xc5, 0x93, 0x89, 0xa2, + 0x16, 0x82, 0x15, 0xfa, 0x8c, 0x05, 0x43, 0x41, 0x68, 0x16, 0x2b, 0xfa, 0x50, 0x91, 0x09, 0x73, + 0x22, 0x5f, 0x47, 0x58, 0xc4, 0xea, 0xd3, 0xcb, 0xcf, 0x21, 0x59, 0x9f, 0x7e, 0x1f, 0x8c, 0x98, + 0x98, 0xfb, 0x19, 0xc5, 0x55, 0xd3, 0x28, 0xfe, 0x9c, 0x39, 0x28, 0x44, 0x4a, 0x60, 0x1f, 0xd3, + 0xed, 0x0a, 0x54, 0x1a, 0x2a, 0x00, 0xe0, 0x8e, 0x4a, 0x8d, 0xaa, 0x92, 0x0f, 0xec, 0x10, 0x88, + 0x53, 0xb3, 0xbf, 0x6b, 0x19, 0xe3, 0x03, 0x93, 0x78, 0xa1, 0x89, 0x22, 0x28, 0xb7, 0xb6, 0xb7, + 0x84, 0x29, 0x7a, 0xb1, 0xa0, 0xee, 0x9d, 0xdf, 0xde, 0xd2, 0x63, 0xdc, 0x6c, 0xc5, 0x94, 0x59, + 0x1f, 0x4e, 0xc0, 0x54, 0xe6, 0x68, 0x79, 0xff, 0xcc, 0x51, 0xfb, 0xad, 0x12, 0x1c, 0xef, 0x1a, + 0x54, 0xe8, 0x75, 0xa8, 0x44, 0xf4, 0x2d, 0xc5, 0xeb, 0x2d, 0x16, 0x96, 0xeb, 0x19, 0x2f, 0x34, + 0xf5, 0xba, 0x9b, 0x6e, 0xc7, 0x9c, 0x25, 0xba, 0x08, 0x48, 0x87, 0xa9, 0x28, 0x0f, 0x24, 0x7f, + 0xe5, 0xd3, 0xe2, 0x51, 0x34, 0xd3, 0x85, 0x81, 0x73, 0x9e, 0x42, 0x2f, 0x64, 0x1d, 0x99, 0xe5, + 0xf4, 0xb9, 0xe5, 0x5e, 0x3e, 0x49, 0xfb, 0x9f, 0x95, 0x60, 0x34, 0x55, 0x3b, 0x0a, 0x79, 0x50, + 0x25, 0x1e, 0x73, 0xea, 0xcb, 0xc5, 0xe6, 0xb0, 0xa5, 0x98, 0xd5, 0x02, 0x79, 0x4e, 0xd0, 0xc5, + 0x8a, 0xc3, 0xfd, 0x71, 0xb8, 0xfe, 0x3c, 0x8c, 0x48, 0x81, 0x3e, 0xe4, 0xb4, 0x3d, 0xd1, 0x81, + 0x6a, 0x8c, 0x9e, 0x33, 0x60, 0x38, 0x85, 0x69, 0xff, 0x6e, 0x19, 0x26, 0xf8, 0x29, 0x48, 0x53, + 0x8d, 0xbc, 0x25, 0xb9, 0xdf, 0xfa, 0x4b, 0xba, 0xc2, 0x1b, 0xef, 0xc8, 0xf5, 0xc3, 0xde, 0x7c, + 0x90, 0xcf, 0xa8, 0xaf, 0xc8, 0xac, 0xaf, 0x65, 0x22, 0xb3, 0xb8, 0xd9, 0xdd, 0x3a, 0x22, 0x89, + 0x7e, 0xb0, 0x42, 0xb5, 0xfe, 0x76, 0x09, 0xc6, 0x33, 0xd7, 0x4a, 0xa0, 0x37, 0xd3, 0x95, 0x88, + 0xad, 0x22, 0x7c, 0xe5, 0x7b, 0xde, 0x34, 0x70, 0xb0, 0x7a, 0xc4, 0xf7, 0x68, 0xaa, 0xd8, 0x7f, + 0x50, 0x82, 0xb1, 0xf4, 0x7d, 0x18, 0xf7, 0x61, 0x4f, 0xbd, 0x07, 0x6a, 0xac, 0xe4, 0x3b, 0xbb, + 0x36, 0x96, 0xbb, 0xe4, 0x79, 0x75, 0x6d, 0xd9, 0x88, 0x35, 0xfc, 0xbe, 0x28, 0xf3, 0x6c, 0xff, + 0x1d, 0x0b, 0x4e, 0xf1, 0xb7, 0xcc, 0x8e, 0xc3, 0xbf, 0x9c, 0xd7, 0xbb, 0xaf, 0x14, 0x2b, 0x60, + 0xa6, 0x32, 0xe1, 0x7e, 0xfd, 0xcb, 0xae, 0xab, 0x14, 0xd2, 0xa6, 0x87, 0xc2, 0x7d, 0x28, 0xec, + 0x81, 0x06, 0x83, 0xfd, 0x07, 0x65, 0xd0, 0x37, 0x74, 0x22, 0x57, 0xe4, 0x38, 0x16, 0x52, 0xa1, + 0x71, 0x75, 0xc7, 0x6f, 0xe8, 0xbb, 0x40, 0xab, 0x99, 0x14, 0xc7, 0x9f, 0xb7, 0x60, 0xd8, 0xf5, + 0xdd, 0xc4, 0x75, 0xd8, 0x36, 0xba, 0x98, 0xeb, 0xde, 0x14, 0xbb, 0x05, 0x4e, 0x39, 0x88, 0xcc, + 0x73, 0x1c, 0xc5, 0x0c, 0x9b, 0x9c, 0xd1, 0x47, 0x45, 0xf0, 0x74, 0xb9, 0xb0, 0x04, 0xe1, 0x6a, + 0x26, 0x62, 0x3a, 0xa4, 0x86, 0x57, 0x12, 0x15, 0x94, 0x57, 0x8f, 0x29, 0x29, 0x55, 0xec, 0x57, + 0xdf, 0x95, 0x4e, 0x9b, 0x31, 0x67, 0x64, 0xc7, 0x80, 0xba, 0xfb, 0xe2, 0x80, 0x81, 0xa9, 0xd3, + 0x50, 0x73, 0x3a, 0x49, 0xd0, 0xa6, 0xdd, 0x24, 0x8e, 0x9a, 0x74, 0xe8, 0xad, 0x04, 0x60, 0x8d, + 0x63, 0x7f, 0x67, 0x10, 0x32, 0x49, 0x87, 0xe8, 0xa6, 0x79, 0xbb, 0xac, 0x55, 0xec, 0xed, 0xb2, + 0x4a, 0x98, 0xbc, 0x1b, 0x66, 0x51, 0x0b, 0x2a, 0xe1, 0xa6, 0x13, 0x4b, 0xb3, 0xfa, 0x25, 0xb5, + 0x8f, 0xa3, 0x8d, 0xb7, 0x77, 0x27, 0x7f, 0xb2, 0x3f, 0xaf, 0x2b, 0x1d, 0xab, 0xd3, 0xbc, 0x82, + 0x8a, 0x66, 0xcd, 0x68, 0x60, 0x4e, 0xff, 0x20, 0x17, 0xde, 0x7d, 0x52, 0xd4, 0xb6, 0xc7, 0x24, + 0xee, 0x78, 0x89, 0x18, 0x0d, 0x2f, 0x15, 0x38, 0xcb, 0x38, 0x61, 0x9d, 0xb1, 0xcf, 0xff, 0x63, + 0x83, 0x29, 0xfa, 0x30, 0xd4, 0xe2, 0xc4, 0x89, 0x92, 0x3b, 0x4c, 0x70, 0xd5, 0xb7, 0xcb, 0x4a, + 0x22, 0x58, 0xd3, 0x43, 0x2f, 0xb3, 0x82, 0xb5, 0x6e, 0xbc, 0x79, 0x87, 0x39, 0x0f, 0xb2, 0xb8, + 0xad, 0xa0, 0x80, 0x0d, 0x6a, 0xe8, 0x2c, 0x00, 0x1b, 0xdb, 0x3c, 0xd0, 0xaf, 0xca, 0xbc, 0x4c, + 0x4a, 0x15, 0x62, 0x05, 0xc1, 0x06, 0x16, 0x7a, 0xc3, 0x02, 0xb8, 0xe1, 0xb8, 0x89, 0xeb, 0xb7, + 0xce, 0xb3, 0x4b, 0x50, 0x0a, 0x38, 0x06, 0xe0, 0x77, 0xde, 0xaa, 0x2c, 0x76, 0x25, 0xc2, 0x35, + 0xc5, 0x07, 0x1b, 0x3c, 0xd1, 0x8b, 0x30, 0xc6, 0xae, 0xd2, 0x25, 0x4d, 0x79, 0xe5, 0x2e, 0x4f, + 0x85, 0x50, 0xfb, 0xa5, 0x7a, 0x0a, 0x8a, 0x33, 0xd8, 0xf6, 0x8f, 0x41, 0xba, 0x6a, 0x06, 0x9a, + 0x94, 0x45, 0x3a, 0xb8, 0x23, 0x9d, 0xa5, 0x5f, 0xa4, 0xea, 0x69, 0xfc, 0x86, 0x05, 0x66, 0x69, + 0x0f, 0xf4, 0x1a, 0xaf, 0x21, 0x62, 0x15, 0x71, 0xf8, 0x69, 0xd0, 0x9d, 0x5a, 0x72, 0xc2, 0xcc, + 0x29, 0xbc, 0x2c, 0x24, 0x72, 0xfa, 0x39, 0xa8, 0x4a, 0xe8, 0x81, 0xec, 0xd2, 0x4f, 0xc0, 0x09, + 0x99, 0x07, 0x29, 0x5d, 0xbf, 0xe2, 0xe0, 0x6c, 0x7f, 0xef, 0x95, 0x74, 0x49, 0x95, 0x7a, 0xb9, + 0xa4, 0xfa, 0xb8, 0x90, 0xf7, 0x37, 0x2d, 0x38, 0x93, 0x15, 0x20, 0x5e, 0x0a, 0x7c, 0x37, 0x09, + 0xa2, 0x55, 0x92, 0xd0, 0x6f, 0xca, 0x0a, 0x9a, 0xdd, 0x70, 0x22, 0x59, 0x0c, 0x9d, 0xe9, 0xfa, + 0x6b, 0x4e, 0xe4, 0x63, 0xd6, 0x8a, 0x76, 0x60, 0x90, 0x07, 0xb4, 0x89, 0x0d, 0xc7, 0x21, 0xa7, + 0x77, 0x4e, 0x77, 0xe8, 0x1d, 0x0f, 0x0f, 0xa6, 0xc3, 0x82, 0xa1, 0xfd, 0x3d, 0x0b, 0xd0, 0xf2, + 0x36, 0x89, 0x22, 0xb7, 0x69, 0x84, 0xe0, 0xb1, 0x6b, 0x6e, 0x8c, 0xeb, 0x6c, 0xcc, 0x2c, 0xdd, + 0xcc, 0x35, 0x37, 0xc6, 0xbf, 0xfc, 0x6b, 0x6e, 0x4a, 0x07, 0xbb, 0xe6, 0x06, 0x2d, 0xc3, 0xa9, + 0x36, 0xdf, 0x31, 0xf1, 0xab, 0x23, 0xf8, 0xf6, 0x49, 0xe5, 0xc4, 0x3d, 0x74, 0x6b, 0x77, 0xf2, + 0xd4, 0x52, 0x1e, 0x02, 0xce, 0x7f, 0xce, 0x7e, 0x0e, 0x10, 0x8f, 0xbc, 0x9b, 0xcd, 0x0b, 0xa3, + 0xea, 0xe9, 0x41, 0xb2, 0xbf, 0x5a, 0x81, 0xf1, 0x4c, 0xa9, 0x5c, 0xba, 0x5b, 0xed, 0x8e, 0xdb, + 0x3a, 0xb4, 0x09, 0xd2, 0x2d, 0x5e, 0x5f, 0x91, 0x60, 0x3e, 0x54, 0x5c, 0x3f, 0xec, 0x24, 0xc5, + 0xa4, 0xc1, 0x72, 0x21, 0x16, 0x28, 0x41, 0xc3, 0xe3, 0x4d, 0xff, 0x62, 0xce, 0xa6, 0xc8, 0xb8, + 0xb2, 0xd4, 0x7e, 0x62, 0xe0, 0x1e, 0x79, 0x34, 0x3e, 0xa9, 0xa3, 0xbc, 0x2a, 0x45, 0xf8, 0x46, + 0x33, 0x83, 0xe5, 0xa8, 0xa3, 0x05, 0x7e, 0xbd, 0x04, 0xc3, 0xc6, 0x47, 0x43, 0xbf, 0x9c, 0x2e, + 0x7c, 0x65, 0x15, 0xf7, 0x4a, 0x8c, 0xfe, 0x94, 0x2e, 0x6d, 0xc5, 0x5f, 0xe9, 0xf1, 0xee, 0x9a, + 0x57, 0xb7, 0x77, 0x27, 0x8f, 0x65, 0xaa, 0x5a, 0xa5, 0xea, 0x60, 0x9d, 0xfe, 0x38, 0x8c, 0x67, + 0xc8, 0xe4, 0xbc, 0xf2, 0x9a, 0xf9, 0xca, 0x87, 0xf6, 0xac, 0x99, 0x5d, 0xf6, 0x4d, 0xda, 0x65, + 0x22, 0x13, 0x30, 0xf0, 0x48, 0x1f, 0x6e, 0xe4, 0x4c, 0xc2, 0x6f, 0xa9, 0xcf, 0x84, 0xdf, 0x27, + 0xa0, 0x1a, 0x06, 0x9e, 0xcb, 0xee, 0x9e, 0xe7, 0x7a, 0x8b, 0xa5, 0x18, 0xaf, 0x88, 0x36, 0xac, + 0xa0, 0xe8, 0x06, 0xd4, 0xae, 0xdf, 0x48, 0xf8, 0x01, 0x96, 0x70, 0xd1, 0x17, 0x75, 0x6e, 0xa5, + 0xec, 0x2e, 0x75, 0x42, 0x86, 0x35, 0x2f, 0x64, 0xc3, 0x20, 0x5b, 0x04, 0x65, 0xf6, 0x02, 0x3b, + 0x3e, 0x60, 0xab, 0x63, 0x8c, 0x05, 0xc4, 0xfe, 0x46, 0x0d, 0x4e, 0xe6, 0xd5, 0x2b, 0x47, 0x1f, + 0x83, 0x41, 0x2e, 0x63, 0x31, 0x57, 0x62, 0xe4, 0xf1, 0x98, 0x67, 0x04, 0x85, 0x58, 0xec, 0x37, + 0x16, 0x3c, 0x05, 0x77, 0xcf, 0x59, 0x17, 0x23, 0xe4, 0x68, 0xb8, 0x2f, 0x3a, 0x9a, 0xfb, 0xa2, + 0xc3, 0xb9, 0x7b, 0xce, 0x3a, 0xba, 0x09, 0x95, 0x96, 0x9b, 0x10, 0x47, 0xf8, 0x41, 0xae, 0x1d, + 0x09, 0x73, 0xe2, 0x70, 0x2b, 0x8d, 0xfd, 0xc4, 0x9c, 0x21, 0xfa, 0xba, 0x05, 0xe3, 0xeb, 0xe9, + 0xec, 0x7e, 0xa1, 0x3c, 0x9d, 0x23, 0xa8, 0x49, 0x9f, 0x66, 0xc4, 0xef, 0x79, 0xca, 0x34, 0xe2, + 0xac, 0x38, 0xe8, 0xd3, 0x16, 0x0c, 0x6d, 0xb8, 0x9e, 0x51, 0x9e, 0xf8, 0x08, 0x3e, 0xce, 0x79, + 0xc6, 0x40, 0x6f, 0x9a, 0xf8, 0xff, 0x18, 0x4b, 0xce, 0xbd, 0x56, 0xaa, 0xc1, 0xc3, 0xae, 0x54, + 0x43, 0xf7, 0x68, 0xa5, 0xfa, 0xac, 0x05, 0x35, 0xd5, 0xd3, 0x22, 0x63, 0xfb, 0xc3, 0x47, 0xf8, + 0xc9, 0xb9, 0xf3, 0x47, 0xfd, 0xc5, 0x9a, 0x39, 0xfa, 0x92, 0x05, 0xc3, 0xce, 0xeb, 0x9d, 0x88, + 0x34, 0xc9, 0x76, 0x10, 0xc6, 0xa2, 0xd8, 0xd7, 0x2b, 0xc5, 0x0b, 0x33, 0x43, 0x99, 0xcc, 0x91, + 0xed, 0xe5, 0x30, 0x16, 0x99, 0x55, 0xba, 0x01, 0x9b, 0x22, 0xd8, 0xbb, 0x25, 0x98, 0xdc, 0x87, + 0x02, 0x7a, 0x1e, 0x46, 0x82, 0xa8, 0xe5, 0xf8, 0xee, 0xeb, 0x66, 0xb9, 0x0e, 0x65, 0x65, 0x2d, + 0x1b, 0x30, 0x9c, 0xc2, 0x34, 0x73, 0xca, 0x4b, 0xfb, 0xe4, 0x94, 0x9f, 0x81, 0x81, 0x88, 0x84, + 0x41, 0x76, 0xb3, 0xc0, 0xb2, 0x1a, 0x18, 0x04, 0x3d, 0x02, 0x65, 0x27, 0x74, 0x45, 0x2c, 0x9d, + 0xda, 0x03, 0xcd, 0xac, 0x2c, 0x60, 0xda, 0x9e, 0x2a, 0x71, 0x51, 0xb9, 0x2b, 0x25, 0x2e, 0xe8, + 0x32, 0x20, 0x8e, 0x5f, 0x06, 0xf5, 0x32, 0x90, 0x3e, 0x16, 0xb1, 0xdf, 0x2a, 0xc3, 0x23, 0x7b, + 0x8e, 0x17, 0x1d, 0x4a, 0x68, 0xed, 0x11, 0x4a, 0x28, 0xbb, 0xa7, 0xb4, 0x5f, 0xf7, 0x94, 0x7b, + 0x74, 0xcf, 0xa7, 0xe9, 0x34, 0x90, 0x65, 0x4e, 0x8a, 0xb9, 0xe6, 0xaf, 0x57, 0xd5, 0x14, 0x31, + 0x03, 0x24, 0x14, 0x6b, 0xbe, 0x74, 0x0f, 0x90, 0xca, 0xa7, 0xae, 0x14, 0xb1, 0x0c, 0xf4, 0x2c, + 0x7b, 0xc2, 0xc7, 0x7e, 0xaf, 0x24, 0x6d, 0xfb, 0x17, 0x4a, 0xf0, 0x58, 0x1f, 0xda, 0xdb, 0x1c, + 0xc5, 0x56, 0x9f, 0xa3, 0xf8, 0x07, 0xfb, 0x33, 0xd9, 0x7f, 0xc5, 0x82, 0xd3, 0xbd, 0x17, 0x0f, + 0xf4, 0x34, 0x0c, 0xaf, 0x47, 0x8e, 0xdf, 0xd8, 0x64, 0x57, 0x97, 0xca, 0x4e, 0x61, 0x7d, 0xad, + 0x9b, 0xb1, 0x89, 0x43, 0xb7, 0xb7, 0x3c, 0xac, 0xc2, 0xc0, 0x90, 0xf9, 0xaf, 0x74, 0x7b, 0xbb, + 0x96, 0x05, 0xe2, 0x6e, 0x7c, 0xfb, 0xcf, 0x4a, 0xf9, 0x62, 0x71, 0x23, 0xe3, 0x20, 0xdf, 0x49, + 0x7c, 0x85, 0x52, 0x1f, 0xba, 0xa4, 0x7c, 0xb7, 0x75, 0xc9, 0x40, 0x2f, 0x5d, 0x82, 0xe6, 0xe0, + 0x98, 0x71, 0xb5, 0x0d, 0xcf, 0x69, 0xe6, 0x31, 0xc3, 0xaa, 0xd0, 0xc7, 0x4a, 0x06, 0x8e, 0xbb, + 0x9e, 0x40, 0x4f, 0x42, 0xd5, 0xf5, 0x63, 0xd2, 0xe8, 0x44, 0x3c, 0x56, 0xdd, 0xc8, 0x23, 0x5b, + 0x10, 0xed, 0x58, 0x61, 0xd8, 0xbf, 0x52, 0x82, 0x87, 0x7a, 0xda, 0x59, 0x77, 0x49, 0x77, 0x99, + 0x9f, 0x63, 0xe0, 0xee, 0x7c, 0x0e, 0xb3, 0x93, 0x2a, 0xfb, 0x76, 0xd2, 0x1f, 0xf6, 0x1e, 0x98, + 0xd4, 0xe6, 0xfe, 0xa1, 0xed, 0xa5, 0x17, 0x60, 0xd4, 0x09, 0x43, 0x8e, 0xc7, 0x42, 0x4e, 0x33, + 0x85, 0x7e, 0x66, 0x4c, 0x20, 0x4e, 0xe3, 0xf6, 0xb5, 0x7a, 0xfe, 0xb1, 0x05, 0x35, 0x4c, 0x36, + 0xb8, 0x76, 0x40, 0xd7, 0x45, 0x17, 0x59, 0x45, 0x94, 0x04, 0xa5, 0x1d, 0x1b, 0xbb, 0xac, 0x54, + 0x66, 0x5e, 0x67, 0x77, 0x5f, 0x81, 0x54, 0x3a, 0xd0, 0x15, 0x48, 0xea, 0x12, 0x9c, 0x72, 0xef, + 0x4b, 0x70, 0xec, 0x6f, 0x0e, 0xd1, 0xd7, 0x0b, 0x83, 0xd9, 0x88, 0x34, 0x63, 0xfa, 0x7d, 0x3b, + 0x91, 0x27, 0x06, 0x89, 0xfa, 0xbe, 0x57, 0xf0, 0x22, 0xa6, 0xed, 0xa9, 0xd3, 0xa4, 0xd2, 0x81, + 0xca, 0x9c, 0x94, 0xf7, 0x2d, 0x73, 0xf2, 0x02, 0x8c, 0xc6, 0xf1, 0xe6, 0x4a, 0xe4, 0x6e, 0x3b, + 0x09, 0xb9, 0x44, 0x76, 0x84, 0x95, 0xa5, 0x4b, 0x13, 0xac, 0x5e, 0xd0, 0x40, 0x9c, 0xc6, 0x45, + 0xf3, 0x70, 0x5c, 0x17, 0x1b, 0x21, 0x51, 0xc2, 0x12, 0x14, 0xf8, 0x48, 0x50, 0x79, 0xc8, 0xba, + 0x3c, 0x89, 0x40, 0xc0, 0xdd, 0xcf, 0x50, 0xfd, 0x96, 0x6a, 0xa4, 0x82, 0x0c, 0xa6, 0xf5, 0x5b, + 0x8a, 0x0e, 0x95, 0xa5, 0xeb, 0x09, 0xb4, 0x04, 0x27, 0xf8, 0xc0, 0x98, 0x09, 0x43, 0xe3, 0x8d, + 0x86, 0xd2, 0xa5, 0x18, 0xe7, 0xbb, 0x51, 0x70, 0xde, 0x73, 0xe8, 0x59, 0x18, 0x56, 0xcd, 0x0b, + 0x73, 0xe2, 0x20, 0x44, 0x79, 0x31, 0x14, 0x99, 0x85, 0x26, 0x36, 0xf1, 0xd0, 0x87, 0xe0, 0x41, + 0xfd, 0x97, 0x67, 0xb1, 0xf1, 0xd3, 0xc1, 0x39, 0x51, 0xc7, 0x49, 0x5d, 0xb9, 0x32, 0x9f, 0x8b, + 0xd6, 0xc4, 0xbd, 0x9e, 0x47, 0xeb, 0x70, 0x5a, 0x81, 0xce, 0xf9, 0x09, 0x4b, 0x49, 0x89, 0x49, + 0xdd, 0x89, 0xc9, 0x95, 0xc8, 0x13, 0x77, 0xd6, 0xaa, 0xdb, 0x30, 0xe7, 0xdd, 0xe4, 0x42, 0x1e, + 0x26, 0x5e, 0xc4, 0x7b, 0x50, 0x41, 0xd3, 0x50, 0x23, 0xbe, 0xb3, 0xee, 0x91, 0xe5, 0xd9, 0x05, + 0x56, 0x0f, 0xca, 0x38, 0x8c, 0x3c, 0x27, 0x01, 0x58, 0xe3, 0xa8, 0x20, 0xd9, 0x91, 0x9e, 0x37, + 0xb3, 0xae, 0xc0, 0xc9, 0x56, 0x23, 0xa4, 0xb6, 0x87, 0xdb, 0x20, 0x33, 0x0d, 0x16, 0x13, 0x48, + 0x3f, 0x0c, 0xaf, 0x91, 0xa9, 0x22, 0xc0, 0xe7, 0x67, 0x57, 0xba, 0x70, 0x70, 0xee, 0x93, 0x2c, + 0x76, 0x34, 0x0a, 0x6e, 0xee, 0x4c, 0x9c, 0xc8, 0xc4, 0x8e, 0xd2, 0x46, 0xcc, 0x61, 0xe8, 0x22, + 0x20, 0x96, 0x4e, 0x70, 0x21, 0x49, 0x42, 0x65, 0xec, 0x4c, 0x9c, 0x64, 0xaf, 0xa4, 0x22, 0xe1, + 0xce, 0x77, 0x61, 0xe0, 0x9c, 0xa7, 0xec, 0x7f, 0x6f, 0xc1, 0xa8, 0x9a, 0xaf, 0x77, 0x21, 0xa1, + 0xc6, 0x4b, 0x27, 0xd4, 0xcc, 0x1f, 0x5e, 0xe3, 0x31, 0xc9, 0x7b, 0x44, 0x65, 0xff, 0xec, 0x30, + 0x80, 0xd6, 0x8a, 0x6a, 0x41, 0xb2, 0x7a, 0x2e, 0x48, 0xf7, 0xad, 0x46, 0xca, 0x2b, 0xfe, 0x52, + 0xb9, 0xb7, 0xc5, 0x5f, 0x56, 0xe1, 0x94, 0x34, 0x17, 0xf8, 0x59, 0xd1, 0x85, 0x20, 0x56, 0x0a, + 0xae, 0x5a, 0x7f, 0x44, 0x10, 0x3a, 0xb5, 0x90, 0x87, 0x84, 0xf3, 0x9f, 0x4d, 0x59, 0x29, 0x43, + 0xfb, 0x59, 0x29, 0x7a, 0x4e, 0x2f, 0x6e, 0xc8, 0xbb, 0x55, 0x32, 0x73, 0x7a, 0xf1, 0xfc, 0x2a, + 0xd6, 0x38, 0xf9, 0x8a, 0xbd, 0x56, 0x90, 0x62, 0x87, 0x03, 0x2b, 0x76, 0xa9, 0x62, 0x86, 0x7b, + 0xaa, 0x18, 0xe9, 0x93, 0x1e, 0xe9, 0xe9, 0x93, 0x7e, 0x11, 0xc6, 0x5c, 0x7f, 0x93, 0x44, 0x6e, + 0x42, 0x9a, 0x6c, 0x2e, 0x30, 0xf5, 0x63, 0x1c, 0x0f, 0x2f, 0xa4, 0xa0, 0x38, 0x83, 0x9d, 0xd6, + 0x8b, 0x63, 0x7d, 0xe8, 0xc5, 0x1e, 0xab, 0xd1, 0x78, 0x31, 0xab, 0xd1, 0xb1, 0xc3, 0xaf, 0x46, + 0xc7, 0x8f, 0x74, 0x35, 0x42, 0x85, 0xac, 0x46, 0x7d, 0x29, 0x7a, 0x63, 0xfb, 0x77, 0x72, 0x9f, + 0xed, 0x5f, 0xaf, 0xa5, 0xe8, 0xd4, 0x1d, 0x2f, 0x45, 0xf9, 0xab, 0xcc, 0x03, 0x77, 0xb4, 0xca, + 0x7c, 0xb6, 0x04, 0xa7, 0xb4, 0x1e, 0xa6, 0xa3, 0xdf, 0xdd, 0xa0, 0x9a, 0x88, 0x5d, 0xcf, 0xc5, + 0xcf, 0x6d, 0x8c, 0xfc, 0x2e, 0x9d, 0x2a, 0xa6, 0x20, 0xd8, 0xc0, 0x62, 0x69, 0x52, 0x24, 0x62, + 0x95, 0x80, 0xb3, 0x4a, 0x7a, 0x56, 0xb4, 0x63, 0x85, 0x41, 0xc7, 0x17, 0xfd, 0x2d, 0x52, 0x4f, + 0xb3, 0xf5, 0xee, 0x66, 0x35, 0x08, 0x9b, 0x78, 0xe8, 0x09, 0xce, 0x84, 0x29, 0x08, 0xaa, 0xa8, + 0x47, 0xc4, 0x7d, 0xbd, 0x52, 0x27, 0x28, 0xa8, 0x14, 0x87, 0xe5, 0xc3, 0x55, 0xba, 0xc5, 0x61, + 0x51, 0x5c, 0x0a, 0xc3, 0xfe, 0x1f, 0x16, 0x3c, 0x94, 0xdb, 0x15, 0x77, 0x61, 0xf1, 0xbd, 0x99, + 0x5e, 0x7c, 0x57, 0x8b, 0xda, 0x6e, 0x18, 0x6f, 0xd1, 0x63, 0x21, 0xfe, 0xb7, 0x16, 0x8c, 0x69, + 0xfc, 0xbb, 0xf0, 0xaa, 0x6e, 0xfa, 0x55, 0x8b, 0xdb, 0x59, 0xd5, 0xba, 0xde, 0xed, 0x77, 0x4b, + 0xa0, 0x6a, 0x50, 0xce, 0x34, 0x64, 0x85, 0xdf, 0x7d, 0x4e, 0x12, 0x77, 0x60, 0x90, 0x1d, 0x84, + 0xc6, 0xc5, 0x04, 0x79, 0xa4, 0xf9, 0xb3, 0x43, 0x55, 0x7d, 0xc8, 0xcc, 0xfe, 0xc6, 0x58, 0x30, + 0x64, 0x75, 0xaa, 0xdd, 0x98, 0x6a, 0xf3, 0xa6, 0xc8, 0x2c, 0xd3, 0x75, 0xaa, 0x45, 0x3b, 0x56, + 0x18, 0x74, 0x79, 0x70, 0x1b, 0x81, 0x3f, 0xeb, 0x39, 0xb1, 0xbc, 0x93, 0x52, 0x2d, 0x0f, 0x0b, + 0x12, 0x80, 0x35, 0x0e, 0x3b, 0x23, 0x75, 0xe3, 0xd0, 0x73, 0x76, 0x8c, 0xfd, 0xb3, 0x51, 0x62, + 0x41, 0x81, 0xb0, 0x89, 0x67, 0xb7, 0x61, 0x22, 0xfd, 0x12, 0x73, 0x64, 0x83, 0xc5, 0x58, 0xf6, + 0xd5, 0x9d, 0xd3, 0x50, 0x73, 0xd8, 0x53, 0x8b, 0x1d, 0x27, 0x7b, 0x95, 0xfc, 0x8c, 0x04, 0x60, + 0x8d, 0x63, 0xff, 0x9a, 0x05, 0x27, 0x72, 0x3a, 0xad, 0xc0, 0xcc, 0xbd, 0x44, 0x6b, 0x9b, 0xbc, + 0x85, 0xfd, 0xdd, 0x30, 0xd4, 0x24, 0x1b, 0x8e, 0x8c, 0xe2, 0x33, 0x74, 0xfb, 0x1c, 0x6f, 0xc6, + 0x12, 0x6e, 0xff, 0x37, 0x0b, 0xc6, 0xd3, 0xb2, 0xc6, 0x2c, 0x1b, 0x86, 0x77, 0x93, 0x1b, 0x37, + 0x82, 0x6d, 0x12, 0xed, 0xd0, 0x37, 0xb7, 0x32, 0xd9, 0x30, 0x5d, 0x18, 0x38, 0xe7, 0x29, 0x56, + 0x81, 0xb6, 0xa9, 0x7a, 0x5b, 0x8e, 0xc8, 0xab, 0x45, 0x8e, 0x48, 0xfd, 0x31, 0xcd, 0xe3, 0x72, + 0xc5, 0x12, 0x9b, 0xfc, 0xed, 0xef, 0x0d, 0x80, 0x4a, 0xed, 0x65, 0xf1, 0x47, 0x05, 0x45, 0x6f, + 0x1d, 0x34, 0x09, 0x4a, 0x0d, 0x86, 0x81, 0xbd, 0x02, 0x02, 0xb8, 0x97, 0xc4, 0x74, 0x5d, 0xaa, + 0x37, 0x5c, 0xd3, 0x20, 0x6c, 0xe2, 0x51, 0x49, 0x3c, 0x77, 0x9b, 0xf0, 0x87, 0x06, 0xd3, 0x92, + 0x2c, 0x4a, 0x00, 0xd6, 0x38, 0x54, 0x92, 0xa6, 0xbb, 0xb1, 0x21, 0xb6, 0xfc, 0x4a, 0x12, 0xda, + 0x3b, 0x98, 0x41, 0x78, 0x51, 0xf1, 0x60, 0x4b, 0x58, 0xc1, 0x46, 0x51, 0xf1, 0x60, 0x0b, 0x33, + 0x08, 0xb5, 0xdb, 0xfc, 0x20, 0x6a, 0xb3, 0xab, 0xfe, 0x9b, 0x8a, 0x8b, 0xb0, 0x7e, 0x95, 0xdd, + 0x76, 0xb9, 0x1b, 0x05, 0xe7, 0x3d, 0x47, 0x47, 0x60, 0x18, 0x91, 0xa6, 0xdb, 0x48, 0x4c, 0x6a, + 0x90, 0x1e, 0x81, 0x2b, 0x5d, 0x18, 0x38, 0xe7, 0x29, 0x34, 0x03, 0xe3, 0x32, 0x35, 0x5b, 0x16, + 0xde, 0x19, 0x4e, 0x17, 0xfa, 0xc0, 0x69, 0x30, 0xce, 0xe2, 0x53, 0xad, 0xd6, 0x16, 0x35, 0xb7, + 0x98, 0xb1, 0x6c, 0x68, 0x35, 0x59, 0x8b, 0x0b, 0x2b, 0x0c, 0xfb, 0x93, 0x65, 0xba, 0x0a, 0xf7, + 0xa8, 0x35, 0x77, 0xd7, 0xa2, 0x05, 0xd3, 0x23, 0x72, 0xa0, 0x8f, 0x11, 0xf9, 0x0c, 0x8c, 0x5c, + 0x8f, 0x03, 0x5f, 0x45, 0xe2, 0x55, 0x7a, 0x46, 0xe2, 0x19, 0x58, 0xf9, 0x91, 0x78, 0x83, 0x45, + 0x45, 0xe2, 0x0d, 0xdd, 0x61, 0x24, 0xde, 0xb7, 0x2b, 0xa0, 0x6e, 0x37, 0xb9, 0x4c, 0x92, 0x1b, + 0x41, 0xb4, 0xe5, 0xfa, 0x2d, 0x96, 0xd2, 0xfe, 0x75, 0x0b, 0x46, 0xf8, 0x7c, 0x59, 0x34, 0x93, + 0xc1, 0x36, 0x0a, 0xba, 0x36, 0x23, 0xc5, 0x6c, 0x6a, 0xcd, 0x60, 0x94, 0xb9, 0x12, 0xd5, 0x04, + 0xe1, 0x94, 0x44, 0xe8, 0xe3, 0x00, 0xd2, 0x3f, 0xba, 0x21, 0x55, 0xe6, 0x42, 0x31, 0xf2, 0x61, + 0xb2, 0xa1, 0x6d, 0xe0, 0x35, 0xc5, 0x04, 0x1b, 0x0c, 0xd1, 0x67, 0x75, 0xa2, 0x1c, 0xcf, 0x3a, + 0xf8, 0xe8, 0x91, 0xf4, 0x4d, 0x3f, 0x69, 0x72, 0x18, 0x86, 0x5c, 0xbf, 0x45, 0xc7, 0x89, 0x88, + 0x58, 0x7a, 0x57, 0x5e, 0x39, 0x88, 0xc5, 0xc0, 0x69, 0xd6, 0x1d, 0xcf, 0xf1, 0x1b, 0x24, 0x5a, + 0xe0, 0xe8, 0xe6, 0x45, 0xe0, 0xac, 0x01, 0x4b, 0x42, 0x5d, 0xf7, 0xc2, 0x54, 0xfa, 0xb9, 0x17, + 0xe6, 0xf4, 0x07, 0xe0, 0x78, 0xd7, 0xc7, 0x3c, 0x50, 0x56, 0xdc, 0x9d, 0x27, 0xd4, 0xd9, 0xff, + 0x7c, 0x50, 0x2f, 0x5a, 0x97, 0x83, 0x26, 0xbf, 0x9d, 0x24, 0xd2, 0x5f, 0x54, 0xd8, 0xb8, 0x05, + 0x0e, 0x11, 0xe3, 0x32, 0x71, 0xd5, 0x88, 0x4d, 0x96, 0x74, 0x8c, 0x86, 0x4e, 0x44, 0xfc, 0xa3, + 0x1e, 0xa3, 0x2b, 0x8a, 0x09, 0x36, 0x18, 0xa2, 0xcd, 0x54, 0x5a, 0xcc, 0xf9, 0xc3, 0xa7, 0xc5, + 0xb0, 0x42, 0x59, 0x79, 0x17, 0x0a, 0x7c, 0xc9, 0x82, 0x31, 0x3f, 0x35, 0x72, 0x8b, 0x09, 0x23, + 0xcd, 0x9f, 0x15, 0xfc, 0x72, 0xac, 0x74, 0x1b, 0xce, 0xf0, 0xcf, 0x5b, 0xd2, 0x2a, 0x07, 0x5c, + 0xd2, 0xf4, 0x35, 0x47, 0x83, 0xbd, 0xae, 0x39, 0x42, 0xbe, 0xba, 0xe7, 0x6d, 0xa8, 0xf0, 0x7b, + 0xde, 0x20, 0xe7, 0x8e, 0xb7, 0x6b, 0x50, 0x6b, 0x44, 0xc4, 0x49, 0xee, 0xf0, 0xca, 0x2f, 0x76, + 0x40, 0x3f, 0x2b, 0x09, 0x60, 0x4d, 0xcb, 0xfe, 0xdf, 0x03, 0x70, 0x4c, 0xf6, 0x88, 0x0c, 0x41, + 0xa7, 0xeb, 0x23, 0xe7, 0xab, 0x8d, 0x5b, 0xb5, 0x3e, 0x5e, 0x90, 0x00, 0xac, 0x71, 0xa8, 0x3d, + 0xd6, 0x89, 0xc9, 0x72, 0x48, 0xfc, 0x45, 0x77, 0x3d, 0x16, 0xe7, 0x9c, 0x6a, 0xa2, 0x5c, 0xd1, + 0x20, 0x6c, 0xe2, 0x51, 0x63, 0x9c, 0xdb, 0xc5, 0x71, 0x36, 0x03, 0x47, 0xd8, 0xdb, 0x58, 0xc2, + 0xd1, 0x2f, 0xe6, 0x16, 0xbf, 0x2d, 0x26, 0xf7, 0xac, 0x2b, 0xf2, 0xfe, 0x80, 0x17, 0x55, 0xfe, + 0x4d, 0x0b, 0x4e, 0xf1, 0x56, 0xd9, 0x93, 0x57, 0xc2, 0xa6, 0x93, 0x90, 0xb8, 0x98, 0x62, 0xf4, + 0x39, 0xf2, 0x69, 0x27, 0x6f, 0x1e, 0x5b, 0x9c, 0x2f, 0x0d, 0x7a, 0xd3, 0x82, 0xf1, 0xad, 0x54, + 0xd9, 0x12, 0xb9, 0x74, 0x1c, 0xb6, 0xa2, 0x40, 0x8a, 0xa8, 0x9e, 0x6a, 0xe9, 0xf6, 0x18, 0x67, + 0xb9, 0xdb, 0x7f, 0x66, 0x81, 0xa9, 0x46, 0xef, 0x7e, 0xb5, 0x93, 0x83, 0x9b, 0x82, 0xd2, 0xba, + 0xac, 0xf4, 0xb4, 0x2e, 0x1f, 0x81, 0x72, 0xc7, 0x6d, 0x8a, 0xfd, 0x85, 0x3e, 0x7d, 0x5d, 0x98, + 0xc3, 0xb4, 0xdd, 0xfe, 0x27, 0x15, 0xed, 0xb7, 0x10, 0xa9, 0x5d, 0x3f, 0x14, 0xaf, 0xbd, 0xa1, + 0xea, 0xa5, 0xf1, 0x37, 0xbf, 0xdc, 0x55, 0x2f, 0xed, 0xfd, 0x07, 0xcf, 0xdc, 0xe3, 0x1d, 0xd4, + 0xab, 0x5c, 0xda, 0xd0, 0x3e, 0x69, 0x7b, 0xd7, 0xa1, 0x4a, 0xb7, 0x60, 0xcc, 0x01, 0x59, 0x4d, + 0x09, 0x55, 0xbd, 0x20, 0xda, 0x6f, 0xef, 0x4e, 0xbe, 0xef, 0xe0, 0x62, 0xc9, 0xa7, 0xb1, 0xa2, + 0x8f, 0x62, 0xa8, 0xd1, 0xdf, 0x2c, 0xc3, 0x50, 0x6c, 0xee, 0xae, 0x28, 0x9d, 0x29, 0x01, 0x85, + 0xa4, 0x2f, 0x6a, 0x3e, 0xc8, 0x87, 0x1a, 0xbb, 0xd3, 0x97, 0x31, 0xe5, 0x7b, 0xc0, 0x15, 0x95, + 0xe7, 0x27, 0x01, 0xb7, 0x77, 0x27, 0x5f, 0x38, 0x38, 0x53, 0xf5, 0x38, 0xd6, 0x2c, 0xec, 0x2f, + 0x0f, 0xe8, 0xb1, 0x2b, 0xca, 0xe4, 0xfd, 0x50, 0x8c, 0xdd, 0xe7, 0x33, 0x63, 0xf7, 0x4c, 0xd7, + 0xd8, 0x1d, 0xd3, 0x17, 0xbf, 0xa6, 0x46, 0xe3, 0xdd, 0x36, 0x04, 0xf6, 0xf7, 0x37, 0x30, 0x0b, + 0xe8, 0xb5, 0x8e, 0x1b, 0x91, 0x78, 0x25, 0xea, 0xf8, 0xae, 0xdf, 0x62, 0xc3, 0xb1, 0x6a, 0x5a, + 0x40, 0x29, 0x30, 0xce, 0xe2, 0xd3, 0x4d, 0x3d, 0xfd, 0xe6, 0xd7, 0x9c, 0x6d, 0x3e, 0xaa, 0x8c, + 0xca, 0x61, 0xab, 0xa2, 0x1d, 0x2b, 0x0c, 0xfb, 0x9b, 0xec, 0x2c, 0xdb, 0x48, 0x6d, 0xa6, 0x63, + 0xc2, 0x63, 0x97, 0x28, 0xf3, 0xb2, 0x63, 0x6a, 0x4c, 0xf0, 0x9b, 0x93, 0x39, 0x0c, 0xdd, 0x80, + 0xa1, 0x75, 0x7e, 0x85, 0x5f, 0x31, 0x25, 0xd6, 0xc5, 0x7d, 0x80, 0xec, 0xa2, 0x16, 0x79, 0x39, + 0xe0, 0x6d, 0xfd, 0x13, 0x4b, 0x6e, 0xf6, 0x77, 0x2a, 0x30, 0x9e, 0xb9, 0xe3, 0x36, 0x55, 0xf0, + 0xb5, 0xb4, 0x6f, 0xc1, 0xd7, 0x8f, 0x00, 0x34, 0x49, 0xe8, 0x05, 0x3b, 0xcc, 0x1c, 0x1b, 0x38, + 0xb0, 0x39, 0xa6, 0x2c, 0xf8, 0x39, 0x45, 0x05, 0x1b, 0x14, 0x45, 0xad, 0x35, 0x5e, 0x3f, 0x36, + 0x53, 0x6b, 0xcd, 0xb8, 0x88, 0x61, 0xf0, 0xee, 0x5e, 0xc4, 0xe0, 0xc2, 0x38, 0x17, 0x51, 0x25, + 0x10, 0xdf, 0x41, 0x9e, 0x30, 0xcb, 0x5f, 0x98, 0x4b, 0x93, 0xc1, 0x59, 0xba, 0xf7, 0xf4, 0x16, + 0xed, 0xf7, 0x40, 0x4d, 0x7e, 0xe7, 0x98, 0xe5, 0x1d, 0x8b, 0x22, 0x0c, 0x72, 0x18, 0xb0, 0xab, + 0xa5, 0xc5, 0xcf, 0xae, 0x5a, 0x08, 0x70, 0xaf, 0x6a, 0x21, 0xd8, 0x5f, 0x2c, 0x51, 0x3b, 0x9e, + 0xcb, 0xa5, 0xca, 0xfa, 0x3c, 0x0e, 0x83, 0x4e, 0x27, 0xd9, 0x0c, 0xba, 0x2e, 0x24, 0x9c, 0x61, + 0xad, 0x58, 0x40, 0xd1, 0x22, 0x0c, 0x34, 0x75, 0xa9, 0x96, 0x83, 0x7c, 0x4f, 0xed, 0x12, 0x75, + 0x12, 0x82, 0x19, 0x15, 0xf4, 0x30, 0x0c, 0x24, 0x4e, 0x4b, 0xa6, 0x5c, 0xb1, 0x34, 0xdb, 0x35, + 0xa7, 0x15, 0x63, 0xd6, 0x6a, 0x2e, 0xdf, 0x03, 0xfb, 0x2c, 0xdf, 0x2f, 0xc0, 0x68, 0xec, 0xb6, + 0x7c, 0x27, 0xe9, 0x44, 0xc4, 0x38, 0xe6, 0xd3, 0x91, 0x1b, 0x26, 0x10, 0xa7, 0x71, 0xed, 0xdf, + 0x1a, 0x81, 0x93, 0xab, 0xb3, 0x4b, 0xb2, 0x00, 0xf9, 0x91, 0x65, 0x4d, 0xe5, 0xf1, 0xb8, 0x7b, + 0x59, 0x53, 0x3d, 0xb8, 0x7b, 0x46, 0xd6, 0x94, 0x67, 0x64, 0x4d, 0xa5, 0x53, 0x58, 0xca, 0x45, + 0xa4, 0xb0, 0xe4, 0x49, 0xd0, 0x4f, 0x0a, 0xcb, 0x91, 0xa5, 0x51, 0xed, 0x29, 0xd0, 0x81, 0xd2, + 0xa8, 0x54, 0x8e, 0x59, 0x21, 0xc9, 0x05, 0x3d, 0x3e, 0x55, 0x6e, 0x8e, 0x99, 0xca, 0xef, 0xe1, + 0x89, 0x33, 0x42, 0xd5, 0xbf, 0x52, 0xbc, 0x00, 0x7d, 0xe4, 0xf7, 0x88, 0xdc, 0x1d, 0x33, 0xa7, + 0x6c, 0xa8, 0x88, 0x9c, 0xb2, 0x3c, 0x71, 0xf6, 0xcd, 0x29, 0x7b, 0x01, 0x46, 0x1b, 0x5e, 0xe0, + 0x93, 0x95, 0x28, 0x48, 0x82, 0x46, 0xe0, 0x09, 0xb3, 0x5e, 0xa9, 0x84, 0x59, 0x13, 0x88, 0xd3, + 0xb8, 0xbd, 0x12, 0xd2, 0x6a, 0x87, 0x4d, 0x48, 0x83, 0x7b, 0x94, 0x90, 0xf6, 0x73, 0x3a, 0x75, + 0x7a, 0x98, 0x7d, 0x91, 0x8f, 0x14, 0xff, 0x45, 0xfa, 0xc9, 0x9f, 0x46, 0x6f, 0xf1, 0x1b, 0x01, + 0xa9, 0x61, 0x3c, 0x1b, 0xb4, 0xa9, 0xe1, 0x37, 0xc2, 0xba, 0xe4, 0xd5, 0x23, 0x18, 0xb0, 0xd7, + 0x56, 0x35, 0x1b, 0x75, 0x4b, 0xa0, 0x6e, 0xc2, 0x69, 0x41, 0x0e, 0x93, 0xda, 0xfd, 0xd5, 0x12, + 0xfc, 0xc8, 0xbe, 0x22, 0xa0, 0x1b, 0x00, 0x89, 0xd3, 0x12, 0x03, 0x55, 0x1c, 0x98, 0x1c, 0x32, + 0xbc, 0x72, 0x4d, 0xd2, 0xe3, 0x65, 0x55, 0xd4, 0x5f, 0x76, 0x14, 0x21, 0x7f, 0xb3, 0xa8, 0xca, + 0xc0, 0xeb, 0xaa, 0x3e, 0x89, 0x03, 0x8f, 0x60, 0x06, 0xa1, 0xcb, 0x7f, 0x44, 0x5a, 0xfa, 0x0a, + 0x6b, 0xf5, 0xf9, 0x30, 0x6b, 0xc5, 0x02, 0x8a, 0x9e, 0x85, 0x61, 0xc7, 0xf3, 0x78, 0x7e, 0x0c, + 0x89, 0xc5, 0x95, 0x40, 0xba, 0x0c, 0x9e, 0x06, 0x61, 0x13, 0xcf, 0xfe, 0xd3, 0x12, 0x4c, 0xee, + 0xa3, 0x53, 0xba, 0x32, 0xfe, 0x2a, 0x7d, 0x67, 0xfc, 0x89, 0x1c, 0x85, 0xc1, 0x1e, 0x39, 0x0a, + 0xcf, 0xc2, 0x70, 0x42, 0x9c, 0xb6, 0x08, 0xc8, 0x12, 0x9e, 0x00, 0x7d, 0x02, 0xac, 0x41, 0xd8, + 0xc4, 0xa3, 0x5a, 0x6c, 0xcc, 0x69, 0x34, 0x48, 0x1c, 0xcb, 0x24, 0x04, 0xe1, 0x4d, 0x2d, 0x2c, + 0xc3, 0x81, 0x39, 0xa9, 0x67, 0x52, 0x2c, 0x70, 0x86, 0x65, 0xb6, 0xc3, 0x6b, 0x7d, 0x76, 0xf8, + 0x37, 0x4a, 0xf0, 0xc8, 0x9e, 0xab, 0x5b, 0xdf, 0xf9, 0x21, 0x9d, 0x98, 0x44, 0xd9, 0x81, 0x73, + 0x25, 0x26, 0x11, 0x66, 0x10, 0xde, 0x4b, 0x61, 0x68, 0x5c, 0x11, 0x5e, 0x74, 0xf2, 0x12, 0xef, + 0xa5, 0x14, 0x0b, 0x9c, 0x61, 0x79, 0xa7, 0xc3, 0xf2, 0xef, 0x96, 0xe0, 0xb1, 0x3e, 0x6c, 0x80, + 0x02, 0x93, 0xbc, 0xd2, 0xa9, 0x76, 0xe5, 0x7b, 0x94, 0x11, 0x79, 0x87, 0xdd, 0xf5, 0xcd, 0x12, + 0x9c, 0xee, 0xbd, 0x14, 0xa3, 0x9f, 0x80, 0xf1, 0x48, 0x45, 0x61, 0x99, 0x59, 0x7a, 0x27, 0xb8, + 0x27, 0x21, 0x05, 0xc2, 0x59, 0x5c, 0x34, 0x05, 0x10, 0x3a, 0xc9, 0x66, 0x7c, 0xee, 0xa6, 0x1b, + 0x27, 0xa2, 0x0a, 0xcd, 0x18, 0x3f, 0xbb, 0x92, 0xad, 0xd8, 0xc0, 0xa0, 0xec, 0xd8, 0xbf, 0xb9, + 0xe0, 0x72, 0x90, 0xf0, 0x87, 0xf8, 0x36, 0xe2, 0x84, 0xbc, 0x76, 0xc4, 0x00, 0xe1, 0x2c, 0x2e, + 0x65, 0xc7, 0x4e, 0x47, 0xb9, 0xa0, 0x7c, 0x7f, 0xc1, 0xd8, 0x2d, 0xaa, 0x56, 0x6c, 0x60, 0x64, + 0xf3, 0x0f, 0x2b, 0xfb, 0xe7, 0x1f, 0xda, 0xff, 0xb8, 0x04, 0x0f, 0xf5, 0x34, 0xe5, 0xfa, 0x9b, + 0x80, 0xf7, 0x5f, 0xce, 0xe0, 0x9d, 0x8d, 0x9d, 0x03, 0xe6, 0xb6, 0xfd, 0x71, 0x8f, 0x91, 0x26, + 0x72, 0xdb, 0xee, 0x3c, 0x39, 0xfc, 0xfe, 0xeb, 0xcf, 0xae, 0x74, 0xb6, 0x81, 0x03, 0xa4, 0xb3, + 0x65, 0x3e, 0x46, 0xa5, 0xcf, 0x89, 0xfc, 0xe7, 0xe5, 0x9e, 0xdd, 0x4b, 0xb7, 0x7e, 0x7d, 0xf9, + 0x69, 0xe7, 0xe0, 0x98, 0xeb, 0xb3, 0x2b, 0xa8, 0x56, 0x3b, 0xeb, 0xa2, 0x30, 0x49, 0x29, 0x7d, + 0x01, 0xfc, 0x42, 0x06, 0x8e, 0xbb, 0x9e, 0xb8, 0x0f, 0xd3, 0x0b, 0xef, 0xac, 0x4b, 0x0f, 0x96, + 0xe0, 0x8a, 0x96, 0xe1, 0x94, 0xec, 0x8a, 0x4d, 0x27, 0x22, 0x4d, 0xb1, 0x8c, 0xc4, 0x22, 0xa1, + 0xe2, 0x21, 0x9e, 0x94, 0x91, 0x83, 0x80, 0xf3, 0x9f, 0x63, 0xb7, 0xfe, 0x04, 0xa1, 0xdb, 0x10, + 0x9b, 0x1c, 0x7d, 0xeb, 0x0f, 0x6d, 0xc4, 0x1c, 0x66, 0x7f, 0x04, 0x6a, 0xea, 0xfd, 0x79, 0x58, + 0xb7, 0x1a, 0x74, 0x5d, 0x61, 0xdd, 0x6a, 0xc4, 0x19, 0x58, 0xf4, 0x6b, 0x51, 0x93, 0x38, 0x33, + 0x7b, 0x2e, 0x91, 0x1d, 0x66, 0x1f, 0xdb, 0xef, 0x85, 0x11, 0xe5, 0x67, 0xe9, 0xf7, 0x2e, 0x24, + 0xfb, 0x33, 0x25, 0x18, 0x4b, 0x97, 0xdb, 0x43, 0x33, 0x30, 0x6e, 0x7c, 0x3c, 0x43, 0x3e, 0xe5, + 0x96, 0x9e, 0x49, 0x83, 0x71, 0x16, 0x1f, 0xad, 0xc0, 0xc9, 0x4c, 0x13, 0x3f, 0x03, 0x28, 0xa5, + 0x83, 0xed, 0x67, 0x72, 0x70, 0x70, 0xee, 0x93, 0xe8, 0x15, 0x18, 0x8e, 0xc8, 0x46, 0x44, 0x44, + 0xdd, 0xc3, 0x83, 0x5f, 0x3a, 0x39, 0xce, 0x03, 0x41, 0x14, 0x09, 0x6c, 0xd2, 0xb3, 0xbf, 0x3c, + 0x08, 0xa3, 0xa9, 0x32, 0x8f, 0x29, 0x3f, 0xb3, 0xb5, 0xaf, 0x9f, 0x99, 0x65, 0x2b, 0x74, 0x7c, + 0x79, 0x5f, 0x9c, 0x91, 0xad, 0xd0, 0xf1, 0x09, 0xe6, 0x30, 0x6a, 0xe5, 0x37, 0xa3, 0x1d, 0xdc, + 0xf1, 0x45, 0x54, 0xb1, 0xb2, 0xf2, 0xe7, 0x58, 0x2b, 0x16, 0x50, 0xf4, 0x86, 0x05, 0x23, 0x31, + 0x3b, 0xc4, 0xe0, 0x5e, 0x7a, 0x31, 0xf7, 0x2e, 0x1e, 0xbe, 0xa8, 0xa2, 0x2a, 0x69, 0xca, 0x02, + 0x85, 0xcc, 0x16, 0x9c, 0xe2, 0x88, 0x3e, 0x63, 0x41, 0x4d, 0x5d, 0x6b, 0x23, 0x2e, 0x75, 0x5c, + 0x2d, 0xb6, 0x8a, 0x26, 0x77, 0xef, 0xaa, 0xf3, 0x20, 0x55, 0x0b, 0x10, 0x6b, 0xc6, 0x28, 0x56, + 0x2e, 0xf4, 0xa1, 0xa3, 0x71, 0xa1, 0x43, 0x8e, 0xfb, 0xfc, 0x3d, 0x50, 0x6b, 0x3b, 0xbe, 0xbb, + 0x41, 0xe2, 0x84, 0x7b, 0xb5, 0x65, 0x71, 0x5f, 0xd9, 0x88, 0x35, 0x9c, 0xda, 0x25, 0x31, 0x7b, + 0xb1, 0xc4, 0x70, 0x43, 0xb3, 0xb1, 0xb6, 0xaa, 0x9b, 0xb1, 0x89, 0x63, 0xfa, 0xcc, 0xe1, 0x9e, + 0xfa, 0xcc, 0x87, 0xf7, 0xf6, 0x99, 0xdb, 0x7f, 0xdf, 0x82, 0x53, 0xb9, 0x5f, 0xed, 0xfe, 0x8d, + 0xff, 0xb4, 0xbf, 0x52, 0x81, 0x13, 0x39, 0xf5, 0x5a, 0xd1, 0x8e, 0x39, 0x9e, 0xad, 0x22, 0x42, + 0x29, 0xd2, 0x91, 0x01, 0xb2, 0x1b, 0x73, 0x06, 0xf1, 0xc1, 0x4e, 0xac, 0xf4, 0xa9, 0x51, 0xf9, + 0xee, 0x9e, 0x1a, 0x19, 0xc3, 0x72, 0xe0, 0x9e, 0x0e, 0xcb, 0xca, 0x3e, 0x47, 0x39, 0xbf, 0x6e, + 0xc1, 0x44, 0xbb, 0xc7, 0x25, 0x01, 0xc2, 0xff, 0x7a, 0xf5, 0x68, 0xae, 0x20, 0xa8, 0x3f, 0x7c, + 0x6b, 0x77, 0xb2, 0xe7, 0xdd, 0x0c, 0xb8, 0xa7, 0x54, 0xf6, 0xf7, 0xca, 0xc0, 0x8a, 0x05, 0xb3, + 0x82, 0x76, 0x3b, 0xe8, 0x13, 0x66, 0xd9, 0x67, 0xab, 0xa8, 0x12, 0xc5, 0x9c, 0xb8, 0x2a, 0x1b, + 0xcd, 0x7b, 0x30, 0xaf, 0x8a, 0x74, 0x56, 0x69, 0x95, 0xfa, 0x50, 0x5a, 0x9e, 0xac, 0xaf, 0x5d, + 0x2e, 0xbe, 0xbe, 0x76, 0x2d, 0x5b, 0x5b, 0x7b, 0xef, 0x4f, 0x3c, 0x70, 0x5f, 0x7e, 0xe2, 0x5f, + 0xb2, 0xb8, 0xe2, 0xc9, 0x7c, 0x05, 0x6d, 0x19, 0x58, 0x7b, 0x58, 0x06, 0x4f, 0x42, 0x35, 0x26, + 0xde, 0xc6, 0x05, 0xe2, 0x78, 0xc2, 0x82, 0xd0, 0xc7, 0xf8, 0xa2, 0x1d, 0x2b, 0x0c, 0x76, 0x01, + 0xaf, 0xe7, 0x05, 0x37, 0x78, 0xad, 0x63, 0x6e, 0x4b, 0xe8, 0x0b, 0x78, 0x15, 0x04, 0x1b, 0x58, + 0xf6, 0xdf, 0x28, 0xf1, 0x11, 0x28, 0x62, 0x41, 0x9e, 0xcf, 0x5c, 0x99, 0xd8, 0x7f, 0x18, 0xc5, + 0xc7, 0x00, 0x1a, 0x41, 0x3b, 0xa4, 0xe6, 0xef, 0x5a, 0x20, 0x8e, 0xc6, 0x2e, 0x1c, 0xfa, 0x26, + 0x74, 0x41, 0x4f, 0xbf, 0x86, 0x6e, 0xc3, 0x06, 0xbf, 0x94, 0x2e, 0x2d, 0xef, 0xab, 0x4b, 0x53, + 0x6a, 0x65, 0x60, 0x9f, 0xd5, 0xee, 0x4f, 0x2d, 0x48, 0x59, 0x44, 0x28, 0x84, 0x0a, 0x15, 0x77, + 0x47, 0xcc, 0xd0, 0xe5, 0xe2, 0xcc, 0x2f, 0xaa, 0x1a, 0xc5, 0xb0, 0x67, 0x3f, 0x31, 0x67, 0x84, + 0x3c, 0x11, 0x32, 0xc2, 0x7b, 0xf5, 0x72, 0x71, 0x0c, 0x2f, 0x04, 0xc1, 0x16, 0x3f, 0xdf, 0xd5, + 0xe1, 0x27, 0xf6, 0xf3, 0x70, 0xbc, 0x4b, 0x28, 0x76, 0x3b, 0x5a, 0x40, 0x57, 0x9f, 0xcc, 0x70, + 0x65, 0x79, 0xac, 0x98, 0xc3, 0xec, 0x6f, 0x5a, 0x70, 0x2c, 0x4b, 0x1e, 0xbd, 0x65, 0xc1, 0xf1, + 0x38, 0x4b, 0xef, 0xa8, 0xfa, 0x4e, 0x85, 0x7d, 0x76, 0x81, 0x70, 0xb7, 0x10, 0xf6, 0xff, 0x11, + 0x83, 0xff, 0x9a, 0xeb, 0x37, 0x83, 0x1b, 0xca, 0x30, 0xb1, 0x7a, 0x1a, 0x26, 0x74, 0x3e, 0x36, + 0x36, 0x49, 0xb3, 0xe3, 0x75, 0x25, 0xd0, 0xae, 0x8a, 0x76, 0xac, 0x30, 0x58, 0xbe, 0x60, 0x47, + 0x14, 0xe0, 0xcf, 0x0c, 0xca, 0x39, 0xd1, 0x8e, 0x15, 0x06, 0x7a, 0x06, 0x46, 0x8c, 0x97, 0x94, + 0xe3, 0x92, 0x19, 0xe4, 0xc6, 0x92, 0x19, 0xe3, 0x14, 0x16, 0x9a, 0x02, 0x50, 0x46, 0x8e, 0x5c, + 0x22, 0x99, 0xbf, 0x4c, 0x69, 0xa2, 0x18, 0x1b, 0x18, 0x2c, 0x3b, 0xd7, 0xeb, 0xc4, 0xec, 0xa8, + 0x63, 0x50, 0x57, 0x54, 0x9d, 0x15, 0x6d, 0x58, 0x41, 0xa9, 0x36, 0x69, 0x3b, 0x7e, 0xc7, 0xf1, + 0x68, 0x0f, 0x89, 0x1d, 0xb0, 0x9a, 0x86, 0x4b, 0x0a, 0x82, 0x0d, 0x2c, 0xfa, 0xc6, 0x89, 0xdb, + 0x26, 0x2f, 0x07, 0xbe, 0x0c, 0xd7, 0xd3, 0xa7, 0x5f, 0xa2, 0x1d, 0x2b, 0x0c, 0xfb, 0xbf, 0x58, + 0x30, 0xae, 0x73, 0xfd, 0xf9, 0x3d, 0xe8, 0xe6, 0x86, 0xdd, 0xda, 0x77, 0xc3, 0x9e, 0x4e, 0x82, + 0x2e, 0xf5, 0x95, 0x04, 0x6d, 0xe6, 0x27, 0x97, 0xf7, 0xcc, 0x4f, 0xfe, 0x51, 0x7d, 0xc7, 0x2e, + 0x4f, 0x64, 0x1e, 0xce, 0xbb, 0x5f, 0x17, 0xd9, 0x30, 0xd8, 0x70, 0x54, 0xa1, 0x9b, 0x11, 0xbe, + 0x77, 0x98, 0x9d, 0x61, 0x48, 0x02, 0x62, 0x2f, 0x43, 0x4d, 0x1d, 0x02, 0xc9, 0xfd, 0xba, 0x95, + 0xbf, 0x5f, 0xef, 0x2b, 0x4f, 0xb2, 0xbe, 0xfe, 0xad, 0xef, 0x3f, 0xfa, 0x8e, 0xdf, 0xff, 0xfe, + 0xa3, 0xef, 0xf8, 0xa3, 0xef, 0x3f, 0xfa, 0x8e, 0x37, 0x6e, 0x3d, 0x6a, 0x7d, 0xeb, 0xd6, 0xa3, + 0xd6, 0xef, 0xdf, 0x7a, 0xd4, 0xfa, 0xa3, 0x5b, 0x8f, 0x5a, 0xdf, 0xbb, 0xf5, 0xa8, 0xf5, 0xa5, + 0xff, 0xf8, 0xe8, 0x3b, 0x5e, 0xce, 0x8d, 0xd7, 0xa4, 0x3f, 0x9e, 0x6a, 0x34, 0xa7, 0xb7, 0xcf, + 0xb2, 0x90, 0x41, 0x3a, 0xbd, 0xa6, 0x8d, 0x31, 0x35, 0x2d, 0xa7, 0xd7, 0xff, 0x0d, 0x00, 0x00, + 0xff, 0xff, 0x21, 0x2d, 0xf5, 0xe3, 0x49, 0xe6, 0x00, 0x00, } func (m *AWSAuthConfig) Marshal() (dAtA []byte, err error) { @@ -5619,6 +5728,73 @@ func (m *ApplicationCondition) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *ApplicationDependency) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ApplicationDependency) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ApplicationDependency) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.RefreshDependencies != nil { + i-- + if *m.RefreshDependencies { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x28 + } + if m.Timeout != nil { + i = encodeVarintGenerated(dAtA, i, uint64(*m.Timeout)) + i-- + dAtA[i] = 0x20 + } + if m.SyncDelay != nil { + i = encodeVarintGenerated(dAtA, i, uint64(*m.SyncDelay)) + i-- + dAtA[i] = 0x18 + } + if m.BlockOnEmpty != nil { + i-- + if *m.BlockOnEmpty { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if len(m.Selectors) > 0 { + for iNdEx := len(m.Selectors) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Selectors[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintGenerated(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func (m *ApplicationDestination) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -5787,6 +5963,50 @@ func (m *ApplicationPreservedFields) MarshalToSizedBuffer(dAtA []byte) (int, err return len(dAtA) - i, nil } +func (m *ApplicationSelector) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ApplicationSelector) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ApplicationSelector) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.NamePattern) > 0 { + for iNdEx := len(m.NamePattern) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.NamePattern[iNdEx]) + copy(dAtA[i:], m.NamePattern[iNdEx]) + i = encodeVarintGenerated(dAtA, i, uint64(len(m.NamePattern[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if m.LabelSelector != nil { + { + size, err := m.LabelSelector.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintGenerated(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *ApplicationSet) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -7547,6 +7767,18 @@ func (m *ApplicationSpec) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.DependsOn != nil { + { + size, err := m.DependsOn.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintGenerated(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } if len(m.Sources) > 0 { for iNdEx := len(m.Sources) - 1; iNdEx >= 0; iNdEx-- { { @@ -10380,6 +10612,28 @@ func (m *OperationState) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + i-- + if m.BlockedOnEmpty { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x50 + if len(m.WaitingFor) > 0 { + for iNdEx := len(m.WaitingFor) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.WaitingFor[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintGenerated(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + } i = encodeVarintGenerated(dAtA, i, uint64(m.RetryCount)) i-- dAtA[i] = 0x40 @@ -13653,6 +13907,51 @@ func (m *SignatureKey) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *SyncDependency) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncDependency) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncDependency) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.RefreshedAt != nil { + { + size, err := m.RefreshedAt.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintGenerated(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + i -= len(m.ApplicationNamespace) + copy(dAtA[i:], m.ApplicationNamespace) + i = encodeVarintGenerated(dAtA, i, uint64(len(m.ApplicationNamespace))) + i-- + dAtA[i] = 0x12 + i -= len(m.ApplicationName) + copy(dAtA[i:], m.ApplicationName) + i = encodeVarintGenerated(dAtA, i, uint64(len(m.ApplicationName))) + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + func (m *SyncOperation) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -14540,6 +14839,33 @@ func (m *ApplicationCondition) Size() (n int) { return n } +func (m *ApplicationDependency) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Selectors) > 0 { + for _, e := range m.Selectors { + l = e.Size() + n += 1 + l + sovGenerated(uint64(l)) + } + } + if m.BlockOnEmpty != nil { + n += 2 + } + if m.SyncDelay != nil { + n += 1 + sovGenerated(uint64(*m.SyncDelay)) + } + if m.Timeout != nil { + n += 1 + sovGenerated(uint64(*m.Timeout)) + } + if m.RefreshDependencies != nil { + n += 2 + } + return n +} + func (m *ApplicationDestination) Size() (n int) { if m == nil { return 0 @@ -14612,6 +14938,25 @@ func (m *ApplicationPreservedFields) Size() (n int) { return n } +func (m *ApplicationSelector) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LabelSelector != nil { + l = m.LabelSelector.Size() + n += 1 + l + sovGenerated(uint64(l)) + } + if len(m.NamePattern) > 0 { + for _, s := range m.NamePattern { + l = len(s) + n += 1 + l + sovGenerated(uint64(l)) + } + } + return n +} + func (m *ApplicationSet) Size() (n int) { if m == nil { return 0 @@ -15291,6 +15636,10 @@ func (m *ApplicationSpec) Size() (n int) { n += 1 + l + sovGenerated(uint64(l)) } } + if m.DependsOn != nil { + l = m.DependsOn.Size() + n += 1 + l + sovGenerated(uint64(l)) + } return n } @@ -16338,6 +16687,13 @@ func (m *OperationState) Size() (n int) { n += 1 + l + sovGenerated(uint64(l)) } n += 1 + sovGenerated(uint64(m.RetryCount)) + if len(m.WaitingFor) > 0 { + for _, e := range m.WaitingFor { + l = e.Size() + n += 1 + l + sovGenerated(uint64(l)) + } + } + n += 2 return n } @@ -17538,6 +17894,23 @@ func (m *SignatureKey) Size() (n int) { return n } +func (m *SyncDependency) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ApplicationName) + n += 1 + l + sovGenerated(uint64(l)) + l = len(m.ApplicationNamespace) + n += 1 + l + sovGenerated(uint64(l)) + if m.RefreshedAt != nil { + l = m.RefreshedAt.Size() + n += 1 + l + sovGenerated(uint64(l)) + } + return n +} + func (m *SyncOperation) Size() (n int) { if m == nil { return 0 @@ -17965,6 +18338,25 @@ func (this *ApplicationCondition) String() string { }, "") return s } +func (this *ApplicationDependency) String() string { + if this == nil { + return "nil" + } + repeatedStringForSelectors := "[]ApplicationSelector{" + for _, f := range this.Selectors { + repeatedStringForSelectors += strings.Replace(strings.Replace(f.String(), "ApplicationSelector", "ApplicationSelector", 1), `&`, ``, 1) + "," + } + repeatedStringForSelectors += "}" + s := strings.Join([]string{`&ApplicationDependency{`, + `Selectors:` + repeatedStringForSelectors + `,`, + `BlockOnEmpty:` + valueToStringGenerated(this.BlockOnEmpty) + `,`, + `SyncDelay:` + valueToStringGenerated(this.SyncDelay) + `,`, + `Timeout:` + valueToStringGenerated(this.Timeout) + `,`, + `RefreshDependencies:` + valueToStringGenerated(this.RefreshDependencies) + `,`, + `}`, + }, "") + return s +} func (this *ApplicationDestination) String() string { if this == nil { return "nil" @@ -18016,6 +18408,17 @@ func (this *ApplicationPreservedFields) String() string { }, "") return s } +func (this *ApplicationSelector) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ApplicationSelector{`, + `LabelSelector:` + strings.Replace(fmt.Sprintf("%v", this.LabelSelector), "LabelSelector", "v1.LabelSelector", 1) + `,`, + `NamePattern:` + fmt.Sprintf("%v", this.NamePattern) + `,`, + `}`, + }, "") + return s +} func (this *ApplicationSet) String() string { if this == nil { return "nil" @@ -18485,6 +18888,7 @@ func (this *ApplicationSpec) String() string { `Info:` + repeatedStringForInfo + `,`, `RevisionHistoryLimit:` + valueToStringGenerated(this.RevisionHistoryLimit) + `,`, `Sources:` + repeatedStringForSources + `,`, + `DependsOn:` + strings.Replace(this.DependsOn.String(), "ApplicationDependency", "ApplicationDependency", 1) + `,`, `}`, }, "") return s @@ -19333,6 +19737,11 @@ func (this *OperationState) String() string { if this == nil { return "nil" } + repeatedStringForWaitingFor := "[]SyncDependency{" + for _, f := range this.WaitingFor { + repeatedStringForWaitingFor += strings.Replace(strings.Replace(f.String(), "SyncDependency", "SyncDependency", 1), `&`, ``, 1) + "," + } + repeatedStringForWaitingFor += "}" s := strings.Join([]string{`&OperationState{`, `Operation:` + strings.Replace(strings.Replace(this.Operation.String(), "Operation", "Operation", 1), `&`, ``, 1) + `,`, `Phase:` + fmt.Sprintf("%v", this.Phase) + `,`, @@ -19341,6 +19750,8 @@ func (this *OperationState) String() string { `StartedAt:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.StartedAt), "Time", "v1.Time", 1), `&`, ``, 1) + `,`, `FinishedAt:` + strings.Replace(fmt.Sprintf("%v", this.FinishedAt), "Time", "v1.Time", 1) + `,`, `RetryCount:` + fmt.Sprintf("%v", this.RetryCount) + `,`, + `WaitingFor:` + repeatedStringForWaitingFor + `,`, + `BlockedOnEmpty:` + fmt.Sprintf("%v", this.BlockedOnEmpty) + `,`, `}`, }, "") return s @@ -20198,6 +20609,18 @@ func (this *SignatureKey) String() string { }, "") return s } +func (this *SyncDependency) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SyncDependency{`, + `ApplicationName:` + fmt.Sprintf("%v", this.ApplicationName) + `,`, + `ApplicationNamespace:` + fmt.Sprintf("%v", this.ApplicationNamespace) + `,`, + `RefreshedAt:` + strings.Replace(fmt.Sprintf("%v", this.RefreshedAt), "Time", "v1.Time", 1) + `,`, + `}`, + }, "") + return s +} func (this *SyncOperation) String() string { if this == nil { return "nil" @@ -21784,7 +22207,7 @@ func (m *ApplicationCondition) Unmarshal(dAtA []byte) error { } return nil } -func (m *ApplicationDestination) Unmarshal(dAtA []byte) error { +func (m *ApplicationDependency) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21807,17 +22230,17 @@ func (m *ApplicationDestination) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplicationDestination: wiretype end group for non-group") + return fmt.Errorf("proto: ApplicationDependency: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplicationDestination: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ApplicationDependency: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Server", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Selectors", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowGenerated @@ -21827,29 +22250,31 @@ func (m *ApplicationDestination) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthGenerated } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthGenerated } if postIndex > l { return io.ErrUnexpectedEOF } - m.Server = string(dAtA[iNdEx:postIndex]) + m.Selectors = append(m.Selectors, ApplicationSelector{}) + if err := m.Selectors[len(m.Selectors)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Namespace", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockOnEmpty", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowGenerated @@ -21859,29 +22284,18 @@ func (m *ApplicationDestination) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthGenerated - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthGenerated - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Namespace = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + b := bool(v != 0) + m.BlockOnEmpty = &b case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncDelay", wireType) } - var stringLen uint64 + var v time.Duration for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowGenerated @@ -21891,24 +22305,53 @@ func (m *ApplicationDestination) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= time.Duration(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthGenerated + m.SyncDelay = &v + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timeout", wireType) } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthGenerated + var v time.Duration + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= time.Duration(b&0x7F) << shift + if b < 0x80 { + break + } } - if postIndex > l { - return io.ErrUnexpectedEOF + m.Timeout = &v + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RefreshDependencies", wireType) } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.RefreshDependencies = &b default: iNdEx = preIndex skippy, err := skipGenerated(dAtA[iNdEx:]) @@ -21930,7 +22373,7 @@ func (m *ApplicationDestination) Unmarshal(dAtA []byte) error { } return nil } -func (m *ApplicationList) Unmarshal(dAtA []byte) error { +func (m *ApplicationDestination) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21953,17 +22396,17 @@ func (m *ApplicationList) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplicationList: wiretype end group for non-group") + return fmt.Errorf("proto: ApplicationDestination: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplicationList: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ApplicationDestination: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ListMeta", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Server", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowGenerated @@ -21973,30 +22416,29 @@ func (m *ApplicationList) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthGenerated } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthGenerated } if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.ListMeta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Server = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Items", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Namespace", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowGenerated @@ -22006,25 +22448,55 @@ func (m *ApplicationList) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthGenerated } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthGenerated } if postIndex > l { return io.ErrUnexpectedEOF } - m.Items = append(m.Items, Application{}) - if err := m.Items[len(m.Items)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.Namespace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF } + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -22047,7 +22519,7 @@ func (m *ApplicationList) Unmarshal(dAtA []byte) error { } return nil } -func (m *ApplicationMatchExpression) Unmarshal(dAtA []byte) error { +func (m *ApplicationList) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -22070,17 +22542,17 @@ func (m *ApplicationMatchExpression) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplicationMatchExpression: wiretype end group for non-group") + return fmt.Errorf("proto: ApplicationList: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplicationMatchExpression: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ApplicationList: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ListMeta", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowGenerated @@ -22090,29 +22562,30 @@ func (m *ApplicationMatchExpression) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthGenerated } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthGenerated } if postIndex > l { return io.ErrUnexpectedEOF } - m.Key = string(dAtA[iNdEx:postIndex]) + if err := m.ListMeta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Operator", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Items", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowGenerated @@ -22122,55 +22595,171 @@ func (m *ApplicationMatchExpression) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthGenerated } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthGenerated } if postIndex > l { return io.ErrUnexpectedEOF } - m.Operator = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Values", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowGenerated - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthGenerated - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthGenerated - } - if postIndex > l { - return io.ErrUnexpectedEOF + m.Items = append(m.Items, Application{}) + if err := m.Items[len(m.Items)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipGenerated(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthGenerated + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ApplicationMatchExpression) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ApplicationMatchExpression: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplicationMatchExpression: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Operator", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Operator = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Values", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -22307,6 +22896,124 @@ func (m *ApplicationPreservedFields) Unmarshal(dAtA []byte) error { } return nil } +func (m *ApplicationSelector) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ApplicationSelector: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplicationSelector: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LabelSelector", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.LabelSelector == nil { + m.LabelSelector = &v1.LabelSelector{} + } + if err := m.LabelSelector.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NamePattern", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NamePattern = append(m.NamePattern, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipGenerated(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthGenerated + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ApplicationSet) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -28010,6 +28717,42 @@ func (m *ApplicationSpec) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DependsOn", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DependsOn == nil { + m.DependsOn = &ApplicationDependency{} + } + if err := m.DependsOn.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipGenerated(dAtA[iNdEx:]) @@ -37207,6 +37950,60 @@ func (m *OperationState) Unmarshal(dAtA []byte) error { break } } + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WaitingFor", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WaitingFor = append(m.WaitingFor, SyncDependency{}) + if err := m.WaitingFor[len(m.WaitingFor)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockedOnEmpty", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.BlockedOnEmpty = bool(v != 0) default: iNdEx = preIndex skippy, err := skipGenerated(dAtA[iNdEx:]) @@ -48439,6 +49236,156 @@ func (m *SignatureKey) Unmarshal(dAtA []byte) error { } return nil } +func (m *SyncDependency) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncDependency: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncDependency: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ApplicationName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ApplicationName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ApplicationNamespace", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ApplicationNamespace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RefreshedAt", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenerated + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthGenerated + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthGenerated + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RefreshedAt == nil { + m.RefreshedAt = &v1.Time{} + } + if err := m.RefreshedAt.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipGenerated(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthGenerated + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *SyncOperation) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/apis/application/v1alpha1/generated.proto b/pkg/apis/application/v1alpha1/generated.proto index 5916e42a539226..2eb459a97e8306 100644 --- a/pkg/apis/application/v1alpha1/generated.proto +++ b/pkg/apis/application/v1alpha1/generated.proto @@ -131,6 +131,24 @@ message ApplicationCondition { optional k8s.io.apimachinery.pkg.apis.meta.v1.Time lastTransitionTime = 3; } +// ApplicationDependency defines +message ApplicationDependency { + // Selectors defines conditions for matching application's dependencies + repeated ApplicationSelector selectors = 1; + + // BlockOnEmpty defines whether to block sync when the list of applications determined by the selector is empty + optional bool blockOnEmpty = 2; + + // SyncDelay specifies the duration in seconds to wait before starting to sync when dependencies are defined + optional int64 syncDelay = 3; + + // Timeout defines the maximum duration in seconds to wait on dependencies before the sync fails + optional int64 timeout = 4; + + // RefreshDependencies defines whether all dependencies should be refreshed before starting a sync + optional bool refreshDependencies = 5; +} + // ApplicationDestination holds information about the application's destination message ApplicationDestination { // Server specifies the URL of the target cluster's Kubernetes control plane API. This must be set if Name is not set. @@ -166,6 +184,15 @@ message ApplicationPreservedFields { repeated string labels = 2; } +// ApplicationSelector specifies which applications this Application depends on +message ApplicationSelector { + // LabelSelector selects applications by their labels + optional k8s.io.apimachinery.pkg.apis.meta.v1.LabelSelector labelSelector = 1; + + // NamePattern selects applications by matching their names + repeated string namePattern = 2; +} + // ApplicationSet is a set of Application resources // +genclient // +genclient:noStatus @@ -587,6 +614,9 @@ message ApplicationSpec { // Sources is a reference to the location of the application's manifests or chart repeated ApplicationSource sources = 8; + + // DependsOn specifies what this application depends on + optional ApplicationDependency dependsOn = 9; } // ApplicationStatus contains status information for the application @@ -1246,6 +1276,12 @@ message OperationState { // RetryCount contains time of operation retries optional int64 retryCount = 8; + + // WaitingFor specifies a list of applications that this operation is waiting for + repeated SyncDependency waitingFor = 9; + + // BlockedOnEmpty is true when the application is waiting for any dependency to be created + optional bool blockedOnEmpty = 10; } message OptionalArray { @@ -2121,6 +2157,14 @@ message SignatureKey { optional string keyID = 1; } +message SyncDependency { + optional string applicationName = 1; + + optional string applicationNamespace = 2; + + optional k8s.io.apimachinery.pkg.apis.meta.v1.Time refreshedAt = 3; +} + // SyncOperation contains details about a sync operation. message SyncOperation { // Revision is the revision (Git) or chart version (Helm) which to sync the application to diff --git a/pkg/apis/application/v1alpha1/openapi_generated.go b/pkg/apis/application/v1alpha1/openapi_generated.go index 32eb8a725f3534..959210ed7c6232 100644 --- a/pkg/apis/application/v1alpha1/openapi_generated.go +++ b/pkg/apis/application/v1alpha1/openapi_generated.go @@ -21,10 +21,12 @@ func GetOpenAPIDefinitions(ref common.ReferenceCallback) map[string]common.OpenA "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.AppProjectStatus": schema_pkg_apis_application_v1alpha1_AppProjectStatus(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.Application": schema_pkg_apis_application_v1alpha1_Application(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationCondition": schema_pkg_apis_application_v1alpha1_ApplicationCondition(ref), + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationDependency": schema_pkg_apis_application_v1alpha1_ApplicationDependency(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationDestination": schema_pkg_apis_application_v1alpha1_ApplicationDestination(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationList": schema_pkg_apis_application_v1alpha1_ApplicationList(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationMatchExpression": schema_pkg_apis_application_v1alpha1_ApplicationMatchExpression(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationPreservedFields": schema_pkg_apis_application_v1alpha1_ApplicationPreservedFields(ref), + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationSelector": schema_pkg_apis_application_v1alpha1_ApplicationSelector(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationSet": schema_pkg_apis_application_v1alpha1_ApplicationSet(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationSetApplicationStatus": schema_pkg_apis_application_v1alpha1_ApplicationSetApplicationStatus(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationSetCondition": schema_pkg_apis_application_v1alpha1_ApplicationSetCondition(ref), @@ -153,6 +155,7 @@ func GetOpenAPIDefinitions(ref common.ReferenceCallback) map[string]common.OpenA "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SCMProviderGeneratorGitlab": schema_pkg_apis_application_v1alpha1_SCMProviderGeneratorGitlab(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SecretRef": schema_pkg_apis_application_v1alpha1_SecretRef(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SignatureKey": schema_pkg_apis_application_v1alpha1_SignatureKey(ref), + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncDependency": schema_pkg_apis_application_v1alpha1_SyncDependency(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncOperation": schema_pkg_apis_application_v1alpha1_SyncOperation(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncOperationResource": schema_pkg_apis_application_v1alpha1_SyncOperationResource(ref), "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncOperationResult": schema_pkg_apis_application_v1alpha1_SyncOperationResult(ref), @@ -598,6 +601,64 @@ func schema_pkg_apis_application_v1alpha1_ApplicationCondition(ref common.Refere } } +func schema_pkg_apis_application_v1alpha1_ApplicationDependency(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "ApplicationDependency defines", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "selectors": { + SchemaProps: spec.SchemaProps{ + Description: "Selectors defines conditions for matching application's dependencies", + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationSelector"), + }, + }, + }, + }, + }, + "blockOnEmpty": { + SchemaProps: spec.SchemaProps{ + Description: "BlockOnEmpty defines whether to block sync when the list of applications determined by the selector is empty", + Type: []string{"boolean"}, + Format: "", + }, + }, + "syncDelay": { + SchemaProps: spec.SchemaProps{ + Description: "SyncDelay specifies the duration in seconds to wait before starting to sync when dependencies are defined", + Type: []string{"integer"}, + Format: "int64", + }, + }, + "timeout": { + SchemaProps: spec.SchemaProps{ + Description: "Timeout defines the maximum duration in seconds to wait on dependencies before the sync fails", + Type: []string{"integer"}, + Format: "int64", + }, + }, + "refreshDependencies": { + SchemaProps: spec.SchemaProps{ + Description: "RefreshDependencies defines whether all dependencies should be refreshed before starting a sync", + Type: []string{"boolean"}, + Format: "", + }, + }, + }, + Required: []string{"selectors"}, + }, + }, + Dependencies: []string{ + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationSelector"}, + } +} + func schema_pkg_apis_application_v1alpha1_ApplicationDestination(ref common.ReferenceCallback) common.OpenAPIDefinition { return common.OpenAPIDefinition{ Schema: spec.Schema{ @@ -759,6 +820,42 @@ func schema_pkg_apis_application_v1alpha1_ApplicationPreservedFields(ref common. } } +func schema_pkg_apis_application_v1alpha1_ApplicationSelector(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "ApplicationSelector specifies which applications this Application depends on", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "labelSelector": { + SchemaProps: spec.SchemaProps{ + Description: "LabelSelector selects applications by their labels", + Ref: ref("k8s.io/apimachinery/pkg/apis/meta/v1.LabelSelector"), + }, + }, + "namePattern": { + SchemaProps: spec.SchemaProps{ + Description: "NamePattern selects applications by matching their names", + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: "", + Type: []string{"string"}, + Format: "", + }, + }, + }, + }, + }, + }, + }, + }, + Dependencies: []string{ + "k8s.io/apimachinery/pkg/apis/meta/v1.LabelSelector"}, + } +} + func schema_pkg_apis_application_v1alpha1_ApplicationSet(ref common.ReferenceCallback) common.OpenAPIDefinition { return common.OpenAPIDefinition{ Schema: spec.Schema{ @@ -2155,12 +2252,18 @@ func schema_pkg_apis_application_v1alpha1_ApplicationSpec(ref common.ReferenceCa }, }, }, + "dependsOn": { + SchemaProps: spec.SchemaProps{ + Description: "DependsOn specifies what this application depends on", + Ref: ref("github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationDependency"), + }, + }, }, Required: []string{"destination", "project"}, }, }, Dependencies: []string{ - "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationDestination", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationSource", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.Info", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ResourceIgnoreDifferences", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncPolicy"}, + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationDependency", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationDestination", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ApplicationSource", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.Info", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.ResourceIgnoreDifferences", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncPolicy"}, } } @@ -4480,12 +4583,33 @@ func schema_pkg_apis_application_v1alpha1_OperationState(ref common.ReferenceCal Format: "int64", }, }, + "waitingFor": { + SchemaProps: spec.SchemaProps{ + Description: "WaitingFor specifies a list of applications that this operation is waiting for", + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncDependency"), + }, + }, + }, + }, + }, + "blockedOnEmpty": { + SchemaProps: spec.SchemaProps{ + Description: "BlockedOnEmpty is true when the application is waiting for any dependency to be created", + Type: []string{"boolean"}, + Format: "", + }, + }, }, Required: []string{"operation", "phase", "startedAt"}, }, }, Dependencies: []string{ - "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.Operation", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncOperationResult", "k8s.io/apimachinery/pkg/apis/meta/v1.Time"}, + "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.Operation", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncDependency", "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1.SyncOperationResult", "k8s.io/apimachinery/pkg/apis/meta/v1.Time"}, } } @@ -7332,6 +7456,40 @@ func schema_pkg_apis_application_v1alpha1_SignatureKey(ref common.ReferenceCallb } } +func schema_pkg_apis_application_v1alpha1_SyncDependency(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "applicationName": { + SchemaProps: spec.SchemaProps{ + Default: "", + Type: []string{"string"}, + Format: "", + }, + }, + "applicationNamespace": { + SchemaProps: spec.SchemaProps{ + Default: "", + Type: []string{"string"}, + Format: "", + }, + }, + "refreshedAt": { + SchemaProps: spec.SchemaProps{ + Ref: ref("k8s.io/apimachinery/pkg/apis/meta/v1.Time"), + }, + }, + }, + Required: []string{"applicationName", "applicationNamespace"}, + }, + }, + Dependencies: []string{ + "k8s.io/apimachinery/pkg/apis/meta/v1.Time"}, + } +} + func schema_pkg_apis_application_v1alpha1_SyncOperation(ref common.ReferenceCallback) common.OpenAPIDefinition { return common.OpenAPIDefinition{ Schema: spec.Schema{ diff --git a/pkg/apis/application/v1alpha1/types.go b/pkg/apis/application/v1alpha1/types.go index fc2908c4643dc5..a657fec569a5ea 100644 --- a/pkg/apis/application/v1alpha1/types.go +++ b/pkg/apis/application/v1alpha1/types.go @@ -25,6 +25,7 @@ import ( v1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + "k8s.io/apimachinery/pkg/labels" "k8s.io/apimachinery/pkg/runtime" "k8s.io/apimachinery/pkg/runtime/schema" "k8s.io/apimachinery/pkg/util/intstr" @@ -38,6 +39,7 @@ import ( "github.com/argoproj/argo-cd/v2/common" "github.com/argoproj/argo-cd/v2/util/collections" "github.com/argoproj/argo-cd/v2/util/env" + "github.com/argoproj/argo-cd/v2/util/glob" "github.com/argoproj/argo-cd/v2/util/helm" utilhttp "github.com/argoproj/argo-cd/v2/util/http" "github.com/argoproj/argo-cd/v2/util/security" @@ -83,6 +85,9 @@ type ApplicationSpec struct { // Sources is a reference to the location of the application's manifests or chart Sources ApplicationSources `json:"sources,omitempty" protobuf:"bytes,8,opt,name=sources"` + + // DependsOn specifies what this application depends on + DependsOn *ApplicationDependency `json:"dependsOn,omitempty" protobuf:"bytes,9,opt,name=dependsOn"` } type IgnoreDifferences []ResourceIgnoreDifferences @@ -91,6 +96,61 @@ func (id IgnoreDifferences) Equals(other IgnoreDifferences) bool { return reflect.DeepEqual(id, other) } +// ApplicationDependency defines +type ApplicationDependency struct { + // Selectors defines conditions for matching application's dependencies + Selectors []ApplicationSelector `json:"selectors" protobuf:"bytes,1,name=selectors"` + // BlockOnEmpty defines whether to block sync when the list of applications determined by the selector is empty + BlockOnEmpty *bool `json:"blockOnEmpty,omitempty" protobuf:"bytes,2,opt,name=blockOnEmpty"` + // SyncDelay specifies the duration in seconds to wait before starting to sync when dependencies are defined + SyncDelay *time.Duration `json:"syncDelay,omitempty" protobuf:"bytes,3,opt,name=syncDelay"` + // Timeout defines the maximum duration in seconds to wait on dependencies before the sync fails + Timeout *time.Duration `json:"timeout,omitempty" protobuf:"bytes,4,opt,name=timeout"` + // RefreshDependencies defines whether all dependencies should be refreshed before starting a sync + RefreshDependencies *bool `json:"refreshDependencies,omitempty" protobuf:"bytes,5,opt,name=refreshDependencies"` +} + +// IsBlockOnEmptyDependencies returns true if the +func (a *Application) IsBlockOnEmptyDependencies() bool { + if a.Spec.DependsOn != nil && a.Spec.DependsOn.BlockOnEmpty != nil { + return *a.Spec.DependsOn.BlockOnEmpty + } else { + return false + } +} + +// ApplicationSelector specifies which applications this Application depends on +type ApplicationSelector struct { + // LabelSelector selects applications by their labels + LabelSelector *metav1.LabelSelector `json:"labelSelector,omitempty" protobuf:"bytes,1,name=labelSelector"` + // NamePattern selects applications by matching their names + NamePattern []string `json:"namePattern,omitempty" protobuf:"bytes,2,name=namePattern"` +} + +// MatchesSelector returns whether an Application matches the conditions in the given selector +func (app *Application) MatchesSelector(s *ApplicationSelector) bool { + logCtx := log.WithField("application", app.QualifiedName()) + + if s.LabelSelector != nil { + sel, err := metav1.LabelSelectorAsSelector(s.LabelSelector) + if err != nil { + logCtx.Warnf("invalid label selector: %v", err) + return false + } + if !sel.Matches(labels.Set(app.ObjectMeta.Labels)) { + return false + } + } + + if len(s.NamePattern) > 0 { + if !glob.MatchStringInList(s.NamePattern, app.GetName(), false) { + return false + } + } + + return true +} + type TrackingMethod string // ResourceIgnoreDifferences contains resource filter and list of json paths which should be ignored during comparison with live state. @@ -1114,6 +1174,10 @@ type OperationState struct { FinishedAt *metav1.Time `json:"finishedAt,omitempty" protobuf:"bytes,7,opt,name=finishedAt"` // RetryCount contains time of operation retries RetryCount int64 `json:"retryCount,omitempty" protobuf:"bytes,8,opt,name=retryCount"` + // WaitingFor specifies a list of applications that this operation is waiting for + WaitingFor []SyncDependency `json:"waitingFor,omitempty" protobuf:"bytes,9,opt,name=waitingFor"` + // BlockedOnEmpty is true when the application is waiting for any dependency to be created + BlockedOnEmpty bool `json:"blockedOnEmpty,omitempty" protobuf:"bytes,10,opt,name=blockedOnEmpty"` } type Info struct { @@ -1496,6 +1560,16 @@ type ComparedTo struct { IgnoreDifferences IgnoreDifferences `json:"ignoreDifferences,omitempty" protobuf:"bytes,4,opt,name=ignoreDifferences"` } +type SyncDependency struct { + ApplicationName string `json:"applicationName" protobuf:"bytes,1,name=applicationName"` + ApplicationNamespace string `json:"applicationNamespace" protobuf:"bytes,2,name=applicationNamespace"` + RefreshedAt *metav1.Time `json:"refreshedAt,omitempty" protobuf:"bytes,3,opt,name=refreshedAt"` +} + +func (sd SyncDependency) QualifiedName() string { + return sd.ApplicationNamespace + "/" + sd.ApplicationName +} + // SyncStatus contains information about the currently observed live and desired states of an application type SyncStatus struct { // Status is the sync state of the comparison @@ -3136,3 +3210,15 @@ func (a *Application) QualifiedName() string { func (a *Application) RBACName(defaultNS string) string { return security.RBACName(defaultNS, a.Spec.GetProject(), a.Namespace, a.Name) } + +// IsWaiting returns whether the application is waiting for dependencies to be +// synced and/or created. +func (a *Application) IsWaiting() bool { + return a.Status.OperationState != nil && + a.Status.OperationState.Phase == synccommon.OperationRunning && + (len(a.Status.OperationState.WaitingFor) > 0 || a.Status.OperationState.BlockedOnEmpty) +} + +func (a *Application) IsAutomated() bool { + return a.Spec.SyncPolicy != nil && a.Spec.SyncPolicy.Automated != nil +} diff --git a/pkg/apis/application/v1alpha1/types_test.go b/pkg/apis/application/v1alpha1/types_test.go index 2374f5fb503e66..e9140fbbc9a43a 100644 --- a/pkg/apis/application/v1alpha1/types_test.go +++ b/pkg/apis/application/v1alpha1/types_test.go @@ -3620,3 +3620,82 @@ func TestOptionalMapEquality(t *testing.T) { }) } } + +func TestAppMatchesSelector(t *testing.T) { + newApp := func(labels map[string]string, name, project string) *Application { + return &Application{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + Namespace: "argocd", + Labels: labels, + }, + Spec: ApplicationSpec{ + Project: project, + }, + } + } + + t.Run("Matches empty", func(t *testing.T) { + app := newApp(map[string]string{"foo": "bar"}, "app", "default") + sel := &ApplicationSelector{} + assert.True(t, app.MatchesSelector(sel)) + }) + + t.Run("Matches label selector", func(t *testing.T) { + app := newApp(map[string]string{"foo": "bar"}, "app", "default") + sel := &ApplicationSelector{ + LabelSelector: &metav1.LabelSelector{ + MatchLabels: map[string]string{"foo": "bar"}, + }, + } + assert.True(t, app.MatchesSelector(sel)) + }) + + t.Run("Matches name", func(t *testing.T) { + app := newApp(map[string]string{"foo": "bar"}, "app", "default") + sel := &ApplicationSelector{ + NamePattern: []string{"app"}, + } + assert.True(t, app.MatchesSelector(sel)) + }) + + t.Run("Does not match label selector", func(t *testing.T) { + app := newApp(map[string]string{"foo": "bar"}, "app", "default") + sel := &ApplicationSelector{ + LabelSelector: &metav1.LabelSelector{ + MatchLabels: map[string]string{"foo": "baz"}, + }, + } + assert.False(t, app.MatchesSelector(sel)) + }) + + t.Run("Does not match name", func(t *testing.T) { + app := newApp(map[string]string{"foo": "bar"}, "app", "default") + sel := &ApplicationSelector{ + NamePattern: []string{"other"}, + } + assert.False(t, app.MatchesSelector(sel)) + }) + + t.Run("Only label selector does not match", func(t *testing.T) { + app := newApp(map[string]string{"foo": "bar"}, "app", "default") + sel := &ApplicationSelector{ + LabelSelector: &metav1.LabelSelector{ + MatchLabels: map[string]string{"foo": "baz"}, + }, + NamePattern: []string{"app"}, + } + assert.False(t, app.MatchesSelector(sel)) + }) + + t.Run("Only name does not match", func(t *testing.T) { + app := newApp(map[string]string{"foo": "bar"}, "app", "default") + sel := &ApplicationSelector{ + LabelSelector: &metav1.LabelSelector{ + MatchLabels: map[string]string{"foo": "bar"}, + }, + NamePattern: []string{"other"}, + } + assert.False(t, app.MatchesSelector(sel)) + }) +} diff --git a/pkg/apis/application/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/application/v1alpha1/zz_generated.deepcopy.go index 8c851067a6be37..8a5ae5e6ca7dff 100644 --- a/pkg/apis/application/v1alpha1/zz_generated.deepcopy.go +++ b/pkg/apis/application/v1alpha1/zz_generated.deepcopy.go @@ -6,6 +6,8 @@ package v1alpha1 import ( + time "time" + corev1 "k8s.io/api/core/v1" apiextensionsv1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1" v1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -245,6 +247,49 @@ func (in *ApplicationCondition) DeepCopy() *ApplicationCondition { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *ApplicationDependency) DeepCopyInto(out *ApplicationDependency) { + *out = *in + if in.Selectors != nil { + in, out := &in.Selectors, &out.Selectors + *out = make([]ApplicationSelector, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + if in.BlockOnEmpty != nil { + in, out := &in.BlockOnEmpty, &out.BlockOnEmpty + *out = new(bool) + **out = **in + } + if in.SyncDelay != nil { + in, out := &in.SyncDelay, &out.SyncDelay + *out = new(time.Duration) + **out = **in + } + if in.Timeout != nil { + in, out := &in.Timeout, &out.Timeout + *out = new(time.Duration) + **out = **in + } + if in.RefreshDependencies != nil { + in, out := &in.RefreshDependencies, &out.RefreshDependencies + *out = new(bool) + **out = **in + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ApplicationDependency. +func (in *ApplicationDependency) DeepCopy() *ApplicationDependency { + if in == nil { + return nil + } + out := new(ApplicationDependency) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *ApplicationDestination) DeepCopyInto(out *ApplicationDestination) { *out = *in @@ -341,6 +386,32 @@ func (in *ApplicationPreservedFields) DeepCopy() *ApplicationPreservedFields { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *ApplicationSelector) DeepCopyInto(out *ApplicationSelector) { + *out = *in + if in.LabelSelector != nil { + in, out := &in.LabelSelector, &out.LabelSelector + *out = new(v1.LabelSelector) + (*in).DeepCopyInto(*out) + } + if in.NamePattern != nil { + in, out := &in.NamePattern, &out.NamePattern + *out = make([]string, len(*in)) + copy(*out, *in) + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ApplicationSelector. +func (in *ApplicationSelector) DeepCopy() *ApplicationSelector { + if in == nil { + return nil + } + out := new(ApplicationSelector) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *ApplicationSet) DeepCopyInto(out *ApplicationSet) { *out = *in @@ -1273,6 +1344,11 @@ func (in *ApplicationSpec) DeepCopyInto(out *ApplicationSpec) { (*in)[i].DeepCopyInto(&(*out)[i]) } } + if in.DependsOn != nil { + in, out := &in.DependsOn, &out.DependsOn + *out = new(ApplicationDependency) + (*in).DeepCopyInto(*out) + } return } @@ -2624,6 +2700,13 @@ func (in *OperationState) DeepCopyInto(out *OperationState) { in, out := &in.FinishedAt, &out.FinishedAt *out = (*in).DeepCopy() } + if in.WaitingFor != nil { + in, out := &in.WaitingFor, &out.WaitingFor + *out = make([]SyncDependency, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } return } @@ -4027,6 +4110,26 @@ func (in *SignatureKey) DeepCopy() *SignatureKey { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *SyncDependency) DeepCopyInto(out *SyncDependency) { + *out = *in + if in.RefreshedAt != nil { + in, out := &in.RefreshedAt, &out.RefreshedAt + *out = (*in).DeepCopy() + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new SyncDependency. +func (in *SyncDependency) DeepCopy() *SyncDependency { + if in == nil { + return nil + } + out := new(SyncDependency) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *SyncOperation) DeepCopyInto(out *SyncOperation) { *out = *in diff --git a/test/e2e/app_dependencies_test.go b/test/e2e/app_dependencies_test.go new file mode 100644 index 00000000000000..d761a644831406 --- /dev/null +++ b/test/e2e/app_dependencies_test.go @@ -0,0 +1,145 @@ +package e2e + +import ( + "testing" + "time" + + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/utils/pointer" + + . "github.com/argoproj/argo-cd/v2/pkg/apis/application/v1alpha1" + . "github.com/argoproj/argo-cd/v2/test/e2e/fixture/app" + "github.com/argoproj/gitops-engine/pkg/sync/common" +) + +func TestSyncWithBlockingDependencies(t *testing.T) { + // Parent app + parent := Given(t). + Path(guestbookPath). + When(). + CreateFromFile(func(app *Application) { + app.Spec.DependsOn = &ApplicationDependency{ + Selectors: []ApplicationSelector{ + { + LabelSelector: &v1.LabelSelector{ + MatchLabels: map[string]string{"name": "dep1"}, + }, + }, + }, + BlockOnEmpty: pointer.Bool(true), + } + }) + + // We sync the parent and make sure it's OutOfSync after some time, and + // that the sync operation is still ongoing, and that the application is + // waiting for any of its dependencies to be created. + parent. + Sync("--async"). + Then(). + Expect(SyncStatusIs(SyncStatusCodeOutOfSync)). + Expect(OperationPhaseIs(common.OperationRunning)). + Expect(OperationMessageContains("Waiting for any app to be created")) + + // Dependency app + dep1 := GivenWithSameState(t). + Name("dep1"). + Path("kustomize"). + When(). + CreateFromFile(func(app *Application) { + app.Labels = map[string]string{ + "name": "dep1", + } + }) + + // Once the dependency is created (but not synced), the parent app should + // change its status to indicate that it's found some dependencies and is + // now waiting for them. + parent.Then(). + Expect(OperationMessageContains("Waiting for dependencies")) + + // We sync the dependency app + dep1.Sync(). + Then(). + Expect(SyncStatusIs(SyncStatusCodeSynced)). + Expect(OperationPhaseIs(common.OperationSucceeded)) + + // After the dependency has been synced, the parent's in-progress sync + // operation must resume and the sync eventually finishes. + parent.Then(). + Expect(SyncStatusIs(SyncStatusCodeSynced)). + Expect(OperationPhaseIs(common.OperationSucceeded)) + +} + +func TestSyncWaitingForDependencies(t *testing.T) { + // Dependency app + dep1 := Given(t). + Name("dep1"). + Path("kustomize"). + When(). + CreateFromFile(func(app *Application) { + app.Labels = map[string]string{ + "name": "dep1", + } + }) + + // Parent app + parent := GivenWithSameState(t). + Path(guestbookPath). + When(). + CreateFromFile(func(app *Application) { + app.Spec.DependsOn = &ApplicationDependency{ + Selectors: []ApplicationSelector{ + { + LabelSelector: &v1.LabelSelector{ + MatchLabels: map[string]string{"name": "dep1"}, + }, + }, + }, + } + }) + + // Parent should be waiting for dependencies after sync + parent. + Sync("--async"). + Then(). + Expect(SyncStatusIs(SyncStatusCodeOutOfSync)). + Expect(OperationPhaseIs(common.OperationRunning)). + Expect(OperationMessageContains("Waiting for dependencies")) + + // We sync the dependency app + dep1.Sync("--async"). + Then(). + Expect(SyncStatusIs(SyncStatusCodeSynced)). + Expect(OperationPhaseIs(common.OperationSucceeded)) + + // After the dependency has been synced, the parent's in-progress sync + // operation must resume and the sync eventually finishes. + parent.Then(). + Expect(SyncStatusIs(SyncStatusCodeSynced)). + Expect(OperationPhaseIs(common.OperationSucceeded)) +} + +func TestSyncBlockingTimeout(t *testing.T) { + Given(t). + Path(guestbookPath). + When(). + CreateFromFile(func(app *Application) { + app.Spec.DependsOn = &ApplicationDependency{ + Selectors: []ApplicationSelector{ + { + LabelSelector: &v1.LabelSelector{ + MatchLabels: map[string]string{"name": "dep1"}, + }, + }, + }, + BlockOnEmpty: pointer.Bool(true), + Timeout: pointer.Duration(3 * time.Second), + } + }). + Sync("--async"). + Then(). + Expect(SyncStatusIs(SyncStatusCodeOutOfSync)). + Expect(OperationPhaseIs(common.OperationFailed)). + Expect(OperationMessageContains("Timeout waiting for dependencies")) +} diff --git a/test/e2e/fixture/app/actions.go b/test/e2e/fixture/app/actions.go index a2b1d5e01371b1..25582ec719aaf6 100644 --- a/test/e2e/fixture/app/actions.go +++ b/test/e2e/fixture/app/actions.go @@ -4,6 +4,7 @@ import ( "encoding/json" "fmt" "os" + "time" log "github.com/sirupsen/logrus" v1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -450,6 +451,12 @@ func (a *Actions) Then() *Consequences { return &Consequences{a.context, a, 15} } +func (a *Actions) Sleep(t time.Duration) *Actions { + log.Infof("sleeping for %v", t) + time.Sleep(t) + return a +} + func (a *Actions) runCli(args ...string) { a.context.t.Helper() a.lastOutput, a.lastError = fixture.RunCli(args...)