From 238d97c1f0ffd348c0fd92925b371debdf13387d Mon Sep 17 00:00:00 2001 From: Bruno Carrier Date: Thu, 26 Oct 2017 10:06:36 -0400 Subject: [PATCH] Merge from upstream (#4) * feat(pipeline_template): Re-save dependent pipelines on template save (#1606) Allows updates to templates that define triggers, params, notifications or concurrency options to cascade changes to downstream pipelines. * chore(expressions): Allow to override global spel version at pipeline level (#1607) - Refactored to allow version override at pipeline level * feat(fastproperties): allow individual stage overrides via trigger (#1608) * fix(gradle): Pin jedis version (#1609) * fix(fastproperty): do not override context on monitor stage (#1610) * fix(events): start/end pipeline events broke because of missing toString (#1611) * fix(timeout): stage timeout overrides cumulative task durations (#1605) * chore(licenses): fix license config and add missing license headers * fix(pipeline_templates): load parent templates when inlining template for plan (#1614) * fix(pipeline_template): Multiple fixes from integration suite (#1616) * fix(pipeline_template): Correctly handle multiple stage injects * fix(pipeline_template): Fix module rendering in partials * chore(queue): removed envelope handling code we don't use * refactor(queue): queue now uses message hash as the message id This makes de-duping easier and means we can easily re-prioritize messages (PR to follow). * chore(queue): renamed things to make intent clearer * feat(pipeline_template): Convert to pipeline template endpoint (#1615) * feat(pipeline_template): Jinja rendering in triggers, params and notifications (#1619) * refactor(metrics): more configurable metrics wiring for thread pools * chore(core): simpler handling of parallel stages * fix(queue): re-prioritize message on queue if an identical one is pushed * fix(metrics): missed migrator in refactor of thread pool metrics (#1625) * feat(cluster-match): implement general purpose cluster matching rule engine (#1579) * fix(canary): target canary cleanup server groups by create date (#1612) * fix(pipeline_template): config level stage replacement (#1622) * feat(queue): Add queue shovel for migrating backends (#1624) * feat(redis_migration): log if there are still pending orchestrations/pipelines (#1626) * fix(pipeline_template): root-level configs mapped if undefined (#1623) * fix(queue): Fix stupid spring wiring (#1628) * feat(servergroup): allow ad-hoc entity tags on deployments (#1627) * chore(oortService): expose /serverGroups endpoint (#1630) * perf(rollingpush): Avoid unnecessarily searching for instance ids (#1633) If a `serverGroupName` or `asgName` is available on the stage context, there is no need to lookup each instance id individually. Also, `getServerGroup()` is more efficient than `getServerGroupFromCluster()` when dealing with clusters containing multiple server groups. * refactor(tag-generator): include server group coordinates in generateTags signature (#1634) * fix(pipeline_template): Allow conditional stages inside of partials (#1631) * feat(pipeline_template): Allow granular inheritance control on params, triggers, notifications (#1632) * fix(stages/bakery): Fix bake stage, ensure deploymentDetails is added to context - Refactor Check Preconditions stage type for parallel stages * feat(pipeline_template): Jinja rendering in partials (#1637) Renders the partials' stage config before rendering partials, to allow for more advanced jinja expressions in the partials. * fix(managed_pipeline): validation of variable types, integration tests for invalid pipelines (#1629) * refactor(pipeline_template): Support multiple MPT schema versions (#1621) Adds the ability to parse different schema versions. This PR doesn't actually change any functionality, except for switching to the new handler API. All existing v1 schema code works the same way it did. Additionally introduced Kotlin to module as new default. * tests(pipeline_templates): integrations tests and removing unique ID (#1638) * feat(pipeline_template): Support template-less configurations (#1636) * fix(timeouts): some tasks inherit stage timeout override (#1640) * fix(pipeline_template): Regression in deserializing tempalted pipeline requests (#1644) * fix(pipeline_template): Check for correct paramConfig field during render (#1642) * fix(web): Enforce limit of pipelines when using previous redis conn (#1646) * fix(redis_migration): fix logging of pending work (#1647) * feat(artifacts): Add receivedArtifacts to Pipeline model. (#1648) * feat(job): decrease job timout and make overridable (#1649) * fix(web): Enforce limit on previous redis for app pipelines endpoint (#1650) * fix(pipeline_template): Do not store state in error handler lol (#1651) * Revert "feat(artifacts): Add receivedArtifacts to Pipeline model. (#1648)" (#1653) This reverts commit 0d9a2f140a8cec07b66e659ce55913bfc570d3ec. * fix(web): Revert take calls; unexpected upstream behaviors (#1654) * fix(rollbacks): support for tolerating some instance failures (#1643) * feat(stage context): Get all stage results in a list (#1655) * refactor(clouddriver): monitor clouddriver tasks every 5 seconds (#1639) * feat(context): Adds trigger contents to stage context (#1659) * feat(clouddriver): Support sharding read-only requests by `user` (#1641) This allows you to serve all pipelines/orchestrations generated by a particular user (or set of users) with a dedicated clouddriver read replica. * feat(moniker): Use a cluster's moniker if available. (#1664) * Use cluster's moniker if available * Format tests for moniker vs frigga cluster names * chore(expressions): Cleanup some noisy logs (#1666) - Include execution id in log message - Remove some noise with debug statements * fix(mine): Search stage definition builders directly instead of depending on stage navigator when trying to cancel canary. * fix(web): Return correct num executions with dual redis (#1668) * perf(build): Removing orca-queue-sqs; unused, unsupported * feat(core): Deploy manifest stage & task (#1674) * feat(pipeline): Resolve received and expected artifacts in trigger. (#1667) * fix(expressions): Include evaluation summary in v2 (#1673) - ensure evaluation errors are included in precondition * fix(execution windows): don't add duplicate execution windows to parallel stages * fix(queue): treat waiting pipelines queue as FIFO if keepWaitingPipelines (#1677) * feat(pipeline_template): Add marker support to disable rendered value expansion (#1665) * feat(get_commits): Display SCM changes in pipeline-triggered pipelines (#1652) The changes tab that displays SCM diff between deploys was not showing if the pipeline was triggered by another pipeline. Now it does. * fix(pipeline_template): Propagate nested render errors (#1679) * feat(moniker): Allow moniker field to pass through to StageData and TargetServerGroups (#1678) * fix(queue): don't keep pushing dead messages indefinitely * feat(qa): dry run pipelines Allow pipelines to run in "dry-run" mode. Stages are proxied and simply test correct evaluation of expressions, optionality, ordering and consistent results. * add executionId to MDC (#1682) * fix(rrb): RRB should determine resize percentages at runtime (#1681) This PR introduces the ability to specify a `scalePct` when scaling based on the capacity of another server group (scale_to_server_group). Previously all RRB capacity adjustments on the target server group were determined at build time. * Oort get server groups (#1680) * Add new clouddriver getServerGroup endpoint * Add new clouddriver getServerGroup endpoint * Removed frigga from oortHelper * Rename oortService method for clearity * Overload getServerGroups and deprecate old prototype * Update test * Also deprecate on the DelegatedOortService * fix(pipeline_template): Fixing regression in test harness (#1686) * feat(pipeline_template): PipelineIdTag checks context for variables defining application and name (#1688) * fix(log): ensure executionId is cleaned up from MDC (#1685) * feat(stages): make FAILED_CONTINUE bubble up from synthetic stages * feat(moniker): Use moniker for instance check task (#1689) * feat(moniker): Use monikers within server-group tasks (#1693) * fix(log) ensure MDC cleanup occurs in correct thread (#1698) * fix(pipeline_template): Do not treat markdown lists as YAML alias (#1645) * feat(artifacts): simplify artifact matching (#1696) * feat(core): Force cache refresh manifest task (#1687) * feat(moniker): use moniker over frigga in ScaleToClusterResize * chore(imports): remove unused frigga import * core(manifest): Delete manifest op (#1700) * feat(artifacts): Check if default artifact was intended (#1701) * feat(rrb): Support for running a pipeline between scale up and disable (#1694) Coordinates for both the new and old server groups are passed along as pipeline parameters. - `delayBeforeDisableSec` -> `getDelayBeforeCleanup()` - support for lookup by stage id in `#stage()` * fix(dryrun): need injected property class to support list of values * Use moniker in DetermineHealthProvidersTask (#1702) * feat(moniker): Use moniker for Job stages. (#1699) * chore(dependencies): updating spinnaker-depenencies (#1707) * feat(queue): update delivery time on runtask (#1676) Adds reschedule method to queue, which updated the delivery time of an existing message on the queue. * feat(logging): ops controller annotation + logstashEncoder dep (#1692) * fix(dryrun): log dry run activity * Properly cast Moniker from context Properly cast Moniker from context Put the try catch in the MonikerHelper Only check for IllegalArgumentException check existence rather than exception * feat(polling): remove polling for wait, manual judgement, and execution window (#1661) * feat(entitytags): Include previous server group image details (#1705) This will ultimately facilitate an orchestrated rollback even if the previous server group no longer exists. It relies on the entity tags feature being enabled (dependency on elastic search and not enabled by default in `clouddriver` / `orca`). This PR also introduces some common retry handling (see `RetrySupport`). * fix(entitytags): Do not fetch previous server group for Titus (#1712) Subsequent PR will make this work properly for Titus. * fix(gae): Support new ExpectedArtifact in GAE deploy. (#1706) * fix(polling): update time left calculation (#1713) * fix(dryrun): send dry run notification correctly * fix(propertyFiles): change wording of error message to indicate that there might be a syntax error in the file (#1715) * fix(fastproperties): always clean up properties marked for rollback (#1717) * feat(core): implement noop stage (#1719) * fix(dryrun): remove pipeline config it for dry runs * feat(rollback): Support rolling back to a server group that no longer exists (#1716) This PR provides a rollback strategy that will clone forward with the image details that were tagged as part of spinnaker/orca#1705. It also supports the `imageName` being explicitly provided, but that's an exceptional case that would not be supported in the UI. The `spinnaker:metadata` tag will include `buildInfo` for the image if it is returned from `clouddriver`. * fix(runJob): retry fetching property files if not found (#1721) * fix(executionWindow): revert to polling (#1722) * fix(canary): gentle canary cleanup (#1711) * fix(rollback): Propagate `interestingHealthProviderNames` (#1723) If `interestingHealthProviderNames` are specified on the parent stage context, propagate them to the clone stage. * fix(dryrun): let Echo remove the pipelineConfigId * fix(dryrun): ignore certain keys in context * fix(cancel): cancel during wait stage (#1726) * fix(logging): updating timeout message w/ timeout value (#1728) * fix(dryrun): strip nested nulls when comparing context * fix(rrb): Only inject pipeline stage if applicaton + pipelineId present (#1729) * fix(tasks): stop using 'shared' task state (#1731) * fix(dryrun): try to cope with values that are sometimes floats or ints * fix(manual judgment): switching back to polling to respect timeout overrides (#1735) * fix(exec window): leave shared state alone (#1737) * Attempt to use moniker before frigga (#1697) * feat(moniker): Use moniker for Rollingpush tasks. (#1703) * feat(moniker): Pass moniker to cleanup stages. (#1732) * feat(moniker): Use moniker for app name over frigga in flex (#1736) * feat(manualJudgment): allow standard notification types for manual judgment (#1739) * fix(dryrun): ignore additional context field --- build.gradle | 12 +- gradle.properties | 10 +- .../orca/bakery/pipeline/BakeStage.groovy | 46 +- .../orca/bakery/tasks/MonitorBakeTask.groovy | 4 +- .../orca/bakery/pipeline/BakeStageSpec.groovy | 55 +- orca-clouddriver/orca-clouddriver.gradle | 1 + .../DelegatingClouddriverService.java | 3 +- .../clouddriver/DelegatingOortService.java | 11 + .../orca/clouddriver/OortService.groovy | 15 +- .../ByAuthenticatedUserServiceSelector.java | 61 ++ .../clouddriver/config/SelectableService.java | 11 +- .../manifest/DeleteManifestStage.java | 39 + .../manifest/DeployManifestStage.java | 39 + .../servergroup/CloneServerGroupStage.groovy | 4 +- .../servergroup/CreateServerGroupStage.groovy | 4 +- .../RollbackServerGroupStage.groovy | 126 +-- .../rollback/ExplicitRollback.groovy | 139 +++ .../rollback/PreviousImageRollback.groovy | 149 ++++ .../servergroup/rollback/Rollback.java | 26 + .../servergroup/rollback/TestRollback.java | 54 ++ .../AbstractDeployStrategyStage.groovy | 4 +- .../strategies/CustomStrategy.groovy | 1 + .../strategies/HighlanderStrategy.groovy | 1 + .../strategies/RedBlackStrategy.groovy | 5 +- .../strategies/RollingRedBlackStrategy.groovy | 204 +++-- .../support/TargetServerGroup.groovy | 6 +- .../support/TargetServerGroupResolver.groovy | 45 +- .../tasks/DetermineHealthProvidersTask.java | 8 +- .../clouddriver/tasks/MonitorKatoTask.groovy | 5 +- .../AbstractClusterWideClouddriverTask.groovy | 13 +- ...ctWaitForClusterWideClouddriverTask.groovy | 9 +- .../ClusterSizePreconditionTask.groovy | 4 +- .../cluster/FindImageFromClusterTask.groovy | 4 +- .../AbstractInstancesCheckTask.groovy | 31 +- ...ractWaitForInstanceHealthChangeTask.groovy | 4 +- ...InstanceAndDecrementServerGroupTask.groovy | 3 + .../instance/TerminateInstancesTask.groovy | 3 + .../WaitForTerminatedInstancesTask.groovy | 4 +- .../tasks/job/WaitOnJobCompletion.groovy | 24 +- .../tasks/manifest/DeleteManifestTask.java | 63 ++ .../tasks/manifest/DeployManifestTask.java | 61 ++ .../ManifestForceCacheRefreshTask.java | 64 ++ ...orAppEngineServerGroupStopStartTask.groovy | 4 +- .../AppEngineServerGroupCreator.groovy | 56 ++ .../AbstractBulkServerGroupTask.java | 17 +- .../AddServerGroupEntityTagsTask.groovy | 99 +++ .../BulkWaitForDestroyedServerGroupTask.java | 10 +- ...extBasedServerGroupEntityTagGenerator.java | 37 + .../ServerGroupEntityTagGenerator.java | 7 +- .../ServerGroupMetadataTagTask.groovy | 79 +- ...nnakerMetadataServerGroupTagGenerator.java | 86 +- .../WaitForDestroyedServerGroupTask.groovy | 4 +- .../clouddriver/utils/ClusterMatchRule.java | 77 ++ .../clouddriver/utils/ClusterMatcher.java | 70 ++ .../orca/clouddriver/utils/MonikerHelper.java | 61 ++ .../orca/clouddriver/utils/OortHelper.groovy | 11 +- .../orca/clouddriver/utils/TrafficGuard.java | 31 +- .../kato/pipeline/ParallelDeployStage.groovy | 29 +- .../orca/kato/pipeline/RollingPushStage.java | 15 + .../ScaleToClusterResizeStrategy.groovy | 4 +- .../ScaleToServerGroupResizeStrategy.groovy | 21 + .../kato/pipeline/support/StageData.groovy | 29 +- .../kato/tasks/DisableInstancesTask.groovy | 3 + .../tasks/rollingpush/CleanUpTagsTask.java | 10 +- .../WaitForNewUpInstancesLaunchTask.groovy | 8 +- .../config/SelectableServiceSpec.groovy | 35 +- .../RollbackServerGroupStageSpec.groovy | 68 ++ .../rollback/ExplicitRollbackSpec.groovy} | 60 +- .../rollback/PreviousImageRollbackSpec.groovy | 161 ++++ .../strategies/RedBlackStrategySpec.groovy | 4 + .../TargetServerGroupResolverSpec.groovy | 19 +- .../DetermineHealthProvidersTaskSpec.groovy | 1 + ...tractClusterWideClouddriverTaskSpec.groovy | 27 +- .../ClusterSizePreconditionTaskSpec.groovy | 23 + .../FindImageFromClusterTaskSpec.groovy | 198 +++-- .../AbstractInstancesCheckTaskSpec.groovy | 6 +- .../AddServerGroupEntityTagsTaskSpec.groovy | 74 ++ .../BulkDestroyServerGroupTaskSpec.groovy | 4 +- .../ServerGroupMetadataTagTaskSpec.groovy | 197 ----- ...MetadataServerGroupTagGeneratorSpec.groovy | 166 ++++ .../WaitForCapacityMatchTaskSpec.groovy | 6 +- .../utils/ClusterMatcherSpec.groovy | 169 ++++ .../clouddriver/utils/TrafficGuardSpec.groovy | 30 +- .../pipeline/ParallelDeployStageSpec.groovy | 14 - .../ScaleToClusterResizeStrategySpec.groovy | 12 +- ...caleToServerGroupResizeStrategySpec.groovy | 32 +- .../support/SourceResolverSpec.groovy | 7 +- .../rollingpush/CleanUpTagsTaskSpec.groovy | 10 + ...WaitForNewUpInstancesLaunchTaskSpec.groovy | 2 +- orca-core/orca-core.gradle | 1 + .../spinnaker/orca/CancellableStage.java | 15 + .../spinnaker/orca/ExecutionContext.java | 8 +- .../orca/OverridableTimeoutRetryableTask.java | 26 + .../netflix/spinnaker/orca/RetrySupport.java | 49 ++ .../netflix/spinnaker/orca/RetryableTask.java | 15 + .../com/netflix/spinnaker/orca/Task.java | 15 + .../netflix/spinnaker/orca/TaskResult.java | 15 + .../orca/config/OrcaConfiguration.java | 60 +- .../orca/jackson/OrcaObjectMapper.java | 17 + .../listeners/ExecutionCleanupListener.java | 15 + .../orca/listeners/ExecutionListener.java | 12 +- .../StageTaskPropagationListener.java | 15 + .../BranchingStageDefinitionBuilder.java | 77 -- .../pipeline/CheckPreconditionsStage.groovy | 49 +- .../DefaultStageDefinitionBuilderFactory.java | 45 + .../orca/pipeline/ExecutionRunner.java | 5 + .../spinnaker/orca/pipeline/NoopStage.java | 29 + .../pipeline/PipelineStarterListener.java | 5 +- .../orca/pipeline/StageDefinitionBuilder.java | 8 +- .../StageDefinitionBuilderFactory.java | 29 + .../ExpressionEvaluationSummary.java | 2 +- .../expressions/ExpressionTransform.groovy | 1 - .../expressions/ExpressionsSupport.java | 4 +- .../PipelineExpressionEvaluator.java | 107 +-- .../spinnaker/orca/pipeline/model/Stage.java | 27 +- .../orca/pipeline/model/StageContext.java | 43 +- .../ExecutionNotFoundException.java | 15 + .../persistence/ExecutionRepository.java | 15 + .../persistence/jedis/JedisConfiguration.java | 60 ++ .../jedis/JedisExecutionRepository.groovy | 56 +- .../tasks/ExpressionPreconditionTask.groovy | 9 + .../orca/pipeline/tasks/WaitTask.groovy | 4 +- .../pipeline/util/ArtifactResolver.groovy | 67 ++ .../util/ContextParameterProcessor.groovy | 5 +- .../ThreadPoolMetricsPostProcessor.java | 68 ++ .../spinnaker/orca/RetrySupportSpec.groovy | 79 ++ .../expressions/ExpressionsSupportSpec.groovy | 73 ++ .../PipelineExpressionEvaluatorSpec.groovy | 90 ++ .../orca-dry-run.gradle | 9 +- .../orca/dryrun/RoundingDoubleSerializer.java | 47 + .../orca/dryrun/RoundingFloatSerializer.java | 47 + .../spinnaker/config/DryRunConfiguration.kt | 43 + .../spinnaker/config/DryRunProperties.kt | 12 +- .../spinnaker/orca/dryrun/DryRunStage.kt | 42 + .../DryRunStageDefinitionBuilderFactory.kt | 56 ++ .../spinnaker/orca/dryrun/DryRunTask.kt | 131 +++ .../spinnaker/orca/dryrun/DryRunStageTest.kt | 117 +++ .../spinnaker/orca/dryrun/DryRunTaskTest.kt | 265 ++++++ orca-echo/orca-echo.gradle | 1 + .../orca/echo/config/EchoConfiguration.groovy | 22 +- .../echo/pipeline/ManualJudgmentStage.groovy | 33 +- .../EchoNotifyingExecutionListener.groovy | 55 +- .../spring/EchoNotifyingStageListener.groovy | 15 + .../pipeline/ManualJudgmentStageSpec.groovy | 38 +- .../EchoNotifyingExecutionListenerSpec.groovy | 3 +- orca-flex/orca-flex.gradle | 1 + .../flex/tasks/AbstractElasticIpTask.groovy | 4 +- ...estrationMigrationNotificationAgent.groovy | 23 +- ...sPipelineMigrationNotificationAgent.groovy | 44 +- .../orca/front50/pipeline/PipelineStage.java | 15 + .../front50/tasks/MonitorPipelineTask.groovy | 4 +- .../orca/igor/pipeline/TravisStage.groovy | 15 + .../orca/igor/tasks/GetCommitsTask.groovy | 9 +- .../igor/tasks/MonitorJenkinsJobTask.groovy | 6 +- .../tasks/MonitorQueuedJenkinsJobTask.groovy | 4 +- .../orca/igor/tasks/GetCommitsTaskSpec.groovy | 44 +- .../tasks/MonitorJenkinsJobTaskSpec.groovy | 2 +- .../kayenta/tasks/MonitorCanaryTask.groovy | 4 +- .../spinnaker/orca/mahe/PropertyAction.groovy | 15 + .../FastPropertyCleanupListener.groovy | 11 +- .../mahe/tasks/CreatePropertiesTask.groovy | 14 +- .../mahe/tasks/MonitorPropertiesTask.groovy | 1 + .../tasks/CreatePropertiesTaskSpec.groovy | 32 + .../tasks/PropertyChangeCleanupSpec.groovy | 59 +- .../orca/mine/pipeline/CanaryStage.groovy | 99 ++- .../mine/pipeline/DeployCanaryStage.groovy | 4 +- .../mine/pipeline/MonitorCanaryStage.groovy | 22 +- .../mine/tasks/DeployedClustersUtil.groovy | 22 + .../orca/mine/tasks/DisableCanaryTask.groovy | 55 ++ .../orca/mine/tasks/MonitorAcaTaskTask.groovy | 4 +- .../orca/mine/tasks/MonitorCanaryTask.groovy | 4 +- .../orca/mine/tasks/RegisterCanaryTask.groovy | 8 +- .../orca/mine/pipeline/CanaryStageSpec.groovy | 108 +++ .../pipeline/MonitorCanaryStageSpec.groovy | 20 +- .../orca-pipelinetemplate.gradle | 4 + .../config/PipelineTemplateConfiguration.java | 11 +- .../PipelineTemplatePipelinePreprocessor.java | 166 ---- .../PipelineTemplatePreprocessor.kt | 95 ++ .../TemplatedPipelineRequest.java | 48 +- ...IllegalTemplateConfigurationException.java | 8 +- .../exceptions/PipelineTemplateException.java | 23 + .../exceptions/TemplateLoaderException.java | 8 +- .../exceptions/TemplateRenderException.java | 2 +- .../orca/pipelinetemplate/handler/Handler.kt | 53 ++ .../handler/PipelineTemplateContext.kt | 63 ++ .../handler/PipelineTemplateErrorHandler.kt | 59 ++ .../handler/SchemaVersionHandler.kt | 36 + .../loader/TemplateLoader.java | 6 +- .../pipeline/UpdatePipelineTemplateStage.java | 77 ++ .../tasks/PlanTemplateDependentsTask.java | 6 +- .../v1schema/TemplateMerge.java | 8 +- .../TemplatedPipelineModelMutator.java | 19 +- .../v1schema/V1SchemaExecutionGenerator.java | 18 + .../converter/PipelineTemplateConverter.java | 171 ++++ .../v1schema/graph/GraphMutator.java | 2 +- .../transform/ConditionalStanzaTransform.java | 9 +- .../ConfigStageInjectionTransform.java | 12 +- .../DefaultVariableAssignmentTransform.java | 40 +- .../graph/transform/RenderTransform.java | 43 +- .../transform/TrimConditionalsTransform.java | 15 + .../v1schema/handler/V1Handlers.kt | 111 +++ .../handler/V1PipelineTemplateContext.kt | 25 + .../handler/V1TemplateLoaderHandler.kt | 98 +++ .../v1schema/model/NamedContent.java | 5 +- .../v1schema/model/NamedHashMap.java | 27 +- .../v1schema/model/PipelineTemplate.java | 51 +- .../v1schema/model/TemplateConfiguration.java | 2 +- .../v1schema/model/TemplateModule.java | 3 +- .../v1schema/render/JinjaRenderer.java | 6 +- .../render/YamlRenderedValueConverter.java | 18 +- .../v1schema/render/tags/ModuleTag.java | 7 +- .../v1schema/render/tags/PipelineIdTag.java | 41 +- ...1TemplateConfigurationSchemaValidator.java | 29 +- .../validator/V1TemplateSchemaValidator.java | 2 - .../pipelinetemplate/validator/Errors.java | 2 +- .../main/resources/pipelineTemplateHeader.txt | 18 + ...ineTemplatePipelinePreprocessorSpec.groovy | 145 +--- .../TemplatedPipelineRequestSpec.groovy | 77 ++ .../UpdatePipelineTemplateStageSpec.groovy | 101 +++ .../PlanTemplateDependentsTaskSpec.groovy | 6 +- .../v1schema/TemplateMergeSpec.groovy | 21 + .../TemplatedPipelineModelMutatorSpec.groovy | 27 + .../v1schema/V1SchemaIntegrationSpec.groovy | 38 +- .../PipelineTemplateConverterSpec.groovy | 36 + .../ConditionalStanzaTransformSpec.groovy | 9 +- .../transform/RenderTransformSpec.groovy | 4 + .../V1TemplateLoaderHandlerSpec.groovy | 237 +++++ .../v1schema/render/JinjaRendererSpec.groovy | 2 + .../v1schema/render/tags/ModuleTagSpec.groovy | 3 +- .../render/tags/PipelineIdTagSpec.groovy | 22 +- ...ateConfigurationSchemaValidatorSpec.groovy | 22 +- .../resources/convertedPipelineTemplate.yml | 199 +++++ .../convertedPipelineTemplateSource.json | 222 +++++ .../v1schema/conditionalStage-config.yml | 7 + .../v1schema/conditionalStage-expected.json | 10 + .../v1schema/conditionalStage-template.yml | 15 + .../integration/v1schema/example-child.yml | 13 + .../integration/v1schema/example-config.yml | 27 + .../v1schema/example-expected.json | 51 ++ .../integration/v1schema/example-root.yml | 14 + .../integration/v1schema/example-template.yml | 16 + .../v1schema/exampleCombined-config.yml | 20 + .../v1schema/exampleCombined-expected.json | 51 ++ .../v1schema/exampleCombined-template.yml | 35 + .../v1schema/inheritance-expected.json | 3 +- .../v1schema/invalidStageFormats-config.yml | 6 + .../invalidStageFormats-expected.json | 29 + .../v1schema/invalidStageFormats-template.yml | 38 + .../integration/v1schema/modules-config.yml | 13 + .../v1schema/modules-expected.json | 40 + .../integration/v1schema/modules-template.yml | 38 + .../nestedConditionalStage-config.yml | 5 + .../nestedConditionalStage-expected.json | 20 + .../nestedConditionalStage-template.yml | 31 + .../v1schema/parameterInheritance-config.yml | 13 + .../parameterInheritance-expected.json | 16 + .../parameterInheritance-template.yml | 14 + .../integration/v1schema/partials-config.yml | 5 + .../v1schema/partials-expected.json | 45 + .../v1schema/partials-template.yml | 43 + .../v1schema/partialsAndModules-config.yml | 5 + .../v1schema/partialsAndModules-expected.json | 36 + .../v1schema/partialsAndModules-template.yml | 42 + .../integration/v1schema/protect-config.yml | 11 + .../v1schema/protect-expected.json | 9 + .../integration/v1schema/protect-template.yml | 13 + .../integration/v1schema/simple-expected.json | 3 +- .../v1schema/stageInjection-config.yml | 31 + .../v1schema/stageInjection-expected.json | 67 ++ .../v1schema/stageInjection-template.yml | 24 + .../v1schema/stageReplacement-config.yml | 24 + .../v1schema/stageReplacement-expected.json | 132 +++ .../v1schema/stageReplacement-template.yml | 114 +++ .../v1schema/unsupportedVersion-config.yml | 5 + .../v1schema/unsupportedVersion-expected.json | 8 + .../v1schema/unsupportedVersion-template.yml | 7 + .../v1schema/variableParameters-config.yml | 7 + .../v1schema/variableParameters-expected.json | 16 + .../v1schema/variableParameters-template.yml | 16 + .../v1schema/variableWrongType-config.yml | 8 + .../v1schema/variableWrongType-expected.json | 9 + .../v1schema/variableWrongType-template.yml | 16 + .../templates/conditional-partials.yml | 9 +- orca-queue-redis/orca-queue-redis.gradle | 2 +- .../config/RedisQueueShovelConfiguration.kt | 86 ++ .../spinnaker/orca/q/redis/RedisQueue.kt | 213 +++-- .../spinnaker/orca/q/redis/RedisQueueTest.kt | 36 - .../spinnaker/config/SqsQueueConfiguration.kt | 48 - .../spinnaker/orca/q/amazon/SqsQueue.kt | 81 -- orca-queue/orca-queue.gradle | 8 +- .../spinnaker/config/QueueConfiguration.kt | 37 +- .../netflix/spinnaker/orca/q/Executions.kt | 5 +- .../com/netflix/spinnaker/orca/q/Message.kt | 69 +- .../spinnaker/orca/q/MessageHandler.kt | 19 + .../com/netflix/spinnaker/orca/q/Queue.kt | 10 + .../spinnaker/orca/q/QueueExecutionRunner.kt | 4 + .../netflix/spinnaker/orca/q/QueueExecutor.kt | 7 +- .../spinnaker/orca/q/QueueProcessor.kt | 17 +- .../netflix/spinnaker/orca/q/QueueShovel.kt | 68 ++ .../orca/q/StageDefinitionBuilders.kt | 66 +- .../orca/q/ThreadPoolQueueExecutor.kt | 9 +- .../orca/q/handler/AbortStageHandler.kt | 53 ++ .../orca/q/handler/AuthenticationAware.kt | 1 + .../orca/q/handler/CancelExecutionHandler.kt | 8 +- .../orca/q/handler/CancelStageHandler.kt | 4 +- .../orca/q/handler/CompleteStageHandler.kt | 28 +- .../orca/q/handler/CompleteTaskHandler.kt | 4 +- .../q/handler/ContinueParentStageHandler.kt | 11 +- .../orca/q/handler/DeadMessageHandler.kt | 28 +- .../q/handler/RescheduleExecutionHandler.kt | 52 ++ .../orca/q/handler/RestartStageHandler.kt | 4 +- .../orca/q/handler/RunTaskHandler.kt | 37 +- .../orca/q/handler/SkipStageHandler.kt | 49 ++ .../orca/q/handler/StageBuilderAware.kt | 7 +- .../orca/q/handler/StartStageHandler.kt | 13 +- .../spinnaker/orca/q/handler/stages.kt | 26 +- .../spinnaker/orca/q/memory/InMemoryQueue.kt | 27 +- .../orca/q/metrics/AtlasQueueMonitor.kt | 22 +- .../orca/q/metrics/MonitorableQueue.kt | 12 +- .../spinnaker/orca/q/metrics/QueueEvent.kt | 3 +- .../q/trafficshaping/TrafficShapingQueue.kt | 2 + .../com/netflix/spinnaker/orca/time/time.kt | 15 + .../spinnaker/orca/q/PipelineBuilder.kt | 29 + .../spinnaker/orca/q/QueueIntegrationTest.kt | 61 +- .../spinnaker/orca/q/QueueProcessorTest.kt | 5 +- .../spinnaker/orca/q/QueueShovelTest.kt | 55 ++ .../com/netflix/spinnaker/orca/q/QueueTest.kt | 131 ++- .../com/netflix/spinnaker/orca/q/Stages.kt | 43 +- .../com/netflix/spinnaker/orca/q/Tasks.kt | 2 + .../orca/q/handler/AbortStageHandlerTest.kt | 175 ++++ .../q/handler/CancelExecutionHandlerTest.kt | 4 + .../orca/q/handler/CancelStageHandlerTest.kt | 3 +- .../q/handler/CompleteStageHandlerTest.kt | 821 ++++++++++-------- .../orca/q/handler/CompleteTaskHandlerTest.kt | 6 +- .../handler/ContinueParentStageHandlerTest.kt | 198 ++--- .../orca/q/handler/DeadMessageHandlerTest.kt | 28 +- .../handler/RescheduleExecutionHandlerTest.kt | 97 +++ .../orca/q/handler/RestartStageHandlerTest.kt | 3 +- .../orca/q/handler/RunTaskHandlerTest.kt | 217 ++++- .../orca/q/handler/SkipStageHandlerTest.kt | 240 +++++ .../orca/q/handler/StartStageHandlerTest.kt | 49 +- .../orca/q/memory/InMemoryQueueTest.kt | 3 +- .../orca/q/metrics/MonitorableQueueTest.kt | 27 +- .../com/netflix/spinnaker/spek/should.kt | 5 + orca-web/orca-web.gradle | 2 + .../controllers/OperationsController.groovy | 9 +- .../PipelineTemplateController.groovy | 7 + .../orca/controllers/TaskController.groovy | 19 +- .../controllers/TaskControllerSpec.groovy | 7 +- settings.gradle | 2 +- 350 files changed, 10878 insertions(+), 2701 deletions(-) create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/config/ByAuthenticatedUserServiceSelector.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/manifest/DeleteManifestStage.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/manifest/DeployManifestStage.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/ExplicitRollback.groovy create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/PreviousImageRollback.groovy create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/Rollback.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/TestRollback.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/DeleteManifestTask.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/DeployManifestTask.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/ManifestForceCacheRefreshTask.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AddServerGroupEntityTagsTask.groovy create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ContextBasedServerGroupEntityTagGenerator.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatchRule.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatcher.java create mode 100644 orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/MonikerHelper.java create mode 100644 orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/RollbackServerGroupStageSpec.groovy rename orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/{RollbackServerGroupStageSpec.groovy => servergroup/rollback/ExplicitRollbackSpec.groovy} (68%) create mode 100644 orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/PreviousImageRollbackSpec.groovy create mode 100644 orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AddServerGroupEntityTagsTaskSpec.groovy delete mode 100644 orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupMetadataTagTaskSpec.groovy create mode 100644 orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/SpinnakerMetadataServerGroupTagGeneratorSpec.groovy create mode 100644 orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatcherSpec.groovy create mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/OverridableTimeoutRetryableTask.java create mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/RetrySupport.java delete mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/BranchingStageDefinitionBuilder.java create mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/DefaultStageDefinitionBuilderFactory.java create mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/NoopStage.java create mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/StageDefinitionBuilderFactory.java create mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/jedis/JedisConfiguration.java create mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/util/ArtifactResolver.groovy create mode 100644 orca-core/src/main/groovy/com/netflix/spinnaker/orca/telemetry/ThreadPoolMetricsPostProcessor.java create mode 100644 orca-core/src/test/groovy/com/netflix/spinnaker/orca/RetrySupportSpec.groovy create mode 100644 orca-core/src/test/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionsSupportSpec.groovy create mode 100644 orca-core/src/test/groovy/com/netflix/spinnaker/orca/pipeline/expressions/PipelineExpressionEvaluatorSpec.groovy rename orca-queue-sqs/orca-queue-sqs.gradle => orca-dry-run/orca-dry-run.gradle (74%) create mode 100644 orca-dry-run/src/main/java/com/netflix/spinnaker/orca/dryrun/RoundingDoubleSerializer.java create mode 100644 orca-dry-run/src/main/java/com/netflix/spinnaker/orca/dryrun/RoundingFloatSerializer.java create mode 100644 orca-dry-run/src/main/kotlin/com/netflix/spinnaker/config/DryRunConfiguration.kt rename orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/config/SqsProperties.kt => orca-dry-run/src/main/kotlin/com/netflix/spinnaker/config/DryRunProperties.kt (77%) create mode 100644 orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStage.kt create mode 100644 orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStageDefinitionBuilderFactory.kt create mode 100644 orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunTask.kt create mode 100644 orca-dry-run/src/test/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStageTest.kt create mode 100644 orca-dry-run/src/test/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunTaskTest.kt create mode 100644 orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/DisableCanaryTask.groovy create mode 100644 orca-mine/src/test/groovy/com/netflix/spinnaker/orca/mine/pipeline/CanaryStageSpec.groovy delete mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePipelinePreprocessor.java create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePreprocessor.kt create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/PipelineTemplateException.java create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/Handler.kt create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/PipelineTemplateContext.kt create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/PipelineTemplateErrorHandler.kt create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/SchemaVersionHandler.kt create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/converter/PipelineTemplateConverter.java create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1Handlers.kt create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1PipelineTemplateContext.kt create mode 100644 orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1TemplateLoaderHandler.kt create mode 100644 orca-pipelinetemplate/src/main/resources/pipelineTemplateHeader.txt create mode 100644 orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/TemplatedPipelineRequestSpec.groovy create mode 100644 orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/pipeline/UpdatePipelineTemplateStageSpec.groovy create mode 100644 orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/converter/PipelineTemplateConverterSpec.groovy create mode 100644 orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1TemplateLoaderHandlerSpec.groovy create mode 100644 orca-pipelinetemplate/src/test/resources/convertedPipelineTemplate.yml create mode 100644 orca-pipelinetemplate/src/test/resources/convertedPipelineTemplateSource.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/example-child.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/example-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/example-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/example-root.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/example-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-template.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-config.yml create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-expected.json create mode 100644 orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-template.yml create mode 100644 orca-queue-redis/src/main/kotlin/com/netflix/spinnaker/config/RedisQueueShovelConfiguration.kt delete mode 100644 orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/config/SqsQueueConfiguration.kt delete mode 100644 orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/orca/q/amazon/SqsQueue.kt create mode 100644 orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueShovel.kt create mode 100644 orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/AbortStageHandler.kt create mode 100644 orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RescheduleExecutionHandler.kt create mode 100644 orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/SkipStageHandler.kt create mode 100644 orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueShovelTest.kt create mode 100644 orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/AbortStageHandlerTest.kt create mode 100644 orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RescheduleExecutionHandlerTest.kt create mode 100644 orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/SkipStageHandlerTest.kt diff --git a/build.gradle b/build.gradle index 15d51c9a0d..c2df8bf1d7 100644 --- a/build.gradle +++ b/build.gradle @@ -34,7 +34,7 @@ allprojects { group = "com.netflix.spinnaker.orca" ext { - spinnakerDependenciesVersion = project.hasProperty('spinnakerDependenciesVersion') ? project.property('spinnakerDependenciesVersion') : '0.109.2' + spinnakerDependenciesVersion = project.hasProperty('spinnakerDependenciesVersion') ? project.property('spinnakerDependenciesVersion') : '0.117.0' } def checkLocalVersions = [spinnakerDependenciesVersion: spinnakerDependenciesVersion] @@ -100,6 +100,16 @@ subprojects { spinnaker.group('spockBase') testCompile spinnaker.dependency('groovy') } + + license { + exclude "**/*.json" + exclude "**/*.md" + exclude "**/*.yml" + mapping { + java = "SLASHSTAR_STYLE" + kt = "SLASHSTAR_STYLE" + } + } } defaultTasks ":orca-web:bootRun" diff --git a/gradle.properties b/gradle.properties index 320d5d6af4..b995693668 100644 --- a/gradle.properties +++ b/gradle.properties @@ -14,8 +14,8 @@ # limitations under the License. # kotlinVersion=1.1.4-3 -junitVersion=1.0.0-M4 -jupiterVersion=5.0.0-M4 -junitLegacyVersion=4.12.0-M4 -spekVersion=1.1.2 - +junitVersion=1.0.0 +jupiterVersion=5.0.0 +junitLegacyVersion=4.12.0 +spekVersion=1.1.5 +jedis.version=2.8.1 diff --git a/orca-bakery/src/main/groovy/com/netflix/spinnaker/orca/bakery/pipeline/BakeStage.groovy b/orca-bakery/src/main/groovy/com/netflix/spinnaker/orca/bakery/pipeline/BakeStage.groovy index db01c68093..acc06da7f7 100644 --- a/orca-bakery/src/main/groovy/com/netflix/spinnaker/orca/bakery/pipeline/BakeStage.groovy +++ b/orca-bakery/src/main/groovy/com/netflix/spinnaker/orca/bakery/pipeline/BakeStage.groovy @@ -16,6 +16,7 @@ package com.netflix.spinnaker.orca.bakery.pipeline +import javax.annotation.Nonnull import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.RestartableStage import com.netflix.spinnaker.orca.Task @@ -23,7 +24,7 @@ import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.bakery.tasks.CompletedBakeTask import com.netflix.spinnaker.orca.bakery.tasks.CreateBakeTask import com.netflix.spinnaker.orca.bakery.tasks.MonitorBakeTask -import com.netflix.spinnaker.orca.pipeline.BranchingStageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.TaskNode import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -32,30 +33,48 @@ import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import org.springframework.stereotype.Component +import static com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFORE + @Slf4j @Component @CompileStatic -class BakeStage implements BranchingStageDefinitionBuilder, RestartableStage { +class BakeStage implements StageDefinitionBuilder, RestartableStage { public static final String PIPELINE_CONFIG_TYPE = "bake" @Override > void taskGraph(Stage stage, TaskNode.Builder builder) { - builder - .withTask("createBake", CreateBakeTask) - .withTask("monitorBake", MonitorBakeTask) - .withTask("completedBake", CompletedBakeTask) + if (isTopLevelStage(stage)) { + builder + .withTask("completeParallel", CompleteParallelBakeTask) + } else { + builder + .withTask("createBake", CreateBakeTask) + .withTask("monitorBake", MonitorBakeTask) + .withTask("completedBake", CompletedBakeTask) + } } @Override - void postBranchGraph(Stage stage, TaskNode.Builder builder) { - builder - .withTask("completeParallel", CompleteParallelBakeTask) + @Nonnull + > List> parallelStages( + @Nonnull Stage stage + ) { + if (isTopLevelStage(stage)) { + return parallelContexts(stage).collect { context -> + newStage(stage.execution, type, "Bake in ${context.region}", context, stage, STAGE_BEFORE) + } + } else { + return Collections.emptyList() + } + } + + private boolean isTopLevelStage(Stage stage) { + stage.parentStageId == null } - @Override @CompileDynamic - public > Collection> parallelContexts(Stage stage) { + > Collection> parallelContexts(Stage stage) { Set deployRegions = stage.context.region ? [stage.context.region] as Set : [] deployRegions.addAll(stage.context.regions as Set ?: []) @@ -100,11 +119,6 @@ class BakeStage implements BranchingStageDefinitionBuilder, RestartableStage { } } - @Override - String parallelStageName(Stage stage, boolean hasParallelFlows) { - return hasParallelFlows ? "Multi-region Bake" : stage.name - } - @Component @CompileStatic static class CompleteParallelBakeTask implements Task { diff --git a/orca-bakery/src/main/groovy/com/netflix/spinnaker/orca/bakery/tasks/MonitorBakeTask.groovy b/orca-bakery/src/main/groovy/com/netflix/spinnaker/orca/bakery/tasks/MonitorBakeTask.groovy index 4f162eb639..e8dd8766aa 100644 --- a/orca-bakery/src/main/groovy/com/netflix/spinnaker/orca/bakery/tasks/MonitorBakeTask.groovy +++ b/orca-bakery/src/main/groovy/com/netflix/spinnaker/orca/bakery/tasks/MonitorBakeTask.groovy @@ -17,7 +17,7 @@ package com.netflix.spinnaker.orca.bakery.tasks import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.bakery.api.BakeStatus import com.netflix.spinnaker.orca.bakery.api.BakeryService @@ -31,7 +31,7 @@ import retrofit.RetrofitError @Slf4j @Component @CompileStatic -class MonitorBakeTask implements RetryableTask { +class MonitorBakeTask implements OverridableTimeoutRetryableTask { long backoffPeriod = 30000 long timeout = 3600000 // 1hr diff --git a/orca-bakery/src/test/groovy/com/netflix/spinnaker/orca/bakery/pipeline/BakeStageSpec.groovy b/orca-bakery/src/test/groovy/com/netflix/spinnaker/orca/bakery/pipeline/BakeStageSpec.groovy index ac6f1bafa3..510b3b27d3 100644 --- a/orca-bakery/src/test/groovy/com/netflix/spinnaker/orca/bakery/pipeline/BakeStageSpec.groovy +++ b/orca-bakery/src/test/groovy/com/netflix/spinnaker/orca/bakery/pipeline/BakeStageSpec.groovy @@ -84,57 +84,28 @@ class BakeStageSpec extends Specification { stage { id = "1" type = "bake" - status = ExecutionStatus.RUNNING - } - stage { - parentStageId = "1" - type = "bake" - name = "Bake" - context = ["ami": 1] - status = ExecutionStatus.RUNNING - } - stage { - parentStageId = "1" - type = "bake" - name = "Bake" - context = ["ami": 2] - status = ExecutionStatus.RUNNING - } - stage { - parentStageId = "1" - type = "bake" - name = "Bake" - context = ["ami": 3] + context = [ + "region": "us-east-1", + "regions": ["us-east-1", "us-west-2", "eu-east-1"] + ] status = ExecutionStatus.RUNNING } } + def bakeStage = pipeline.stageById("1") + def parallelStages = new BakeStage().parallelStages(bakeStage) + parallelStages.eachWithIndex { it, idx -> it.context.ami = idx + 1 } + pipeline.stages.addAll(parallelStages) + when: def taskResult = new BakeStage.CompleteParallelBakeTask().execute(pipeline.stageById("1")) then: - taskResult.outputs == [ - deploymentDetails: [ - ["ami": 1], ["ami": 2], ["ami": 3] - ] - ] - } - - @Unroll - def "should return a different stage name when parallel flows are present"() { - given: - def stage = stage { - type = "type" - name = stageName + with(taskResult.outputs) { + deploymentDetails[0].ami == 1 + deploymentDetails[1].ami == 2 + deploymentDetails[2].ami == 3 } - - expect: - new BakeStage().parallelStageName(stage, hasParallelFlows) == expectedStageName - - where: - stageName | hasParallelFlows || expectedStageName - "Default" | false || "Default" - "Default" | true || "Multi-region Bake" } private diff --git a/orca-clouddriver/orca-clouddriver.gradle b/orca-clouddriver/orca-clouddriver.gradle index bb4255b1d5..c755a2250c 100644 --- a/orca-clouddriver/orca-clouddriver.gradle +++ b/orca-clouddriver/orca-clouddriver.gradle @@ -19,5 +19,6 @@ dependencies { compile project(":orca-retrofit") compile project(":orca-front50") compile project(":orca-bakery") + compile 'com.netflix.spinnaker.moniker:moniker:0.2.0' testCompile project(":orca-test") } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/DelegatingClouddriverService.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/DelegatingClouddriverService.java index fd0cdce271..6549dea7b5 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/DelegatingClouddriverService.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/DelegatingClouddriverService.java @@ -27,12 +27,13 @@ class DelegatingClouddriverService { } T getService() { - SelectableService.Criteria criteria = new SelectableService.Criteria(null, null, null, null); + SelectableService.Criteria criteria = new SelectableService.Criteria(null, null, null, null, null); ExecutionContext executionContext = ExecutionContext.get(); if (executionContext != null) { criteria = new SelectableService.Criteria( executionContext.getApplication(), + executionContext.getAuthenticatedUser(), executionContext.getExecutionType(), executionContext.getExecutionId(), executionContext.getOrigin() diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/DelegatingOortService.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/DelegatingOortService.java index 27cd2292ee..c4267c36c9 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/DelegatingOortService.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/DelegatingOortService.java @@ -40,11 +40,22 @@ public Response getServerGroupFromCluster(String app, String account, String clu return getService().getServerGroupFromCluster(app, account, cluster, serverGroup, region, cloudProvider); } + @Override + public Response getServerGroups(String app) { + return getService().getServerGroups(app); + } + + @Deprecated @Override public Response getServerGroup(String app, String account, String region, String serverGroup) { return getService().getServerGroup(app, account, region, serverGroup); } + @Override + public Response getServerGroup(String account, String serverGroup, String region) { + return getService().getServerGroup(account, serverGroup, region); + } + @Override public Response getTargetServerGroup(String app, String account, String cluster, String cloudProvider, String scope, String target) { if (app == null) { diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/OortService.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/OortService.groovy index 88adbdb5b7..2585d0920c 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/OortService.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/OortService.groovy @@ -30,6 +30,9 @@ interface OortService { @Path("cluster") String cluster, @Path("cloudProvider") String cloudProvider) + @GET("/applications/{app}/serverGroups") + Response getServerGroups(@Path("app") String app) + @GET("/applications/{app}/clusters/{account}/{cluster}/{cloudProvider}/serverGroups/{serverGroup}") Response getServerGroupFromCluster(@Path("app") String app, @Path("account") String account, @@ -38,11 +41,17 @@ interface OortService { @Query("region") String region, @Path("cloudProvider") String cloudProvider) + @Deprecated @GET("/applications/{app}/serverGroups/{account}/{region}/{serverGroup}") Response getServerGroup(@Path("app") String app, - @Path("account") String account, - @Path("region") String region, - @Path("serverGroup") String serverGroup) + @Path("account") String account, + @Path("region") String region, + @Path("serverGroup") String serverGroup) + + @GET("/serverGroups/{account}/{region}/{serverGroup}") + Response getServerGroup(@Path("account") String account, + @Path("region") String region, + @Path("serverGroup") String serverGroup) @GET("/applications/{app}/clusters/{account}/{cluster}/{cloudProvider}/{scope}/serverGroups/target/{target}") Response getTargetServerGroup(@Path("app") String app, diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/config/ByAuthenticatedUserServiceSelector.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/config/ByAuthenticatedUserServiceSelector.java new file mode 100644 index 0000000000..e832540168 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/config/ByAuthenticatedUserServiceSelector.java @@ -0,0 +1,61 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.config; + +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +public class ByAuthenticatedUserServiceSelector implements ServiceSelector { + private final Object service; + private final int priority; + private final List userPatterns; + + public ByAuthenticatedUserServiceSelector(Object service, Integer priority, Map config) { + this.service = service; + this.priority = priority; + + Collection users = new HashSet( + ((Map) config.get("users")).values() + ); + this.userPatterns = users.stream().map(Pattern::compile).collect(Collectors.toList()); + } + + @Override + public Object getService() { + return service; + } + + @Override + public int getPriority() { + return priority; + } + + @Override + public boolean supports(SelectableService.Criteria criteria) { + if (criteria.getAuthenticatedUser() == null) { + return false; + } + + return userPatterns + .stream() + .anyMatch(userPattern -> userPattern.matcher(criteria.getAuthenticatedUser()).matches()); + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/config/SelectableService.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/config/SelectableService.java index 48ae2f9306..25ca6474b7 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/config/SelectableService.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/config/SelectableService.java @@ -41,19 +41,22 @@ public Object getService(Criteria criteria) { public static class Criteria { private final String application; + private final String authenticatedUser; private final String executionType; private final String executionId; private final String origin; - public Criteria(String application, String executionType, String origin) { - this(application, executionType, null, origin); + public Criteria(String application, String authenticatedUser, String executionType, String origin) { + this(application, authenticatedUser, executionType, null, origin); } public Criteria(String application, + String authenticatedUser, String executionType, String executionId, String origin) { this.application = application; + this.authenticatedUser = authenticatedUser; this.executionType = executionType; this.executionId = executionId; this.origin = origin; @@ -63,6 +66,10 @@ public String getApplication() { return application; } + public String getAuthenticatedUser() { + return authenticatedUser; + } + public String getExecutionType() { return executionType; } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/manifest/DeleteManifestStage.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/manifest/DeleteManifestStage.java new file mode 100644 index 0000000000..30b7c2c4e2 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/manifest/DeleteManifestStage.java @@ -0,0 +1,39 @@ +/* + * Copyright 2017 Google, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package com.netflix.spinnaker.orca.clouddriver.pipeline.manifest; + +import com.netflix.spinnaker.orca.clouddriver.tasks.MonitorKatoTask; +import com.netflix.spinnaker.orca.clouddriver.tasks.manifest.DeleteManifestTask; +import com.netflix.spinnaker.orca.clouddriver.tasks.manifest.ManifestForceCacheRefreshTask; +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder; +import com.netflix.spinnaker.orca.pipeline.TaskNode; +import com.netflix.spinnaker.orca.pipeline.model.Execution; +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.springframework.stereotype.Component; + +@Component +public class DeleteManifestStage implements StageDefinitionBuilder { + public static final String PIPELINE_CONFIG_TYPE = "deleteManifest"; + + @Override + public > void taskGraph(Stage stage, TaskNode.Builder builder) { + builder.withTask(DeleteManifestTask.TASK_NAME, DeleteManifestTask.class) + .withTask("monitorDelete", MonitorKatoTask.class) + .withTask(ManifestForceCacheRefreshTask.TASK_NAME, ManifestForceCacheRefreshTask.class); + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/manifest/DeployManifestStage.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/manifest/DeployManifestStage.java new file mode 100644 index 0000000000..72f18c5de1 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/manifest/DeployManifestStage.java @@ -0,0 +1,39 @@ +/* + * Copyright 2017 Google, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package com.netflix.spinnaker.orca.clouddriver.pipeline.manifest; + +import com.netflix.spinnaker.orca.clouddriver.tasks.MonitorKatoTask; +import com.netflix.spinnaker.orca.clouddriver.tasks.manifest.DeployManifestTask; +import com.netflix.spinnaker.orca.clouddriver.tasks.manifest.ManifestForceCacheRefreshTask; +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder; +import com.netflix.spinnaker.orca.pipeline.TaskNode; +import com.netflix.spinnaker.orca.pipeline.model.Execution; +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.springframework.stereotype.Component; + +@Component +public class DeployManifestStage implements StageDefinitionBuilder { + public static final String PIPELINE_CONFIG_TYPE = "deployManifest"; + + @Override + public > void taskGraph(Stage stage, TaskNode.Builder builder) { + builder.withTask(DeployManifestTask.TASK_NAME, DeployManifestTask.class) + .withTask("monitorDeploy", MonitorKatoTask.class) + .withTask(ManifestForceCacheRefreshTask.TASK_NAME, ManifestForceCacheRefreshTask.class); + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/CloneServerGroupStage.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/CloneServerGroupStage.groovy index 0aa7d5300f..9ff9f2955f 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/CloneServerGroupStage.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/CloneServerGroupStage.groovy @@ -22,7 +22,7 @@ import com.netflix.spinnaker.orca.clouddriver.tasks.MonitorKatoTask import com.netflix.spinnaker.orca.clouddriver.tasks.instance.WaitForUpInstancesTask import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.CloneServerGroupTask import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.ServerGroupCacheForceRefreshTask -import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.ServerGroupMetadataTagTask +import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.AddServerGroupEntityTagsTask import com.netflix.spinnaker.orca.pipeline.TaskNode import com.netflix.spinnaker.orca.pipeline.model.Stage import groovy.util.logging.Slf4j @@ -58,7 +58,7 @@ class CloneServerGroupStage extends AbstractDeployStrategyStage { if (taggingEnabled) { tasks += [ - new TaskNode.TaskDefinition("tagServerGroup", ServerGroupMetadataTagTask) + new TaskNode.TaskDefinition("tagServerGroup", AddServerGroupEntityTagsTask) ] } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/CreateServerGroupStage.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/CreateServerGroupStage.groovy index 78af3f3b3b..07b6efc373 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/CreateServerGroupStage.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/CreateServerGroupStage.groovy @@ -22,7 +22,7 @@ import com.netflix.spinnaker.orca.clouddriver.tasks.MonitorKatoTask import com.netflix.spinnaker.orca.clouddriver.tasks.instance.WaitForUpInstancesTask import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.CreateServerGroupTask import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.ServerGroupCacheForceRefreshTask -import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.ServerGroupMetadataTagTask +import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.AddServerGroupEntityTagsTask import com.netflix.spinnaker.orca.pipeline.TaskNode import com.netflix.spinnaker.orca.pipeline.model.Stage import org.springframework.beans.factory.annotation.Autowired @@ -50,7 +50,7 @@ class CreateServerGroupStage extends AbstractDeployStrategyStage { ] if (taggingEnabled) { - tasks << TaskNode.task("tagServerGroup", ServerGroupMetadataTagTask) + tasks << TaskNode.task("tagServerGroup", AddServerGroupEntityTagsTask) } tasks << TaskNode.task("waitForUpInstances", WaitForUpInstancesTask) diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/RollbackServerGroupStage.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/RollbackServerGroupStage.groovy index f78fd148b4..43cf34bc30 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/RollbackServerGroupStage.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/RollbackServerGroupStage.groovy @@ -18,9 +18,13 @@ package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup import com.fasterxml.jackson.annotation.JsonIgnore -import com.netflix.spinnaker.orca.clouddriver.pipeline.providers.aws.ApplySourceServerGroupCapacityStage -import com.netflix.spinnaker.orca.clouddriver.pipeline.providers.aws.CaptureSourceServerGroupCapacityStage -import com.netflix.spinnaker.orca.kato.pipeline.support.ResizeStrategy +import com.netflix.frigga.Names +import com.netflix.spinnaker.orca.RetrySupport +import com.netflix.spinnaker.orca.clouddriver.OortService +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback.ExplicitRollback +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback.PreviousImageRollback +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback.Rollback +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback.TestRollback import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -37,7 +41,7 @@ class RollbackServerGroupStage implements StageDefinitionBuilder { AutowireCapableBeanFactory autowireCapableBeanFactory @Override - def > List> aroundStages(Stage stage) { + > List> aroundStages(Stage stage) { def stageData = stage.mapTo(StageData) if (!stageData.rollbackType) { @@ -50,7 +54,9 @@ class RollbackServerGroupStage implements StageDefinitionBuilder { } static enum RollbackType { - EXPLICIT(ExplicitRollback) + EXPLICIT(ExplicitRollback), + PREVIOUS_IMAGE(PreviousImageRollback), + TEST(TestRollback) final Class implementationClass @@ -62,114 +68,4 @@ class RollbackServerGroupStage implements StageDefinitionBuilder { static class StageData { RollbackType rollbackType } - - static interface Rollback { - def > List> buildStages(Stage parentStage) - } - - static class ExplicitRollback implements Rollback { - String rollbackServerGroupName - String restoreServerGroupName - - @Autowired - @JsonIgnore - EnableServerGroupStage enableServerGroupStage - - @Autowired - @JsonIgnore - DisableServerGroupStage disableServerGroupStage - - @Autowired - @JsonIgnore - ResizeServerGroupStage resizeServerGroupStage - - @Autowired - @JsonIgnore - CaptureSourceServerGroupCapacityStage captureSourceServerGroupCapacityStage - - @Autowired - @JsonIgnore - ApplySourceServerGroupCapacityStage applySourceServerGroupCapacityStage - - @JsonIgnore - def > List> buildStages(Stage parentStage) { - def stages = [] - - Map enableServerGroupContext = new HashMap(parentStage.context) - enableServerGroupContext.serverGroupName = restoreServerGroupName - stages << newStage( - parentStage.execution, enableServerGroupStage.type, "enable", enableServerGroupContext, parentStage, SyntheticStageOwner.STAGE_AFTER - ) - - stages << buildCaptureSourceServerGroupCapacityStage(parentStage, parentStage.mapTo(ResizeStrategy.Source)) - - Map resizeServerGroupContext = new HashMap(parentStage.context) + [ - action : ResizeStrategy.ResizeAction.scale_to_server_group.toString(), - source : { - def source = parentStage.mapTo(ResizeStrategy.Source) - source.serverGroupName = rollbackServerGroupName - return source - }.call(), - asgName : restoreServerGroupName, - pinMinimumCapacity: true - ] - stages << newStage( - parentStage.execution, resizeServerGroupStage.type, "resize", resizeServerGroupContext, parentStage, SyntheticStageOwner.STAGE_AFTER - ) - - Map disableServerGroupContext = new HashMap(parentStage.context) - disableServerGroupContext.serverGroupName = rollbackServerGroupName - stages << newStage( - parentStage.execution, disableServerGroupStage.type, "disable", disableServerGroupContext, parentStage, SyntheticStageOwner.STAGE_AFTER - ) - - stages << buildApplySourceServerGroupCapacityStage(parentStage, parentStage.mapTo(ResizeStrategy.Source)) - return stages - } - - Stage buildCaptureSourceServerGroupCapacityStage(Stage parentStage, - ResizeStrategy.Source source) { - Map captureSourceServerGroupCapacityContext = [ - useSourceCapacity: true, - source : [ - asgName : rollbackServerGroupName, - serverGroupName: rollbackServerGroupName, - region : source.region, - account : source.credentials, - cloudProvider : source.cloudProvider - ] - ] - return newStage( - parentStage.execution, - captureSourceServerGroupCapacityStage.type, - "snapshot", - captureSourceServerGroupCapacityContext, - parentStage, - SyntheticStageOwner.STAGE_AFTER - ) - } - - Stage buildApplySourceServerGroupCapacityStage(Stage parentStage, - ResizeStrategy.Source source) { - Map applySourceServerGroupCapacityContext = [ - credentials: source.credentials, - target : [ - asgName : restoreServerGroupName, - serverGroupName: restoreServerGroupName, - region : source.region, - account : source.credentials, - cloudProvider : source.cloudProvider - ] - ] - return newStage( - parentStage.execution, - applySourceServerGroupCapacityStage.type, - "apply", - applySourceServerGroupCapacityContext, - parentStage, - SyntheticStageOwner.STAGE_AFTER - ) - } - } - } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/ExplicitRollback.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/ExplicitRollback.groovy new file mode 100644 index 0000000000..e429422dd0 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/ExplicitRollback.groovy @@ -0,0 +1,139 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback + +import com.fasterxml.jackson.annotation.JsonIgnore +import com.netflix.spinnaker.orca.clouddriver.pipeline.providers.aws.ApplySourceServerGroupCapacityStage +import com.netflix.spinnaker.orca.clouddriver.pipeline.providers.aws.CaptureSourceServerGroupCapacityStage +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.DisableServerGroupStage +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.EnableServerGroupStage +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.ResizeServerGroupStage +import com.netflix.spinnaker.orca.kato.pipeline.support.ResizeStrategy +import com.netflix.spinnaker.orca.pipeline.model.Execution +import com.netflix.spinnaker.orca.pipeline.model.Stage +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner +import org.springframework.beans.factory.annotation.Autowired; + +import static com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder.newStage + +class ExplicitRollback implements Rollback { + String rollbackServerGroupName + String restoreServerGroupName + Integer targetHealthyRollbackPercentage + + @Autowired + @JsonIgnore + EnableServerGroupStage enableServerGroupStage + + @Autowired + @JsonIgnore + DisableServerGroupStage disableServerGroupStage + + @Autowired + @JsonIgnore + ResizeServerGroupStage resizeServerGroupStage + + @Autowired + @JsonIgnore + CaptureSourceServerGroupCapacityStage captureSourceServerGroupCapacityStage + + @Autowired + @JsonIgnore + ApplySourceServerGroupCapacityStage applySourceServerGroupCapacityStage + + @JsonIgnore + def > List> buildStages(Stage parentStage) { + def stages = [] + + Map enableServerGroupContext = new HashMap(parentStage.context) + enableServerGroupContext.targetHealthyDeployPercentage = targetHealthyRollbackPercentage + enableServerGroupContext.serverGroupName = restoreServerGroupName + stages << newStage( + parentStage.execution, enableServerGroupStage.type, "enable", enableServerGroupContext, parentStage, SyntheticStageOwner.STAGE_AFTER + ) + + stages << buildCaptureSourceServerGroupCapacityStage(parentStage, parentStage.mapTo(ResizeStrategy.Source)) + + Map resizeServerGroupContext = new HashMap(parentStage.context) + [ + action : ResizeStrategy.ResizeAction.scale_to_server_group.toString(), + source : { + def source = parentStage.mapTo(ResizeStrategy.Source) + source.serverGroupName = rollbackServerGroupName + return source + }.call(), + asgName : restoreServerGroupName, + pinMinimumCapacity : true, + targetHealthyDeployPercentage: targetHealthyRollbackPercentage + ] + stages << newStage( + parentStage.execution, resizeServerGroupStage.type, "resize", resizeServerGroupContext, parentStage, SyntheticStageOwner.STAGE_AFTER + ) + + Map disableServerGroupContext = new HashMap(parentStage.context) + disableServerGroupContext.serverGroupName = rollbackServerGroupName + stages << newStage( + parentStage.execution, disableServerGroupStage.type, "disable", disableServerGroupContext, parentStage, SyntheticStageOwner.STAGE_AFTER + ) + + stages << buildApplySourceServerGroupCapacityStage(parentStage, parentStage.mapTo(ResizeStrategy.Source)) + return stages + } + + Stage buildCaptureSourceServerGroupCapacityStage(Stage parentStage, + ResizeStrategy.Source source) { + Map captureSourceServerGroupCapacityContext = [ + useSourceCapacity: true, + source : [ + asgName : rollbackServerGroupName, + serverGroupName: rollbackServerGroupName, + region : source.region, + account : source.credentials, + cloudProvider : source.cloudProvider + ] + ] + return newStage( + parentStage.execution, + captureSourceServerGroupCapacityStage.type, + "snapshot", + captureSourceServerGroupCapacityContext, + parentStage, + SyntheticStageOwner.STAGE_AFTER + ) + } + + Stage buildApplySourceServerGroupCapacityStage(Stage parentStage, + ResizeStrategy.Source source) { + Map applySourceServerGroupCapacityContext = [ + credentials: source.credentials, + target : [ + asgName : restoreServerGroupName, + serverGroupName: restoreServerGroupName, + region : source.region, + account : source.credentials, + cloudProvider : source.cloudProvider + ] + ] + return newStage( + parentStage.execution, + applySourceServerGroupCapacityStage.type, + "apply", + applySourceServerGroupCapacityContext, + parentStage, + SyntheticStageOwner.STAGE_AFTER + ) + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/PreviousImageRollback.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/PreviousImageRollback.groovy new file mode 100644 index 0000000000..23745ed52b --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/PreviousImageRollback.groovy @@ -0,0 +1,149 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback + +import com.fasterxml.jackson.annotation.JsonIgnore +import com.netflix.frigga.Names +import com.netflix.spinnaker.orca.RetrySupport +import com.netflix.spinnaker.orca.clouddriver.OortService +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.CloneServerGroupStage +import com.netflix.spinnaker.orca.pipeline.model.Execution +import com.netflix.spinnaker.orca.pipeline.model.Stage +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner +import org.springframework.beans.factory.annotation.Autowired; + +import static com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder.newStage + +class PreviousImageRollback implements Rollback { + String rollbackServerGroupName + String imageName + String imageId + Integer targetHealthyRollbackPercentage + + @Autowired + @JsonIgnore + CloneServerGroupStage cloneServerGroupStage + + @Autowired + @JsonIgnore + OortService oortService + + @Autowired + @JsonIgnore + RetrySupport retrySupport + + @Override + > List> buildStages(Stage parentStage) { + def stages = [] + + def parentStageContext = parentStage.context + + def imageName = this.imageName + def imageId = this.imageId + + if (!imageName) { + def imageDetails = getImageDetailsFromEntityTags( + parentStageContext.cloudProvider as String, + parentStageContext.credentials as String, + parentStageContext.region as String + ) + + imageName = imageDetails?.imageName + imageId = imageDetails?.imageId ?: imageId + } + + if (!imageName) { + throw new IllegalStateException("Unable to determine rollback image (serverGroupName: ${rollbackServerGroupName})") + } + + def names = Names.parseName(rollbackServerGroupName) + + Map cloneServerGroupContext = [ + targetHealthyDeployPercentage: targetHealthyRollbackPercentage, + imageId : imageId, + imageName : imageName, + amiName : imageName, + strategy : "redblack", + application : parentStageContext.moniker?.app ?: names.app, + stack : parentStageContext.moniker?.stack ?: names.stack, + freeFormDetails : parentStageContext.moniker?.detail ?: names.detail, + region : parentStageContext.region, + credentials : parentStageContext.credentials, + cloudProvider : parentStageContext.cloudProvider, + source: [ + asgName : rollbackServerGroupName, + serverGroupName : rollbackServerGroupName, + account : parentStageContext.credentials, + region : parentStageContext.region, + cloudProvider : parentStageContext.cloudProvider, + useSourceCapacity: true + ] + ] + + if (parentStageContext.containsKey("interestingHealthProviderNames")) { + cloneServerGroupContext.interestingHealthProviderNames = parentStageContext.interestingHealthProviderNames + } + + stages << newStage( + parentStage.execution, cloneServerGroupStage.type, "clone", cloneServerGroupContext, parentStage, SyntheticStageOwner.STAGE_AFTER + ) + + return stages + } + + private ImageDetails getImageDetailsFromEntityTags(String cloudProvider, + String credentials, + String region) { + def entityTags = retrySupport.retry({ + oortService.getEntityTags( + cloudProvider, + "serverGroup", + rollbackServerGroupName, + credentials, + region + ) + }, 15, 2000, false) + + if (entityTags?.size() > 1) { + // this should _not_ happen + String id = Arrays.asList( + cloudProvider, + "serverGroup", + rollbackServerGroupName, + credentials, + region + ).join(":") + throw new IllegalStateException("More than one set of entity tags found for " + id); + } + + if (!entityTags) { + return null + } + + def previousServerGroup = entityTags[0].tags.find { it.name == "spinnaker:metadata" }?.value?.previousServerGroup + if (!previousServerGroup?.imageName) { + return null + } + + return new ImageDetails(imageId: previousServerGroup.imageId, imageName: previousServerGroup.imageName) + } + + static class ImageDetails { + String imageId + String imageName + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/Rollback.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/Rollback.java new file mode 100644 index 0000000000..c621081b0d --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/Rollback.java @@ -0,0 +1,26 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback; + +import com.netflix.spinnaker.orca.pipeline.model.Execution; +import com.netflix.spinnaker.orca.pipeline.model.Stage; + +import java.util.List; + +public interface Rollback { + > List> buildStages(Stage parentStage); +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/TestRollback.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/TestRollback.java new file mode 100644 index 0000000000..eaa01d0f12 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/TestRollback.java @@ -0,0 +1,54 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.EnableServerGroupStage; +import com.netflix.spinnaker.orca.pipeline.WaitStage; +import com.netflix.spinnaker.orca.pipeline.model.Execution; +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner; +import org.springframework.beans.factory.annotation.Autowired; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder.newStage; + +public class TestRollback implements Rollback { + private Long waitTime; + + @Autowired + @JsonIgnore + WaitStage waitStage; + + @Override + public > List> buildStages(Stage parentStage) { + Map waitContext = Collections.singletonMap("waitTime", waitTime); + + return Collections.singletonList( + newStage( + parentStage.getExecution(), waitStage.getType(), "wait", waitContext, parentStage, SyntheticStageOwner.STAGE_AFTER + ) + ); + } + + public void setWaitTime(Long waitTime) { + this.waitTime = waitTime; + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/AbstractDeployStrategyStage.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/AbstractDeployStrategyStage.groovy index cb2bbfd239..e88e2ffd2f 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/AbstractDeployStrategyStage.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/AbstractDeployStrategyStage.groovy @@ -16,6 +16,7 @@ package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.strategies +import com.netflix.spinnaker.moniker.Moniker import com.netflix.spinnaker.orca.clouddriver.pipeline.AbstractCloudProviderAwareStage import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.Location import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroup @@ -153,10 +154,10 @@ abstract class AbstractDeployStrategyStage extends AbstractCloudProviderAwareSta return className } - @Immutable static class CleanupConfig { String account String cluster + Moniker moniker String cloudProvider Location location @@ -166,6 +167,7 @@ abstract class AbstractDeployStrategyStage extends AbstractCloudProviderAwareSta new CleanupConfig( account: stageData.account, cluster: stageData.cluster, + moniker: stageData.moniker, cloudProvider: stageData.cloudProvider, location: loc ) diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/CustomStrategy.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/CustomStrategy.groovy index 0fdc82aeac..8c5c4a3896 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/CustomStrategy.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/CustomStrategy.groovy @@ -46,6 +46,7 @@ class CustomStrategy implements Strategy, ApplicationContextAware { application : stage.context.application, credentials : cleanupConfig.account, cluster : cleanupConfig.cluster, + moniker : cleanupConfig.moniker, (cleanupConfig.location.singularType()): cleanupConfig.location.value, cloudProvider : cleanupConfig.cloudProvider, strategy : true, diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/HighlanderStrategy.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/HighlanderStrategy.groovy index 08db8d35e0..036f6ed894 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/HighlanderStrategy.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/HighlanderStrategy.groovy @@ -44,6 +44,7 @@ class HighlanderStrategy implements Strategy, ApplicationContextAware { Map shrinkContext = [ (cleanupConfig.location.singularType()): cleanupConfig.location.value, cluster : cleanupConfig.cluster, + moniker : cleanupConfig.moniker, credentials : cleanupConfig.account, cloudProvider : cleanupConfig.cloudProvider, shrinkToSize : 1, diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RedBlackStrategy.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RedBlackStrategy.groovy index a0311c5d41..47e072a4ea 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RedBlackStrategy.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RedBlackStrategy.groovy @@ -59,6 +59,7 @@ class RedBlackStrategy implements Strategy, ApplicationContextAware { (cleanupConfig.location.singularType()): cleanupConfig.location.value, cluster : cleanupConfig.cluster, credentials : cleanupConfig.account, + moniker : cleanupConfig.moniker, cloudProvider : cleanupConfig.cloudProvider, ] @@ -84,8 +85,8 @@ class RedBlackStrategy implements Strategy, ApplicationContextAware { ) } - if(stageData?.delayBeforeDisableSec) { - def waitContext = [waitTime: stageData?.delayBeforeDisableSec] + if(stageData?.getDelayBeforeCleanup()) { + def waitContext = [waitTime: stageData?.getDelayBeforeCleanup()] stages << newStage( stage.execution, waitStage.type, diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RollingRedBlackStrategy.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RollingRedBlackStrategy.groovy index 5bcbe36f78..267e7ca706 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RollingRedBlackStrategy.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RollingRedBlackStrategy.groovy @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.strategies import com.netflix.spinnaker.orca.clouddriver.pipeline.cluster.DisableClusterStage @@ -5,7 +20,8 @@ import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.ResizeServerG import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.DetermineTargetServerGroupStage import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroup import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroupResolver -import com.netflix.spinnaker.orca.kato.pipeline.support.SourceResolver +import com.netflix.spinnaker.orca.front50.pipeline.PipelineStage +import com.netflix.spinnaker.orca.kato.pipeline.support.ResizeStrategy import com.netflix.spinnaker.orca.pipeline.WaitStage import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -31,6 +47,9 @@ class RollingRedBlackStrategy implements Strategy, ApplicationContextAware { @Autowired WaitStage waitStage + @Autowired + PipelineStage pipelineStage + @Autowired DetermineTargetServerGroupStage determineTargetServerGroupStage @@ -44,30 +63,18 @@ class RollingRedBlackStrategy implements Strategy, ApplicationContextAware { def cleanupConfig = AbstractDeployStrategyStage.CleanupConfig.fromStage(stage) Map baseContext = [ - (cleanupConfig.location.singularType()): cleanupConfig.location.value, - cluster : cleanupConfig.cluster, - credentials : cleanupConfig.account, - cloudProvider : cleanupConfig.cloudProvider, + (cleanupConfig.location.singularType()): cleanupConfig.location.value, + cluster : cleanupConfig.cluster, + moniker : cleanupConfig.moniker, + credentials : cleanupConfig.account, + cloudProvider : cleanupConfig.cloudProvider, ] - Map originalCapacity = stageData.originalCapacity ?: stageData.capacity - if (!originalCapacity) { - originalCapacity = [ - min: stageData.targetSize, - max: stageData.targetSize, - desired: stageData.targetSize - ] - } - if (stageData.targetSize) { stage.context.targetSize = 0 } if (stage.context.useSourceCapacity) { - List target = targetServerGroupResolver.resolve(new Stage(null, null, null, baseContext + [target: TargetServerGroup.Params.Target.current_asg_dynamic])) - if (target.size() > 0) { - originalCapacity = target.get(0).capacity - } stage.context.useSourceCapacity = false } @@ -84,80 +91,145 @@ class RollingRedBlackStrategy implements Strategy, ApplicationContextAware { } def findContext = baseContext + [ - target: TargetServerGroup.Params.Target.current_asg_dynamic, - targetLocation: cleanupConfig.location, + target : TargetServerGroup.Params.Target.current_asg_dynamic, + targetLocation: cleanupConfig.location, ] stages << newStage( - stage.execution, - determineTargetServerGroupStage.type, - "Determine Deployed Server Group", - findContext, - stage, - SyntheticStageOwner.STAGE_AFTER + stage.execution, + determineTargetServerGroupStage.type, + "Determine Deployed Server Group", + findContext, + stage, + SyntheticStageOwner.STAGE_AFTER ) // java .forEach rather than groovy .each, since the nested .each closure sometimes omits parent context targetPercentages.forEach({ p -> + def source = getSource(targetServerGroupResolver, stageData, baseContext) def resizeContext = baseContext + [ - target: TargetServerGroup.Params.Target.current_asg_dynamic, - targetLocation: cleanupConfig.location, - capacity: makeIncrementalCapacity(originalCapacity, p) + target : TargetServerGroup.Params.Target.current_asg_dynamic, + action : ResizeStrategy.ResizeAction.scale_to_server_group, + source : source, + targetLocation: cleanupConfig.location, + scalePct : p, + pinCapacity : p < 100 // if p = 100, capacity should be unpinned ] - stages << newStage( - stage.execution, - resizeServerGroupStage.type, - "Grow to $p% Desired Size", - resizeContext, - stage, - SyntheticStageOwner.STAGE_AFTER + def resizeStage = newStage( + stage.execution, + resizeServerGroupStage.type, + "Grow to $p% Desired Size", + resizeContext, + stage, + SyntheticStageOwner.STAGE_AFTER ) + stages << resizeStage + + // an expression to grab newly deployed server group at runtime (ie. the server group being resized up) + def deployedServerGroupName = '${' + "#stage('${resizeStage.id}')['context']['asgName']" + '}'.toString() + stages.addAll(getBeforeCleanupStages(stage, stageData, source, deployedServerGroupName, p)) def disableContext = baseContext + [ - desiredPercentage : p, - remainingEnabledServerGroups: 1, - preferLargerOverNewer : false + desiredPercentage : p, + remainingEnabledServerGroups: 1, + preferLargerOverNewer : false ] - if(stageData?.delayBeforeDisableSec) { - def waitContext = [waitTime: stageData?.delayBeforeDisableSec] - stages << newStage( - stage.execution, - waitStage.type, - "wait", - waitContext, - stage, - SyntheticStageOwner.STAGE_AFTER - ) - } + stages << newStage( + stage.execution, + disableClusterStage.type, + "Disable $p% of Traffic", + disableContext, + stage, + SyntheticStageOwner.STAGE_AFTER + ) + }) + + return stages + } + + List getBeforeCleanupStages(Stage parentStage, + RollingRedBlackStageData stageData, + ResizeStrategy.Source source, + String deployedServerGroupName, + int percentageComplete) { + def stages = [] + if (stageData.getDelayBeforeCleanup()) { + def waitContext = [waitTime: stageData.getDelayBeforeCleanup()] stages << newStage( - stage.execution, - disableClusterStage.type, - "Disable $p% of Traffic", - disableContext, - stage, - SyntheticStageOwner.STAGE_AFTER + parentStage.execution, + waitStage.type, + "wait", + waitContext, + parentStage, + SyntheticStageOwner.STAGE_AFTER ) + } - }) + if (stageData.pipelineBeforeCleanup?.application && stageData.pipelineBeforeCleanup?.pipelineId) { + def serverGroupCoordinates = [ + region : source.region, + serverGroupName: source.serverGroupName, + account : source.credentials, + cloudProvider : source.cloudProvider + ] + + def pipelineContext = [ + pipelineApplication: stageData.pipelineBeforeCleanup.application, + pipelineId : stageData.pipelineBeforeCleanup.pipelineId, + pipelineParameters : [ + "deployedServerGroup": serverGroupCoordinates + [ + serverGroupName: deployedServerGroupName + ], + "sourceServerGroup" : serverGroupCoordinates + [ + serverGroupName: source.serverGroupName + ], + "percentageComplete" : percentageComplete + ] + ] + + stages << newStage( + parentStage.execution, + pipelineStage.type, + "Run Validation Pipeline", + pipelineContext, + parentStage, + SyntheticStageOwner.STAGE_AFTER + ) + } return stages } - private Map makeIncrementalCapacity(Map originalCapacity, Integer p) { + static ResizeStrategy.Source getSource(TargetServerGroupResolver targetServerGroupResolver, + RollingRedBlackStageData stageData, + Map baseContext) { + if (stageData.source) { + return new ResizeStrategy.Source( + region: stageData.source.region, + serverGroupName: stageData.source.serverGroupName ?: stageData.source.asgName, + credentials: stageData.credentials ?: stageData.account, + cloudProvider: stageData.cloudProvider + ) + } + + // no source server group specified, lookup current server group + TargetServerGroup target = targetServerGroupResolver.resolve( + new Stage(null, null, null, baseContext + [target: TargetServerGroup.Params.Target.current_asg_dynamic]) + )?.get(0) - if (p == 100) { - return originalCapacity + if (!target) { + throw new IllegalStateException("No target server groups found (${baseContext})") } - def desired = (Integer) originalCapacity.desired * (p / 100d) - return [ - min: desired, - max: desired, - desired: desired - ] + return new ResizeStrategy.Source( + region: target.getLocation().value, + serverGroupName: target.getName(), + credentials: stageData.credentials ?: stageData.account, + cloudProvider: stageData.cloudProvider + ) } ApplicationContext applicationContext diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroup.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroup.groovy index b5e2cba7c4..b462cf07d7 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroup.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroup.groovy @@ -20,6 +20,7 @@ import groovy.transform.InheritConstructors import groovy.transform.ToString import groovy.util.logging.Slf4j import com.netflix.frigga.Names +import com.netflix.spinnaker.moniker.Moniker import com.netflix.spinnaker.orca.kato.pipeline.support.StageData import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -195,6 +196,7 @@ class TargetServerGroup { // TODO(ttomsu): This feels dirty - consider structuring to enable an 'exact' Target that just specifies the exact // server group name to fetch? String serverGroupName + Moniker moniker // Alternatively to asgName, the combination of target and cluster can be used. Target target @@ -205,11 +207,11 @@ class TargetServerGroup { String cloudProvider = "aws" String getApp() { - Names.parseName(serverGroupName ?: cluster)?.app + moniker?.app ?: Names.parseName(serverGroupName ?: cluster)?.app } String getCluster() { - cluster ?: Names.parseName(serverGroupName)?.cluster + moniker?.cluster ?: cluster ?: Names.parseName(serverGroupName)?.cluster } static Params fromStage(Stage stage) { diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroupResolver.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroupResolver.groovy index 03903378cb..f1be56f269 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroupResolver.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroupResolver.groovy @@ -16,6 +16,7 @@ package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support +import com.netflix.spinnaker.orca.RetrySupport import groovy.util.logging.Slf4j import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spinnaker.orca.clouddriver.OortService @@ -37,6 +38,9 @@ class TargetServerGroupResolver { @Autowired ObjectMapper mapper + @Autowired + RetrySupport retrySupport + List resolve(Stage stage) { resolveByParams(TargetServerGroup.Params.fromStage(stage)) } @@ -159,38 +163,23 @@ class TargetServerGroupResolver { } private T fetchWithRetries(Class responseType, int maxRetries, long retryBackoff, Closure fetchClosure) { - def converter = new JacksonConverter(mapper) + return retrySupport.retry({ + def converter = new JacksonConverter(mapper) - def lastException = null - for (int i = 1; i <= maxRetries; i++) { + Response response try { - Response response - try { - response = fetchClosure.call() - } catch (RetrofitError re) { - if (re.kind == RetrofitError.Kind.HTTP && re.response.status == 404) { - return null - } - throw re - } - try { - return (T) converter.fromBody(response.body, responseType) - } catch (ConversionException ce) { - throw RetrofitError.conversionError(response.url, response, converter, responseType, ce) - } + response = fetchClosure.call() } catch (RetrofitError re) { - lastException = re - - if (re.kind == RetrofitError.Kind.NETWORK) { - Thread.sleep(retryBackoff) - } else if (re.kind == RetrofitError.Kind.HTTP && re.response.status == 429) { - Thread.sleep(retryBackoff) - } else { - throw re + if (re.kind == RetrofitError.Kind.HTTP && re.response.status == 404) { + return null } + throw re } - } - - throw lastException + try { + return (T) converter.fromBody(response.body, responseType) + } catch (ConversionException ce) { + throw RetrofitError.conversionError(response.url, response, converter, responseType, ce) + } + }, maxRetries, retryBackoff, false) } } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/DetermineHealthProvidersTask.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/DetermineHealthProvidersTask.java index ca6f554e8d..4a2fbb0c15 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/DetermineHealthProvidersTask.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/DetermineHealthProvidersTask.java @@ -23,15 +23,18 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import com.netflix.frigga.Names; +import com.netflix.spinnaker.moniker.Moniker; import com.netflix.spinnaker.orca.ExecutionStatus; import com.netflix.spinnaker.orca.RetryableTask; import com.netflix.spinnaker.orca.TaskResult; import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.InterestingHealthProviderNamesSupplier; import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.ServerGroupCreator; import com.netflix.spinnaker.orca.clouddriver.utils.CloudProviderAware; +import com.netflix.spinnaker.orca.clouddriver.utils.MonikerHelper; import com.netflix.spinnaker.orca.front50.Front50Service; import com.netflix.spinnaker.orca.front50.model.Application; import com.netflix.spinnaker.orca.pipeline.model.Stage; +import com.thoughtworks.xstream.mapper.Mapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -97,7 +100,10 @@ public TaskResult execute(Stage stage) { try { String applicationName = (String) stage.getContext().get("application"); - if (applicationName == null && stage.getContext().containsKey("serverGroupName")) { + Moniker moniker = MonikerHelper.monikerFromStage(stage); + if (applicationName == null && moniker != null && moniker.getApp() != null) { + applicationName = moniker.getApp(); + } else if (applicationName == null && stage.getContext().containsKey("serverGroupName")) { applicationName = Names.parseName((String) stage.getContext().get("serverGroupName")).getApp(); } else if (applicationName == null && stage.getContext().containsKey("asgName")) { applicationName = Names.parseName((String) stage.getContext().get("asgName")).getApp(); diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/MonitorKatoTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/MonitorKatoTask.groovy index 6906bbac34..e29995e2ee 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/MonitorKatoTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/MonitorKatoTask.groovy @@ -57,7 +57,7 @@ class MonitorKatoTask implements RetryableTask { this.clock = clock } - long getBackoffPeriod() { 10000L } + long getBackoffPeriod() { 5000L } long getTimeout() { 3600000L } @@ -120,6 +120,9 @@ class MonitorKatoTask implements RetryableTask { if (!stage.context.containsKey("deploy.jobs") && deployed) { outputs["deploy.jobs"] = deployed } + if (!stage.context.containsKey("deploy.outputs") && deployed) { + outputs["deploy.outputs"] = deployed + } } if (status == ExecutionStatus.SUCCEEDED || status == ExecutionStatus.TERMINAL || status == ExecutionStatus.RUNNING) { List> katoTasks = [] diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractClusterWideClouddriverTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractClusterWideClouddriverTask.groovy index 3a71dced25..9c2ca48896 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractClusterWideClouddriverTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractClusterWideClouddriverTask.groovy @@ -17,6 +17,7 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.cluster import com.netflix.frigga.Names +import com.netflix.spinnaker.moniker.Moniker import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.RetryableTask import com.netflix.spinnaker.orca.TaskResult @@ -62,13 +63,19 @@ abstract class AbstractClusterWideClouddriverTask extends AbstractCloudProviderA @Canonical static class ClusterSelection { String cluster + Moniker moniker; String cloudProvider = 'aws' String credentials @Override String toString() { - "Cluster $cloudProvider/$credentials/$cluster" + "Cluster $cloudProvider/$credentials/$cluster/$moniker" } + + String getApplication() { + moniker?.app ?: Names.parseName(cluster).app + } + } protected TaskResult missingClusterResult(Stage stage, ClusterSelection clusterSelection) { @@ -83,9 +90,7 @@ abstract class AbstractClusterWideClouddriverTask extends AbstractCloudProviderA @Override TaskResult execute(Stage stage) { def clusterSelection = stage.mapTo(ClusterSelection) - def names = Names.parseName(clusterSelection.cluster) - - Optional cluster = oortHelper.getCluster(names.app, + Optional cluster = oortHelper.getCluster(clusterSelection.getApplication(), clusterSelection.credentials, clusterSelection.cluster, clusterSelection.cloudProvider) diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractWaitForClusterWideClouddriverTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractWaitForClusterWideClouddriverTask.groovy index 82eefbfcf0..45b674e23c 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractWaitForClusterWideClouddriverTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractWaitForClusterWideClouddriverTask.groovy @@ -16,9 +16,8 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.cluster -import com.netflix.frigga.Names import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroup import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask @@ -30,7 +29,7 @@ import groovy.util.logging.Slf4j import org.springframework.beans.factory.annotation.Autowired @Slf4j -abstract class AbstractWaitForClusterWideClouddriverTask extends AbstractCloudProviderAwareTask implements RetryableTask { +abstract class AbstractWaitForClusterWideClouddriverTask extends AbstractCloudProviderAwareTask implements OverridableTimeoutRetryableTask { @Override public long getBackoffPeriod() { 10000 } @@ -101,9 +100,7 @@ abstract class AbstractWaitForClusterWideClouddriverTask extends AbstractCloudPr return TaskResult.SUCCEEDED } - def names = Names.parseName(clusterSelection.cluster) - - Optional cluster = oortHelper.getCluster(names.app, clusterSelection.credentials, clusterSelection.cluster, clusterSelection.cloudProvider) + Optional cluster = oortHelper.getCluster(clusterSelection.getApplication(), clusterSelection.credentials, clusterSelection.cluster, clusterSelection.cloudProvider) if (!cluster.isPresent()) { return missingClusterResult(stage, clusterSelection) } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/ClusterSizePreconditionTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/ClusterSizePreconditionTask.groovy index cab54c712f..829322468a 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/ClusterSizePreconditionTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/ClusterSizePreconditionTask.groovy @@ -18,6 +18,7 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.cluster import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.frigga.Names +import com.netflix.spinnaker.moniker.Moniker import com.netflix.spinnaker.orca.RetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.OortService @@ -48,13 +49,14 @@ class ClusterSizePreconditionTask extends AbstractCloudProviderAwareTask impleme @Canonical static class ComparisonConfig { String cluster + Moniker moniker String comparison = '==' int expected = 1 String credentials Set regions public String getApplication() { - Names.parseName(cluster)?.app + moniker?.app ?: Names.parseName(cluster)?.app } public Operator getOp() { diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/FindImageFromClusterTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/FindImageFromClusterTask.groovy index bf70c58d15..05c308a384 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/FindImageFromClusterTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/FindImageFromClusterTask.groovy @@ -19,6 +19,7 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.cluster import com.fasterxml.jackson.core.type.TypeReference import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.frigga.Names +import com.netflix.spinnaker.moniker.Moniker import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.RetryableTask import com.netflix.spinnaker.orca.TaskResult @@ -80,6 +81,7 @@ class FindImageFromClusterTask extends AbstractCloudProviderAwareTask implements @Canonical static class FindImageConfiguration { String cluster + Moniker moniker List regions List zones List namespaces @@ -89,7 +91,7 @@ class FindImageFromClusterTask extends AbstractCloudProviderAwareTask implements String imageNamePattern String getApplication() { - Names.parseName(cluster).app + moniker?.app ?: Names.parseName(cluster).app } Set getRequiredLocations() { diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractInstancesCheckTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractInstancesCheckTask.groovy index e997c0fbc7..f8c777c447 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractInstancesCheckTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractInstancesCheckTask.groovy @@ -16,12 +16,15 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.instance +import com.netflix.spinnaker.orca.clouddriver.utils.MonikerHelper + import java.time.Duration import java.util.concurrent.TimeUnit import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.frigga.Names +import com.netflix.spinnaker.moniker.Moniker import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.OortService import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask @@ -34,7 +37,7 @@ import org.springframework.beans.factory.annotation.Autowired import retrofit.RetrofitError @Slf4j -abstract class AbstractInstancesCheckTask extends AbstractCloudProviderAwareTask implements RetryableTask { +abstract class AbstractInstancesCheckTask extends AbstractCloudProviderAwareTask implements OverridableTimeoutRetryableTask { long backoffPeriod = TimeUnit.SECONDS.toMillis(10) long timeout = TimeUnit.HOURS.toMillis(2) long serverGroupWaitTime = TimeUnit.MINUTES.toMillis(10) @@ -59,7 +62,8 @@ abstract class AbstractInstancesCheckTask extends AbstractCloudProviderAwareTask */ abstract protected Map> getServerGroups(Stage stage) - abstract protected boolean hasSucceeded(Stage stage, Map serverGroup, List instances, Collection interestingHealthProviderNames) + abstract + protected boolean hasSucceeded(Stage stage, Map serverGroup, List instances, Collection interestingHealthProviderNames) protected Map getAdditionalRunningStageContext(Stage stage, Map serverGroup) { [:] @@ -96,7 +100,8 @@ abstract class AbstractInstancesCheckTask extends AbstractCloudProviderAwareTask } try { - def serverGroups = fetchServerGroups(account, getCloudProvider(stage), serverGroupsByRegion) + Moniker moniker = MonikerHelper.monikerFromStage(stage) + def serverGroups = fetchServerGroups(account, getCloudProvider(stage), serverGroupsByRegion, moniker) if (!serverGroups) { return new TaskResult(ExecutionStatus.RUNNING) } @@ -121,9 +126,9 @@ abstract class AbstractInstancesCheckTask extends AbstractCloudProviderAwareTask if (seenServerGroup && !stage.context.capacitySnapshot) { newContext.zeroDesiredCapacityCount = 0 newContext.capacitySnapshot = [ - minSize : serverGroup.capacity.min, - desiredCapacity: serverGroup.capacity.desired, - maxSize : serverGroup.capacity.max + minSize : serverGroup.capacity.min, + desiredCapacity: serverGroup.capacity.desired, + maxSize : serverGroup.capacity.max ] } if (seenServerGroup) { @@ -196,16 +201,18 @@ abstract class AbstractInstancesCheckTask extends AbstractCloudProviderAwareTask } } - private List fetchServerGroups(String account, String cloudProvider, Map> serverGroupsByRegion) { - Names names = Names.parseName(serverGroupsByRegion.values().flatten()[0]) - + private List fetchServerGroups(String account, String cloudProvider, Map> serverGroupsByRegion, Moniker moniker) { if (serverGroupsByRegion.values().flatten().size() > 1) { - def response = oortService.getCluster(names.app, account, names.cluster, cloudProvider) + Names names = Names.parseName(serverGroupsByRegion.values().flatten()[0]) + def appName = moniker?.app ?: names.app + def clusterName = moniker?.cluster ?: names.cluster + def response = oortService.getCluster(appName, account, clusterName, cloudProvider) def cluster = objectMapper.readValue(response.body.in().text, Map) return cluster.serverGroups ?: [] } else { def region = serverGroupsByRegion.keySet()[0] - def response = oortService.getServerGroup(names.app, account, region, names.group) + def serverGroupName = serverGroupsByRegion[region][0] + def response = oortService.getServerGroup(account, region, serverGroupName) def serverGroup = objectMapper.readValue(response.body.in().text, Map) return [serverGroup] } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractWaitForInstanceHealthChangeTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractWaitForInstanceHealthChangeTask.groovy index 62aa7ec9dd..5bea68489a 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractWaitForInstanceHealthChangeTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractWaitForInstanceHealthChangeTask.groovy @@ -18,13 +18,13 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.instance import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.OortService import com.netflix.spinnaker.orca.pipeline.model.Stage import org.springframework.beans.factory.annotation.Autowired -abstract class AbstractWaitForInstanceHealthChangeTask implements RetryableTask { +abstract class AbstractWaitForInstanceHealthChangeTask implements OverridableTimeoutRetryableTask { long backoffPeriod = 5000 long timeout = 3600000 diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/TerminateInstanceAndDecrementServerGroupTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/TerminateInstanceAndDecrementServerGroupTask.groovy index 4e3eea1afe..a4693fdd8f 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/TerminateInstanceAndDecrementServerGroupTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/TerminateInstanceAndDecrementServerGroupTask.groovy @@ -49,7 +49,10 @@ class TerminateInstanceAndDecrementServerGroupTask extends AbstractCloudProvider List remainingInstances = instanceSupport.remainingInstances(stage) + def serverGroupName = stage.context.serverGroupName ?: stage.context.asgName + trafficGuard.verifyInstanceTermination( + serverGroupName, [stage.context.instance] as List, account, Location.region(stage.context.region as String), diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/TerminateInstancesTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/TerminateInstancesTask.groovy index 5409371ad8..8e60a970eb 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/TerminateInstancesTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/TerminateInstancesTask.groovy @@ -49,7 +49,10 @@ class TerminateInstancesTask extends AbstractCloudProviderAwareTask implements T List remainingInstances = instanceSupport.remainingInstances(stage) + def serverGroupName = stage.context.serverGroupName ?: stage.context.asgName + trafficGuard.verifyInstanceTermination( + serverGroupName, stage.context.instanceIds as List, account, Location.region(stage.context.region as String), diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/WaitForTerminatedInstancesTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/WaitForTerminatedInstancesTask.groovy index fc69147d66..28d43572ee 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/WaitForTerminatedInstancesTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/WaitForTerminatedInstancesTask.groovy @@ -17,7 +17,7 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.instance import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.pipeline.instance.TerminatingInstance import com.netflix.spinnaker.orca.clouddriver.pipeline.instance.TerminatingInstanceSupport @@ -29,7 +29,7 @@ import org.springframework.stereotype.Component @Slf4j @Component -class WaitForTerminatedInstancesTask extends AbstractCloudProviderAwareTask implements RetryableTask { +class WaitForTerminatedInstancesTask extends AbstractCloudProviderAwareTask implements OverridableTimeoutRetryableTask { long backoffPeriod = 10000 long timeout = 3600000 diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/job/WaitOnJobCompletion.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/job/WaitOnJobCompletion.groovy index 8c7cf0a154..358ee105af 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/job/WaitOnJobCompletion.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/job/WaitOnJobCompletion.groovy @@ -16,12 +16,14 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.job +import com.netflix.spinnaker.orca.RetrySupport + import java.util.concurrent.TimeUnit import com.fasterxml.jackson.core.type.TypeReference import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.frigga.Names import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.KatoRestService import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask @@ -30,9 +32,9 @@ import org.springframework.beans.factory.annotation.Autowired import org.springframework.stereotype.Component @Component -public class WaitOnJobCompletion extends AbstractCloudProviderAwareTask implements RetryableTask { +public class WaitOnJobCompletion extends AbstractCloudProviderAwareTask implements OverridableTimeoutRetryableTask { long backoffPeriod = TimeUnit.SECONDS.toMillis(10) - long timeout = TimeUnit.DAYS.toMillis(1) + long timeout = TimeUnit.HOURS.toMillis(2) @Autowired KatoRestService katoRestService @@ -40,6 +42,9 @@ public class WaitOnJobCompletion extends AbstractCloudProviderAwareTask implemen @Autowired ObjectMapper objectMapper + @Autowired + RetrySupport retrySupport + static final String REFRESH_TYPE = "Job" @Override @@ -69,8 +74,9 @@ public class WaitOnJobCompletion extends AbstractCloudProviderAwareTask implemen def name = names[0] def parsedName = Names.parseName(name) + String appName = stage.context.moniker?.app ?: stage.context.applicaton ?: parsedName.app - Map job = objectMapper.readValue(katoRestService.collectJob(parsedName.app, account, location, name, "delete").body.in(), new TypeReference() {}) + Map job = objectMapper.readValue(katoRestService.collectJob(appName, account, location, name, "delete").body.in(), new TypeReference() {}) outputs.jobStatus = job switch ((String) job.jobState) { @@ -80,10 +86,12 @@ public class WaitOnJobCompletion extends AbstractCloudProviderAwareTask implemen if (stage.context.propertyFile) { Map properties = [:] - properties = katoRestService.getFileContents(parsedName.app, account, location, name, stage.context.propertyFile) - if (properties.size() == 0) { - throw new IllegalStateException("expected properties file ${stage.context.propertyFile} but one was not found or was empty") - } + retrySupport.retry({ + properties = katoRestService.getFileContents(appName, account, location, name, stage.context.propertyFile) + if (properties.size() == 0) { + throw new IllegalStateException("Expected properties file ${stage.context.propertyFile} but it was either missing, empty or contained invalid syntax") + } + }, 6, 5000, false) // retry for 30 seconds outputs << properties outputs.propertyFileContents = properties } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/DeleteManifestTask.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/DeleteManifestTask.java new file mode 100644 index 0000000000..98923e06da --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/DeleteManifestTask.java @@ -0,0 +1,63 @@ +/* + * Copyright 2017 Google, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package com.netflix.spinnaker.orca.clouddriver.tasks.manifest; + +import com.google.common.collect.ImmutableMap; +import com.netflix.spinnaker.orca.ExecutionStatus; +import com.netflix.spinnaker.orca.Task; +import com.netflix.spinnaker.orca.TaskResult; +import com.netflix.spinnaker.orca.clouddriver.KatoService; +import com.netflix.spinnaker.orca.clouddriver.model.TaskId; +import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask; +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import javax.annotation.Nonnull; +import java.util.Collections; +import java.util.Map; + +@Component +public class DeleteManifestTask extends AbstractCloudProviderAwareTask implements Task { + @Autowired + KatoService kato; + + public static final String TASK_NAME = "deleteManifest"; + + @Nonnull + @Override + public TaskResult execute(@Nonnull Stage stage) { + String credentials = getCredentials(stage); + String cloudProvider = getCloudProvider(stage); + Map operation = new ImmutableMap.Builder() + .put(TASK_NAME, stage.getContext()) + .build(); + + TaskId taskId = kato.requestOperations(cloudProvider, Collections.singletonList(operation)).toBlocking().first(); + + Map outputs = new ImmutableMap.Builder() + .put("kato.result.expected", true) + .put("kato.last.task.id", taskId) + .put("delete.account.name", credentials) + .put("delete.name", stage.getContext().get("name")) + .put("delete.location", stage.getContext().get("location")) + .build(); + + return new TaskResult(ExecutionStatus.SUCCEEDED, outputs); + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/DeployManifestTask.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/DeployManifestTask.java new file mode 100644 index 0000000000..59ea758291 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/DeployManifestTask.java @@ -0,0 +1,61 @@ +/* + * Copyright 2017 Google, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package com.netflix.spinnaker.orca.clouddriver.tasks.manifest; + +import com.google.common.collect.ImmutableMap; +import com.netflix.spinnaker.orca.ExecutionStatus; +import com.netflix.spinnaker.orca.Task; +import com.netflix.spinnaker.orca.TaskResult; +import com.netflix.spinnaker.orca.clouddriver.KatoService; +import com.netflix.spinnaker.orca.clouddriver.model.TaskId; +import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask; +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import javax.annotation.Nonnull; +import java.util.Collections; +import java.util.Map; + +@Component +public class DeployManifestTask extends AbstractCloudProviderAwareTask implements Task { + @Autowired + KatoService kato; + + public static final String TASK_NAME = "deployManifest"; + + @Nonnull + @Override + public TaskResult execute(@Nonnull Stage stage) { + String credentials = getCredentials(stage); + String cloudProvider = getCloudProvider(stage); + Map operation = new ImmutableMap.Builder() + .put(TASK_NAME, stage.getContext()) + .build(); + + TaskId taskId = kato.requestOperations(cloudProvider, Collections.singletonList(operation)).toBlocking().first(); + + Map outputs = new ImmutableMap.Builder() + .put("kato.result.expected", true) + .put("kato.last.task.id", taskId) + .put("deploy.account.name", credentials) + .build(); + + return new TaskResult(ExecutionStatus.SUCCEEDED, outputs); + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/ManifestForceCacheRefreshTask.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/ManifestForceCacheRefreshTask.java new file mode 100644 index 0000000000..3decbdceab --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/manifest/ManifestForceCacheRefreshTask.java @@ -0,0 +1,64 @@ +/* + * Copyright 2017 Google, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package com.netflix.spinnaker.orca.clouddriver.tasks.manifest; + +import com.google.common.collect.ImmutableMap; +import com.netflix.spinnaker.orca.ExecutionStatus; +import com.netflix.spinnaker.orca.Task; +import com.netflix.spinnaker.orca.TaskResult; +import com.netflix.spinnaker.orca.clouddriver.CloudDriverCacheService; +import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask; +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import java.util.List; +import java.util.Map; + +@Component +public class ManifestForceCacheRefreshTask extends AbstractCloudProviderAwareTask implements Task { + private final static String REFRESH_TYPE = "Manifest"; + public final static String TASK_NAME = "forceCacheRefresh"; + + @Autowired + CloudDriverCacheService cacheService; + + @Override + public TaskResult execute(Stage stage) { + String cloudProvider = getCloudProvider(stage); + String account = getCredentials(stage); + Map> deployedManifests = (Map>) stage.getContext().get("deploy.outputs"); + + for (Map.Entry> entry : deployedManifests.entrySet()) { + String location = entry.getKey(); + entry.getValue().forEach(name -> { + Map request = new ImmutableMap.Builder() + .put("account", account) + .put("name", name) + .put("location", location) + .build(); + + cacheService.forceCacheUpdate(cloudProvider, REFRESH_TYPE, request); + }); + + // TODO(lwander): make sure cache refresh is processed + } + + return new TaskResult(ExecutionStatus.SUCCEEDED); + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/providers/appengine/AbstractWaitForAppEngineServerGroupStopStartTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/providers/appengine/AbstractWaitForAppEngineServerGroupStopStartTask.groovy index f46479f0b5..c3966e4789 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/providers/appengine/AbstractWaitForAppEngineServerGroupStopStartTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/providers/appengine/AbstractWaitForAppEngineServerGroupStopStartTask.groovy @@ -48,8 +48,10 @@ abstract class AbstractWaitForAppEngineServerGroupStopStartTask extends Abstract String account = getCredentials(stage) String serverGroupName = (stage.context.serverGroupName ?: stage.context.asgName) as String Names names = Names.parseName(serverGroupName) + String appName = stage.context.moniker?.app ?: names.app + String clusterName = stage.context.moniker?.cluster ?: names.cluster try { - def response = oortService.getCluster(names.app, account, names.cluster, cloudProvider) + def response = oortService.getCluster(appName, account, clusterName, cloudProvider) Map cluster = objectMapper.readValue(response.body.in().text, Map) def serverGroup = cluster.serverGroups.find { it.name == serverGroupName } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/providers/appengine/AppEngineServerGroupCreator.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/providers/appengine/AppEngineServerGroupCreator.groovy index 12725fb05a..54d27a5715 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/providers/appengine/AppEngineServerGroupCreator.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/providers/appengine/AppEngineServerGroupCreator.groovy @@ -16,9 +16,16 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.providers.appengine +import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spinnaker.kork.artifacts.model.Artifact +import com.netflix.spinnaker.kork.artifacts.model.ExpectedArtifact import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.ServerGroupCreator +import com.netflix.spinnaker.orca.pipeline.model.Execution +import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.model.Stage +import com.netflix.spinnaker.orca.pipeline.util.ArtifactResolver import groovy.util.logging.Slf4j +import org.springframework.beans.factory.annotation.Autowired import org.springframework.stereotype.Component @Slf4j @@ -27,6 +34,9 @@ class AppEngineServerGroupCreator implements ServerGroupCreator { boolean katoResultExpected = false String cloudProvider = 'appengine' + @Autowired + ObjectMapper objectMapper + @Override List getOperations(Stage stage) { def operation = [:] @@ -38,6 +48,7 @@ class AppEngineServerGroupCreator implements ServerGroupCreator { operation.putAll(stage.context) } + resolveArtifacts(stage, operation) operation.branch = AppEngineBranchFinder.findInStage(operation, stage) ?: operation.branch return [[(OPERATION): operation]] @@ -47,4 +58,49 @@ class AppEngineServerGroupCreator implements ServerGroupCreator { Optional getHealthProviderName() { return Optional.empty() } + + private void resolveArtifacts(Stage stage, Map operation) { + if (operation.repositoryUrl) { + return + } + + Map expectedArtifact = operation.expectedArtifact + // NOTE: expectedArtifact is a Map, and fragile to field changes in the underlying data structures. + // If a field changes in the ExpectedArtifact model, change it here. + if (operation.fromArtifact && expectedArtifact && expectedArtifact.matchArtifact) { + Execution execution = stage.getExecution() + Map trigger = [:] + if (execution instanceof Pipeline) { + trigger = ((Pipeline) execution).getTrigger() + } + + List artifacts = (List) trigger.artifacts + def foundArtifact = artifacts.find { a -> + ExpectedArtifact e = objectMapper.convertValue(expectedArtifact, ExpectedArtifact) + e.matches(objectMapper.convertValue(a, Artifact)) + } + Artifact artifact = objectMapper.convertValue(foundArtifact, Artifact) + if (artifact?.reference) { + String repositoryUrl = '' + switch (artifact.type) { + // TODO(jacobkiefer): These object types are pretty fragile, we need to harden this somehow. + case 'gcs/object': + if (!artifact.reference.startsWith('gs://')) { + repositoryUrl = "gs://${artifact.reference}" + } else { + repositoryUrl = artifact.reference + } + operation.repositoryUrl = repositoryUrl + break + default: + throw new ArtifactResolver.ArtifactResolutionException('Unknown artifact type') + break + } + } else { + throw new ArtifactResolver.ArtifactResolutionException('Missing artifact reference for artifact: ${artifact}') + } + } else { + throw new ArtifactResolver.ArtifactResolutionException('AppEngine Deploy description missing repositoryUrl but misconfigured for resolving Artifacts') + } + } } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AbstractBulkServerGroupTask.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AbstractBulkServerGroupTask.java index be581f06e1..94e8b3a7aa 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AbstractBulkServerGroupTask.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AbstractBulkServerGroupTask.java @@ -19,6 +19,7 @@ import java.util.*; import java.util.stream.Collectors; import com.netflix.frigga.Names; +import com.netflix.spinnaker.moniker.Moniker; import com.netflix.spinnaker.orca.ExecutionStatus; import com.netflix.spinnaker.orca.RetryableTask; import com.netflix.spinnaker.orca.TaskResult; @@ -27,6 +28,7 @@ import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.Location; import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroup; import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask; +import com.netflix.spinnaker.orca.clouddriver.utils.MonikerHelper; import com.netflix.spinnaker.orca.clouddriver.utils.OortHelper; import com.netflix.spinnaker.orca.pipeline.model.Stage; import org.slf4j.Logger; @@ -39,6 +41,9 @@ public abstract class AbstractBulkServerGroupTask extends AbstractCloudProviderA @Autowired protected OortHelper oortHelper; + @Autowired + protected MonikerHelper monikerHelper; + @Autowired protected KatoService katoService; @@ -62,21 +67,19 @@ public TaskResult execute(Stage stage) { if (request.getServerGroupNames() == null || request.getServerGroupNames().isEmpty()) { throw new IllegalArgumentException("Server group names must be provided"); } - - Names names = Names.parseName(request.getServerGroupNames().get(0)); - + String clusterName = monikerHelper.getClusterNameFromStage(stage, request.getServerGroupNames().get(0)); Map cluster = oortHelper.getCluster( - names.getApp(), + monikerHelper.getAppNameFromStage(stage, request.getServerGroupNames().get(0)), request.getCredentials(), - names.getCluster(), + clusterName, request.getCloudProvider() ).orElseThrow( - () -> new IllegalArgumentException(String.format("No Cluster details found for %s", names.getCluster())) + () -> new IllegalArgumentException(String.format("No Cluster details found for %s", clusterName)) ); List serverGroups = Optional.ofNullable((List) cluster.get("serverGroups")) .orElseThrow( - () -> new IllegalArgumentException(String.format("No server groups found for cluster %s", names.getCluster())) + () -> new IllegalArgumentException(String.format("No server groups found for cluster %s", clusterName)) ); Location location = Optional.ofNullable(Location.region(request.getRegion())) diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AddServerGroupEntityTagsTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AddServerGroupEntityTagsTask.groovy new file mode 100644 index 0000000000..e9df2d8dc0 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AddServerGroupEntityTagsTask.groovy @@ -0,0 +1,99 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.tasks.servergroup + +import java.util.concurrent.TimeUnit +import com.fasterxml.jackson.annotation.JsonProperty +import com.netflix.spinnaker.orca.ExecutionStatus +import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.TaskResult +import com.netflix.spinnaker.orca.clouddriver.KatoService +import com.netflix.spinnaker.orca.clouddriver.model.TaskId +import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask +import com.netflix.spinnaker.orca.pipeline.model.Stage +import groovy.util.logging.Slf4j +import org.springframework.beans.factory.annotation.Autowired +import org.springframework.stereotype.Component + +@Component +@Slf4j +class AddServerGroupEntityTagsTask extends AbstractCloudProviderAwareTask implements RetryableTask { + long backoffPeriod = TimeUnit.SECONDS.toMillis(5) + long timeout = TimeUnit.MINUTES.toMillis(5) + + @Autowired + KatoService kato + + @Autowired + Collection tagGenerators + + @Override + TaskResult execute(Stage stage) { + try { + List tagOperations = buildTagOperations(stage) + if (!tagOperations) { + return new TaskResult(ExecutionStatus.SKIPPED) + } + TaskId taskId = kato.requestOperations(tagOperations).toBlocking().first() + return new TaskResult(ExecutionStatus.SUCCEEDED, new HashMap() { + { + put("notification.type", "upsertentitytags") + put("kato.last.task.id", taskId) + } + }) + } catch (Exception e) { + log.error("Failed to tag deployed server groups (stageId: ${stage.id}, executionId: ${stage.execution.id})", e) + return new TaskResult(ExecutionStatus.FAILED_CONTINUE) + } + } + + private List buildTagOperations(Stage stage) { + def operations = [] + ((StageData) stage.mapTo(StageData)).deployServerGroups.each { String region, Set serverGroups -> + serverGroups.each { String serverGroup -> + Collection> tags = tagGenerators ? + tagGenerators.findResults { + it.generateTags(stage, serverGroup, getCredentials(stage), region, getCloudProvider(stage)) + }.flatten() : + [] + if (!tags) { + return [] + } + operations << + [ + "upsertEntityTags": [ + tags : tags, + entityRef: [ + entityType : "servergroup", + entityId : serverGroup, + account : getCredentials(stage), + region : region, + cloudProvider: getCloudProvider(stage) + ] + ] + ] + } + } + + return operations + } + + static class StageData { + @JsonProperty("deploy.server.groups") + Map> deployServerGroups = [:] + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/BulkWaitForDestroyedServerGroupTask.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/BulkWaitForDestroyedServerGroupTask.java index 2e90efd55e..0feb33ba07 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/BulkWaitForDestroyedServerGroupTask.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/BulkWaitForDestroyedServerGroupTask.java @@ -21,11 +21,13 @@ import java.util.stream.Collectors; import com.fasterxml.jackson.databind.ObjectMapper; import com.netflix.frigga.Names; +import com.netflix.spinnaker.moniker.Moniker; import com.netflix.spinnaker.orca.ExecutionStatus; import com.netflix.spinnaker.orca.RetryableTask; import com.netflix.spinnaker.orca.TaskResult; import com.netflix.spinnaker.orca.clouddriver.OortService; import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask; +import com.netflix.spinnaker.orca.clouddriver.utils.MonikerHelper; import com.netflix.spinnaker.orca.pipeline.model.Stage; import com.netflix.spinnaker.orca.retrofit.exceptions.RetrofitExceptionHandler; import org.slf4j.Logger; @@ -45,17 +47,19 @@ public class BulkWaitForDestroyedServerGroupTask extends AbstractCloudProviderAw @Autowired private ObjectMapper objectMapper; + @Autowired + private MonikerHelper monikerHelper; + @Override public TaskResult execute(Stage stage) { String region = (String) stage.getContext().get("region"); Map> regionToServerGroups = (Map>) stage.getContext().get("deploy.server.groups"); List serverGroupNames = regionToServerGroups.get(region); - Names names = Names.parseName(serverGroupNames.get(0)); try { Response response = oortService.getCluster( - names.getApp(), + monikerHelper.getAppNameFromStage(stage, serverGroupNames.get(0)), getCredentials(stage), - names.getCluster(), + monikerHelper.getClusterNameFromStage(stage, serverGroupNames.get(0)), getCloudProvider(stage) ); diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ContextBasedServerGroupEntityTagGenerator.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ContextBasedServerGroupEntityTagGenerator.java new file mode 100644 index 0000000000..32b7fb6604 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ContextBasedServerGroupEntityTagGenerator.java @@ -0,0 +1,37 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.tasks.servergroup; + +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.springframework.stereotype.Component; + +import java.util.List; +import java.util.Map; + +@Component +public class ContextBasedServerGroupEntityTagGenerator implements ServerGroupEntityTagGenerator { + + @Override + public List> generateTags(Stage stage, String serverGroup, String account, String location, String cloudProvider) { + Map context = stage.getContext(); + + if (context.containsKey("entityTags")) { + return (List>) context.getOrDefault("entityTags", null); + } + return null; + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupEntityTagGenerator.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupEntityTagGenerator.java index 7dbb22511a..3914152c72 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupEntityTagGenerator.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupEntityTagGenerator.java @@ -18,14 +18,15 @@ import com.netflix.spinnaker.orca.pipeline.model.Stage; +import java.util.Collection; import java.util.Map; public interface ServerGroupEntityTagGenerator { /** - * Generates an entity tag (e.g. server group provenance metadata) to be applied to a server group after deployment + * Generates a collection of entity tags (e.g. server group provenance metadata) to be applied to a server group after deployment * @param stage the stage that performed the deployment - * @return a map representing a tag to send to Clouddriver + * @return a collection of maps representing tags to send to Clouddriver */ - Map generateTag(Stage stage); + Collection> generateTags(Stage stage, String serverGroup, String account, String location, String cloudProvider); } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupMetadataTagTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupMetadataTagTask.groovy index 39cfd817d2..107aa9dd13 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupMetadataTagTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupMetadataTagTask.groovy @@ -1,11 +1,11 @@ /* * Copyright 2017 Netflix, Inc. * - * Licensed under the Apache License, Version 2.0 (the "License"); + * Licensed under the Apache License, Version 2.0 (the "License") * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -16,80 +16,13 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.servergroup -import java.util.concurrent.TimeUnit -import com.fasterxml.jackson.annotation.JsonProperty -import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask -import com.netflix.spinnaker.orca.TaskResult -import com.netflix.spinnaker.orca.clouddriver.KatoService -import com.netflix.spinnaker.orca.clouddriver.model.TaskId -import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask -import com.netflix.spinnaker.orca.pipeline.model.Stage import groovy.util.logging.Slf4j -import org.springframework.beans.factory.annotation.Autowired import org.springframework.stereotype.Component @Component @Slf4j -class ServerGroupMetadataTagTask extends AbstractCloudProviderAwareTask implements RetryableTask { - long backoffPeriod = TimeUnit.SECONDS.toMillis(5) - long timeout = TimeUnit.MINUTES.toMillis(5) - - @Autowired - KatoService kato - - @Autowired - Collection tagGenerators - - @Override - TaskResult execute(Stage stage) { - try { - List tagOperations = buildTagOperations(stage) - if (!tagOperations) { - return new TaskResult(ExecutionStatus.SKIPPED) - } - TaskId taskId = kato.requestOperations(tagOperations).toBlocking().first() - return new TaskResult(ExecutionStatus.SUCCEEDED, new HashMap() { - { - put("notification.type", "upsertentitytags") - put("kato.last.task.id", taskId) - } - }) - } catch (Exception e) { - log.error("Failed to tag deployed server groups (stageId: ${stage.id}, executionId: ${stage.execution.id})", e) - return new TaskResult(ExecutionStatus.FAILED_CONTINUE) - } - } - - private List buildTagOperations(Stage stage) { - def operations = [] - ((StageData) stage.mapTo(StageData)).deployServerGroups.each { String region, Set serverGroups -> - serverGroups.each { String serverGroup -> - Collection> tags = tagGenerators ? tagGenerators.findResults { it.generateTag(stage) } : [] - if (!tags) { - return [] - } - operations << - [ - "upsertEntityTags": [ - tags : tags, - entityRef: [ - entityType : "servergroup", - entityId : serverGroup, - account : getCredentials(stage), - region : region, - cloudProvider: getCloudProvider(stage) - ] - ] - ] - } - } - - return operations - } - - static class StageData { - @JsonProperty("deploy.server.groups") - Map> deployServerGroups = [:] - } +@Deprecated +class ServerGroupMetadataTagTask extends AddServerGroupEntityTagsTask { + // TODO: Remove after 11/11/17 - only here to smooth over initial round of deployments + // Just use AddServerGroupEntityTagsTask } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/SpinnakerMetadataServerGroupTagGenerator.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/SpinnakerMetadataServerGroupTagGenerator.java index 3b3fb3a550..4fa9d030e2 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/SpinnakerMetadataServerGroupTagGenerator.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/SpinnakerMetadataServerGroupTagGenerator.java @@ -16,24 +16,40 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.servergroup; +import com.netflix.frigga.Names; +import com.netflix.spinnaker.orca.RetrySupport; +import com.netflix.spinnaker.orca.clouddriver.OortService; import com.netflix.spinnaker.orca.pipeline.model.Execution; import com.netflix.spinnaker.orca.pipeline.model.Orchestration; import com.netflix.spinnaker.orca.pipeline.model.Pipeline; import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.springframework.stereotype.Component; +import retrofit.RetrofitError; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @Component public class SpinnakerMetadataServerGroupTagGenerator implements ServerGroupEntityTagGenerator { + private final Logger log = LoggerFactory.getLogger(this.getClass()); + private final OortService oortService; + private final RetrySupport retrySupport; + + public SpinnakerMetadataServerGroupTagGenerator(OortService oortService, RetrySupport retrySupport) { + this.oortService = oortService; + this.retrySupport = retrySupport; + } @Override - public Map generateTag(Stage stage) { + public Collection> generateTags(Stage stage, String serverGroup, String account, String location, String cloudProvider) { Execution execution = stage.getExecution(); Map context = stage.getContext(); - Map value = new HashMap<>(); + Map value = new HashMap<>(); value.put("stageId", stage.getId()); value.put("executionId", execution.getId()); value.put("executionType", execution.getClass().getSimpleName().toLowerCase()); @@ -57,11 +73,75 @@ public Map generateTag(Stage stage) { value.put("comments", (String) context.get("comments")); } + String cluster = null; + try { + cluster = Names.parseName(serverGroup).getCluster(); + + Map previousServerGroup = getPreviousServerGroup( + execution.getApplication(), + account, + cluster, + cloudProvider, + location + ); + + if (previousServerGroup != null) { + value.put("previousServerGroup", previousServerGroup); + } + } catch (Exception e) { + // failure to populate `previousServerGroup` is not considered a fatal error that would cause this task to fail + log.error("Unable to determine ancestor image details for {}:{}:{}:{}", cloudProvider, account, location, cluster, e); + } + Map tag = new HashMap<>(); tag.put("name", "spinnaker:metadata"); tag.put("value", value); - return tag; + return Collections.singletonList(tag); } + Map getPreviousServerGroup(String application, + String account, + String cluster, + String cloudProvider, + String location) { + if (cloudProvider.equals("titus")) { + // TODO-AJ titus does not force cache refresh so `ANCESTOR` will return inconsistent results + return null; + } + + return retrySupport.retry(() -> { + try { + Map targetServerGroup = oortService.getServerGroupSummary( + application, + account, + cluster, + cloudProvider, + location, + "ANCESTOR", + "image", + "true" + ); + + Map previousServerGroup = new HashMap<>(); + previousServerGroup.put("name", targetServerGroup.get("serverGroupName")); + previousServerGroup.put("imageName", targetServerGroup.get("imageName")); + previousServerGroup.put("imageId", targetServerGroup.get("imageId")); + previousServerGroup.put("cloudProvider", cloudProvider); + + if (targetServerGroup.containsKey("buildInfo")) { + previousServerGroup.put("buildInfo", targetServerGroup.get("buildInfo")); + } + + return previousServerGroup; + } catch (RetrofitError e) { + if (e.getKind() == RetrofitError.Kind.HTTP && e.getResponse().getStatus() == 404) { + // it's ok if the previous server group does not exist + return null; + } + + throw e; + } + }, 12, 5000, false); // retry for up to one minute + } } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/WaitForDestroyedServerGroupTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/WaitForDestroyedServerGroupTask.groovy index b9253ae985..a2fa1a3c82 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/WaitForDestroyedServerGroupTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/WaitForDestroyedServerGroupTask.groovy @@ -49,8 +49,10 @@ class WaitForDestroyedServerGroupTask extends AbstractCloudProviderAwareTask imp String serverGroupRegion = (stage.context.regions as Collection)?.getAt(0) ?: stage.context.region String serverGroupName = (stage.context.serverGroupName ?: stage.context.asgName) as String // TODO: Retire asgName Names names = Names.parseName(serverGroupName) + String appName = stage.context.moniker?.app ?: names.app + String clusterName = stage.context.moniker?.cluster ?: names.cluster try { - def response = oortService.getCluster(names.app, account, names.cluster, cloudProvider) + def response = oortService.getCluster(appName, account, clusterName, cloudProvider) if (response.status != 200) { return new TaskResult(ExecutionStatus.RUNNING) diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatchRule.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatchRule.java new file mode 100644 index 0000000000..8288629500 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatchRule.java @@ -0,0 +1,77 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.utils; + +public class ClusterMatchRule { + private String account; + private String location; + private String stack; + private String detail; + private Integer priority; + + public ClusterMatchRule() { + } + + public ClusterMatchRule(String account, String location, String stack, String detail, Integer priority) { + this.account = account; + this.location = location; + this.stack = stack; + this.detail = detail; + this.priority = priority; + } + + public String getAccount() { + return account == null ? "" : account; + } + + public void setAccount(String account) { + this.account = account; + } + + public String getLocation() { + return location == null ? "" : location; + } + + public void setLocation(String location) { + this.location = location; + } + + public String getStack() { + return stack == null ? "" : stack; + } + + public void setStack(String stack) { + this.stack = stack; + } + + public String getDetail() { + return detail == null ? "" : detail; + } + + public void setDetail(String detail) { + this.detail = detail; + } + + public Integer getPriority() { + return priority == null ? 0 : priority; + } + + public void setPriority(Integer priority) { + this.priority = priority; + } + +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatcher.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatcher.java new file mode 100644 index 0000000000..fc551e0b37 --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatcher.java @@ -0,0 +1,70 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.utils; + +import com.netflix.frigga.Names; + +import java.util.Comparator; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +public class ClusterMatcher { + + public static ClusterMatchRule getMatchingRule(String account, String location, String clusterName, List rules) { + if (!Optional.ofNullable(rules).isPresent()) { + return null; + } + Names nameParts = Names.parseName(clusterName); + + String stack = nameParts.getStack() == null ? "" : nameParts.getStack(); + String detail = nameParts.getDetail() == null ? "" : nameParts.getDetail(); + + List candidates = rules.stream().filter(rule -> { + String ruleAccount = rule.getAccount(); + String ruleLocation = rule.getLocation(); + String ruleStack = rule.getStack(); + String ruleDetail = rule.getDetail(); + return (ruleAccount.equals("*") || ruleAccount.equals(account)) && + (ruleLocation.equals("*") || ruleLocation.equals(location)) && + (ruleStack.equals("*") || ruleStack.equals(stack) || ruleStack.isEmpty() && stack.isEmpty()) && + (ruleDetail.equals("*") || ruleDetail.equals(detail) || ruleDetail.isEmpty() && detail.isEmpty()); + }).sorted((o1, o2) -> { + if (!o1.getAccount().equals(o2.getAccount())) { + return "*".equals(o1.getAccount()) ? 1 : -1; + } + if (!o1.getLocation().equals(o2.getLocation())) { + return "*".equals(o1.getLocation()) ? 1 : -1; + } + if (!o1.getStack().equals(o2.getStack())) { + return "*".equals(o1.getStack()) ? 1 : -1; + } + if (!o1.getDetail().equals(o2.getDetail())) { + return "*".equals(o1.getDetail()) ? 1 : -1; + } + return o1.getPriority() - o2.getPriority(); + }) + .collect(Collectors.toList()); + + if (candidates.isEmpty()) { + return null; + } + + return candidates.get(0); + } + +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/MonikerHelper.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/MonikerHelper.java new file mode 100644 index 0000000000..b1ba4497fb --- /dev/null +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/MonikerHelper.java @@ -0,0 +1,61 @@ +/* + * Copyright 2017 Armory, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.clouddriver.utils; + + +import com.netflix.frigga.Names; +import com.netflix.spinnaker.moniker.Moniker; +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.springframework.stereotype.Component; + + +/** + * Helper methods for getting the app, cluster, etc from a moniker. When a moniker is not available use frigga. + */ +@Component +public class MonikerHelper { + public String getAppNameFromStage(Stage stage, String fallbackFriggaName) { + Names names = Names.parseName(fallbackFriggaName); + Moniker moniker = monikerFromStage(stage); + String appName; + if (moniker != null && moniker.getApp() != null) { + appName = moniker.getApp(); + } else { + appName = names.getApp(); + } + return appName; + } + + public String getClusterNameFromStage(Stage stage, String fallbackFriggaName) { + Names names = Names.parseName(fallbackFriggaName); + Moniker moniker = monikerFromStage(stage); + String clusterName; + if (moniker != null && moniker.getCluster() != null) { + clusterName = moniker.getCluster(); + } else { + clusterName = names.getCluster(); + } + return clusterName; + } + + static public Moniker monikerFromStage(Stage stage) { + if (stage.getContext().containsKey("moniker")) { + return (Moniker) stage.mapTo("/moniker", Moniker.class); + } else { + return null; + } + } +} diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/OortHelper.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/OortHelper.groovy index 0f68158022..3889da18bb 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/OortHelper.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/OortHelper.groovy @@ -17,7 +17,6 @@ package com.netflix.spinnaker.orca.clouddriver.utils import com.fasterxml.jackson.databind.ObjectMapper -import com.netflix.frigga.Names import com.netflix.spinnaker.orca.clouddriver.OortService import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroup import org.springframework.beans.factory.annotation.Autowired @@ -59,15 +58,7 @@ class OortHelper { String serverGroupName, String location, String cloudProvider) { - def name = Names.parseName(serverGroupName) - return convertedResponse(List) { oortService.getServerGroupFromCluster(name.app, account, name.cluster, serverGroupName, null, cloudProvider) } - .map({ List serverGroups -> - serverGroups.find { - it.region == location || it.zones?.contains(location) || it.namespace == location - } - }).map({ Map serverGroup -> - new TargetServerGroup(serverGroup) - }) + return new TargetServerGroup(convert(oortService.getServerGroup(account, location, serverGroupName) , Map)) } public Optional convertedResponse(Class type, Closure request) { diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/TrafficGuard.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/TrafficGuard.java index 68a54eef1a..c617887038 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/TrafficGuard.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/clouddriver/utils/TrafficGuard.java @@ -46,13 +46,24 @@ public TrafficGuard(OortHelper oortHelper, Optional front50Servi this.front50Service = front50Service.orElse(null); } - public void verifyInstanceTermination(List instanceIds, String account, Location location, String cloudProvider, String operationDescriptor) { + public void verifyInstanceTermination(String serverGroupNameFromStage, + List instanceIds, + String account, + Location location, + String cloudProvider, + String operationDescriptor) { Map> instancesPerServerGroup = new HashMap<>(); - instanceIds.forEach(instanceId -> { + for (String instanceId : instanceIds) { + String serverGroupName = serverGroupNameFromStage; + if (serverGroupName == null) { + Optional resolvedServerGroupName = resolveServerGroupNameForInstance(instanceId, account, location.getValue(), cloudProvider); + serverGroupName = resolvedServerGroupName.orElse(null); + } - Optional resolvedServerGroupName = resolveServerGroupNameForInstance(instanceId, account, location.getValue(), cloudProvider); - resolvedServerGroupName.ifPresent(name -> instancesPerServerGroup.computeIfAbsent(name, serverGroup -> new ArrayList<>()).add(instanceId)); - }); + if (serverGroupName != null) { + instancesPerServerGroup.computeIfAbsent(serverGroupName, serverGroup -> new ArrayList<>()).add(instanceId); + } + } instancesPerServerGroup.entrySet().forEach(entry -> { String serverGroupName = entry.getKey(); @@ -133,11 +144,9 @@ public boolean hasDisableLock(String cluster, String account, Location location) return false; } List> trafficGuards = (List>) application.details().get("trafficGuards"); - return trafficGuards.stream().anyMatch(guard -> - ("*".equals(guard.get("account")) || account.equals(guard.get("account"))) && - ("*".equals(guard.get("location")) || location.getValue().equals(guard.get("location"))) && - ("*".equals(guard.get("stack")) || StringUtils.equals(names.getStack(), guard.get("stack"))) && - ("*".equals(guard.get("detail")) || StringUtils.equals(names.getDetail(), guard.get("detail"))) - ); + List rules = trafficGuards.stream().map(guard -> + new ClusterMatchRule(guard.get("account"), guard.get("location"), guard.get("stack"), guard.get("detail"), 1) + ).collect(Collectors.toList()); + return ClusterMatcher.getMatchingRule(account, location.getValue(), cluster, rules) != null; } } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/ParallelDeployStage.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/ParallelDeployStage.groovy index 8d4c24fa5a..9494274979 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/ParallelDeployStage.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/ParallelDeployStage.groovy @@ -16,12 +16,13 @@ package com.netflix.spinnaker.orca.kato.pipeline +import javax.annotation.Nonnull import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.Task import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.CloneServerGroupStage import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.CreateServerGroupStage -import com.netflix.spinnaker.orca.pipeline.BranchingStageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.TaskNode import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Pipeline @@ -30,11 +31,12 @@ import groovy.transform.CompileDynamic import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import org.springframework.stereotype.Component +import static com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFORE @Component @Slf4j @CompileStatic -class ParallelDeployStage implements BranchingStageDefinitionBuilder { +class ParallelDeployStage implements StageDefinitionBuilder { @Deprecated public static final String PIPELINE_CONFIG_TYPE = "deploy" @@ -46,16 +48,15 @@ class ParallelDeployStage implements BranchingStageDefinitionBuilder { @Override > void taskGraph(Stage stage, TaskNode.Builder builder) { - } - - @Override - void postBranchGraph(Stage stage, TaskNode.Builder builder) { builder.withTask("completeParallelDeploy", CompleteParallelDeployTask) } - @Override - String getChildStageType(Stage childStage) { - return isClone(childStage) ? CloneServerGroupStage.PIPELINE_CONFIG_TYPE : PIPELINE_CONFIG_TYPE + @Nonnull > List> parallelStages( + @Nonnull Stage stage) { + parallelContexts(stage).collect { context -> + def type = isClone(stage) ? CloneServerGroupStage.PIPELINE_CONFIG_TYPE : CreateServerGroupStage.PIPELINE_CONFIG_TYPE + newStage(stage.execution, type, context.name as String, context, stage, STAGE_BEFORE) + } } @CompileDynamic @@ -77,9 +78,8 @@ class ParallelDeployStage implements BranchingStageDefinitionBuilder { ] } - @Override @CompileDynamic - > Collection> parallelContexts(Stage stage) { + protected > Collection> parallelContexts(Stage stage) { if (stage.execution instanceof Pipeline) { Map trigger = ((Pipeline) stage.execution).trigger if (trigger.parameters?.strategy == true) { @@ -145,11 +145,6 @@ class ParallelDeployStage implements BranchingStageDefinitionBuilder { } } - @Override - String parallelStageName(Stage stage, boolean hasParallelFlows) { - return isClone(stage) ? "Clone" : stage.name - } - @CompileDynamic private > boolean isClone(Stage stage) { if (stage.execution instanceof Pipeline) { @@ -166,7 +161,7 @@ class ParallelDeployStage implements BranchingStageDefinitionBuilder { @Component @Slf4j @CompileStatic - public static class CompleteParallelDeployTask implements Task { + static class CompleteParallelDeployTask implements Task { TaskResult execute(Stage stage) { log.info("Completed Parallel Deploy") new TaskResult(ExecutionStatus.SUCCEEDED, [:], [:]) diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/RollingPushStage.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/RollingPushStage.java index 5e2d88ba08..99cb290bef 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/RollingPushStage.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/RollingPushStage.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.kato.pipeline; import java.util.Map; diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToClusterResizeStrategy.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToClusterResizeStrategy.groovy index 23d9292b93..a9b825b2f5 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToClusterResizeStrategy.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToClusterResizeStrategy.groovy @@ -41,8 +41,8 @@ class ScaleToClusterResizeStrategy implements ResizeStrategy{ @Override Capacity capacityForOperation(Stage stage, String account, String serverGroupName, String cloudProvider, Location location, OptionalConfiguration resizeConfig) { def names = Names.parseName(serverGroupName) - def appName = names.app - def clusterName = names.cluster + def appName = stage?.context?.moniker?.app ?: names.app + def clusterName = stage?.context?.moniker?.cluster ?:names.cluster def cluster = oortHelper.getCluster(appName, account, clusterName, cloudProvider) List targetServerGroups = cluster diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToServerGroupResizeStrategy.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToServerGroupResizeStrategy.groovy index 903ee7a7d4..d7c5149580 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToServerGroupResizeStrategy.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToServerGroupResizeStrategy.groovy @@ -60,6 +60,25 @@ class ScaleToServerGroupResizeStrategy implements ResizeStrategy { def currentDesired = Integer.parseInt(tsg.capacity.desired.toString()) def currentMax = Integer.parseInt(tsg.capacity.max.toString()) + def scalePct = resizeConfig.scalePct + if (scalePct != null) { + double factor = scalePct / 100.0d + + // scalePct only applies to the desired capacity + currentDesired = (Integer) Math.ceil(currentDesired * factor) + + // min capacity may need adjusting iff scalePct pushed desired below current min + currentMin = Math.min(currentMin, currentDesired) + } + + if (stageData.pinCapacity) { + return new ResizeStrategy.Capacity( + currentDesired, + currentDesired, + currentDesired + ) + } + return new ResizeStrategy.Capacity( currentMax, currentDesired, @@ -72,5 +91,7 @@ class ScaleToServerGroupResizeStrategy implements ResizeStrategy { // whether or not `min` capacity should be set to `desired` capacity boolean pinMinimumCapacity + + boolean pinCapacity } } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/StageData.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/StageData.groovy index ef9e458d25..f8754924f4 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/StageData.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/StageData.groovy @@ -17,6 +17,7 @@ package com.netflix.spinnaker.orca.kato.pipeline.support import com.netflix.frigga.autoscaling.AutoScalingGroupNameBuilder +import com.netflix.spinnaker.moniker.Moniker class StageData { String strategy @@ -27,6 +28,7 @@ class StageData { String freeFormDetails String application String stack + Moniker moniker @Deprecated String providerType = "aws" String cloudProvider = "aws" boolean scaleDown @@ -35,14 +37,23 @@ class StageData { Boolean useSourceCapacity Boolean preferSourceCapacity Source source + + @Deprecated long delayBeforeDisableSec + long delayBeforeCleanup + PipelineBeforeCleanup pipelineBeforeCleanup + String getCluster() { - def builder = new AutoScalingGroupNameBuilder() - builder.appName = application - builder.stack = stack - builder.detail = freeFormDetails - return builder.buildGroupName() + if (moniker?.cluster) { + return moniker.cluster + } else { + def builder = new AutoScalingGroupNameBuilder() + builder.appName = application + builder.stack = stack + builder.detail = freeFormDetails + return builder.buildGroupName() + } } String getAccount() { @@ -52,6 +63,10 @@ class StageData { return account ?: credentials } + long getDelayBeforeCleanup() { + return this.delayBeforeCleanup ?: this.delayBeforeDisableSec + } + @Deprecated List getRegions() { availabilityZones?.keySet()?.toList() ?: (region ? [region] : []) @@ -86,4 +101,8 @@ class StageData { Boolean preferSourceCapacity } + static class PipelineBeforeCleanup { + String application + String pipelineId + } } diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/DisableInstancesTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/DisableInstancesTask.groovy index a1803fccf5..cff3aafdd7 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/DisableInstancesTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/DisableInstancesTask.groovy @@ -42,7 +42,10 @@ class DisableInstancesTask implements CloudProviderAware, Task { String cloudProvider = getCloudProvider(stage) String account = getCredentials(stage) + def serverGroupName = stage.context.serverGroupName ?: stage.context.asgName + trafficGuard.verifyInstanceTermination( + serverGroupName, stage.context.instanceIds as List, account, Location.region(stage.context.region as String), diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/CleanUpTagsTask.java b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/CleanUpTagsTask.java index 4ae6bda9b2..07c5520fb8 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/CleanUpTagsTask.java +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/CleanUpTagsTask.java @@ -21,7 +21,6 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import com.fasterxml.jackson.databind.ObjectMapper; -import com.netflix.frigga.Names; import com.netflix.spinnaker.orca.ExecutionStatus; import com.netflix.spinnaker.orca.RetryableTask; import com.netflix.spinnaker.orca.TaskResult; @@ -29,6 +28,7 @@ import com.netflix.spinnaker.orca.clouddriver.OortService; import com.netflix.spinnaker.orca.clouddriver.model.TaskId; import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask; +import com.netflix.spinnaker.orca.clouddriver.utils.MonikerHelper; import com.netflix.spinnaker.orca.kato.pipeline.support.SourceResolver; import com.netflix.spinnaker.orca.kato.pipeline.support.StageData; import com.netflix.spinnaker.orca.pipeline.model.Stage; @@ -55,18 +55,20 @@ public class CleanUpTagsTask extends AbstractCloudProviderAwareTask implements R @Autowired ObjectMapper objectMapper; + @Autowired + MonikerHelper monikerHelper; + @Override public TaskResult execute(Stage stage) { try { StageData.Source source = sourceResolver.getSource(stage); String serverGroupName = Optional.ofNullable(source.getServerGroupName()).orElse(source.getAsgName()); - Names name = Names.parseName(serverGroupName); String cloudProvider = getCloudProvider(stage); Response serverGroupResponse = oortService.getServerGroupFromCluster( - name.getApp(), + monikerHelper.getAppNameFromStage(stage, serverGroupName), source.getAccount(), - name.getCluster(), + monikerHelper.getClusterNameFromStage(stage, serverGroupName), serverGroupName, source.getRegion(), cloudProvider diff --git a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/WaitForNewUpInstancesLaunchTask.groovy b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/WaitForNewUpInstancesLaunchTask.groovy index ff6a186fc0..19f7b86099 100644 --- a/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/WaitForNewUpInstancesLaunchTask.groovy +++ b/orca-clouddriver/src/main/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/WaitForNewUpInstancesLaunchTask.groovy @@ -42,7 +42,13 @@ class WaitForNewUpInstancesLaunchTask implements RetryableTask { @Override TaskResult execute(Stage stage) { StageData stageData = stage.mapTo(StageData) - def response = oortService.getServerGroupFromCluster(stageData.application, stageData.account, stageData.cluster, stage.context.asgName as String, stage.context.region as String, stageData.cloudProvider ?: 'aws' ) + + // similar check in `AbstractInstancesCheckTask` + def response = oortService.getServerGroup( + stageData.account, + stage.context.region as String, + stage.context.asgName as String + ) Map serverGroup = objectMapper.readValue(response.body.in(), Map) diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/config/SelectableServiceSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/config/SelectableServiceSpec.groovy index ab28e716c9..2273878455 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/config/SelectableServiceSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/config/SelectableServiceSpec.groovy @@ -16,26 +16,25 @@ package com.netflix.spinnaker.orca.clouddriver.config -import com.netflix.spinnaker.orca.clouddriver.InstanceService -import com.netflix.spinnaker.orca.clouddriver.KatoService -import com.netflix.spinnaker.orca.clouddriver.MortService -import com.netflix.spinnaker.orca.clouddriver.OortService import spock.lang.Shared import spock.lang.Specification import spock.lang.Unroll; class SelectableServiceSpec extends Specification { @Shared - def mortService = Mock(MortService) + def mortService = "mort" @Shared - def oortService = Mock(OortService) + def oortService = "oort" @Shared - def katoService = Mock(KatoService) + def katoService = "kato" @Shared - def instanceService = Mock(InstanceService) + def instanceService = "instance" + + @Shared + def bakeryService = "bakery" @Unroll def "should lookup service by application or executionType"() { @@ -45,6 +44,7 @@ class SelectableServiceSpec extends Specification { new ByApplicationServiceSelector(mortService, 10, ["applicationPattern": ".*spindemo.*"]), new ByExecutionTypeServiceSelector(oortService, 5, ["executionTypes": [0: "orchestration"]]), new ByOriginServiceSelector(instanceService, 20, ["origin": "deck", "executionTypes": [0: "orchestration"]]), + new ByAuthenticatedUserServiceSelector(bakeryService, 25, ["users": [0: "user1@email.com", 1: ".*user2.*"]]), new DefaultServiceSelector(katoService, 1, [:]) ] ) @@ -56,14 +56,17 @@ class SelectableServiceSpec extends Specification { service == expectedService where: - criteria || expectedService - new SelectableService.Criteria(null, null, null) || katoService // the default service selector - new SelectableService.Criteria("spindemo", "orchestration", "api") || mortService - new SelectableService.Criteria("1-spindemo-1", "orchestration", "api") || mortService - new SelectableService.Criteria("spindemo", "orchestration", "deck") || instanceService // origin selector is higher priority - new SelectableService.Criteria("spindemo", "pipeline", "deck") || mortService // fall back to application selector as origin selector does not support pipeline - new SelectableService.Criteria("spintest", "orchestration", "api") || oortService - new SelectableService.Criteria("spintest", "pipeline", "api") || katoService + criteria || expectedService + new SelectableService.Criteria(null, null, null, null) || katoService // the default service selector + new SelectableService.Criteria("spindemo", null, "orchestration", "api") || mortService + new SelectableService.Criteria("1-spindemo-1", null, "orchestration", "api") || mortService + new SelectableService.Criteria("spindemo", null, "orchestration", "deck") || instanceService // origin selector is higher priority + new SelectableService.Criteria("spindemo", null, "pipeline", "deck") || mortService // fall back to application selector as origin selector does not support pipeline + new SelectableService.Criteria("spintest", null, "orchestration", "api") || oortService + new SelectableService.Criteria("spintest", null, "pipeline", "api") || katoService + new SelectableService.Criteria("spintest", "user1@unsupported.com", "orchestration", "api") || oortService + new SelectableService.Criteria("spintest", "user1@email.com", "orchestration", "api") || bakeryService // user selector is highest priority + new SelectableService.Criteria("spintest", "user2@random.com", "orchestration", "api") || bakeryService // user selector supports regex patterns } def "should default to all execution types if none configured (by origin selector)"() { diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/RollbackServerGroupStageSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/RollbackServerGroupStageSpec.groovy new file mode 100644 index 0000000000..0633743cef --- /dev/null +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/RollbackServerGroupStageSpec.groovy @@ -0,0 +1,68 @@ +/* + * Copyright 2015 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup + +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback.TestRollback +import com.netflix.spinnaker.orca.pipeline.WaitStage +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner +import org.springframework.beans.factory.config.AutowireCapableBeanFactory +import spock.lang.Shared +import spock.lang.Specification + +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.stage + +class RollbackServerGroupStageSpec extends Specification { + @Shared + def waitStage = new WaitStage() + + def "should build stages appropriate for strategy"() { + given: + def autowireCapableBeanFactory = Stub(AutowireCapableBeanFactory) { + autowireBean(_) >> { TestRollback rollback -> + rollback.waitStage = waitStage + } + } + + def rollbackServerGroupStage = new RollbackServerGroupStage() + rollbackServerGroupStage.autowireCapableBeanFactory = autowireCapableBeanFactory + + def stage = stage { + type = "rollbackServerGroup" + context = [ + rollbackType : "TEST", + rollbackContext : [ + waitTime: 100, + ] + ] + } + + when: + def tasks = rollbackServerGroupStage.buildTaskGraph(stage) + def allStages = rollbackServerGroupStage.aroundStages(stage) + def beforeStages = allStages.findAll { it.syntheticStageOwner == SyntheticStageOwner.STAGE_BEFORE } + def afterStages = allStages.findAll { it.syntheticStageOwner == SyntheticStageOwner.STAGE_AFTER } + + then: + tasks.iterator().size() == 0 + beforeStages.isEmpty() + afterStages*.type == [ + "wait", + ] + afterStages[0].context == [waitTime: 100] + } +} diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/RollbackServerGroupStageSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/ExplicitRollbackSpec.groovy similarity index 68% rename from orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/RollbackServerGroupStageSpec.groovy rename to orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/ExplicitRollbackSpec.groovy index 82e106e3f2..ea2a4ae449 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/RollbackServerGroupStageSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/ExplicitRollbackSpec.groovy @@ -1,7 +1,7 @@ /* - * Copyright 2015 Netflix, Inc. + * Copyright 2017 Netflix, Inc. * - * Licensed under the Apache License, Version 2.0 (the "License") + * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * @@ -14,23 +14,22 @@ * limitations under the License. */ - -package com.netflix.spinnaker.orca.clouddriver.pipeline +package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback import com.netflix.spinnaker.orca.clouddriver.pipeline.providers.aws.ApplySourceServerGroupCapacityStage import com.netflix.spinnaker.orca.clouddriver.pipeline.providers.aws.CaptureSourceServerGroupCapacityStage import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.DisableServerGroupStage import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.EnableServerGroupStage import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.ResizeServerGroupStage -import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.RollbackServerGroupStage import com.netflix.spinnaker.orca.kato.pipeline.support.ResizeStrategy import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner -import org.springframework.beans.factory.config.AutowireCapableBeanFactory import spock.lang.Shared import spock.lang.Specification -import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.stage +import spock.lang.Subject + +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.stage; -class RollbackServerGroupStageSpec extends Specification { +class ExplicitRollbackSpec extends Specification { @Shared def enableServerGroupStage = new EnableServerGroupStage() @@ -46,43 +45,36 @@ class RollbackServerGroupStageSpec extends Specification { @Shared def applySourceServerGroupCapacityStage = new ApplySourceServerGroupCapacityStage() + @Subject + def rollback = new ExplicitRollback( + enableServerGroupStage: enableServerGroupStage, + disableServerGroupStage: disableServerGroupStage, + resizeServerGroupStage: resizeServerGroupStage, + captureSourceServerGroupCapacityStage: captureSourceServerGroupCapacityStage, + applySourceServerGroupCapacityStage: applySourceServerGroupCapacityStage + ) + def "should inject enable, resize and disable stages corresponding to the server group being restored and rollbacked"() { given: - def autowireCapableBeanFactory = Stub(AutowireCapableBeanFactory) { - autowireBean(_) >> { RollbackServerGroupStage.ExplicitRollback rollback -> - rollback.enableServerGroupStage = enableServerGroupStage - rollback.disableServerGroupStage = disableServerGroupStage - rollback.resizeServerGroupStage = resizeServerGroupStage - rollback.captureSourceServerGroupCapacityStage = captureSourceServerGroupCapacityStage - rollback.applySourceServerGroupCapacityStage = applySourceServerGroupCapacityStage - } - } - - def rollbackServerGroupStage = new RollbackServerGroupStage() - rollbackServerGroupStage.autowireCapableBeanFactory = autowireCapableBeanFactory + rollback.rollbackServerGroupName = "servergroup-v002" + rollback.restoreServerGroupName = "servergroup-v001" + rollback.targetHealthyRollbackPercentage = 95 def stage = stage { type = "rollbackServerGroup" context = [ - rollbackType : "EXPLICIT", - rollbackContext: [ - restoreServerGroupName : "servergroup-v001", - rollbackServerGroupName: "servergroup-v002" - ], - credentials : "test", - cloudProvider : "aws", - "region" : "us-west-1" + credentials : "test", + cloudProvider : "aws", + "region" : "us-west-1" ] } when: - def tasks = rollbackServerGroupStage.buildTaskGraph(stage) - def allStages = rollbackServerGroupStage.aroundStages(stage) + def allStages = rollback.buildStages(stage) def beforeStages = allStages.findAll { it.syntheticStageOwner == SyntheticStageOwner.STAGE_BEFORE } def afterStages = allStages.findAll { it.syntheticStageOwner == SyntheticStageOwner.STAGE_AFTER } then: - tasks.iterator().size() == 0 beforeStages.isEmpty() afterStages*.type == [ "enableServerGroup", @@ -92,7 +84,8 @@ class RollbackServerGroupStageSpec extends Specification { "applySourceServerGroupCapacity" ] afterStages[0].context == stage.context + [ - serverGroupName: "servergroup-v001" + serverGroupName: "servergroup-v001", + targetHealthyDeployPercentage: 95 ] afterStages[1].context == [ source : [ @@ -108,7 +101,8 @@ class RollbackServerGroupStageSpec extends Specification { action : "scale_to_server_group", source : new ResizeStrategy.Source(null, null, "us-west-1", null, "servergroup-v002", "test", "aws"), asgName : "servergroup-v001", - pinMinimumCapacity: true + pinMinimumCapacity: true, + targetHealthyDeployPercentage: 95 ] afterStages[3].context == stage.context + [ serverGroupName: "servergroup-v002" diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/PreviousImageRollbackSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/PreviousImageRollbackSpec.groovy new file mode 100644 index 0000000000..4c1e2c25d4 --- /dev/null +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/rollback/PreviousImageRollbackSpec.groovy @@ -0,0 +1,161 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.rollback + +import com.netflix.spinnaker.orca.RetrySupport +import com.netflix.spinnaker.orca.clouddriver.OortService +import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.CloneServerGroupStage +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner +import spock.lang.Specification +import spock.lang.Subject +import spock.lang.Unroll + +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.stage; + +class PreviousImageRollbackSpec extends Specification { + def cloneServerGroupStage = new CloneServerGroupStage() + def oortService = Mock(OortService) + + @Subject + def rollback = new PreviousImageRollback( + cloneServerGroupStage: cloneServerGroupStage, + oortService: oortService, + retrySupport: Spy(RetrySupport) { + _ * sleep(_) >> { /* do nothing */ } + } + ) + + def stage = stage { + type = "rollbackServerGroup" + context = [ + credentials : "test", + cloudProvider: "aws", + region : "us-west-1" + ] + } + + @Unroll + def "should inject clone stage with #imageSource"() { + given: + rollback.rollbackServerGroupName = "application-v002" + rollback.targetHealthyRollbackPercentage = 95 + rollback.imageName = imageName + + when: + def allStages = rollback.buildStages(stage) + def beforeStages = allStages.findAll { it.syntheticStageOwner == SyntheticStageOwner.STAGE_BEFORE } + def afterStages = allStages.findAll { it.syntheticStageOwner == SyntheticStageOwner.STAGE_AFTER } + + then: + (imageName ? 0 : 1) * oortService.getEntityTags(_, _, _, _, _) >> { + // should only call `getEntityTags()` if an image was not explicitly provided to the rollback + return [[ + tags: [ + [ + name : "spinnaker:metadata", + value: [ + previousServerGroup: [ + imageName: "previous_image_from_entity_tags", + imageId : "previous_image_from_entity_tags_id", + ] + ] + ] + ] + ]] + + } + + beforeStages.isEmpty() + afterStages*.type == [ + "cloneServerGroup", + ] + afterStages[0].context == [ + amiName : expectedImageName, + imageId : expectedImageId, + imageName : expectedImageName, + strategy : "redblack", + application : "application", + stack : null, + freeFormDetails : null, + targetHealthyDeployPercentage: 95, + region : "us-west-1", + credentials : "test", + cloudProvider : "aws", + source : [ + asgName : "application-v002", + serverGroupName : "application-v002", + account : "test", + region : "us-west-1", + cloudProvider : "aws", + useSourceCapacity: true + ] + ] + + where: + imageName | imageSource || expectedImageName || expectedImageId + "explicit_image" | "explicitly provided image" || "explicit_image" || null + null | "image fetched from `spinnaker:metadata` entity tag" || "previous_image_from_entity_tags" || "previous_image_from_entity_tags_id" + } + + @Unroll + def "should include interestingHealthProviderNames in clone stage context when present in parent"() { + given: + rollback.imageName = "explicit_image" + stage.context.putAll(additionalContext) + + when: + def allStages = rollback.buildStages(stage) + + then: + allStages[0].context.containsKey("interestingHealthProviderNames") == hasInterestingHealthProviderNames + + where: + additionalContext || hasInterestingHealthProviderNames + [:] || false + [interestingHealthProviderNames: null] || true + [interestingHealthProviderNames: ["Amazon"]] || true + [interestingHealthProviderNames: []] || true + + } + + def "should raise exception if multiple entity tags found"() { + when: + rollback.rollbackServerGroupName = "application-v002" + rollback.getImageDetailsFromEntityTags("aws", "test", "us-west-2") + + then: + 1 * oortService.getEntityTags(*_) >> { + return [ + [id: "1"], + [id: "2"] + ] + } + + def e = thrown(IllegalStateException) + e.message == "More than one set of entity tags found for aws:serverGroup:application-v002:test:us-west-2" + } + + def "should raise exception if no image found"() { + when: + rollback.rollbackServerGroupName = "application-v002" + rollback.buildStages(stage) + + then: + def e = thrown(IllegalStateException) + e.message == "Unable to determine rollback image (serverGroupName: application-v002)" + } +} diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RedBlackStrategySpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RedBlackStrategySpec.groovy index 83fad5a36e..50fdf39e98 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RedBlackStrategySpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/strategies/RedBlackStrategySpec.groovy @@ -16,6 +16,7 @@ package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.strategies +import com.netflix.spinnaker.moniker.Moniker import com.netflix.spinnaker.orca.clouddriver.pipeline.cluster.DisableClusterStage import com.netflix.spinnaker.orca.clouddriver.pipeline.cluster.ScaleDownClusterStage import com.netflix.spinnaker.orca.clouddriver.pipeline.cluster.ShrinkClusterStage @@ -32,10 +33,12 @@ class RedBlackStrategySpec extends Specification { def "should compose flow"() { given: + Moniker moniker = new Moniker(app: "unit", stack: "test"); def ctx = [ account : "testAccount", application : "unit", stack : "tests", + moniker : moniker, cloudProvider : "aws", region : "north", availabilityZones: [ @@ -60,6 +63,7 @@ class RedBlackStrategySpec extends Specification { credentials : "testAccount", cloudProvider : "aws", cluster : "unit-tests", + moniker : moniker, region : "north", remainingEnabledServerGroups : 1, preferLargerOverNewer : false, diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroupResolverSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroupResolverSpec.groovy index 1621585092..66db09c66f 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroupResolverSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/pipeline/servergroup/support/TargetServerGroupResolverSpec.groovy @@ -17,6 +17,7 @@ package com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spinnaker.orca.RetrySupport import com.netflix.spinnaker.orca.clouddriver.OortService import com.netflix.spinnaker.orca.pipeline.model.Stage import retrofit.RetrofitError @@ -32,8 +33,16 @@ class TargetServerGroupResolverSpec extends Specification { OortService oort = Mock(OortService) ObjectMapper mapper = new ObjectMapper() + RetrySupport retrySupport = Spy(RetrySupport) { + _ * sleep(_) >> { /* do nothing */ } + } + @Subject - TargetServerGroupResolver subject = new TargetServerGroupResolver(oortService: oort, mapper: mapper) + TargetServerGroupResolver subject = new TargetServerGroupResolver( + oortService: oort, + mapper: mapper, + retrySupport: retrySupport + ) def "should resolve to target server groups"() { when: @@ -131,7 +140,7 @@ class TargetServerGroupResolverSpec extends Specification { } @Unroll - def "should retry on network + 429 errors"() { + def "should retry on non 404 errors"() { given: def invocationCount = 0 def capturedResult @@ -152,9 +161,9 @@ class TargetServerGroupResolverSpec extends Specification { where: exception || expectNull || expectedInvocationCount - new IllegalStateException("should not retry") || false || 1 - retrofitError(RetrofitError.Kind.UNEXPECTED, 400) || false || 1 - retrofitError(RetrofitError.Kind.HTTP, 500) || false || 1 + new IllegalStateException("should retry") || false || 10 + retrofitError(RetrofitError.Kind.UNEXPECTED, 400) || false || 10 + retrofitError(RetrofitError.Kind.HTTP, 500) || false || 10 retrofitError(RetrofitError.Kind.HTTP, 404) || true || 1 // a 404 should short-circuit and return null retrofitError(RetrofitError.Kind.NETWORK, 0) || false || 10 retrofitError(RetrofitError.Kind.HTTP, 429) || false || 10 diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/DetermineHealthProvidersTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/DetermineHealthProvidersTaskSpec.groovy index e33809b4b0..9de672b436 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/DetermineHealthProvidersTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/DetermineHealthProvidersTaskSpec.groovy @@ -66,6 +66,7 @@ class DetermineHealthProvidersTaskSpec extends Specification { [application: "app"] | bA("app", true, true) || [:] // no health provider names when platformHealthOnlyShowOverride is true [application: "app", cloudProvider: "random"] | null || [:] // no health provider names when cloud provider is unsupported/unknown [application: "app"] | null || [:] // no health provider names when an exception is raised + [moniker: [app: "app"]] | null || [:] } private static bA(String applicationName, Boolean platformHealthOnly, Boolean platformHealthOnlyShowOverride) { diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractClusterWideClouddriverTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractClusterWideClouddriverTaskSpec.groovy index e2dfecc0af..be330070de 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractClusterWideClouddriverTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/AbstractClusterWideClouddriverTaskSpec.groovy @@ -23,6 +23,7 @@ import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.CreateServerG import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.DisableServerGroupStage import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.Location import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroup +import com.netflix.spinnaker.orca.clouddriver.tasks.cluster.AbstractClusterWideClouddriverTask.ClusterSelection import com.netflix.spinnaker.orca.clouddriver.utils.OortHelper import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -98,7 +99,7 @@ class AbstractClusterWideClouddriverTaskSpec extends Specification { given: def pipeline = new Pipeline("orca") def stage = new Stage<>(pipeline, DisableServerGroupStage.PIPELINE_CONFIG_TYPE, [ - continueIfClusterNotFound: true + continueIfClusterNotFound: true ]) def task = new AbstractClusterWideClouddriverTask() { @Override @@ -122,7 +123,7 @@ class AbstractClusterWideClouddriverTaskSpec extends Specification { given: def pipeline = new Pipeline("orca") def stage = new Stage<>(pipeline, DisableServerGroupStage.PIPELINE_CONFIG_TYPE, [ - continueIfClusterNotFound: true + continueIfClusterNotFound: true ]) def task = new AbstractClusterWideClouddriverTask() { @Override @@ -137,11 +138,31 @@ class AbstractClusterWideClouddriverTaskSpec extends Specification { def result = task.execute(stage) then: - 1 * oortHelper.getCluster(_, _, _, _) >> Optional.of([ serverGroups: [] ]) + 1 * oortHelper.getCluster(_, _, _, _) >> Optional.of([serverGroups: []]) result == TaskResult.SUCCEEDED } + @Unroll + 'cluster with name "#cluster" and moniker "#moniker" should have application name "#expected"'() { + given: + def stage = new Stage<>(new Pipeline("orca"), 'clusterSelection', [ + cluster: cluster, + moniker: moniker, + ]) + when: + ClusterSelection selection = stage.mapTo(ClusterSelection) + + then: + selection.getApplication() == expected + + where: + cluster | moniker | expected + 'clustername' | ['app': 'appname'] | 'appname' + 'app-stack' | null | 'app' + + } + static TargetServerGroup sg(String name, String region = "us-west-1", int createdTime = System.currentTimeMillis()) { new TargetServerGroup(name: name, region: region, createdTime: createdTime) } diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/ClusterSizePreconditionTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/ClusterSizePreconditionTaskSpec.groovy index b150f607fd..e2481455ba 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/ClusterSizePreconditionTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/ClusterSizePreconditionTaskSpec.groovy @@ -120,6 +120,29 @@ class ClusterSizePreconditionTaskSpec extends Specification { } + @Unroll + 'cluster with name "#cluster" and moniker "#moniker" should have application name "#expected"'() { + given: + def stage = new Stage<>(new Pipeline("orca"), 'checkCluster', [ + context: [ + cluster: cluster, + moniker: moniker, + ] + ]) + when: + ClusterSizePreconditionTask.ComparisonConfig config = stage.mapTo("/context", ClusterSizePreconditionTask.ComparisonConfig) + + then: + config.getApplication() == expected + + where: + cluster | moniker | expected + 'clustername' | ['app': 'appname'] | 'appname' + 'app-stack' | null | 'app' + + } + + Map mkSg(String region) { [name: "foo-v${cnt.incrementAndGet()}".toString(), region: region] } diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/FindImageFromClusterTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/FindImageFromClusterTaskSpec.groovy index 057d9de513..21583f4bc6 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/FindImageFromClusterTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/cluster/FindImageFromClusterTaskSpec.groovy @@ -33,7 +33,7 @@ import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.pipeline class FindImageFromClusterTaskSpec extends Specification { @Subject - task = new FindImageFromClusterTask() + task = new FindImageFromClusterTask() OortService oortService = Mock(OortService) def setup() { @@ -47,23 +47,23 @@ class FindImageFromClusterTaskSpec extends Specification { def pipe = pipeline { application = "contextAppName" // Should be ignored. } - def stage = new Stage<>(pipe, "findImage", [ - cloudProvider : "cloudProvider", - cluster : "foo-test", - account : "test", - selectionStrategy: "LARGEST", - onlyEnabled : "false", - regions : [location1.value, location2.value] - ]) + def stage = new Stage<>(pipe, "findImage", [ + cloudProvider : "cloudProvider", + cluster : "foo-test", + account : "test", + selectionStrategy: "LARGEST", + onlyEnabled : "false", + regions : [location1.value, location2.value] + ]) when: - def result = task.execute(stage) + def result = task.execute(stage) then: - 1 * oortService.getServerGroupSummary("foo", "test", "foo-test", "cloudProvider", location1.value, - "LARGEST", FindImageFromClusterTask.SUMMARY_TYPE, false.toString()) >> oortResponse1 - 1 * oortService.getServerGroupSummary("foo", "test", "foo-test", "cloudProvider", location2.value, - "LARGEST", FindImageFromClusterTask.SUMMARY_TYPE, false.toString()) >> oortResponse2 + 1 * oortService.getServerGroupSummary("foo", "test", "foo-test", "cloudProvider", location1.value, + "LARGEST", FindImageFromClusterTask.SUMMARY_TYPE, false.toString()) >> oortResponse1 + 1 * oortService.getServerGroupSummary("foo", "test", "foo-test", "cloudProvider", location2.value, + "LARGEST", FindImageFromClusterTask.SUMMARY_TYPE, false.toString()) >> oortResponse2 assertNorth result.outputs?.deploymentDetails?.find { it.region == "north" } as Map @@ -72,28 +72,28 @@ class FindImageFromClusterTaskSpec extends Specification { } as Map where: - location1 = new Location(type: Location.Type.REGION, value: "north") - location2 = new Location(type: Location.Type.REGION, value: "south") - - oortResponse1 = [ - summaries: [[ - serverGroupName: "foo-test-v000", - imageId: "ami-012", - imageName: "ami-012-name", - image: [ imageId: "ami-012", name: "ami-012-name", foo: "bar" ], - buildInfo: [ job: "foo-build", buildNumber: 1 ] - ]] - ] + location1 = new Location(type: Location.Type.REGION, value: "north") + location2 = new Location(type: Location.Type.REGION, value: "south") - oortResponse2 = [ - summaries: [[ - serverGroupName: "foo-test-v002", - imageId: "ami-234", - imageName: "ami-234-name", - image: [ imageId: "ami-234", name: "ami-234-name", foo: "baz" ], - buildInfo: [ job: "foo-build", buildNumber: 1 ] - ]] - ] + oortResponse1 = [ + summaries: [[ + serverGroupName: "foo-test-v000", + imageId : "ami-012", + imageName : "ami-012-name", + image : [imageId: "ami-012", name: "ami-012-name", foo: "bar"], + buildInfo : [job: "foo-build", buildNumber: 1] + ]] + ] + + oortResponse2 = [ + summaries: [[ + serverGroupName: "foo-test-v002", + imageId : "ami-234", + imageName : "ami-234-name", + image : [imageId: "ami-234", name: "ami-234-name", foo: "baz"], + buildInfo : [job: "foo-build", buildNumber: 1] + ]] + ] } def "should be RUNNING if summary does not include imageId"() { @@ -123,17 +123,17 @@ class FindImageFromClusterTaskSpec extends Specification { oortResponse1 = [ summaries: [[ - serverGroupName: "foo-test-v000", - imageId: "ami-012", - imageName: "ami-012-name", - image: [ imageId: "ami-012", name: "ami-012-name", foo: "bar" ], - buildInfo: [ job: "foo-build", buildNumber: 1 ] + serverGroupName: "foo-test-v000", + imageId : "ami-012", + imageName : "ami-012-name", + image : [imageId: "ami-012", name: "ami-012-name", foo: "bar"], + buildInfo : [job: "foo-build", buildNumber: 1] ]] ] oortResponse2 = [ summaries: [[ - serverGroupName: "foo-test-v002" + serverGroupName: "foo-test-v002" ]] ] } @@ -181,12 +181,12 @@ class FindImageFromClusterTaskSpec extends Specification { } def stage = new Stage<>(pipe, "findImage", [ resolveMissingLocations: true, - cloudProvider : "cloudProvider", - cluster : "foo-test", - account : "test", - selectionStrategy: "LARGEST", - onlyEnabled : "false", - regions : [location1.value, location2.value] + cloudProvider : "cloudProvider", + cluster : "foo-test", + account : "test", + selectionStrategy : "LARGEST", + onlyEnabled : "false", + regions : [location1.value, location2.value] ]) when: @@ -213,12 +213,12 @@ class FindImageFromClusterTaskSpec extends Specification { oortResponse1 = [ summaries: [[ - serverGroupName: "foo-test-v000", - imageId : "ami-012", - imageName : "ami-012-name-ebs", - image : [imageId: "ami-012", name: "ami-012-name-ebs", foo: "bar"], - buildInfo : [job: "foo-build", buildNumber: 1] - ]] + serverGroupName: "foo-test-v000", + imageId : "ami-012", + imageName : "ami-012-name-ebs", + image : [imageId: "ami-012", name: "ami-012-name-ebs", foo: "bar"], + buildInfo : [job: "foo-build", buildNumber: 1] + ]] ] imageSearchResult = [ @@ -230,7 +230,7 @@ class FindImageFromClusterTaskSpec extends Specification { ], [ imageName: "ami-012-name-ebs1", - amis : [ + amis : [ "south": ["ami-234"] ] ] @@ -244,12 +244,12 @@ class FindImageFromClusterTaskSpec extends Specification { } def stage = new Stage<>(pipe, "findImage", [ resolveMissingLocations: true, - cloudProvider : "cloudProvider", - cluster : "foo-test", - account : "test", - selectionStrategy: "LARGEST", - onlyEnabled : "false", - regions : [location1.value, location2.value] + cloudProvider : "cloudProvider", + cluster : "foo-test", + account : "test", + selectionStrategy : "LARGEST", + onlyEnabled : "false", + regions : [location1.value, location2.value] ]) when: @@ -276,11 +276,11 @@ class FindImageFromClusterTaskSpec extends Specification { oortResponse1 = [ summaries: [[ - serverGroupName: "foo-test-v000", - imageId : "ami-012", - imageName : "ami-012-name-ebs", - image : [imageId: "ami-012", name: "ami-012-name-ebs", foo: "bar"] - ]] + serverGroupName: "foo-test-v000", + imageId : "ami-012", + imageName : "ami-012-name-ebs", + image : [imageId: "ami-012", name: "ami-012-name-ebs", foo: "bar"] + ]] ] imageSearchResult = [ @@ -292,7 +292,7 @@ class FindImageFromClusterTaskSpec extends Specification { ], [ imageName: "ami-012-name-ebs1", - amis : [ + amis : [ "south": ["ami-234"] ] ] @@ -307,12 +307,12 @@ class FindImageFromClusterTaskSpec extends Specification { } def stage = new Stage<>(pipe, "findImage", [ resolveMissingLocations: true, - cloudProvider : "aws", - cluster : "foo-test", - account : "test", - selectionStrategy: "LARGEST", - onlyEnabled : "false", - regions : [location1.value, location2.value] + cloudProvider : "aws", + cluster : "foo-test", + account : "test", + selectionStrategy : "LARGEST", + onlyEnabled : "false", + regions : [location1.value, location2.value] ]) when: @@ -356,7 +356,7 @@ class FindImageFromClusterTaskSpec extends Specification { ], [ imageName: "ami-012-name-ebs1", - amis : [ + amis : [ "south": ["ami-234"] ] ] @@ -367,30 +367,30 @@ class FindImageFromClusterTaskSpec extends Specification { def "should parse fail strategy error message"() { given: def stage = new Stage<>(new Pipeline("orca"), "whatever", [ - cloudProvider : "cloudProvider", - cluster : "foo-test", - account : "test", - selectionStrategy: "FAIL", - onlyEnabled : "false", - zones : [location.value] - ]) + cloudProvider : "cloudProvider", + cluster : "foo-test", + account : "test", + selectionStrategy: "FAIL", + onlyEnabled : "false", + zones : [location.value] + ]) - Response response = new Response("http://oort", 404, "NOT_FOUND", [], new TypedString(oortResponse)) + Response response = new Response("http://oort", 404, "NOT_FOUND", [], new TypedString(oortResponse)) when: task.execute(stage) then: - 1 * oortService.getServerGroupSummary("foo", "test", "foo-test", "cloudProvider", location.value, - "FAIL", FindImageFromClusterTask.SUMMARY_TYPE, false.toString()) >> { - throw new RetrofitError(null, null, response, null, null, null, null) - } - IllegalStateException ise = thrown() - ise.message == "Multiple possible server groups present in ${location.value}".toString() + 1 * oortService.getServerGroupSummary("foo", "test", "foo-test", "cloudProvider", location.value, + "FAIL", FindImageFromClusterTask.SUMMARY_TYPE, false.toString()) >> { + throw new RetrofitError(null, null, response, null, null, null, null) + } + IllegalStateException ise = thrown() + ise.message == "Multiple possible server groups present in ${location.value}".toString() where: - location = new Location(type: Location.Type.ZONE, value: "north-pole-1a") - oortResponse = """\ + location = new Location(type: Location.Type.ZONE, value: "north-pole-1a") + oortResponse = """\ { "error" : "target.fail.strategy", "message" : "target.fail.strategy", @@ -398,4 +398,24 @@ class FindImageFromClusterTaskSpec extends Specification { } """.stripIndent() } + + @Unroll + 'cluster with name "#cluster" and moniker "#moniker" should have application name "#expected"'() { + given: + def stage = new Stage<>(new Pipeline("orca"), 'findImageFromCluster', [ + cluster: cluster, + moniker: moniker, + ]) + when: + FindImageFromClusterTask.FindImageConfiguration config = stage.mapTo(FindImageFromClusterTask.FindImageConfiguration) + + then: + config.getApplication() == expected + + where: + cluster | moniker | expected + 'clustername' | ['app': 'appname'] | 'appname' + 'app-stack' | null | 'app' + + } } diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractInstancesCheckTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractInstancesCheckTaskSpec.groovy index ccecba86e9..72aac5e3c4 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractInstancesCheckTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/instance/AbstractInstancesCheckTaskSpec.groovy @@ -87,7 +87,7 @@ class AbstractInstancesCheckTaskSpec extends Specification { task.execute(stage) then: - 1 * task.oortService.getServerGroup("front50", "test", "us-west-1", "front50-v000") >> constructResponse(200, ''' + 1 * task.oortService.getServerGroup("test", "us-west-1", "front50-v000") >> constructResponse(200, ''' { "name": "front50-v000", "region": "us-west-1", @@ -133,7 +133,7 @@ class AbstractInstancesCheckTaskSpec extends Specification { then: result.context.zeroDesiredCapacityCount == expected - 1 * task.oortService.getServerGroup("front50", "test", "us-west-1", "front50-v000") >> constructResponse(200, ''' + 1 * task.oortService.getServerGroup("test", "us-west-1", "front50-v000") >> constructResponse(200, ''' { "name": "front50-v000", "region": "us-west-1", @@ -179,7 +179,7 @@ class AbstractInstancesCheckTaskSpec extends Specification { then: result.context.zeroDesiredCapacityCount == 1 - 1 * task.oortService.getServerGroup("front50", "test", "us-west-1", "front50-v000") >> constructResponse(200, ''' + 1 * task.oortService.getServerGroup("test", "us-west-1", "front50-v000") >> constructResponse(200, ''' { "name": "front50-v000", "region": "us-west-1", diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AddServerGroupEntityTagsTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AddServerGroupEntityTagsTaskSpec.groovy new file mode 100644 index 0000000000..2e7e3a1056 --- /dev/null +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/AddServerGroupEntityTagsTaskSpec.groovy @@ -0,0 +1,74 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.tasks.servergroup + +import com.netflix.spinnaker.orca.ExecutionStatus +import com.netflix.spinnaker.orca.RetrySupport +import com.netflix.spinnaker.orca.clouddriver.KatoService +import com.netflix.spinnaker.orca.clouddriver.OortService +import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.model.Stage +import retrofit.RetrofitError +import retrofit.client.Response +import spock.lang.Specification +import spock.lang.Subject +import static java.net.HttpURLConnection.HTTP_NOT_FOUND + +class AddServerGroupEntityTagsTaskSpec extends Specification { + + def katoService = Mock(KatoService) + def oortService = Mock(OortService) + def retrySupport = Spy(RetrySupport) { + _ * sleep(_) >> { /* do nothing */ } + } + + ServerGroupEntityTagGenerator tagGenerator = new SpinnakerMetadataServerGroupTagGenerator(oortService, retrySupport) + + @Subject + AddServerGroupEntityTagsTask task = new AddServerGroupEntityTagsTask(kato: katoService, tagGenerators: [tagGenerator]) + + void "should return with failed/continue status if tagging operation fails"() { + when: + def stage = new Stage<>(new Pipeline("orca"), "whatever", [ + "deploy.server.groups": [ + "us-east-1": ["foo-v001"] + ] + ]) + def result = task.execute(stage) + + then: + result.status == ExecutionStatus.FAILED_CONTINUE + 1 * katoService.requestOperations(_) >> { throw new RuntimeException("something went wrong") } + } + + void "skips tagging when no tag generators or generators do not produce any tags"() { + given: + AddServerGroupEntityTagsTask emptyTask = new AddServerGroupEntityTagsTask(kato: katoService, tagGenerators: []) + + when: + def stage = new Stage<>(new Pipeline("orca"), "whatever", [ + "deploy.server.groups": [ + "us-east-1": ["foo-v001"], + ] + ]) + def result = emptyTask.execute(stage) + + then: + result.status == ExecutionStatus.SKIPPED + 0 * _ + } +} diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/BulkDestroyServerGroupTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/BulkDestroyServerGroupTaskSpec.groovy index 6fb9627c83..65b22786b7 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/BulkDestroyServerGroupTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/BulkDestroyServerGroupTaskSpec.groovy @@ -3,6 +3,7 @@ package com.netflix.spinnaker.orca.clouddriver.tasks.servergroup import com.netflix.spinnaker.orca.clouddriver.KatoService import com.netflix.spinnaker.orca.clouddriver.model.TaskId import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroup +import com.netflix.spinnaker.orca.clouddriver.utils.MonikerHelper import com.netflix.spinnaker.orca.clouddriver.utils.OortHelper import com.netflix.spinnaker.orca.clouddriver.utils.TrafficGuard import com.netflix.spinnaker.orca.pipeline.model.Pipeline @@ -13,7 +14,7 @@ class BulkDestroyServerGroupTaskSpec extends Specification { def "should create multiple destroy operations on bulk destroy server group task"() { given: - def task = new BulkDestroyServerGroupTask(trafficGuard: Mock(TrafficGuard)) + def task = new BulkDestroyServerGroupTask(trafficGuard: Mock(TrafficGuard), monikerHelper: new MonikerHelper()) def stage = new Stage<>(new Pipeline("orca"), "") stage.context = [ cloudProvider: "titus", @@ -87,4 +88,5 @@ class BulkDestroyServerGroupTaskSpec extends Specification { then: thrown(TargetServerGroup.NotFoundException) } + } diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupMetadataTagTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupMetadataTagTaskSpec.groovy deleted file mode 100644 index 7bb0925519..0000000000 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/ServerGroupMetadataTagTaskSpec.groovy +++ /dev/null @@ -1,197 +0,0 @@ -/* - * Copyright 2017 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License") - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.netflix.spinnaker.orca.clouddriver.tasks.servergroup - -import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.clouddriver.KatoService -import com.netflix.spinnaker.orca.clouddriver.model.TaskId -import com.netflix.spinnaker.orca.pipeline.model.Execution -import com.netflix.spinnaker.orca.pipeline.model.Pipeline -import com.netflix.spinnaker.orca.pipeline.model.Stage -import spock.lang.Specification -import spock.lang.Subject -import spock.lang.Unroll -import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.orchestration -import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.pipeline - -class ServerGroupMetadataTagTaskSpec extends Specification { - - KatoService katoService = Mock(KatoService) - - ServerGroupEntityTagGenerator tagGenerator = new SpinnakerMetadataServerGroupTagGenerator() - - @Subject - ServerGroupMetadataTagTask task = new ServerGroupMetadataTagTask(kato: katoService, tagGenerators: [tagGenerator]) - - List taggingOps = null - - void "should return with failed/continue status if tagging operation fails"() { - when: - def stage = new Stage<>(new Pipeline("orca"), "whatever", [ - "deploy.server.groups": [ - "us-east-1": ["foo-v001"] - ] - ]) - def result = task.execute(stage) - - then: - result.status == ExecutionStatus.FAILED_CONTINUE - 1 * katoService.requestOperations(_) >> { throw new RuntimeException("something went wrong") } - } - - void "sends tag with all relevant metadata for each server group"() { - given: - mockTaggingOperation() - List expectedTags = [ - [ - stageId: "x", - executionType: "pipeline", - description: "Deploy to us-east-1", - pipelineConfigId: "config-id", - application: "foo", - executionId: "ex-id", - user: "chris" - ], - [ - stageId: "x", - executionType: "pipeline", - description: "Deploy to us-east-1", - pipelineConfigId: "config-id", - application: "foo", - executionId: "ex-id", - user: "chris" - ], - [ - stageId: "x", - executionType: "pipeline", - description: "Deploy to us-east-1", - pipelineConfigId: "config-id", - application: "foo", - executionId: "ex-id", - user: "chris" - ] - ] - - when: - def pipeline = pipeline { - pipelineConfigId = "config-id" - name = "Deploy to us-east-1" - application = "foo" - id = "ex-id" - authentication = new Execution.AuthenticationDetails("chris") - } - def stage = new Stage<>(pipeline, "whatever", [ - "deploy.server.groups": [ - "us-east-1": ["foo-v001"], - "us-west-1": ["foo-v001", "foo-v002"] - ] - ]) - stage.id = "x" - def result = task.execute(stage) - - then: - result.status == ExecutionStatus.SUCCEEDED - taggingOps == expectedTags - } - - void "omits user if authentication not found"() { - given: - mockTaggingOperation() - - when: - def stage = new Stage<>(new Pipeline("orca"), "whatever", [ - "deploy.server.groups": [ - "us-east-1": ["foo-v001"], - ] - ]) - def result = task.execute(stage) - - then: - result.status == ExecutionStatus.SUCCEEDED - taggingOps[0].user == null - } - - void "includes description on tasks"() { - given: - mockTaggingOperation() - - when: - def orchestration = orchestration { - description = "some description" - } - def stage = new Stage<>(orchestration, "zzz", [ - "deploy.server.groups": [ - "us-east-1": ["foo-v001"], - ] - ]) - task.execute(stage) - - then: - taggingOps[0].description == "some description" - } - - void "skips tagging when no tag generators or generators do not produce any tags"() { - given: - ServerGroupMetadataTagTask emptyTask = new ServerGroupMetadataTagTask(kato: katoService, tagGenerators: []) - - when: - def stage = new Stage<>(new Pipeline("orca"), "whatever", [ - "deploy.server.groups": [ - "us-east-1": ["foo-v001"], - ] - ]) - def result = emptyTask.execute(stage) - - then: - result.status == ExecutionStatus.SKIPPED - 0 * _ - } - - @Unroll - void "prefers comments to reason from context and applies as 'comments' field"() { - given: - mockTaggingOperation() - - when: - def stage = new Stage<>(new Pipeline("orca"), "whatever", [ - "deploy.server.groups": [ - "us-east-1": ["foo-v001"], - ], - comments: comments, - reason: reason - ]) - def result = task.execute(stage) - - then: - result.status == ExecutionStatus.SUCCEEDED - taggingOps[0].comments == expected - - where: - comments | reason || expected - null | null || null - null | "r" || "r" - "c" | null || "c" - "c" | "r" || "c" - } - - private void mockTaggingOperation() { - 1 * katoService.requestOperations({ ops -> - taggingOps = ops.collect { it.upsertEntityTags.tags[0].value } - true - }) >> rx.Observable.just(new TaskId("eh")) - } -} diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/SpinnakerMetadataServerGroupTagGeneratorSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/SpinnakerMetadataServerGroupTagGeneratorSpec.groovy new file mode 100644 index 0000000000..c5ee097ad9 --- /dev/null +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/SpinnakerMetadataServerGroupTagGeneratorSpec.groovy @@ -0,0 +1,166 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.tasks.servergroup + +import com.netflix.spinnaker.orca.RetrySupport +import com.netflix.spinnaker.orca.clouddriver.OortService +import com.netflix.spinnaker.orca.pipeline.model.Stage +import retrofit.RetrofitError +import retrofit.client.Response +import spock.lang.Shared +import spock.lang.Specification +import spock.lang.Unroll + +import static com.netflix.spinnaker.orca.pipeline.model.Execution.* +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.orchestration +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.pipeline +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.stage +import static java.net.HttpURLConnection.HTTP_NOT_FOUND + +class SpinnakerMetadataServerGroupTagGeneratorSpec extends Specification { + def oortService = Mock(OortService) + def retrySupport = Spy(RetrySupport) { + _ * sleep(_) >> { /* do nothing */ } + } + + @Shared + def notFoundError = RetrofitError.httpError( + null, + new Response("http://google.com", HTTP_NOT_FOUND, "Not Found", [], null), + null, + null + ) + + @Unroll + def "should build spinnaker:metadata tag for pipeline"() { + given: + def tagGenerator = Spy(SpinnakerMetadataServerGroupTagGenerator, constructorArgs: [oortService, retrySupport]) { + 1 * getPreviousServerGroup(_, _, _, _, _) >> { return previousServerGroup } + } + + def pipeline = pipeline { + name = "my pipeline" + application = "application" + pipelineConfigId = "configId" + authentication = authenticatedUser ? new AuthenticationDetails(authenticatedUser) : null + + stage { + type = "wait" + context = [comments: "this is a wait stage"] + } + } + + when: + def tags = tagGenerator.generateTags(pipeline.stages[0], "application-v002", "account", "us-west-2", "aws") + + then: + tags == [[ + name : "spinnaker:metadata", + value: [ + executionId : pipeline.id, + pipelineConfigId: "configId", + application : "application", + executionType : "pipeline", + description : "my pipeline", + stageId : pipeline.stages[0].id, + comments : "this is a wait stage", + ] + (previousServerGroup ? [previousServerGroup: previousServerGroup] : [:]) + + (authenticatedUser ? [user: authenticatedUser] : [:]) + ]] + + where: + previousServerGroup | authenticatedUser || _ + null | null || _ // metadata tag should NOT include `previousServerGroup` + null | "username" || _ // include user if non-null + [serverGroupName: "application-v001"] | null || _ + } + + @Unroll + def "should build spinnaker:metadata tag for orchestration"() { + given: + def tagGenerator = Spy(SpinnakerMetadataServerGroupTagGenerator, constructorArgs: [oortService, retrySupport]) { + 1 * getPreviousServerGroup(_, _, _, _, _) >> { return previousServerGroup } + } + + def orchestration = orchestration { + name = "my orchestration" + application = "application" + authentication = authenticatedUser ? new AuthenticationDetails(authenticatedUser) : null + description = "this is my orchestration" + + stages << new Stage<>(delegate, "wait") + } + + when: + def tags = tagGenerator.generateTags(orchestration.stages[0], "application-v002", "account", "us-west-2", "aws") + + then: + tags == [[ + name : "spinnaker:metadata", + value: [ + executionId : orchestration.id, + application : "application", + executionType: "orchestration", + description : "this is my orchestration", + stageId : orchestration.stages[0].id, + ] + (previousServerGroup ? [previousServerGroup: previousServerGroup] : [:]) + + (authenticatedUser ? [user: authenticatedUser] : [:]) + ]] + + where: + previousServerGroup | authenticatedUser || _ + null | null || _ // metadata tag should NOT include `previousServerGroup` + null | "username" || _ // include user if non-null + [serverGroupName: "application-v001"] | null || _ + } + + def "should construct previous server group metadata when present"() { + given: + def tagGenerator = new SpinnakerMetadataServerGroupTagGenerator(oortService, retrySupport) + + when: "previous server does exist" + def previousServerGroupMetadata = tagGenerator.getPreviousServerGroup( + "application", "account", "cluster", "aws", "us-west-2" + ) + + then: "metadata should be returned" + 1 * oortService.getServerGroupSummary("application", "account", "cluster", "aws", "us-west-2", "ANCESTOR", "image", "true") >> { + return [ + serverGroupName: "application-v001", + imageId : "ami-1234567", + imageName : "my_image" + ] + } + previousServerGroupMetadata == [ + name : "application-v001", + imageId : "ami-1234567", + imageName : "my_image", + cloudProvider: "aws" + ] + + when: "previous server group does NOT exist" + previousServerGroupMetadata = tagGenerator.getPreviousServerGroup( + "application", "account", "cluster", "aws", "us-west-2" + ) + + then: "no metadata should be returned" + 1 * oortService.getServerGroupSummary("application", "account", "cluster", "aws", "us-west-2", "ANCESTOR", "image", "true") >> { + throw notFoundError + } + previousServerGroupMetadata == null + } +} diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/WaitForCapacityMatchTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/WaitForCapacityMatchTaskSpec.groovy index 893b6f5a0e..f1b410bf5b 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/WaitForCapacityMatchTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/tasks/servergroup/WaitForCapacityMatchTaskSpec.groovy @@ -47,7 +47,7 @@ class WaitForCapacityMatchTaskSpec extends Specification { void "should properly wait for a scale up operation"() { setup: oort = Stub(OortService) - oort.getServerGroup("kato", "test", "us-east-1", "kato-main-v000") >> { new Response('kato', 200, 'ok', [], new TypedString(mapper.writeValueAsString(serverGroup))) } + oort.getServerGroup("test", "us-east-1", "kato-main-v000") >> { new Response('kato', 200, 'ok', [], new TypedString(mapper.writeValueAsString(serverGroup))) } task.oortService = oort def context = [account: "test", "deploy.server.groups": ["us-east-1": ["kato-main-v000"]]] def stage = new Stage<>(new Orchestration("orca"), "resizeAsg", context) @@ -103,7 +103,7 @@ class WaitForCapacityMatchTaskSpec extends Specification { ] ] - oort.getServerGroup("kato", "test", "us-east-1", "kato-main-v000") >> { new Response('kato', 200, 'ok', [], new TypedString(mapper.writeValueAsString(serverGroup))) } + oort.getServerGroup("test", "us-east-1", "kato-main-v000") >> { new Response('kato', 200, 'ok', [], new TypedString(mapper.writeValueAsString(serverGroup))) } task.oortService = oort def context = [account: "test", "deploy.server.groups": ["us-east-1": ["kato-main-v000"]]] def stage = new Stage<>(new Orchestration("orca"), "resizeAsg", context) @@ -136,7 +136,7 @@ class WaitForCapacityMatchTaskSpec extends Specification { void "should properly wait for a scale down operation"() { setup: oort = Stub(OortService) - oort.getServerGroup("kato", "test", "us-east-1", "kato-main-v000") >> { new Response('kato', 200, 'ok', [], new TypedString(mapper.writeValueAsString(serverGroup))) } + oort.getServerGroup("test", "us-east-1", "kato-main-v000") >> { new Response('kato', 200, 'ok', [], new TypedString(mapper.writeValueAsString(serverGroup))) } task.oortService = oort def context = [account: "test", "deploy.server.groups": ["us-east-1": ["kato-main-v000"]]] def stage = new Stage<>(new Orchestration("orca"), "resizeAsg", context) diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatcherSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatcherSpec.groovy new file mode 100644 index 0000000000..ca5ad693e4 --- /dev/null +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/utils/ClusterMatcherSpec.groovy @@ -0,0 +1,169 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.clouddriver.utils + +import spock.lang.Specification +import spock.lang.Unroll + +class ClusterMatcherSpec extends Specification { + + String account = "test" + String location = "us-east-1" + String stack = "stack" + String detail = "detail" + String clusterName = "myapp-stack-detail" + + void "returns null when no rules provided"() { + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, null) == null + } + + void "returns null when no rules match on account, location, or stack/detail"() { + given: + List rules = [ + new ClusterMatchRule(account: account, location: location, stack: "*", detail: "*", priority: 1) + ] + + expect: + ClusterMatcher.getMatchingRule("prod", location, clusterName, rules) == null + } + + void "returns rule based on location if accounts are identical"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: account, location: location, stack: "*", detail: "*", priority: 1) + List rules = [ + new ClusterMatchRule(account: account, location: "*", stack: "*", detail: "*", priority: 1), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, rules) == expected + } + + void "returns rule based on stack if account and location are identical"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: account, location: location, stack: stack, detail: "*", priority: 1) + List rules = [ + new ClusterMatchRule(account: account, location: location, stack: "*", detail: "*", priority: 1), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, rules) == expected + } + + void "returns rule based on detail if account, location, and stack are identical"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: account, location: location, stack: stack, detail: detail, priority: 1) + List rules = [ + new ClusterMatchRule(account: account, location: location, stack: "*", detail: "*", priority: 1), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, rules) == expected + } + + void "returns rule based on priority if all other fields match"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: account, location: location, stack: stack, detail: detail, priority: 1) + List rules = [ + new ClusterMatchRule(account: account, location: location, stack: stack, detail: detail, priority: 2), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, rules) == expected + } + + void "specific account takes priority over all other wildcard fields"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: account, location: "*", stack: "*", detail: "*", priority: 2) + List rules = [ + new ClusterMatchRule(account: "*", location: location, stack: stack, detail: detail, priority: 1), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, rules) == expected + } + + void "specific location takes priority over wildcard stack, detail"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: account, location: location, stack: "*", detail: "*", priority: 2) + List rules = [ + new ClusterMatchRule(account: account, location: "*", stack: stack, detail: detail, priority: 1), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, rules) == expected + } + + void "specific stack takes priority over wildcard detail"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: account, location: location, stack: stack, detail: "*", priority: 2) + List rules = [ + new ClusterMatchRule(account: account, location: location, stack: "*", detail: detail, priority: 1), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, rules) == expected + } + + void "specific detail takes priority over priority"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: account, location: location, stack: stack, detail: detail, priority: 2) + List rules = [ + new ClusterMatchRule(account: account, location: location, stack: stack, detail: "*", priority: 1), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, clusterName, rules) == expected + } + + void "handles clusters without account or details values"() { + given: + ClusterMatchRule expected = new ClusterMatchRule(account: "*", location: "*", stack: "*", detail: "*", priority: 1) + List rules = [ + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, "myapp", rules) == expected + } + + @Unroll + void "handles rules without account or details values, preferring them to wildcards"() { + given: + List rules = [ + new ClusterMatchRule(account: "*", location: "*", stack: "*", detail: "*", priority: 1), + expected + ] + + expect: + ClusterMatcher.getMatchingRule(account, location, "myapp", rules) == expected + + where: + expected << [ + new ClusterMatchRule(account: "*", location: "*", stack: null, detail: null, priority: 1), + new ClusterMatchRule(account: "*", location: "*", stack: "", detail: "", priority: 1) + ] + } +} diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/utils/TrafficGuardSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/utils/TrafficGuardSpec.groovy index 94f50459c8..7485a9fa97 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/utils/TrafficGuardSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/clouddriver/utils/TrafficGuardSpec.groovy @@ -231,8 +231,9 @@ class TrafficGuardSpec extends Specification { given: addGuard([account: "test", location: "us-east-1", stack: "foo"]) targetServerGroup.instances = [[name: "i-1", healthState: "Up"], [name: "i-2", healthState: "Down"]] + when: - trafficGuard.verifyInstanceTermination(["i-1"], "test", location, "aws", "x") + trafficGuard.verifyInstanceTermination(null, ["i-1"], "test", location, "aws", "x") then: thrown(IllegalStateException) @@ -249,8 +250,9 @@ class TrafficGuardSpec extends Specification { addGuard([account: "test", location: "us-east-1", stack: "foo"]) targetServerGroup.instances = [[name: "i-1", healthState: "Up"], [name: "i-2", healthState: "Down"]] otherServerGroup.instances = [[name: "i-1", healthState: "Down"]] + when: - trafficGuard.verifyInstanceTermination(["i-1"], "test", location, "aws", "x") + trafficGuard.verifyInstanceTermination(null, ["i-1"], "test", location, "aws", "x") then: thrown(IllegalStateException) @@ -267,8 +269,9 @@ class TrafficGuardSpec extends Specification { addGuard([account: "test", location: "us-east-1", stack: "foo"]) targetServerGroup.instances = [[name: "i-1", healthState: "Up"], [name: "i-2", healthState: "Down"]] otherServerGroup.instances = [[name: "i-1", healthState: "Up"]] + when: - trafficGuard.verifyInstanceTermination(["i-1"], "test", location, "aws", "x") + trafficGuard.verifyInstanceTermination(null, ["i-1"], "test", location, "aws", "x") then: notThrown(IllegalStateException) @@ -285,8 +288,9 @@ class TrafficGuardSpec extends Specification { addGuard([account: "test", location: "us-east-1", stack: "foo"]) targetServerGroup.instances = [[name: "i-1", healthState: "Up"], [name: "i-2", healthState: "Up"]] otherServerGroup.instances = [[name: "i-1", healthState: "Down"]] + when: - trafficGuard.verifyInstanceTermination(["i-1", "i-2"], "test", location, "aws", "x") + trafficGuard.verifyInstanceTermination(null, ["i-1", "i-2"], "test", location, "aws", "x") then: thrown(IllegalStateException) @@ -303,8 +307,9 @@ class TrafficGuardSpec extends Specification { given: addGuard([account: "test", location: "us-east-1", stack: "foo"]) targetServerGroup.instances = [[name: "i-1"]] + when: - trafficGuard.verifyInstanceTermination(["i-1"], "test", location, "aws", "x") + trafficGuard.verifyInstanceTermination(null, ["i-1"], "test", location, "aws", "x") then: notThrown(IllegalStateException) @@ -314,6 +319,21 @@ class TrafficGuardSpec extends Specification { 0 * _ } + void "should avoid searching for instance ids when server group provided"() { + given: + addGuard([account: "test", location: "us-east-1", stack: "foo"]) + targetServerGroup.instances = [[name: "i-1"]] + + when: + trafficGuard.verifyInstanceTermination("app-foo-v001", ["i-1"], "test", location, "aws", "x") + + then: + notThrown(IllegalStateException) + 1 * front50Service.get("app") >> application + 1 * oortHelper.getTargetServerGroup("test", "app-foo-v001", location.value, "aws") >> (targetServerGroup as TargetServerGroup) + 0 * _ + } + private void addGuard(Map guard) { applicationDetails.putIfAbsent("trafficGuards", []) applicationDetails.get("trafficGuards") << guard diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/ParallelDeployStageSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/ParallelDeployStageSpec.groovy index ac4e9212b7..101c382840 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/ParallelDeployStageSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/ParallelDeployStageSpec.groovy @@ -45,20 +45,6 @@ class ParallelDeployStageSpec extends Specification { [account: "prod", restrictedExecutionWindow: [:], cluster: [availabilityZones: ["europe-west1-b": []], cloudProvider: "gce"], type: "createServerGroup", name: "Deploy in europe-west1-b"]] } - @Unroll - def "should return stage name regardless of whether parallel flows are present"() { - given: - def stage = new Stage<>(new Pipeline("orca"), "type", stageName, [:]) - - expect: - new ParallelDeployStage().parallelStageName(stage, hasParallelFlows) == expectedStageName - - where: - stageName | hasParallelFlows || expectedStageName - "Default" | false || "Default" - "Default" | true || "Default" - } - Map deployStageContext(String account, String cloudProvider, String... availabilityZones) { def context = ["account": account, restrictedExecutionWindow: [:]] if (availabilityZones.size() == 1) { diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToClusterResizeStrategySpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToClusterResizeStrategySpec.groovy index 430664cd51..aa650e22fb 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToClusterResizeStrategySpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToClusterResizeStrategySpec.groovy @@ -17,7 +17,6 @@ package com.netflix.spinnaker.orca.kato.pipeline.support import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.Location -import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroup import com.netflix.spinnaker.orca.clouddriver.utils.OortHelper import com.netflix.spinnaker.orca.kato.pipeline.support.ResizeStrategy.Capacity import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -28,11 +27,14 @@ import java.util.concurrent.atomic.AtomicInteger class ScaleToClusterResizeStrategySpec extends Specification { - Stage stage = Mock(Stage) + OortHelper oortHelper = Mock(OortHelper) @Subject ScaleToClusterResizeStrategy strategy = new ScaleToClusterResizeStrategy(oortHelper: oortHelper) def 'empty or missing cluster fails on scale_to_cluster'() { + given: + Stage stage = new Stage(null, "stage", [moniker: [app: application, cluster: clusterName]]) + when: strategy.capacityForOperation(stage, account, serverGroupName, cloudProvider, location, resizeConfig) @@ -48,6 +50,9 @@ class ScaleToClusterResizeStrategySpec extends Specification { } def 'capacity is the maximum value of min/max/desired across the cluster for scale_to_cluster'() { + given: + Stage stage = new Stage(null, "stage", [moniker: [app: application, cluster: clusterName]]) + when: def cap = strategy.capacityForOperation(stage, account, serverGroupName, cloudProvider, location, resizeConfig) @@ -69,6 +74,9 @@ class ScaleToClusterResizeStrategySpec extends Specification { } def 'desired capacity is increased by scalePct or scaleNum for scale_to_cluster within the min/max bounds'() { + given: + Stage stage = new Stage(null, "stage", [moniker: [app: application, cluster: clusterName]]) + when: def cap = strategy.capacityForOperation(stage, account, serverGroupName, cloudProvider, location, resizeConfig) diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToServerGroupResizeStrategySpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToServerGroupResizeStrategySpec.groovy index fc8d27af5d..31538397e6 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToServerGroupResizeStrategySpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/ScaleToServerGroupResizeStrategySpec.groovy @@ -29,6 +29,8 @@ class ScaleToServerGroupResizeStrategySpec extends Specification { Stage stage = new Stage<>(new Pipeline("orca"), "Scale", [:]) OortHelper oortHelper = Mock(OortHelper) + def resizeConfig = new ResizeStrategy.OptionalConfiguration() + @Subject def strategy = new ScaleToServerGroupResizeStrategy(oortHelper: oortHelper) @@ -71,6 +73,8 @@ class ScaleToServerGroupResizeStrategySpec extends Specification { @Unroll def "should return source server group capacity"() { given: + resizeConfig.scalePct = scalePct + stage.context = [ source : [ credentials : "test", @@ -78,11 +82,12 @@ class ScaleToServerGroupResizeStrategySpec extends Specification { region : "us-west-1", cloudProvider : "aws" ], - pinMinimumCapacity: pinMinimumCapacity + pinMinimumCapacity: pinMinimumCapacity, + pinCapacity : pinCapacity ] when: - def capacity = strategy.capacityForOperation(stage, "test", "s-v001", "aws", null, null) + def capacity = strategy.capacityForOperation(stage, "test", "s-v001", "aws", null, resizeConfig) then: 1 * oortHelper.getTargetServerGroup("test", "s-v001", "us-west-1", "aws") >> { @@ -95,15 +100,24 @@ class ScaleToServerGroupResizeStrategySpec extends Specification { )) } - capacity.min == expectedMinimumCapacity - capacity.max == 3 - capacity.desired == 3 + capacity.min == expectedMin + capacity.desired == expectedDesired + capacity.max == expectedMax where: - pinMinimumCapacity || expectedMinimumCapacity - null || 1 - false || 1 - true || 3 + scalePct | pinCapacity | pinMinimumCapacity || expectedMin || expectedDesired || expectedMax + null | null | null || 1 || 3 || 3 + null | null | false || 1 || 3 || 3 + null | null | true || 3 || 3 || 3 + 100 | null | null || 1 || 3 || 3 + 50 | null | null || 1 || 2 || 3 + 25 | null | null || 1 || 1 || 3 + 0 | null | null || 0 || 0 || 3 + 0 | false | null || 0 || 0 || 3 + 100 | true | null || 3 || 3 || 3 + 50 | true | null || 2 || 2 || 2 + 25 | true | null || 1 || 1 || 1 + 0 | true | null || 0 || 0 || 0 } @Unroll diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/SourceResolverSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/SourceResolverSpec.groovy index 28f353909d..9c939c51d1 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/SourceResolverSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/pipeline/support/SourceResolverSpec.groovy @@ -17,6 +17,7 @@ package com.netflix.spinnaker.orca.kato.pipeline.support import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spinnaker.orca.RetrySupport import com.netflix.spinnaker.orca.clouddriver.OortService import com.netflix.spinnaker.orca.clouddriver.pipeline.servergroup.support.TargetServerGroupResolver import com.netflix.spinnaker.orca.pipeline.model.Pipeline @@ -94,10 +95,14 @@ class SourceResolverSpec extends Specification { given: OortService oort = Mock(OortService) ObjectMapper mapper = new ObjectMapper() + RetrySupport retrySupport = Spy(RetrySupport) { + _ * sleep(_) >> { /* do nothing */ } + } + SourceResolver resolver = new SourceResolver( oortService: oort, mapper: mapper, - resolver: new TargetServerGroupResolver(oortService: oort, mapper: mapper) + resolver: new TargetServerGroupResolver(oortService: oort, mapper: mapper, retrySupport: retrySupport) ) when: diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/CleanUpTagsTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/CleanUpTagsTaskSpec.groovy index fff1683be8..9c25f48251 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/CleanUpTagsTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/CleanUpTagsTaskSpec.groovy @@ -4,6 +4,7 @@ import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spinnaker.orca.clouddriver.KatoService import com.netflix.spinnaker.orca.clouddriver.OortService import com.netflix.spinnaker.orca.clouddriver.model.TaskId +import com.netflix.spinnaker.orca.clouddriver.utils.MonikerHelper import com.netflix.spinnaker.orca.kato.pipeline.support.SourceResolver import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -70,6 +71,15 @@ class CleanUpTagsTaskSpec extends Specification { List operations = [] task.objectMapper = new ObjectMapper(); + task.monikerHelper = Mock(MonikerHelper) { + 1* getAppNameFromStage(stage, "app-v00") >> { + "app" + } + 1* getClusterNameFromStage(stage, "app-v00") >> { + "app" + } + 0 * _ + } task.oortService = Mock(OortService) { 1* getServerGroupFromCluster("app","test", "app", "app-v00", "us-east-1", "aws") >> { oortResponse diff --git a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/WaitForNewUpInstancesLaunchTaskSpec.groovy b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/WaitForNewUpInstancesLaunchTaskSpec.groovy index cec6990a4b..368b3ae010 100644 --- a/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/WaitForNewUpInstancesLaunchTaskSpec.groovy +++ b/orca-clouddriver/src/test/groovy/com/netflix/spinnaker/orca/kato/tasks/rollingpush/WaitForNewUpInstancesLaunchTaskSpec.groovy @@ -54,7 +54,7 @@ class WaitForNewUpInstancesLaunchTaskSpec extends Specification { def response = task.execute(stage) then: - 1 * oortService.getServerGroupFromCluster(application, account, cluster, serverGroup, region, cloudProvider) >> oortResponse + 1 * oortService.getServerGroup(account, region, serverGroup) >> oortResponse response.status == expectedStatus diff --git a/orca-core/orca-core.gradle b/orca-core/orca-core.gradle index 2f3fa3f18d..f1a90ca485 100644 --- a/orca-core/orca-core.gradle +++ b/orca-core/orca-core.gradle @@ -35,6 +35,7 @@ dependencies { compile spinnaker.dependency('spectatorApi') compile spinnaker.dependency('kork') compile spinnaker.dependency('korkExceptions') + compile spinnaker.dependency('korkArtifacts') compile "com.fasterxml.jackson.datatype:jackson-datatype-jdk8:${spinnaker.version('jackson')}" compile "com.fasterxml.jackson.datatype:jackson-datatype-jsr310:${spinnaker.version('jackson')}" diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/CancellableStage.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/CancellableStage.java index e8c82df1dd..bc30040581 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/CancellableStage.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/CancellableStage.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca; import java.util.Map; diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/ExecutionContext.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/ExecutionContext.java index c392009b1c..0a23065234 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/ExecutionContext.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/ExecutionContext.java @@ -20,12 +20,14 @@ public class ExecutionContext { private static final ThreadLocal threadLocal = new ThreadLocal<>(); private final String application; + private final String authenticatedUser; private final String executionType; private final String executionId; private final String origin; - public ExecutionContext(String application, String executionType, String executionId, String origin) { + public ExecutionContext(String application, String authenticatedUser, String executionType, String executionId, String origin) { this.application = application; + this.authenticatedUser = authenticatedUser; this.executionType = executionType; this.executionId = executionId; this.origin = origin; @@ -47,6 +49,10 @@ public String getApplication() { return application; } + public String getAuthenticatedUser() { + return authenticatedUser; + } + public String getExecutionType() { return executionType; } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/OverridableTimeoutRetryableTask.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/OverridableTimeoutRetryableTask.java new file mode 100644 index 0000000000..c08d7eec7c --- /dev/null +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/OverridableTimeoutRetryableTask.java @@ -0,0 +1,26 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca; + +/** + * A retryable task whose timeout is taken from the top level stage + * if that value has been overridden. + * + * These are typically wait/monitor stages + */ +public interface OverridableTimeoutRetryableTask extends RetryableTask { + +} diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/RetrySupport.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/RetrySupport.java new file mode 100644 index 0000000000..4a75597c18 --- /dev/null +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/RetrySupport.java @@ -0,0 +1,49 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca; + +import java.util.function.Supplier; + +public class RetrySupport { + public T retry(Supplier fn, int maxRetries, long retryBackoff, boolean exponential) { + int retries = 0; + while (true) { + try { + return fn.get(); + } catch (Exception e) { + if (retries >= (maxRetries - 1)) { + throw e; + } + + long timeout = !exponential ? retryBackoff : (long) Math.pow(2, retries) * retryBackoff; + sleep(timeout); + + retries++; + } + } + } + + /** + * Overridable by test cases to avoid Thread.sleep() + */ + void sleep(long duration) { + try { + Thread.sleep(duration); + } catch (InterruptedException ignored) { + } + } +} diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/RetryableTask.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/RetryableTask.java index cfa5aa3cf7..d481b7b6e3 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/RetryableTask.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/RetryableTask.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca; import java.time.Duration; diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/Task.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/Task.java index 0303b4e69a..acabd772c8 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/Task.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/Task.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca; import javax.annotation.Nonnull; diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/TaskResult.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/TaskResult.java index cd84a76bc4..a92d0850cd 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/TaskResult.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/TaskResult.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca; import java.util.Map; diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/config/OrcaConfiguration.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/config/OrcaConfiguration.java index 1995cdde2e..5e48a02dc0 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/config/OrcaConfiguration.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/config/OrcaConfiguration.java @@ -1,14 +1,27 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.config; import java.time.Clock; import java.time.Duration; +import java.util.Collection; import java.util.Map; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.function.BiConsumer; -import java.util.function.Function; import com.fasterxml.jackson.databind.ObjectMapper; -import com.netflix.spectator.api.Id; import com.netflix.spectator.api.Registry; +import com.netflix.spinnaker.orca.RetrySupport; import com.netflix.spinnaker.orca.events.ExecutionEvent; import com.netflix.spinnaker.orca.events.ExecutionListenerAdapter; import com.netflix.spinnaker.orca.exceptions.DefaultExceptionHandler; @@ -19,8 +32,7 @@ import com.netflix.spinnaker.orca.listeners.ExecutionListener; import com.netflix.spinnaker.orca.listeners.MetricsExecutionListener; import com.netflix.spinnaker.orca.notifications.scheduling.SuspendedPipelinesNotificationHandler; -import com.netflix.spinnaker.orca.pipeline.PipelineStartTracker; -import com.netflix.spinnaker.orca.pipeline.PipelineStarterListener; +import com.netflix.spinnaker.orca.pipeline.*; import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository; import com.netflix.spinnaker.orca.pipeline.persistence.PipelineStack; import com.netflix.spinnaker.orca.pipeline.persistence.memory.InMemoryPipelineStack; @@ -39,10 +51,8 @@ import org.springframework.context.annotation.Scope; import org.springframework.core.Ordered; import org.springframework.core.annotation.Order; -import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor; import rx.Scheduler; import rx.schedulers.Schedulers; -import static java.lang.String.format; import static java.time.temporal.ChronoUnit.MINUTES; import static org.springframework.beans.factory.config.BeanDefinition.SCOPE_PROTOTYPE; @@ -51,7 +61,8 @@ "com.netflix.spinnaker.orca.pipeline", "com.netflix.spinnaker.orca.notifications.scheduling", "com.netflix.spinnaker.orca.deprecation", - "com.netflix.spinnaker.orca.pipeline.util" + "com.netflix.spinnaker.orca.pipeline.util", + "com.netflix.spinnaker.orca.telemetry" }) @EnableConfigurationProperties public class OrcaConfiguration { @@ -124,7 +135,8 @@ UserConfiguredUrlRestrictions userConfiguredUrlRestrictions(UserConfiguredUrlRes @Bean public ContextFunctionConfiguration contextFunctionConfiguration(UserConfiguredUrlRestrictions userConfiguredUrlRestrictions, - @Value("${spelEvaluator:v1}") String spelEvaluator) { + @Value("${spelEvaluator:v1}") + String spelEvaluator) { return new ContextFunctionConfiguration(userConfiguredUrlRestrictions, spelEvaluator); } @@ -138,25 +150,15 @@ public ApplicationListener onCompleteMetricExecutionListenerAdap return new ExecutionListenerAdapter(new MetricsExecutionListener(registry), repository); } - // TODO: this is a weird place to have this, feels like it should be a bean configurer or something - public static ThreadPoolTaskExecutor applyThreadPoolMetrics(Registry registry, - ThreadPoolTaskExecutor executor, - String threadPoolName) { - BiConsumer> createGauge = - (name, valueCallback) -> { - Id id = registry - .createId(format("threadpool.%s", name)) - .withTag("id", threadPoolName); - - registry.gauge(id, executor, ref -> valueCallback.apply(ref.getThreadPoolExecutor())); - }; - - createGauge.accept("activeCount", ThreadPoolExecutor::getActiveCount); - createGauge.accept("maximumPoolSize", ThreadPoolExecutor::getMaximumPoolSize); - createGauge.accept("corePoolSize", ThreadPoolExecutor::getCorePoolSize); - createGauge.accept("poolSize", ThreadPoolExecutor::getPoolSize); - createGauge.accept("blockingQueueSize", e -> e.getQueue().size()); + @Bean + @ConditionalOnMissingBean(StageDefinitionBuilderFactory.class) + public StageDefinitionBuilderFactory stageDefinitionBuilderFactory(Collection stageDefinitionBuilders) { + return new DefaultStageDefinitionBuilderFactory(stageDefinitionBuilders); + } - return executor; + @Bean + public RetrySupport retrySupport() { + return new RetrySupport(); } + } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/jackson/OrcaObjectMapper.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/jackson/OrcaObjectMapper.java index a6ac61850b..71db9966df 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/jackson/OrcaObjectMapper.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/jackson/OrcaObjectMapper.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.jackson; import com.fasterxml.jackson.databind.ObjectMapper; @@ -5,6 +20,7 @@ import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; import static com.fasterxml.jackson.annotation.JsonInclude.Include.NON_NULL; import static com.fasterxml.jackson.databind.DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES; +import static com.fasterxml.jackson.databind.SerializationFeature.WRITE_NULL_MAP_VALUES; public class OrcaObjectMapper { private OrcaObjectMapper() {} @@ -14,6 +30,7 @@ public static ObjectMapper newInstance() { instance.registerModule(new Jdk8Module()); instance.registerModule(new GuavaModule()); instance.disable(FAIL_ON_UNKNOWN_PROPERTIES); + instance.disable(WRITE_NULL_MAP_VALUES); instance.setSerializationInclusion(NON_NULL); return instance; } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/ExecutionCleanupListener.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/ExecutionCleanupListener.java index 6105eaa1b5..a6d190aeb1 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/ExecutionCleanupListener.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/ExecutionCleanupListener.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.listeners; import com.netflix.spinnaker.orca.ExecutionStatus; diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/ExecutionListener.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/ExecutionListener.java index 07c5651248..fc9a6196d2 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/ExecutionListener.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/ExecutionListener.java @@ -16,18 +16,20 @@ package com.netflix.spinnaker.orca.listeners; +import javax.annotation.Nonnull; import com.netflix.spinnaker.orca.ExecutionStatus; import com.netflix.spinnaker.orca.pipeline.model.Execution; import org.springframework.core.Ordered; public interface ExecutionListener extends Ordered, Comparable { - default void beforeExecution(Persister persister, Execution execution) { + default void beforeExecution(@Nonnull Persister persister, + @Nonnull Execution execution) { // do nothing } - default void afterExecution(Persister persister, - Execution execution, - ExecutionStatus executionStatus, + default void afterExecution(@Nonnull Persister persister, + @Nonnull Execution execution, + @Nonnull ExecutionStatus executionStatus, boolean wasSuccessful) { // do nothing } @@ -37,7 +39,7 @@ default int getOrder() { } @Override - default int compareTo(ExecutionListener o) { + default int compareTo(@Nonnull ExecutionListener o) { return o.getOrder() - getOrder(); } } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/StageTaskPropagationListener.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/StageTaskPropagationListener.java index 2d80c538f3..558b800c38 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/StageTaskPropagationListener.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/listeners/StageTaskPropagationListener.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.listeners; import java.util.Optional; diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/BranchingStageDefinitionBuilder.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/BranchingStageDefinitionBuilder.java deleted file mode 100644 index 9988200b4f..0000000000 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/BranchingStageDefinitionBuilder.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Copyright 2016 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License") - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.netflix.spinnaker.orca.pipeline; - -import java.util.Collection; -import java.util.Map; -import com.netflix.spinnaker.orca.pipeline.model.Execution; -import com.netflix.spinnaker.orca.pipeline.model.Stage; -import static com.netflix.spinnaker.orca.pipeline.TaskNode.Builder; -import static com.netflix.spinnaker.orca.pipeline.TaskNode.GraphType.HEAD; -import static com.netflix.spinnaker.orca.pipeline.TaskNode.GraphType.TAIL; - -/** - * Implement for stages that will create parallel branches to perform the same - * tasks for multiple contexts. For example, a multi-region bake or deploy. - */ -public interface BranchingStageDefinitionBuilder extends StageDefinitionBuilder { - - /** - * Produce the different contexts for each parallel branch. - */ - > Collection> parallelContexts(Stage stage); - - default TaskNode.TaskGraph buildPreGraph(Stage stage) { - TaskNode.Builder graphBuilder = Builder(HEAD); - preBranchGraph(stage, graphBuilder); - return graphBuilder.build(); - } - - /** - * Define any tasks that should run _before_ the parallel split. - */ - default void preBranchGraph(Stage stage, TaskNode.Builder builder) { - } - - default TaskNode.TaskGraph buildPostGraph(Stage stage) { - Builder graphBuilder = Builder(TAIL); - postBranchGraph(stage, graphBuilder); - return graphBuilder.build(); - } - - /** - * Define any tasks that should run _after_ the parallel split. - */ - default void postBranchGraph(Stage stage, TaskNode.Builder builder) { - } - - /** - * Override this to rename the stage if it has parallel flows. - * This affects the base stage not the individual parallel synthetic stages. - */ - default String parallelStageName(Stage stage, boolean hasParallelFlows) { - return stage.getName(); - } - - /** - * Determines the type of child stage. - */ - default String getChildStageType(Stage childStage) { - return childStage.getType(); - } -} - diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/CheckPreconditionsStage.groovy b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/CheckPreconditionsStage.groovy index b24cdd654d..7de2fbaf74 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/CheckPreconditionsStage.groovy +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/CheckPreconditionsStage.groovy @@ -16,17 +16,22 @@ package com.netflix.spinnaker.orca.pipeline +import javax.annotation.Nonnull import com.netflix.spinnaker.orca.Task import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage import com.netflix.spinnaker.orca.pipeline.tasks.PreconditionTask +import groovy.transform.CompileStatic import org.springframework.beans.factory.annotation.Autowired import org.springframework.stereotype.Component +import static com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFORE +import static java.util.Collections.emptyList @Component -class CheckPreconditionsStage implements BranchingStageDefinitionBuilder { +@CompileStatic +class CheckPreconditionsStage implements StageDefinitionBuilder { - static final String PIPELINE_CONFIG_TYPE = "checkPreconditions" + public static final String PIPELINE_CONFIG_TYPE = "checkPreconditions" private final List preconditionTasks @@ -37,21 +42,38 @@ class CheckPreconditionsStage implements BranchingStageDefinitionBuilder { @Override def > void taskGraph(Stage stage, TaskNode.Builder builder) { - String preconditionType = stage.context.preconditionType - if (!preconditionType) { - throw new IllegalStateException("no preconditionType specified for stage $stage.id") - } - Task preconditionTask = preconditionTasks.find { - it.preconditionType == preconditionType + if (!isTopLevelStage(stage)) { + String preconditionType = stage.context.preconditionType + if (!preconditionType) { + throw new IllegalStateException("no preconditionType specified for stage $stage.id") + } + Task preconditionTask = preconditionTasks.find { + it.preconditionType == preconditionType + } + if (!preconditionTask) { + throw new IllegalStateException("no Precondition implementation for type $preconditionType") + } + builder.withTask("checkPrecondition", preconditionTask.getClass() as Class) } - if (!preconditionTask) { - throw new IllegalStateException("no Precondition implementation for type $preconditionType") + } + + @Nonnull + > List> parallelStages( + @Nonnull Stage stage) { + if (isTopLevelStage(stage)) { + return parallelContexts(stage).collect { context -> + newStage(stage.execution, type, "Check precondition (${context.preconditionType})", context, stage, STAGE_BEFORE) + } + } else { + return emptyList() } - builder.withTask("checkPrecondition", preconditionTask.getClass() as Class) } - @Override - def > Collection> parallelContexts(Stage stage) { + private boolean isTopLevelStage(Stage stage) { + return stage.parentStageId == null + } + + private > Collection> parallelContexts(Stage stage) { stage.resolveStrategyParams() def baseContext = new HashMap(stage.context) List preconditions = baseContext.remove('preconditions') as List @@ -67,7 +89,6 @@ class CheckPreconditionsStage implements BranchingStageDefinitionBuilder { context['context'][it] = context['context'][it] ?: baseContext[it] } - context.name = context.name ?: "Check precondition (${context.preconditionType})".toString() return context } } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/DefaultStageDefinitionBuilderFactory.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/DefaultStageDefinitionBuilderFactory.java new file mode 100644 index 0000000000..7ad81002b3 --- /dev/null +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/DefaultStageDefinitionBuilderFactory.java @@ -0,0 +1,45 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.pipeline; + +import java.util.Arrays; +import java.util.Collection; +import javax.annotation.Nonnull; +import com.netflix.spinnaker.orca.pipeline.ExecutionRunner.NoSuchStageDefinitionBuilder; +import com.netflix.spinnaker.orca.pipeline.model.Stage; + +public class DefaultStageDefinitionBuilderFactory implements StageDefinitionBuilderFactory { + private final Collection stageDefinitionBuilders; + + public DefaultStageDefinitionBuilderFactory(Collection stageDefinitionBuilders) { + this.stageDefinitionBuilders = stageDefinitionBuilders; + } + + public DefaultStageDefinitionBuilderFactory(StageDefinitionBuilder... stageDefinitionBuilders) { + this(Arrays.asList(stageDefinitionBuilders)); + } + + @Override + public @Nonnull StageDefinitionBuilder builderFor( + @Nonnull Stage stage) throws NoSuchStageDefinitionBuilder { + return stageDefinitionBuilders + .stream() + .filter((it) -> it.getType().equals(stage.getType()) || it.getType().equals(stage.getContext().get("alias"))) + .findFirst() + .orElseThrow(() -> new NoSuchStageDefinitionBuilder(stage.getType())); + } +} diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/ExecutionRunner.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/ExecutionRunner.java index d506b08d96..5b4c8220df 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/ExecutionRunner.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/ExecutionRunner.java @@ -29,6 +29,11 @@ default > void restart( throw new UnsupportedOperationException(); } + default > void reschedule( + @Nonnull T execution) throws Exception { + throw new UnsupportedOperationException(); + } + default > void unpause( @Nonnull T execution) throws Exception { throw new UnsupportedOperationException(); diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/NoopStage.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/NoopStage.java new file mode 100644 index 0000000000..5546ee9c74 --- /dev/null +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/NoopStage.java @@ -0,0 +1,29 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.pipeline; + +import com.netflix.spinnaker.orca.pipeline.model.Execution; +import com.netflix.spinnaker.orca.pipeline.model.Stage; +import org.springframework.stereotype.Component; + +@Component +public class NoopStage implements StageDefinitionBuilder { + @Override + public > void taskGraph(Stage stage, TaskNode.Builder builder) { + + } +} diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/PipelineStarterListener.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/PipelineStarterListener.java index a36396e687..fb13744499 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/PipelineStarterListener.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/PipelineStarterListener.java @@ -75,7 +75,10 @@ private void processPipelines(Pipeline execution) { if (execution.getPipelineConfigId() != null) { List queuedPipelines = startTracker.getQueuedPipelines(execution.getPipelineConfigId()); if (!queuedPipelines.isEmpty()) { - String nextPipelineId = queuedPipelines.get(0); + // pipelines are stored in a stack... + // if we are keeping waiting pipelines, take the oldest one; otherwise, take the most recent + int nextIndex = execution.isKeepWaitingPipelines() ? queuedPipelines.size() - 1 : 0; + String nextPipelineId = queuedPipelines.get(nextIndex); queuedPipelines.forEach(id -> { if (Objects.equals(id, nextPipelineId)) { Pipeline queuedExecution = executionRepository.retrievePipeline(id); diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/StageDefinitionBuilder.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/StageDefinitionBuilder.java index 0effba8710..3277acbc5d 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/StageDefinitionBuilder.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/StageDefinitionBuilder.java @@ -20,6 +20,7 @@ import java.util.Map; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import com.netflix.spinnaker.orca.pipeline.TaskNode.TaskGraph; import com.netflix.spinnaker.orca.pipeline.model.Execution; import com.netflix.spinnaker.orca.pipeline.model.Stage; import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner; @@ -29,7 +30,7 @@ public interface StageDefinitionBuilder { - default @Nonnull TaskNode.TaskGraph buildTaskGraph(@Nonnull Stage stage) { + default @Nonnull TaskGraph buildTaskGraph(@Nonnull Stage stage) { Builder graphBuilder = Builder(FULL); taskGraph(stage, graphBuilder); return graphBuilder.build(); @@ -44,6 +45,11 @@ default > void taskGraph( return emptyList(); } + default @Nonnull > List> parallelStages( + @Nonnull Stage stage) { + return emptyList(); + } + /** * @return the stage type this builder handles. */ diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/StageDefinitionBuilderFactory.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/StageDefinitionBuilderFactory.java new file mode 100644 index 0000000000..5409dac4e8 --- /dev/null +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/StageDefinitionBuilderFactory.java @@ -0,0 +1,29 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.pipeline; + +import javax.annotation.Nonnull; +import com.netflix.spinnaker.orca.pipeline.ExecutionRunner.NoSuchStageDefinitionBuilder; +import com.netflix.spinnaker.orca.pipeline.model.Stage; + +@FunctionalInterface +public interface StageDefinitionBuilderFactory { + + @Nonnull StageDefinitionBuilder builderFor( + @Nonnull Stage stage) throws NoSuchStageDefinitionBuilder; + +} diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionEvaluationSummary.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionEvaluationSummary.java index a011fa5a30..88797eb750 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionEvaluationSummary.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionEvaluationSummary.java @@ -69,7 +69,7 @@ public void appendAttempted(String expression) { public String toString() { String attempted = attempts.stream().collect(Collectors.joining(",")); String failed = expressionResult.keySet().stream().collect(Collectors.joining(",")); - return String.format("Evaluated %d expression(s) - (%s), %d failed - (%s)", + return String.format("%d expression(s) - (%s), %d failed - (%s)", getTotalEvaluated(), attempted, getFailureCount(), diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionTransform.groovy b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionTransform.groovy index ce6fb8cefb..f8e88be356 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionTransform.groovy +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionTransform.groovy @@ -61,7 +61,6 @@ class ExpressionTransform { } as T } else if ((source instanceof String || source instanceof GString) && source.toString().contains(parserContext.getExpressionPrefix())) { String literalExpression = source.toString() - log.debug("Processing expression {}", literalExpression) literalExpression = includeExecutionObjectForStageFunctions(literalExpression) T result diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionsSupport.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionsSupport.java index 75d43f58f9..ea918f8679 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionsSupport.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionsSupport.java @@ -252,7 +252,7 @@ static Map readProperties(String text) throws IOException { /** * Finds a Stage by id * @param obj #root.execution - * @param id the name of the stage to find + * @param id the name or id of the stage to find * @return a stage specified by id */ static Object stage(Object obj, String id) { @@ -264,7 +264,7 @@ static Object stage(Object obj, String id) { return stages .stream() - .filter(i -> id != null && id.equals(i.getName())) + .filter(i -> id != null && (id.equals(i.getName()) || id.equals(i.getId()))) .findFirst() .orElseThrow( () -> new SpelHelperFunctionException( diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/PipelineExpressionEvaluator.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/PipelineExpressionEvaluator.java index 270f44fceb..8e5f9bda4b 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/PipelineExpressionEvaluator.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/expressions/PipelineExpressionEvaluator.java @@ -25,10 +25,7 @@ import org.springframework.expression.spel.standard.SpelExpressionParser;; import org.springframework.expression.spel.support.StandardEvaluationContext; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; +import java.util.*; import static com.netflix.spinnaker.orca.pipeline.expressions.PipelineExpressionEvaluator.ExpressionEvaluationVersion.V2; import static com.netflix.spinnaker.orca.pipeline.expressions.PipelineExpressionEvaluator.ExpressionEvaluationVersion.V1; @@ -38,7 +35,7 @@ public class PipelineExpressionEvaluator extends ExpressionsSupport implements E public static final String SUMMARY = "expressionEvaluationSummary"; private static final String SPEL_EVALUATOR = "spelEvaluator"; private final ExpressionParser parser = new SpelExpressionParser(); - private static String spelEvaluator = V1; + private static String spelEvaluator; public static final String ERROR = "Failed Expression Evaluation"; public interface ExpressionEvaluationVersion { @@ -58,51 +55,13 @@ public Map evaluate(Map source, Object rootObjec } public static boolean shouldUseV2Evaluator(Object obj) { - if (V2.equals(spelEvaluator)) { - return true; - } - try { - if (obj instanceof Map) { - Map pipelineConfig = (Map) obj; - //if using v2, add version to stage context. - if (V2.equals(pipelineConfig.get(SPEL_EVALUATOR))) { - updateSpelEvaluatorVersion(pipelineConfig); - return true; - } - - List stages = (List) Optional.ofNullable(pipelineConfig.get("stages")).orElse(Collections.emptyList()); - boolean useV2 = stages - .stream() - .filter(i -> i.containsKey(SPEL_EVALUATOR) && V2.equals(i.get(SPEL_EVALUATOR))) - .findFirst() - .orElse(null) != null; - - if (useV2) { - updateSpelEvaluatorVersion(pipelineConfig); - } - } else if (obj instanceof Pipeline) { - Pipeline pipeline = (Pipeline) obj; - List stages = Optional.ofNullable(pipeline.getStages()).orElse(Collections.emptyList()); - return stages - .stream() - .filter(s -> hasV2InContext(s)) - .findFirst() - .orElse(null) != null; - - } else if (obj instanceof Stage) { - if (hasV2InContext(obj)) { - return true; - } - - Stage stage = (Stage) obj; - // if any using v2 - return stage.getExecution().getStages() - .stream() - .filter(s -> hasV2InContext(s)) - .findFirst() - .orElse(null) != null; + String versionInPipeline = getSpelVersion(obj); + if (Arrays.asList(V1, V2).contains(versionInPipeline) && obj instanceof Map) { + updateSpelEvaluatorVersion((Map) obj, versionInPipeline); } + + return !V1.equals(versionInPipeline) && (V2.equals(spelEvaluator) || V2.equals(versionInPipeline)); } catch (Exception e) { LOGGER.error("Failed to determine whether to use v2 expression evaluator. using V1.", e); } @@ -110,13 +69,57 @@ public static boolean shouldUseV2Evaluator(Object obj) { return false; } - private static boolean hasV2InContext(Object obj) { - return obj instanceof Stage && ((Stage) obj).getContext() != null && V2.equals(((Stage) obj).getContext().get(SPEL_EVALUATOR)); + private static boolean hasVersionInContext(Object obj) { + return obj instanceof Stage && ((Stage) obj).getContext().containsKey(SPEL_EVALUATOR); + } + + private static String getSpelVersion(Object obj) { + if (obj instanceof Map) { + Map pipelineConfig = (Map) obj; + if (pipelineConfig.containsKey(SPEL_EVALUATOR)) { + return (String) pipelineConfig.get(SPEL_EVALUATOR); + } + + List stages = (List) Optional.ofNullable(pipelineConfig.get("stages")).orElse(Collections.emptyList()); + Map stage = stages + .stream() + .filter(i -> i.containsKey(SPEL_EVALUATOR)) + .findFirst() + .orElse(null); + + return (stage != null) ? (String) stage.get(SPEL_EVALUATOR) : null; + } else if (obj instanceof Pipeline) { + Pipeline pipeline = (Pipeline) obj; + Stage stage = pipeline.getStages() + .stream() + .filter(PipelineExpressionEvaluator::hasVersionInContext) + .findFirst() + .orElse(null); + + return (stage != null) ? (String) stage.getContext().get(SPEL_EVALUATOR) : null; + + } else if (obj instanceof Stage) { + Stage stage = (Stage) obj; + if (hasVersionInContext(obj)) { + return (String) stage.getContext().get(SPEL_EVALUATOR); + } + + // if any using v2 + List stages = stage.getExecution().getStages(); + Stage withVersion = (Stage) stages.stream() + .filter(PipelineExpressionEvaluator::hasVersionInContext) + .findFirst() + .orElse(null); + + return (withVersion != null) ? (String) withVersion.getContext().get(SPEL_EVALUATOR) : null; + } + + return null; } - private static void updateSpelEvaluatorVersion(Map rawPipeline) { + private static void updateSpelEvaluatorVersion(Map rawPipeline, String versionInPipeline) { Optional.ofNullable((List) rawPipeline.get("stages")).orElse(Collections.emptyList()) - .forEach(i -> i.put(SPEL_EVALUATOR, V2)); + .forEach(i -> i.put(SPEL_EVALUATOR, versionInPipeline)); } } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/model/Stage.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/model/Stage.java index b44ca6fb3e..6351d4e01b 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/model/Stage.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/model/Stage.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.pipeline.model; import java.io.IOException; @@ -395,9 +410,9 @@ public void resolveStrategyParams() { } /** - * Returns the top-most stage timeout value if present. + * Returns the top-most stage. */ - @JsonIgnore public Optional getTopLevelTimeout() { + @JsonIgnore public Stage getTopLevelStage() { Stage topLevelStage = this; while (topLevelStage.parentStageId != null) { String sid = topLevelStage.parentStageId; @@ -408,6 +423,14 @@ public void resolveStrategyParams() { throw new IllegalStateException("Could not find stage by parentStageId (stage: " + topLevelStage.getId() + ", parentStageId:" + sid + ")"); } } + return topLevelStage; + } + + /** + * Returns the top-most stage timeout value if present. + */ + @JsonIgnore public Optional getTopLevelTimeout() { + Stage topLevelStage = getTopLevelStage(); Object timeout = topLevelStage.getContext().get("stageTimeoutMs"); if (timeout instanceof Integer) { return Optional.of((Integer) timeout).map(Long::new); diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/model/StageContext.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/model/StageContext.java index b6ad880ddc..5ad9972532 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/model/StageContext.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/model/StageContext.java @@ -16,11 +16,15 @@ package com.netflix.spinnaker.orca.pipeline.model; +import com.google.common.collect.ForwardingMap; + +import javax.annotation.Nullable; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; -import javax.annotation.Nullable; -import com.google.common.collect.ForwardingMap; +import java.util.stream.Collectors; public class StageContext extends ForwardingMap { @@ -35,6 +39,15 @@ public StageContext(Stage stage) { return delegate; } + private Map getTrigger() { + Execution execution = stage.getExecution(); + if (execution instanceof Pipeline) { + return ((Pipeline) execution).getTrigger(); + } else { + return Collections.emptyMap(); + } + } + @Override public Object get(@Nullable Object key) { if (delegate().containsKey(key)) { return super.get(key); @@ -49,8 +62,32 @@ public StageContext(Stage stage) { Optional .ofNullable(stage.getExecution()) .map(execution -> execution.getContext().get(key)) - .orElse(null) + .orElse(getTrigger().get(key)) ); } } + + /* + * Gets all objects matching 'key', sorted by proximity to the current stage. + * If the key exists in the current context, it will be the first element returned + */ + public List getAll(Object key) { + List result = stage + .ancestors() + .stream() + .filter(it -> it.getOutputs().containsKey(key)) + .map(it -> it.getOutputs().get(key)) + .collect(Collectors.toList()); + + if (delegate.containsKey(key)) { + result.add(0, delegate.get(key)); + } + + Map trigger = getTrigger(); + if (trigger.containsKey(key)) { + result.add(key); + } + + return result; + } } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/ExecutionNotFoundException.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/ExecutionNotFoundException.java index b96c000263..cb6cd038f2 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/ExecutionNotFoundException.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/ExecutionNotFoundException.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.pipeline.persistence; import com.netflix.spinnaker.kork.web.exceptions.NotFoundException; diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/ExecutionRepository.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/ExecutionRepository.java index 485ea86f9c..31b7106896 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/ExecutionRepository.java +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/ExecutionRepository.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.pipeline.persistence; import com.netflix.spinnaker.orca.ExecutionStatus; diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/jedis/JedisConfiguration.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/jedis/JedisConfiguration.java new file mode 100644 index 0000000000..5c1ef1470c --- /dev/null +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/jedis/JedisConfiguration.java @@ -0,0 +1,60 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.pipeline.persistence.jedis; + +import org.springframework.beans.factory.annotation.Qualifier; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor; +import rx.Scheduler; +import rx.schedulers.Schedulers; + +@Configuration +public class JedisConfiguration { + + @Bean("QueryAll") + public ThreadPoolTaskExecutor queryAll() { + return newFixedThreadPool(10); + } + + @Bean + public Scheduler queryAllScheduler( + @Qualifier("QueryAll") ThreadPoolTaskExecutor executor) { + return Schedulers.from(executor); + } + + @Bean("QueryByApp") + public ThreadPoolTaskExecutor queryByApp( + @Value("${threadPool.executionRepository:150}") int threadPoolSize + ) { + return newFixedThreadPool(threadPoolSize); + } + + @Bean + public Scheduler queryByAppScheduler( + @Qualifier("QueryByApp") ThreadPoolTaskExecutor executor) { + return Schedulers.from(executor); + } + + private static ThreadPoolTaskExecutor newFixedThreadPool(int threadPoolSize) { + ThreadPoolTaskExecutor executor = new ThreadPoolTaskExecutor(); + executor.setCorePoolSize(threadPoolSize); + executor.setMaxPoolSize(threadPoolSize); + return executor; + } +} diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/jedis/JedisExecutionRepository.groovy b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/jedis/JedisExecutionRepository.groovy index eb317f01c0..50d651c5bf 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/jedis/JedisExecutionRepository.groovy +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/persistence/jedis/JedisExecutionRepository.groovy @@ -16,12 +16,12 @@ package com.netflix.spinnaker.orca.pipeline.persistence.jedis +import java.util.concurrent.Executors import java.util.function.Function import com.fasterxml.jackson.core.type.TypeReference import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spectator.api.Registry import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.config.OrcaConfiguration import com.netflix.spinnaker.orca.jackson.OrcaObjectMapper import com.netflix.spinnaker.orca.pipeline.model.* import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionNotFoundException @@ -32,7 +32,6 @@ import groovy.util.logging.Slf4j import org.springframework.beans.factory.annotation.Autowired import org.springframework.beans.factory.annotation.Qualifier import org.springframework.beans.factory.annotation.Value -import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor import org.springframework.stereotype.Component import redis.clients.jedis.Jedis import redis.clients.jedis.Response @@ -70,51 +69,38 @@ class JedisExecutionRepository implements ExecutionRepository { @Autowired JedisExecutionRepository( - Registry registry, - @Qualifier("jedisPool") Pool jedisPool, - @Qualifier("jedisPoolPrevious") Optional> jedisPoolPrevious, - @Value('${threadPool.executionRepository:150}') int threadPoolSize, - @Value('${chunkSize.executionRepository:75}') int threadPoolChunkSize + Registry registry, + @Qualifier("jedisPool") Pool jedisPool, + @Qualifier("jedisPoolPrevious") Optional> jedisPoolPrevious, + @Qualifier("queryAllScheduler") Scheduler queryAllScheduler, + @Qualifier("queryByAppScheduler") Scheduler queryByAppScheduler, + @Value('${chunkSize.executionRepository:75}') int threadPoolChunkSize ) { - this( - jedisPool, - jedisPoolPrevious, - Schedulers.from(newFixedThreadPool(registry, 10, "QueryAll")), - Schedulers.from(newFixedThreadPool(registry, threadPoolSize, "QueryByApp")), - threadPoolChunkSize - ) + this.jedisPool = jedisPool + this.jedisPoolPrevious = jedisPoolPrevious + this.queryAllScheduler = queryAllScheduler + this.queryByAppScheduler = queryByAppScheduler + this.chunkSize = threadPoolChunkSize this.registry = registry } JedisExecutionRepository( Registry registry, Pool jedisPool, + Optional> jedisPoolPrevious, int threadPoolSize, int threadPoolChunkSize ) { this( + registry, jedisPool, - Optional.empty(), - Schedulers.from(newFixedThreadPool(registry, 10, "QueryAll")), - Schedulers.from(newFixedThreadPool(registry, threadPoolSize, "QueryByApp")), + jedisPoolPrevious, + Schedulers.from(Executors.newFixedThreadPool(10)), + Schedulers.from(Executors.newFixedThreadPool(threadPoolSize)), threadPoolChunkSize ) } - JedisExecutionRepository( - Pool jedisPool, - Optional> jedisPoolPrevious, - Scheduler queryAllScheduler, - Scheduler queryByAppScheduler, - int threadPoolChunkSize - ) { - this.jedisPool = jedisPool - this.jedisPoolPrevious = jedisPoolPrevious - this.queryAllScheduler = queryAllScheduler - this.queryByAppScheduler = queryByAppScheduler - this.chunkSize = threadPoolChunkSize - } - @Override void store(Orchestration orchestration) { withJedis(getJedisPoolForId(orchestration.id)) { Jedis jedis -> @@ -825,12 +811,4 @@ class JedisExecutionRepository implements ExecutionRepository { private Collection> allJedis() { return ([jedisPool] + (jedisPoolPrevious.present ? [jedisPoolPrevious.get()] : [])) } - - private static ThreadPoolTaskExecutor newFixedThreadPool(Registry registry, - int threadPoolSize, - String threadPoolName) { - def executor = new ThreadPoolTaskExecutor(maxPoolSize: threadPoolSize, corePoolSize: threadPoolSize) - executor.afterPropertiesSet() - return OrcaConfiguration.applyThreadPoolMetrics(registry, executor, threadPoolName) - } } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/tasks/ExpressionPreconditionTask.groovy b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/tasks/ExpressionPreconditionTask.groovy index d61179c97e..d2ccf15792 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/tasks/ExpressionPreconditionTask.groovy +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/tasks/ExpressionPreconditionTask.groovy @@ -19,6 +19,7 @@ package com.netflix.spinnaker.orca.pipeline.tasks import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.TaskResult +import com.netflix.spinnaker.orca.pipeline.expressions.PipelineExpressionEvaluator import com.netflix.spinnaker.orca.pipeline.model.Stage import com.netflix.spinnaker.orca.pipeline.util.ContextParameterProcessor import org.springframework.beans.factory.annotation.Autowired @@ -49,6 +50,7 @@ class ExpressionPreconditionTask implements PreconditionTask { expression = matcher.group(1) } + ensureEvaluationSummaryIncluded(result, stage, expression) def status = Boolean.valueOf(expression) ? ExecutionStatus.SUCCEEDED : ExecutionStatus.TERMINAL return new TaskResult(status, [ context: new HashMap(stage.context.context as Map) + [ @@ -57,6 +59,13 @@ class ExpressionPreconditionTask implements PreconditionTask { ]) } + private static void ensureEvaluationSummaryIncluded(Map result, Stage stage, String expression) { + if (!expression.trim().startsWith('$') && PipelineExpressionEvaluator.SUMMARY in result) { + stage.context[PipelineExpressionEvaluator.SUMMARY] = PipelineExpressionEvaluator.SUMMARY in stage.context ? + stage.context[PipelineExpressionEvaluator.SUMMARY] + result[PipelineExpressionEvaluator.SUMMARY] : result[PipelineExpressionEvaluator.SUMMARY] + } + } + static class StageData { String expression = "false" } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/tasks/WaitTask.groovy b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/tasks/WaitTask.groovy index 0e3400296a..3e63d38cfc 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/tasks/WaitTask.groovy +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/tasks/WaitTask.groovy @@ -28,8 +28,8 @@ import static com.netflix.spinnaker.orca.ExecutionStatus.SUCCEEDED @Component @CompileStatic class WaitTask implements RetryableTask { - long backoffPeriod = 15000 - long timeout = Integer.MAX_VALUE + final long backoffPeriod = 15000 + final long timeout = Integer.MAX_VALUE TimeProvider timeProvider = new TimeProvider() diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/util/ArtifactResolver.groovy b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/util/ArtifactResolver.groovy new file mode 100644 index 0000000000..931285237d --- /dev/null +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/util/ArtifactResolver.groovy @@ -0,0 +1,67 @@ +/* + * Copyright 2017 Google, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.pipeline.util + +import com.netflix.spinnaker.kork.artifacts.model.Artifact +import com.netflix.spinnaker.kork.artifacts.model.ExpectedArtifact +import groovy.util.logging.Slf4j + +import java.lang.reflect.Field + +@Slf4j +class ArtifactResolver { + + static void resolveArtifacts(Map pipeline) { + Set resolvedArtifacts = [] + List receivedArtifacts = pipeline.receivedArtifacts ?: [] + List expectedArtifacts = pipeline.trigger.expectedArtifacts ?: [] + List unresolvedExpectedArtifacts = [] + + for (ExpectedArtifact expectedArtifact : expectedArtifacts) { + List matches = receivedArtifacts.findAll { a -> expectedArtifact.matches((Artifact) a) } + switch (matches.size()) { + case 0: + unresolvedExpectedArtifacts.add(expectedArtifact) + continue + case 1: + resolvedArtifacts.add(matches[0]) + continue + default: + throw new IllegalStateException("Expected artifact ${expectedArtifact} matches multiple incoming artifacts ${matches}") + } + } + + for (ExpectedArtifact expectedArtifact : unresolvedExpectedArtifacts) { + if (expectedArtifact.usePriorArtifact) { + throw new UnsupportedOperationException("'usePriorArtifact' is not supported yet") + } else if (expectedArtifact.useDefaultArtifact && expectedArtifact.defaultArtifact) { + resolvedArtifacts.add(expectedArtifact.defaultArtifact) + } else { + throw new IllegalStateException("Unmatched expected artifact ${expectedArtifact} with no fallback behavior specified") + } + } + + pipeline.trigger.artifacts = resolvedArtifacts as List + } + + static class ArtifactResolutionException extends RuntimeException { + ArtifactResolutionException(String message) { + super(message) + } + + } +} diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/util/ContextParameterProcessor.groovy b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/util/ContextParameterProcessor.groovy index 1116da264b..51887d8fb3 100644 --- a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/util/ContextParameterProcessor.groovy +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/pipeline/util/ContextParameterProcessor.groovy @@ -90,8 +90,8 @@ class ContextParameterProcessor { allowUnknownKeys, ) - if (summary.totalEvaluated > 0) { - log.debug(summary.toString()) + if (summary.totalEvaluated > 0 && context.execution) { + log.info("Evaluated {} in execution {}", summary, context.execution.id) } if (summary.failureCount > 0) { @@ -103,7 +103,6 @@ class ContextParameterProcessor { Map process(Map parameters, Map context, boolean allowUnknownKeys) { if (PipelineExpressionEvaluator.shouldUseV2Evaluator(parameters) || PipelineExpressionEvaluator.shouldUseV2Evaluator(context)) { - log.debug("Using V2 expression evaluation") return processV2(parameters, context, allowUnknownKeys) } diff --git a/orca-core/src/main/groovy/com/netflix/spinnaker/orca/telemetry/ThreadPoolMetricsPostProcessor.java b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/telemetry/ThreadPoolMetricsPostProcessor.java new file mode 100644 index 0000000000..8aa1eedca0 --- /dev/null +++ b/orca-core/src/main/groovy/com/netflix/spinnaker/orca/telemetry/ThreadPoolMetricsPostProcessor.java @@ -0,0 +1,68 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.telemetry; + +import java.util.concurrent.ThreadPoolExecutor; +import java.util.function.BiConsumer; +import java.util.function.Function; +import com.netflix.spectator.api.Id; +import com.netflix.spectator.api.Registry; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.config.BeanPostProcessor; +import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor; +import org.springframework.stereotype.Component; +import static java.lang.String.format; + +@Component +public class ThreadPoolMetricsPostProcessor implements BeanPostProcessor { + + private final Registry registry; + + @Autowired + public ThreadPoolMetricsPostProcessor(Registry registry) {this.registry = registry;} + + @Override + public Object postProcessBeforeInitialization(Object bean, String beanName) { + if (bean instanceof ThreadPoolTaskExecutor) { + applyThreadPoolMetrics((ThreadPoolTaskExecutor) bean, beanName); + } + return bean; + } + + @Override + public Object postProcessAfterInitialization(Object bean, String beanName) { + return bean; + } + + private void applyThreadPoolMetrics(ThreadPoolTaskExecutor executor, + String threadPoolName) { + BiConsumer> createGauge = + (name, fn) -> { + Id id = registry + .createId(format("threadpool.%s", name)) + .withTag("id", threadPoolName); + + registry.gauge(id, executor, ref -> fn.apply(ref.getThreadPoolExecutor())); + }; + + createGauge.accept("activeCount", ThreadPoolExecutor::getActiveCount); + createGauge.accept("maximumPoolSize", ThreadPoolExecutor::getMaximumPoolSize); + createGauge.accept("corePoolSize", ThreadPoolExecutor::getCorePoolSize); + createGauge.accept("poolSize", ThreadPoolExecutor::getPoolSize); + createGauge.accept("blockingQueueSize", e -> e.getQueue().size()); + } +} diff --git a/orca-core/src/test/groovy/com/netflix/spinnaker/orca/RetrySupportSpec.groovy b/orca-core/src/test/groovy/com/netflix/spinnaker/orca/RetrySupportSpec.groovy new file mode 100644 index 0000000000..d6d63ba954 --- /dev/null +++ b/orca-core/src/test/groovy/com/netflix/spinnaker/orca/RetrySupportSpec.groovy @@ -0,0 +1,79 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca + +import spock.lang.Specification +import spock.lang.Unroll + +class RetrySupportSpec extends Specification { + @Unroll + def "should retry until success or #maxRetries attempts is reached"() { + given: + def retrySupport = Spy(RetrySupport) { + Math.min(maxRetries - 1, failures) * sleep(10000) >> { /* do nothing */ } + } + + int attemptCounter = 0; + + when: + def exceptionMessage + try { + retrySupport.retry({ + if (attemptCounter++ < failures) { + throw new IllegalStateException("Failed after " + attemptCounter + " attempts"); + } + }, maxRetries, 10000, false) + } catch (Exception e) { + exceptionMessage = e.message + } + + then: + attemptCounter == expectedAttempts + exceptionMessage == expectedExceptionMessage + + where: + failures || maxRetries || expectedAttempts || expectedExceptionMessage + 3 || 10 || 4 || null + 11 || 10 || 10 || "Failed after 10 attempts" + } + + def "should sleep exponentially"() { + given: + def retrySupport = Spy(RetrySupport) { + 1 * sleep(10000) >> { /* do nothing */ } + 1 * sleep(20000) >> { /* do nothing */ } + 1 * sleep(40000) >> { /* do nothing */ } + 1 * sleep(80000) >> { /* do nothing */ } + } + + int attemptCounter = 0; + + when: + retrySupport.retry({ + if (attemptCounter++ < failures) { + throw new IllegalStateException("Failed after " + attemptCounter + " attempts"); + } + }, maxRetries, 10000, true) + + then: + attemptCounter == expectedAttempts + + where: + failures || maxRetries || expectedAttempts + 4 || 10 || 5 + } +} diff --git a/orca-core/src/test/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionsSupportSpec.groovy b/orca-core/src/test/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionsSupportSpec.groovy new file mode 100644 index 0000000000..c7c3ed5caf --- /dev/null +++ b/orca-core/src/test/groovy/com/netflix/spinnaker/orca/pipeline/expressions/ExpressionsSupportSpec.groovy @@ -0,0 +1,73 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.pipeline.expressions + +import com.netflix.spinnaker.orca.ExecutionStatus +import spock.lang.Shared +import spock.lang.Specification +import spock.lang.Unroll + +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.pipeline +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.stage; + +class ExpressionsSupportSpec extends Specification { + @Shared + def pipeline = pipeline { + stage { + id = "1" + name = "My First Stage" + context = [ + "region": "us-east-1", + ] + } + + stage { + id = "2" + name = "My Second Stage" + context = [ + "region": "us-west-1", + ] + } + } + + @Unroll + def "stage() should match on #matchedAttribute"() { + expect: + ExpressionsSupport.stage(pipeline, stageCriteria).context.region == expectedRegion + + where: + stageCriteria || matchedAttribute || expectedRegion + "My Second Stage" || "name" || "us-west-1" + "1" || "id" || "us-east-1" + "2" || "id" || "us-west-1" + } + + def "stage() should raise exception if stage not found"() { + when: + ExpressionsSupport.stage(pipeline, "does_not_exist") + + then: + thrown(SpelHelperFunctionException) + + when: + ExpressionsSupport.stage("not_an_expression", "does_not_matter") + + then: + // raise exception when not passed an Execution + thrown(SpelHelperFunctionException) + } +} diff --git a/orca-core/src/test/groovy/com/netflix/spinnaker/orca/pipeline/expressions/PipelineExpressionEvaluatorSpec.groovy b/orca-core/src/test/groovy/com/netflix/spinnaker/orca/pipeline/expressions/PipelineExpressionEvaluatorSpec.groovy new file mode 100644 index 0000000000..43b5d8fecb --- /dev/null +++ b/orca-core/src/test/groovy/com/netflix/spinnaker/orca/pipeline/expressions/PipelineExpressionEvaluatorSpec.groovy @@ -0,0 +1,90 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.pipeline.expressions + +import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.model.Stage +import spock.lang.Specification +import spock.lang.Unroll + +import static com.netflix.spinnaker.orca.pipeline.expressions.PipelineExpressionEvaluator.ExpressionEvaluationVersion.* + +class PipelineExpressionEvaluatorSpec extends Specification { + + @Unroll + def "should determine Evaluator version from pipeline map"() { + expect: + PipelineExpressionEvaluator.shouldUseV2Evaluator(pipeline) == useV2 + + where: + pipeline | useV2 + [:] | false + [spelEvaluator: V1] | false + [spelEvaluator: V2] | true + [stages: [ [spelEvaluator: V2]]] | true + } + + @Unroll + def "should update Evaluator version if v2 used anywhere"() { + when: + PipelineExpressionEvaluator.shouldUseV2Evaluator(pipeline) + + then: + def stages = pipeline.stages + (stages != null && stages[0].containsKey("spelEvaluator")) == hasVersion + + where: + pipeline | hasVersion + [:] | false + [spelEvaluator: V2, stages: [[:]]] | true + [spelEvaluator: V2 ] | false + [spelEvaluator: V1, stages: [[:]]] | true + } + + @Unroll + def "should be able to get version from a stage"() { + given: + def stage = new Stage<>(new Pipeline("orca"), "type", stageContext) + + expect: + PipelineExpressionEvaluator.shouldUseV2Evaluator(stage) == useV2 + + where: + stageContext | useV2 + [:] | false + [spelEvaluator: V2] | true + [spelEvaluator: V2] | true + [spelEvaluator: V1] | false + } + + @Unroll + def "should be able to override global version per at pipeline level"() { + given: "global is set to v2" + PipelineExpressionEvaluator.spelEvaluator = V2 + + expect: + PipelineExpressionEvaluator.shouldUseV2Evaluator(pipeline) == useV2 + + where: + pipeline | useV2 + [:] | true + [spelEvaluator: V1] | false + [spelEvaluator: null] | true + [stages: [ [spelEvaluator: V2]]] | true + [stages: [ [spelEvaluator: V1]]] | false + } +} diff --git a/orca-queue-sqs/orca-queue-sqs.gradle b/orca-dry-run/orca-dry-run.gradle similarity index 74% rename from orca-queue-sqs/orca-queue-sqs.gradle rename to orca-dry-run/orca-dry-run.gradle index c30706e64f..40ebaa5a01 100644 --- a/orca-queue-sqs/orca-queue-sqs.gradle +++ b/orca-dry-run/orca-dry-run.gradle @@ -22,10 +22,9 @@ repositories { } dependencies { - compile project(":orca-queue") - compile "com.fasterxml.jackson.module:jackson-module-kotlin:${spinnaker.version("jackson")}" + compile project(":orca-core") - compile spinnaker.dependency("clouddriverAws") - - testCompile project(path: ":orca-queue", configuration: "testOutput") + testCompile "org.springframework.boot:spring-boot-test:${spinnaker.version('springBoot')}" + testCompile project(":orca-test") + testCompile project(path: ":orca-queue", configuration: "testArtifacts") } diff --git a/orca-dry-run/src/main/java/com/netflix/spinnaker/orca/dryrun/RoundingDoubleSerializer.java b/orca-dry-run/src/main/java/com/netflix/spinnaker/orca/dryrun/RoundingDoubleSerializer.java new file mode 100644 index 0000000000..371f59a8af --- /dev/null +++ b/orca-dry-run/src/main/java/com/netflix/spinnaker/orca/dryrun/RoundingDoubleSerializer.java @@ -0,0 +1,47 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.dryrun; + +import java.io.IOException; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; + +/** + * Serializes a double as an integer if possible. + *

+ * Written in Java as Kotlin's Double doesn't work with Jackson at runtime. + */ +public class RoundingDoubleSerializer extends JsonSerializer { + @Override public Class handledType() { + return Double.class; + } + + @Override + public void serialize( + Double value, + JsonGenerator gen, + SerializerProvider serializers + ) throws IOException, JsonProcessingException { + if (value % 1 == 0d) { + gen.writeNumber(value.intValue()); + } else { + gen.writeNumber(value); + } + } +} diff --git a/orca-dry-run/src/main/java/com/netflix/spinnaker/orca/dryrun/RoundingFloatSerializer.java b/orca-dry-run/src/main/java/com/netflix/spinnaker/orca/dryrun/RoundingFloatSerializer.java new file mode 100644 index 0000000000..f1ff49433b --- /dev/null +++ b/orca-dry-run/src/main/java/com/netflix/spinnaker/orca/dryrun/RoundingFloatSerializer.java @@ -0,0 +1,47 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.dryrun; + +import java.io.IOException; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; + +/** + * Serializes a float as an integer if possible. + *

+ * Written in Java as Kotlin's Float doesn't work with Jackson at runtime. + */ +public class RoundingFloatSerializer extends JsonSerializer { + @Override public Class handledType() { + return Float.class; + } + + @Override + public void serialize( + Float value, + JsonGenerator gen, + SerializerProvider serializers + ) throws IOException, JsonProcessingException { + if (value % 1 == 0f) { + gen.writeNumber(value.intValue()); + } else { + gen.writeNumber(value); + } + } +} diff --git a/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/config/DryRunConfiguration.kt b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/config/DryRunConfiguration.kt new file mode 100644 index 0000000000..be1b245d56 --- /dev/null +++ b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/config/DryRunConfiguration.kt @@ -0,0 +1,43 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.config + +import com.netflix.spinnaker.orca.dryrun.DryRunStageDefinitionBuilderFactory +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilderFactory +import org.slf4j.LoggerFactory +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty +import org.springframework.boot.context.properties.EnableConfigurationProperties +import org.springframework.context.annotation.Bean +import org.springframework.context.annotation.ComponentScan +import org.springframework.context.annotation.Configuration + +@Configuration +@ConditionalOnProperty("dryrun.enabled") +@ComponentScan("com.netflix.spinnaker.orca.dryrun") +@EnableConfigurationProperties(DryRunProperties::class) +open class DryRunConfiguration { + @Bean + open fun dryRunStageDefinitionBuilderFactory( + stageDefinitionBuilders: Collection + ): StageDefinitionBuilderFactory { + log.info("Dry run trigger support enabled") + return DryRunStageDefinitionBuilderFactory(stageDefinitionBuilders) + } + + private val log = LoggerFactory.getLogger(javaClass) +} diff --git a/orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/config/SqsProperties.kt b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/config/DryRunProperties.kt similarity index 77% rename from orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/config/SqsProperties.kt rename to orca-dry-run/src/main/kotlin/com/netflix/spinnaker/config/DryRunProperties.kt index 9e1c32480c..c22ba52d17 100644 --- a/orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/config/SqsProperties.kt +++ b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/config/DryRunProperties.kt @@ -1,11 +1,11 @@ /* * Copyright 2017 Netflix, Inc. * - * Licensed under the Apache License, Version 2.0 (the "License"); + * Licensed under the Apache License, Version 2.0 (the "License") * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,12 +13,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.netflix.spinnaker.config import org.springframework.boot.context.properties.ConfigurationProperties -@ConfigurationProperties("queue.sqs") -class SqsProperties { - var queueName: String = "orca-task-queue" - var region: String = "us-west-2" +@ConfigurationProperties("dryrun") +class DryRunProperties { + val pipelineIds: MutableList = mutableListOf() } diff --git a/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStage.kt b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStage.kt new file mode 100644 index 0000000000..689f57911d --- /dev/null +++ b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStage.kt @@ -0,0 +1,42 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.dryrun + +import com.netflix.spinnaker.orca.Task +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.TaskNode.Builder +import com.netflix.spinnaker.orca.pipeline.model.Execution +import com.netflix.spinnaker.orca.pipeline.model.Stage +import kotlin.reflect.KClass + +class DryRunStage(private val delegate: StageDefinitionBuilder) : StageDefinitionBuilder { + + override fun > taskGraph(stage: Stage, builder: Builder) { + builder.withTask("dry run", DryRunTask::class) + } + + override fun > aroundStages(stage: Stage): List> + = delegate.aroundStages(stage) + + override fun > parallelStages(stage: Stage): List> + = delegate.parallelStages(stage) + + override fun getType() = delegate.type + + private fun Builder.withTask(name: String, type: KClass) = + withTask(name, type.java) +} diff --git a/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStageDefinitionBuilderFactory.kt b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStageDefinitionBuilderFactory.kt new file mode 100644 index 0000000000..6f235f72da --- /dev/null +++ b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStageDefinitionBuilderFactory.kt @@ -0,0 +1,56 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.dryrun + +import com.netflix.spinnaker.orca.pipeline.CheckPreconditionsStage +import com.netflix.spinnaker.orca.pipeline.DefaultStageDefinitionBuilderFactory +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.model.Stage + +class DryRunStageDefinitionBuilderFactory( + stageDefinitionBuilders: Collection +) : DefaultStageDefinitionBuilderFactory(stageDefinitionBuilders) { + + override fun builderFor(stage: Stage<*>): StageDefinitionBuilder = + stage.getExecution().let { execution -> + super.builderFor(stage).let { + if (stage.isExpressionPreconditionStage()) { + it + } else if (execution is Pipeline && execution.trigger["type"] == "dryrun") { + DryRunStage(it) + } else { + it + } + } + } + + private fun Stage<*>.isExpressionPreconditionStage() = + isPreconditionStage() && (isExpressionChild() || isExpressionParent()) + + private fun Stage<*>.isPreconditionStage() = + getType() == CheckPreconditionsStage.PIPELINE_CONFIG_TYPE + + private fun Stage<*>.isExpressionChild() = + getContext()["preconditionType"] == "expression" + + @Suppress("UNCHECKED_CAST") + private fun Stage<*>.isExpressionParent() = + (getContext()["preconditions"] as Iterable>?)?.run { + all { it["type"] == "expression" } + } == true +} diff --git a/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunTask.kt b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunTask.kt new file mode 100644 index 0000000000..fd6d1ebd99 --- /dev/null +++ b/orca-dry-run/src/main/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunTask.kt @@ -0,0 +1,131 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.dryrun + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.databind.module.SimpleModule +import com.netflix.spinnaker.orca.ExecutionStatus +import com.netflix.spinnaker.orca.ExecutionStatus.SKIPPED +import com.netflix.spinnaker.orca.ExecutionStatus.TERMINAL +import com.netflix.spinnaker.orca.Task +import com.netflix.spinnaker.orca.TaskResult +import com.netflix.spinnaker.orca.jackson.OrcaObjectMapper +import com.netflix.spinnaker.orca.pipeline.model.Execution +import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.model.Stage +import org.slf4j.LoggerFactory +import org.springframework.stereotype.Component + +@Component +class DryRunTask : Task { + private val blacklistKeyPatterns = + setOf( + "amiSuffix", + "kato\\..*", + "spelEvaluator", + "stageDetails", + "useSourceCapacity" + ) + .map(String::toRegex) + + override fun execute(stage: Stage>): TaskResult = + stage + .getExecution() + .let { execution -> + when (execution) { + is Pipeline -> { + realStage(execution, stage) + .let { realStage -> + stage.evaluateStage(realStage) + } + } + else -> { + log.error("Dry run is only supported for pipelines") + TaskResult(TERMINAL) + } + } + } + + private fun Stage>.evaluateStage(realStage: Stage): TaskResult { + val dryRunResult = mutableMapOf() + var status: ExecutionStatus? = null + + val diff = getContext().removeNulls().diffKeys(realStage.context.removeNulls()) + + if (diff.isNotEmpty()) { + dryRunResult["context"] = diff + .mapValues { (key, value) -> + "Expected \"${realStage.context[key]}\" but found \"$value\"." + } + status = TERMINAL + } + + if (realStage.status == SKIPPED) { + dryRunResult["errors"] = listOf("Expected stage to be skipped.") + status = TERMINAL + } + + return TaskResult( + status ?: realStage.status, + realStage.context + diff, + realStage.outputs + if (dryRunResult.isNotEmpty()) mapOf("dryRunResult" to dryRunResult) else emptyMap() + ) + } + + private fun Map.removeNulls() = + // this is a quick way to remove nested null values + mapper.writeValueAsString(this).let { json -> + mapper.readValue>(json) + } + + private inline fun ObjectMapper.readValue(src: String): T = readValue(src, T::class.java) + + private fun Map.diffKeys(other: Map): Map = + filterKeys { key -> + blacklistKeyPatterns.none(key::matches) + } + .filter { (key, value) -> + value != other[key] + } + + private fun realStage(execution: Pipeline, stage: Stage>): Stage { + return execution + .trigger["lastSuccessfulExecution"] + .let { realPipeline -> + when (realPipeline) { + is Pipeline -> realPipeline + is Map<*, *> -> mapper.convertValue(realPipeline) + else -> throw IllegalStateException("No triggering pipeline execution found") + } + } + .stageByRef(stage.getRefId()) + } + + private val mapper = OrcaObjectMapper + .newInstance() + .apply { + SimpleModule() + .addSerializer(RoundingFloatSerializer()) + .addSerializer(RoundingDoubleSerializer()) + .let(this::registerModule) + } + + private val log = LoggerFactory.getLogger(javaClass) + + private inline fun ObjectMapper.convertValue(fromValue: Any): T = + convertValue(fromValue, T::class.java) +} diff --git a/orca-dry-run/src/test/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStageTest.kt b/orca-dry-run/src/test/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStageTest.kt new file mode 100644 index 0000000000..51f6381037 --- /dev/null +++ b/orca-dry-run/src/test/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunStageTest.kt @@ -0,0 +1,117 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.dryrun + +import com.natpryce.hamkrest.greaterThan +import com.natpryce.hamkrest.should.shouldMatch +import com.netflix.spinnaker.orca.q.* +import com.netflix.spinnaker.orca.q.handler.plan +import com.netflix.spinnaker.spek.shouldEqual +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on + +object DryRunStageTest : Spek({ + + setOf(zeroTaskStage, singleTaskStage, multiTaskStage, stageWithSyntheticBefore, stageWithSyntheticBeforeAndNoTasks).forEach { proxiedStage -> + describe("building tasks for a ${proxiedStage.type}") { + val pipeline = pipeline { + stage { + refId = "1" + type = proxiedStage.type + } + } + + val subject = DryRunStage(proxiedStage) + + on("planning the stage") { + subject.plan(pipeline.stageByRef("1")) + } + + it("constructs a single task") { + pipeline.stageByRef("1").tasks.let { + it.size shouldEqual 1 + it.first().implementingClass shouldEqual DryRunTask::class.qualifiedName + } + } + } + } + + setOf(zeroTaskStage, singleTaskStage, multiTaskStage).forEach { proxiedStage -> + describe("building synthetic stages for a ${proxiedStage.type}") { + val pipeline = pipeline { + stage { + refId = "1" + type = proxiedStage.type + } + } + + val subject = DryRunStage(proxiedStage) + + on("planning the stage") { + subject.plan(pipeline.stageByRef("1")) + } + + it("does not build any synthetic stages") { + pipeline.stages.size shouldEqual 1 + } + } + } + + setOf(stageWithSyntheticBefore, stageWithSyntheticBeforeAndNoTasks).forEach { proxiedStage -> + describe("building synthetic stages for a ${proxiedStage.type}") { + val pipeline = pipeline { + stage { + refId = "1" + type = proxiedStage.type + } + } + + val subject = DryRunStage(proxiedStage) + + on("planning the stage") { + subject.plan(pipeline.stageByRef("1")) + } + + it("builds the usual synthetic stages") { + pipeline.stages.size shouldMatch greaterThan(1) + } + } + } + + setOf(stageWithParallelBranches).forEach { proxiedStage -> + describe("building parallel stages for a ${proxiedStage.type}") { + val pipeline = pipeline { + stage { + refId = "1" + type = proxiedStage.type + } + } + + val subject = DryRunStage(proxiedStage) + + on("planning the stage") { + subject.plan(pipeline.stageByRef("1")) + } + + it("builds the usual parallel stages") { + pipeline.stages.size shouldMatch greaterThan(1) + } + } + } +}) diff --git a/orca-dry-run/src/test/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunTaskTest.kt b/orca-dry-run/src/test/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunTaskTest.kt new file mode 100644 index 0000000000..9711938b27 --- /dev/null +++ b/orca-dry-run/src/test/kotlin/com/netflix/spinnaker/orca/dryrun/DryRunTaskTest.kt @@ -0,0 +1,265 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.dryrun + +import com.netflix.spinnaker.orca.ExecutionStatus.* +import com.netflix.spinnaker.orca.q.pipeline +import com.netflix.spinnaker.orca.q.singleTaskStage +import com.netflix.spinnaker.orca.q.stage +import com.netflix.spinnaker.spek.shouldEqual +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on + +object DryRunTaskTest : Spek({ + + val subject = DryRunTask() + + describe("running the task") { + given("a stage that will evaluate successfully") { + val realPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["something"] = "covfefe" + context["fromConfig"] = "covfefe" + outputs["result"] = "covfefe" + status = SUCCEEDED + } + } + val realStage = realPipeline.stageByRef("1") + + val testPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["fromConfig"] = "covfefe" + } + trigger["lastSuccessfulExecution"] = realPipeline + } + val stage = testPipeline.stageByRef("1") + + on("running the stage") { + val result = subject.execute(stage) + + it("returns the same status as the real stage") { + result.status shouldEqual realStage.status + } + + it("duplicates the outputs of the real stage") { + result.outputs shouldEqual realStage.outputs + } + + it("replicates the stage context after execution") { + result.context shouldEqual realStage.context + } + } + } + + given("a mismatch in context values") { + val realPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["something"] = "covfefe" + status = SUCCEEDED + } + } + + val testPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["something"] = "dotard" + } + trigger["lastSuccessfulExecution"] = realPipeline + } + val stage = testPipeline.stageByRef("1") + + on("running the stage") { + val result = subject.execute(stage) + + it("fails") { + result.status shouldEqual TERMINAL + } + + it("puts the new value in the context") { + result.context["something"] shouldEqual stage.context["something"] + } + + it("adds an error to the stage outputs") { + result.outputs["dryRunResult"] shouldEqual mapOf( + "context" to mapOf( + "something" to "Expected \"covfefe\" but found \"dotard\"." + ) + ) + } + } + } + + given("a mismatch in context values in a blacklisted key") { + val realPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["amiSuffix"] = "1234" + status = SUCCEEDED + } + } + + val testPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["amiSuffix"] = "5678" + } + trigger["lastSuccessfulExecution"] = realPipeline + } + val stage = testPipeline.stageByRef("1") + + on("running the stage") { + val result = subject.execute(stage) + + it("succeeds") { + result.status shouldEqual SUCCEEDED + } + } + } + + given("a mismatch in context values in a key matching a blacklisted pattern") { + val realPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["kato.whatever"] = "1234" + status = SUCCEEDED + } + } + + val testPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["kato.whatever"] = "5678" + } + trigger["lastSuccessfulExecution"] = realPipeline + } + val stage = testPipeline.stageByRef("1") + + on("running the stage") { + val result = subject.execute(stage) + + it("succeeds") { + result.status shouldEqual SUCCEEDED + } + } + } + + given("a mismatch between a null and a missing key") { + val realPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["whatever"] = mapOf("foo" to "bar", "baz" to null) + status = SUCCEEDED + } + } + + val testPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["whatever"] = mapOf("foo" to "bar") + } + trigger["lastSuccessfulExecution"] = realPipeline + } + val stage = testPipeline.stageByRef("1") + + on("running the stage") { + val result = subject.execute(stage) + + it("succeeds") { + result.status shouldEqual SUCCEEDED + } + } + } + + given("a mismatch between a deeply nested null and a missing key") { + val realPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["whatever"] = listOf(mapOf("foo" to "bar", "baz" to null)) + status = SUCCEEDED + } + } + + val testPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + context["whatever"] = listOf(mapOf("foo" to "bar")) + } + trigger["lastSuccessfulExecution"] = realPipeline + } + val stage = testPipeline.stageByRef("1") + + on("running the stage") { + val result = subject.execute(stage) + + it("succeeds") { + result.status shouldEqual SUCCEEDED + } + } + } + + given("a stage that was skipped previously") { + val realPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + status = SKIPPED + } + } + + val testPipeline = pipeline { + stage { + refId = "1" + type = singleTaskStage.type + } + trigger["lastSuccessfulExecution"] = realPipeline + } + val stage = testPipeline.stageByRef("1") + + on("running the stage") { + val result = subject.execute(stage) + + it("fails") { + result.status shouldEqual TERMINAL + } + + it("adds an error to the stage outputs") { + result.outputs["dryRunResult"] shouldEqual mapOf( + "errors" to listOf("Expected stage to be skipped.") + ) + } + } + } + } +}) diff --git a/orca-echo/orca-echo.gradle b/orca-echo/orca-echo.gradle index 387b8b4c2f..0134f9b316 100644 --- a/orca-echo/orca-echo.gradle +++ b/orca-echo/orca-echo.gradle @@ -20,6 +20,7 @@ dependencies { compile project(":orca-retrofit") compile project(":orca-igor") compile project(":orca-front50") + compile project(":orca-dry-run") compile spinnaker.dependency('kork') compile spinnaker.dependency('bootAutoConfigure') compile "net.lariverosc:jesque-spring:1.0.1" diff --git a/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/config/EchoConfiguration.groovy b/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/config/EchoConfiguration.groovy index 8b9aa43a87..f08e6f427d 100644 --- a/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/config/EchoConfiguration.groovy +++ b/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/config/EchoConfiguration.groovy @@ -17,6 +17,7 @@ package com.netflix.spinnaker.orca.echo.config import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spinnaker.config.DryRunProperties import com.netflix.spinnaker.orca.echo.EchoService import com.netflix.spinnaker.orca.echo.spring.EchoNotifyingExecutionListener import com.netflix.spinnaker.orca.echo.spring.EchoNotifyingStageListener @@ -83,11 +84,22 @@ class EchoConfiguration { } @Bean - EchoNotifyingExecutionListener echoNotifyingPipelineExecutionListener(EchoService echoService, - Front50Service front50Service, - ObjectMapper objectMapper, - ContextParameterProcessor contextParameterProcessor) { - new EchoNotifyingExecutionListener(echoService, front50Service, objectMapper, contextParameterProcessor) + EchoNotifyingExecutionListener echoNotifyingPipelineExecutionListener( + EchoService echoService, + Front50Service front50Service, + ObjectMapper objectMapper, + ContextParameterProcessor contextParameterProcessor, + Optional dryRunProperties) { + new EchoNotifyingExecutionListener( + echoService, + front50Service, + objectMapper, + contextParameterProcessor, + dryRunProperties + .map { it.pipelineIds } + .orElse([]) + .toSet() + ) } @Bean diff --git a/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/pipeline/ManualJudgmentStage.groovy b/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/pipeline/ManualJudgmentStage.groovy index 2ba04dee1c..a17922a3c4 100644 --- a/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/pipeline/ManualJudgmentStage.groovy +++ b/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/pipeline/ManualJudgmentStage.groovy @@ -16,6 +16,7 @@ package com.netflix.spinnaker.orca.echo.pipeline +import java.time.Duration import java.util.concurrent.TimeUnit import com.google.common.annotations.VisibleForTesting import com.netflix.spinnaker.orca.* @@ -62,8 +63,8 @@ class ManualJudgmentStage implements StageDefinitionBuilder, RestartableStage, A @Component @VisibleForTesting public static class WaitForManualJudgmentTask implements RetryableTask { - long backoffPeriod = 15000 - long timeout = TimeUnit.DAYS.toMillis(3) + final long backoffPeriod = 15000 + final long timeout = TimeUnit.DAYS.toMillis(3) @Autowired(required = false) EchoService echoService @@ -71,20 +72,20 @@ class ManualJudgmentStage implements StageDefinitionBuilder, RestartableStage, A @Override TaskResult execute(Stage stage) { StageData stageData = stage.mapTo(StageData) - NotificationState notificationState + String notificationState ExecutionStatus executionStatus switch (stageData.state) { case StageData.State.CONTINUE: - notificationState = NotificationState.manualJudgmentContinue + notificationState = "manualJudgmentContinue" executionStatus = ExecutionStatus.SUCCEEDED break case StageData.State.STOP: - notificationState = NotificationState.manualJudgmentStop + notificationState = "manualJudgmentStop" executionStatus = ExecutionStatus.TERMINAL break default: - notificationState = NotificationState.manualJudgment + notificationState = "manualJudgment" executionStatus = ExecutionStatus.RUNNING break } @@ -94,12 +95,12 @@ class ManualJudgmentStage implements StageDefinitionBuilder, RestartableStage, A return new TaskResult(executionStatus, outputs) } - Map processNotifications(Stage stage, StageData stageData, NotificationState notificationState) { + Map processNotifications(Stage stage, StageData stageData, String notificationState) { if (echoService) { // sendNotifications will be true if using the new scheme for configuration notifications. // The new scheme matches the scheme used by the other stages. // If the deprecated scheme is in use, only the original 'awaiting judgment' notification is supported. - if (notificationState != NotificationState.manualJudgment && !stage.context.sendNotifications) { + if (notificationState != "manualJudgment" && !stage.context.sendNotifications) { return [:] } @@ -118,12 +119,6 @@ class ManualJudgmentStage implements StageDefinitionBuilder, RestartableStage, A } } - static enum NotificationState { - manualJudgment, - manualJudgmentContinue, - manualJudgmentStop - } - static class StageData { String judgmentStatus = "" List notifications = [] @@ -151,13 +146,13 @@ class ManualJudgmentStage implements StageDefinitionBuilder, RestartableStage, A String address String cc String type - List when - Map message + List when + Map message - Map lastNotifiedByNotificationState = [:] + Map lastNotifiedByNotificationState = [:] Long notifyEveryMs = -1 - boolean shouldNotify(NotificationState notificationState, Date now = new Date()) { + boolean shouldNotify(String notificationState, Date now = new Date()) { // The new scheme for configuring notifications requires the use of the when list (just like the other stages). // If this list is present, but does not contain an entry for this particular notification state, do not notify. if (when && !when.contains(notificationState)) { @@ -177,7 +172,7 @@ class ManualJudgmentStage implements StageDefinitionBuilder, RestartableStage, A return new Date(lastNotified.time + notifyEveryMs) <= now } - void notify(EchoService echoService, Stage stage, NotificationState notificationState) { + void notify(EchoService echoService, Stage stage, String notificationState) { echoService.create(new EchoService.Notification( notificationType: EchoService.Notification.Type.valueOf(type.toUpperCase()), to: address ? [address] : null, diff --git a/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingExecutionListener.groovy b/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingExecutionListener.groovy index 9baf891f80..321b071aeb 100644 --- a/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingExecutionListener.groovy +++ b/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingExecutionListener.groovy @@ -1,32 +1,51 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.echo.spring import com.fasterxml.jackson.databind.ObjectMapper -import com.netflix.spinnaker.orca.front50.Front50Service - -import com.netflix.spinnaker.orca.front50.model.ApplicationNotifications -import com.netflix.spinnaker.orca.pipeline.util.ContextParameterProcessor -import groovy.transform.CompileStatic -import groovy.util.logging.Slf4j import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.echo.EchoService +import com.netflix.spinnaker.orca.front50.Front50Service +import com.netflix.spinnaker.orca.front50.model.ApplicationNotifications import com.netflix.spinnaker.orca.listeners.ExecutionListener import com.netflix.spinnaker.orca.listeners.Persister import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.util.ContextParameterProcessor +import groovy.transform.CompileDynamic +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j @Slf4j @CompileStatic class EchoNotifyingExecutionListener implements ExecutionListener { private final EchoService echoService - private final Front50Service front50Service - private final ObjectMapper objectMapper - private final ContextParameterProcessor contextParameterProcessor + private final Set dryRunPipelineIds - EchoNotifyingExecutionListener(EchoService echoService, Front50Service front50Service, ObjectMapper objectMapper, ContextParameterProcessor contextParameterProcessor) { + EchoNotifyingExecutionListener( + EchoService echoService, + Front50Service front50Service, + ObjectMapper objectMapper, + ContextParameterProcessor contextParameterProcessor, + Set dryRunPipelineIds) { + this.dryRunPipelineIds = dryRunPipelineIds this.echoService = echoService this.front50Service = front50Service this.objectMapper = objectMapper @@ -43,7 +62,7 @@ class EchoNotifyingExecutionListener implements ExecutionListener { echoService.recordEvent( details: [ source : "orca", - type : "orca:${execution.getClass().simpleName.toLowerCase()}:starting", + type : "orca:${execution.getClass().simpleName.toLowerCase()}:starting".toString(), application: execution.application, ], content: [ @@ -66,6 +85,7 @@ class EchoNotifyingExecutionListener implements ExecutionListener { if (execution.status != ExecutionStatus.SUSPENDED) { if (execution instanceof Pipeline) { addApplicationNotifications(execution as Pipeline) + addDryRunNotifications(execution as Pipeline) } echoService.recordEvent( details: [ @@ -117,4 +137,17 @@ class EchoNotifyingExecutionListener implements ExecutionListener { } } } + + /** + * Adds a notification for dry run enabled pipelines. + * @param pipeline + */ + @CompileDynamic + private void addDryRunNotifications(Pipeline pipeline) { + if (pipeline.pipelineConfigId in dryRunPipelineIds) { + log.info("Sending dry run notification for $pipeline.application $pipeline.name") + pipeline.notifications << [type: "dryrun", when: "pipeline.complete"] + } + } + } diff --git a/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingStageListener.groovy b/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingStageListener.groovy index df211bd376..17582c3ead 100644 --- a/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingStageListener.groovy +++ b/orca-echo/src/main/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingStageListener.groovy @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.echo.spring import com.netflix.spinnaker.orca.ExecutionStatus diff --git a/orca-echo/src/test/groovy/com/netflix/spinnaker/orca/echo/pipeline/ManualJudgmentStageSpec.groovy b/orca-echo/src/test/groovy/com/netflix/spinnaker/orca/echo/pipeline/ManualJudgmentStageSpec.groovy index 310fa25b40..3685e1ac04 100644 --- a/orca-echo/src/test/groovy/com/netflix/spinnaker/orca/echo/pipeline/ManualJudgmentStageSpec.groovy +++ b/orca-echo/src/test/groovy/com/netflix/spinnaker/orca/echo/pipeline/ManualJudgmentStageSpec.groovy @@ -62,7 +62,7 @@ class ManualJudgmentStageSpec extends Specification { then: result.status == ExecutionStatus.RUNNING result.context.notifications.findAll { - it.lastNotifiedByNotificationState[NotificationState.manualJudgment] + it.lastNotifiedByNotificationState["manualJudgment"] }*.type == ["email", "hipchat", "sms"] } @@ -85,13 +85,13 @@ class ManualJudgmentStageSpec extends Specification { if (sent) result.context.notifications?.getAt(0)?.lastNotifiedByNotificationState?.containsKey(notificationState) where: - sendNotifications | notificationState | judgmentStatus | executionStatus || sent - true | NotificationState.manualJudgment | null | ExecutionStatus.RUNNING || true - false | NotificationState.manualJudgment | null | ExecutionStatus.RUNNING || true - true | NotificationState.manualJudgmentContinue | "continue" | ExecutionStatus.SUCCEEDED || true - false | NotificationState.manualJudgmentContinue | "continue" | ExecutionStatus.SUCCEEDED || false - true | NotificationState.manualJudgmentStop | "stop" | ExecutionStatus.TERMINAL || true - false | NotificationState.manualJudgmentStop | "stop" | ExecutionStatus.TERMINAL || false + sendNotifications | notificationState | judgmentStatus | executionStatus || sent + true | "manualJudgment" | null | ExecutionStatus.RUNNING || true + false | "manualJudgment" | null | ExecutionStatus.RUNNING || true + true | "manualJudgmentContinue" | "continue" | ExecutionStatus.SUCCEEDED || true + false | "manualJudgmentContinue" | "continue" | ExecutionStatus.SUCCEEDED || false + true | "manualJudgmentStop" | "stop" | ExecutionStatus.TERMINAL || true + false | "manualJudgmentStop" | "stop" | ExecutionStatus.TERMINAL || false } @Unroll @@ -100,18 +100,18 @@ class ManualJudgmentStageSpec extends Specification { notification.shouldNotify(notificationState, now) == shouldNotify where: - notification | notificationState | now || shouldNotify - new Notification() | NotificationState.manualJudgment | new Date() || true + notification | notificationState | now || shouldNotify + new Notification() | "manualJudgment" | new Date() || true new Notification( - lastNotifiedByNotificationState: [(NotificationState.manualJudgment): new Date(1)]) | NotificationState.manualJudgment | new Date() || false + lastNotifiedByNotificationState: [("manualJudgment"): new Date(1)]) | "manualJudgment" | new Date() || false new Notification( - lastNotifiedByNotificationState: [(NotificationState.manualJudgment): new Date(1)]) | NotificationState.manualJudgmentContinue | new Date() || true + lastNotifiedByNotificationState: [("manualJudgment"): new Date(1)]) | "manualJudgmentContinue" | new Date() || true new Notification( - lastNotifiedByNotificationState: [(NotificationState.manualJudgment): new Date(1), - (NotificationState.manualJudgmentContinue): new Date(1)]) | NotificationState.manualJudgmentContinue | new Date() || false + lastNotifiedByNotificationState: [("manualJudgment"): new Date(1), + ("manualJudgmentContinue"): new Date(1)]) | "manualJudgmentContinue" | new Date() || false new Notification( - lastNotifiedByNotificationState: [(NotificationState.manualJudgment): new Date(1)], - notifyEveryMs: 60000) | NotificationState.manualJudgment | new Date(60001) || true + lastNotifiedByNotificationState: [("manualJudgment"): new Date(1)], + notifyEveryMs: 60000) | "manualJudgment" | new Date(60001) || true } @Unroll @@ -144,11 +144,7 @@ class ManualJudgmentStageSpec extends Specification { 0 * _ where: - notificationState << [ - NotificationState.manualJudgment, - NotificationState.manualJudgmentContinue, - NotificationState.manualJudgmentStop - ] + notificationState << [ "manualJudgment", "manualJudgmentContinue", "manualJudgmentStop" ] } @Unroll diff --git a/orca-echo/src/test/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingExecutionListenerSpec.groovy b/orca-echo/src/test/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingExecutionListenerSpec.groovy index ae630f7ecc..5d0106a149 100644 --- a/orca-echo/src/test/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingExecutionListenerSpec.groovy +++ b/orca-echo/src/test/groovy/com/netflix/spinnaker/orca/echo/spring/EchoNotifyingExecutionListenerSpec.groovy @@ -27,6 +27,7 @@ import com.netflix.spinnaker.orca.pipeline.util.ContextParameterProcessor import spock.lang.Shared import spock.lang.Specification import spock.lang.Subject +import static java.util.Collections.emptySet class EchoNotifyingExecutionListenerSpec extends Specification { @@ -37,7 +38,7 @@ class EchoNotifyingExecutionListenerSpec extends Specification { @Shared ContextParameterProcessor contextParameterProcessor = new ContextParameterProcessor() @Subject - def echoListener = new EchoNotifyingExecutionListener(echoService, front50Service, objectMapper, contextParameterProcessor) + def echoListener = new EchoNotifyingExecutionListener(echoService, front50Service, objectMapper, contextParameterProcessor, emptySet()) @Shared ApplicationNotifications notifications = new ApplicationNotifications() diff --git a/orca-flex/orca-flex.gradle b/orca-flex/orca-flex.gradle index 6e2696e207..1c872d9ccb 100644 --- a/orca-flex/orca-flex.gradle +++ b/orca-flex/orca-flex.gradle @@ -16,6 +16,7 @@ dependencies { compile spinnaker.dependency('frigga') + compile 'com.netflix.spinnaker.moniker:moniker:0.2.0' compile project(":orca-retrofit") testCompile project(":orca-test") } diff --git a/orca-flex/src/main/groovy/com/netflix/spinnaker/orca/flex/tasks/AbstractElasticIpTask.groovy b/orca-flex/src/main/groovy/com/netflix/spinnaker/orca/flex/tasks/AbstractElasticIpTask.groovy index f094df73a9..a81d42c666 100644 --- a/orca-flex/src/main/groovy/com/netflix/spinnaker/orca/flex/tasks/AbstractElasticIpTask.groovy +++ b/orca-flex/src/main/groovy/com/netflix/spinnaker/orca/flex/tasks/AbstractElasticIpTask.groovy @@ -17,6 +17,7 @@ package com.netflix.spinnaker.orca.flex.tasks import com.netflix.frigga.Names +import com.netflix.spinnaker.moniker.Moniker import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.Task import com.netflix.spinnaker.orca.TaskResult @@ -49,11 +50,12 @@ abstract class AbstractElasticIpTask implements Task { static class StageData { String account String cluster + Moniker moniker String region ElasticIpRequest elasticIp String getApplication() { - return cluster ? Names.parseName(cluster).app : null + return moniker ? moniker.getApp() : cluster ? Names.parseName(cluster).app : null } } } diff --git a/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/migrations/MultiRedisOrchestrationMigrationNotificationAgent.groovy b/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/migrations/MultiRedisOrchestrationMigrationNotificationAgent.groovy index 4ec61b5ada..955755fb37 100644 --- a/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/migrations/MultiRedisOrchestrationMigrationNotificationAgent.groovy +++ b/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/migrations/MultiRedisOrchestrationMigrationNotificationAgent.groovy @@ -17,6 +17,9 @@ package com.netflix.spinnaker.orca.front50.migrations +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit +import java.util.function.Function import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spectator.api.Registry import com.netflix.spinnaker.orca.ExecutionStatus @@ -39,9 +42,6 @@ import redis.clients.util.Pool import rx.Observable import rx.schedulers.Schedulers -import java.util.concurrent.TimeUnit -import java.util.function.Function - @Slf4j @Component @ConditionalOnExpression(value = '${pollers.multiRedisOrchestrationMigration.enabled:false}') @@ -68,9 +68,9 @@ class MultiRedisOrchestrationMigrationNotificationAgent extends AbstractPollingN this.jedisPool = jedisPool this.jedisPoolPrevious = jedisPoolPrevious - def queryAllScheduler = Schedulers.from(JedisExecutionRepository.newFixedThreadPool(registry, 1, "QueryAll")) - def queryByAppScheduler = Schedulers.from(JedisExecutionRepository.newFixedThreadPool(registry, 1, "QueryByApp")) - this.executionRepositoryPrevious = new JedisExecutionRepository(jedisPoolPrevious, Optional.empty(), queryAllScheduler, queryByAppScheduler, 75) + def queryAllScheduler = Schedulers.from(Executors.newFixedThreadPool(1)) + def queryByAppScheduler = Schedulers.from(Executors.newFixedThreadPool(1)) + this.executionRepositoryPrevious = new JedisExecutionRepository(registry, jedisPoolPrevious, Optional.empty(), queryAllScheduler, queryByAppScheduler, 75) } @Override @@ -110,13 +110,20 @@ class MultiRedisOrchestrationMigrationNotificationAgent extends AbstractPollingN allApplications.eachWithIndex { Application application, int index -> def applicationName = application.name.toLowerCase() - def migratableOrchestrations = executionRepositoryPrevious + def unmigratedOrchestrations = executionRepositoryPrevious .retrieveOrchestrationsForApplication(applicationName, executionCriteria) - .filter({ orchestration -> orchestration.status.isComplete() && !previouslyMigratedOrchestrationIds.contains(orchestration.id) }) + .filter({ orchestration -> !previouslyMigratedOrchestrationIds.contains(orchestration.id) }) .toList() .toBlocking() .single() + def migratableOrchestrations = unmigratedOrchestrations.findAll { it.status.isComplete() } + def pendingOrchestrations = unmigratedOrchestrations.findAll { !it.status.isComplete() } + + if (!pendingOrchestrations.isEmpty()) { + log.info("${pendingOrchestrations.size()} orchestrations yet to complete ${applicationName}) [${index}/${allApplications.size()}]") + } + if (migratableOrchestrations.isEmpty()) { return } diff --git a/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/migrations/MultiRedisPipelineMigrationNotificationAgent.groovy b/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/migrations/MultiRedisPipelineMigrationNotificationAgent.groovy index 736b18b322..62cfd5781a 100644 --- a/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/migrations/MultiRedisPipelineMigrationNotificationAgent.groovy +++ b/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/migrations/MultiRedisPipelineMigrationNotificationAgent.groovy @@ -14,9 +14,10 @@ * limitations under the License. */ - package com.netflix.spinnaker.orca.front50.migrations +import java.util.concurrent.TimeUnit +import java.util.function.Function import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spectator.api.Registry import com.netflix.spinnaker.orca.ExecutionStatus @@ -36,10 +37,7 @@ import org.springframework.stereotype.Component import redis.clients.jedis.Jedis import redis.clients.util.Pool import rx.Observable -import rx.schedulers.Schedulers - -import java.util.concurrent.TimeUnit -import java.util.function.Function +import rx.Scheduler @Slf4j @Component @@ -56,20 +54,22 @@ class MultiRedisPipelineMigrationNotificationAgent extends AbstractPollingNotifi long pollingIntervalMs @Autowired - MultiRedisPipelineMigrationNotificationAgent(ObjectMapper objectMapper, - Client jesqueClient, - Registry registry, - @Qualifier("jedisPool") Pool jedisPool, - @Qualifier("jedisPoolPrevious") Pool jedisPoolPrevious, - Front50Service front50Service) { + MultiRedisPipelineMigrationNotificationAgent( + ObjectMapper objectMapper, + Client jesqueClient, + Registry registry, + @Qualifier("jedisPool") Pool jedisPool, + @Qualifier("jedisPoolPrevious") Pool jedisPoolPrevious, + @Qualifier("queryAllScheduler") Scheduler queryAllScheduler, + @Qualifier("queryByAppScheduler") Scheduler queryByAppScheduler, + Front50Service front50Service + ) { super(objectMapper, jesqueClient) this.jedisPool = jedisPool this.jedisPoolPrevious = jedisPoolPrevious this.front50Service = front50Service - def queryAllScheduler = Schedulers.from(JedisExecutionRepository.newFixedThreadPool(registry, 1, "QueryAll")) - def queryByAppScheduler = Schedulers.from(JedisExecutionRepository.newFixedThreadPool(registry, 1, "QueryByApp")) - this.executionRepositoryPrevious = new JedisExecutionRepository(jedisPoolPrevious, Optional.empty(), queryAllScheduler, queryByAppScheduler, 75) + this.executionRepositoryPrevious = new JedisExecutionRepository(registry, jedisPoolPrevious, Optional.empty(), queryAllScheduler, queryByAppScheduler, 75) } @Override @@ -99,19 +99,28 @@ class MultiRedisPipelineMigrationNotificationAgent extends AbstractPollingNotifi } def executionCriteria = new ExecutionRepository.ExecutionCriteria(limit: 50) - executionCriteria.statuses = ExecutionStatus.values().findAll { it.complete }.collect { it.name() } + executionCriteria.statuses = ExecutionStatus.values().findAll { + it.complete + }.collect { it.name() } def allPipelineConfigIds = front50Service.allPipelines*.id + front50Service.allStrategies*.id log.info("Found ${allPipelineConfigIds.size()} pipeline configs") allPipelineConfigIds.eachWithIndex { String pipelineConfigId, int index -> - def migratablePipelines = executionRepositoryPrevious + def unmigratedPipelines = executionRepositoryPrevious .retrievePipelinesForPipelineConfigId(pipelineConfigId, executionCriteria) - .filter({ pipeline -> pipeline.status.isComplete() && !previouslyMigratedPipelineIds.contains(pipeline.id) }) + .filter({ pipeline -> !previouslyMigratedPipelineIds.contains(pipeline.id) }) .toList() .toBlocking() .single() + def migratablePipelines = unmigratedPipelines.findAll { it.status.isComplete() } + def pendingPipelines = unmigratedPipelines.findAll { !it.status.isComplete() } + + if (!pendingPipelines.isEmpty()) { + log.info("${pendingPipelines.size()} pipelines yet to complete (${pipelineConfigId}) [${index}/${allPipelineConfigIds.size()}]") + } + if (migratablePipelines.isEmpty()) { return } @@ -144,7 +153,6 @@ class MultiRedisPipelineMigrationNotificationAgent extends AbstractPollingNotifi log.info("${migratablePipelines.size()} pipelines migrated (${pipelineConfigId}) [${index}/${allPipelineConfigIds.size()}]") } - } private T withJedis(Pool jedisPool, Function action) { diff --git a/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/pipeline/PipelineStage.java b/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/pipeline/PipelineStage.java index 2d704c00c6..5ffdad22cd 100644 --- a/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/pipeline/PipelineStage.java +++ b/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/pipeline/PipelineStage.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.front50.pipeline; import java.util.List; diff --git a/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/tasks/MonitorPipelineTask.groovy b/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/tasks/MonitorPipelineTask.groovy index 344b8764f6..a0dbc5fc1d 100644 --- a/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/tasks/MonitorPipelineTask.groovy +++ b/orca-front50/src/main/groovy/com/netflix/spinnaker/orca/front50/tasks/MonitorPipelineTask.groovy @@ -18,7 +18,7 @@ package com.netflix.spinnaker.orca.front50.tasks import java.util.concurrent.TimeUnit import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -29,7 +29,7 @@ import org.springframework.stereotype.Component @Slf4j @Component -class MonitorPipelineTask implements RetryableTask { +class MonitorPipelineTask implements OverridableTimeoutRetryableTask { @Autowired ExecutionRepository executionRepository diff --git a/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/pipeline/TravisStage.groovy b/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/pipeline/TravisStage.groovy index b770bec4d9..72e03ca80a 100644 --- a/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/pipeline/TravisStage.groovy +++ b/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/pipeline/TravisStage.groovy @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.igor.pipeline import org.springframework.stereotype.Component diff --git a/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/GetCommitsTask.groovy b/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/GetCommitsTask.groovy index 5195fea5e2..2a8c92104c 100644 --- a/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/GetCommitsTask.groovy +++ b/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/GetCommitsTask.groovy @@ -92,6 +92,13 @@ class GetCommitsTask implements DiffTask { //figure out the new asg/ami/commit String targetAmi = getTargetAmi(stage.context, region) + if (!targetAmi) { + def parentPipeline = stage.execution.trigger?.parentExecution + while (!targetAmi && parentPipeline?.context) { + targetAmi = getTargetAmi(parentPipeline.context, region) + parentPipeline = parentPipeline.trigger?.parentExecution + } + } //get commits from igor sourceInfo = resolveInfoFromAmi(ancestorAmi, account, region) @@ -203,7 +210,7 @@ class GetCommitsTask implements DiffTask { def globalAccount = front50Service.credentials.find { it.global } def applicationAccount = globalAccount?.name ?: account Application app = front50Service.get(application) - return [repoType : app?.details().repoType, projectKey : app?.details().repoProjectKey, repositorySlug : app?.details().repoSlug] + return [repoType : app?.details()?.repoType, projectKey : app?.details()?.repoProjectKey, repositorySlug : app?.details()?.repoSlug] } String getBuildFromAppVersion(String appVersion) { diff --git a/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorJenkinsJobTask.groovy b/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorJenkinsJobTask.groovy index 7e24dbfec0..2c170f731d 100644 --- a/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorJenkinsJobTask.groovy +++ b/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorJenkinsJobTask.groovy @@ -18,7 +18,7 @@ package com.netflix.spinnaker.orca.igor.tasks import java.util.concurrent.TimeUnit import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.igor.BuildArtifactFilter import com.netflix.spinnaker.orca.igor.BuildService @@ -30,7 +30,7 @@ import retrofit.RetrofitError @Slf4j @Component -class MonitorJenkinsJobTask implements RetryableTask { +class MonitorJenkinsJobTask implements OverridableTimeoutRetryableTask { long backoffPeriod = 10000 long timeout = TimeUnit.HOURS.toMillis(2) @@ -79,7 +79,7 @@ class MonitorJenkinsJobTask implements RetryableTask { if (stage.context.propertyFile) { properties = buildService.getPropertyFile(buildNumber, stage.context.propertyFile, master, job) if (properties.size() == 0 && result == 'SUCCESS') { - throw new IllegalStateException("expected properties file ${stage.context.propertyFile} but one was not found or was empty") + throw new IllegalStateException("Expected properties file ${stage.context.propertyFile} but it was either missing, empty or contained invalid syntax") } outputs << properties outputs.propertyFileContents = properties diff --git a/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorQueuedJenkinsJobTask.groovy b/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorQueuedJenkinsJobTask.groovy index 245b1ae4b9..773e5254fa 100644 --- a/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorQueuedJenkinsJobTask.groovy +++ b/orca-igor/src/main/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorQueuedJenkinsJobTask.groovy @@ -18,7 +18,7 @@ package com.netflix.spinnaker.orca.igor.tasks import java.util.concurrent.TimeUnit import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.igor.BuildService import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -29,7 +29,7 @@ import retrofit.RetrofitError @Slf4j @Component -class MonitorQueuedJenkinsJobTask implements RetryableTask { +class MonitorQueuedJenkinsJobTask implements OverridableTimeoutRetryableTask { long backoffPeriod = 10000 long timeout = TimeUnit.HOURS.toMillis(2) diff --git a/orca-igor/src/test/groovy/com/netflix/spinnaker/orca/igor/tasks/GetCommitsTaskSpec.groovy b/orca-igor/src/test/groovy/com/netflix/spinnaker/orca/igor/tasks/GetCommitsTaskSpec.groovy index 7e648ab6ca..93145c9f0a 100644 --- a/orca-igor/src/test/groovy/com/netflix/spinnaker/orca/igor/tasks/GetCommitsTaskSpec.groovy +++ b/orca-igor/src/test/groovy/com/netflix/spinnaker/orca/igor/tasks/GetCommitsTaskSpec.groovy @@ -207,7 +207,7 @@ class GetCommitsTaskSpec extends Specification { jobState = 'SUCCESS' } - Stage setupGetCommits(Map contextMap, String account, String app, String sourceImage, String targetImage, String region, String cluster, String serverGroup, int serverGroupCalls = 1, int oortCalls = 1) { + Stage setupGetCommits(Map contextMap, String account, String app, String sourceImage, String targetImage, String region, String cluster, String serverGroup, int serverGroupCalls = 1, int oortCalls = 1, Pipeline pipeline = this.pipeline) { def stage = new Stage<>(pipeline, "stash", contextMap) task.buildService = Stub(BuildService) { @@ -629,4 +629,46 @@ class GetCommitsTaskSpec extends Specification { ancestorBuild | targetBuild "216" | "217" } + + @Unroll + def "get commits from parent pipelines"() { + given: + Pipeline parentPipeline = new Pipeline("parentPipeline") + Pipeline childPipeline = new Pipeline("childPipeline") + String katoTasks = "[{\"resultObjects\": [" + + "{\"ancestorServerGroupNameByRegion\": { \"${region}\":\"${serverGroup}\"}}," + + "{\"messages\" : [ ], \"serverGroupNameByRegion\": {\"${region}\": \"${targetServerGroup}\"},\"serverGroupNames\": [\"${region}:${targetServerGroup}\"]}],\"status\": {\"completed\": true,\"failed\": false}}]" + def katoMap = getObjectMapper().readValue(katoTasks, List) + def contextMap = [application: app, account: account, + source : [asgName: serverGroup, region: region, account: account], "deploy.server.groups": ["us-west-1": [targetServerGroup]], "kato.tasks" : katoMap] + + parentPipeline.context = [deploymentDetails: [[imageId: "ami-foo", ami: "amiFooName", region: "us-east-1"], [imageId: targetImage, ami: targetImageName, region: region]]] + Stage parentStage = setupGetCommits(contextMap, account, app, sourceImage, targetImage, region, cluster, serverGroup, 1, 1, parentPipeline) + Stage childStage = new Stage<>(childPipeline, "stash", [application: app, account: account, source: [asgName: serverGroup, region: region, account: account], "deploy.server.groups": ["us-west-1": [targetServerGroup]], "kato.tasks" : katoMap]) + + parentPipeline.stages << parentStage + childStage.execution.trigger.put("parentPipelineId", parentStage.execution.id) + childStage.execution.trigger.put("parentExecution", parentStage.execution) + + when: + def result = task.execute(childStage) + + then: + assertResults(result, ExecutionStatus.SUCCEEDED) + + where: + app = "myapp" + account = "test" + region = "us-west-1" + sourceImage = "ami-source" + targetImage = "ami-target" + targetImageName = "amiTargetName" + jobState = 'SUCCESS' + + cluster | serverGroup | targetServerGroup + "myapp" | "myapp" | "myapp-v000" + "myapp" | "myapp-v001" | "myapp-v002" + "myapp-stack" | "myapp-stack-v002" | "myapp-stack-v003" + "myapp-stack-detail" | "myapp-stack-detail-v002" | "myapp-stack-detail-v003" + } } diff --git a/orca-igor/src/test/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorJenkinsJobTaskSpec.groovy b/orca-igor/src/test/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorJenkinsJobTaskSpec.groovy index 5ca70571c1..77719e6d7a 100644 --- a/orca-igor/src/test/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorJenkinsJobTaskSpec.groovy +++ b/orca-igor/src/test/groovy/com/netflix/spinnaker/orca/igor/tasks/MonitorJenkinsJobTaskSpec.groovy @@ -175,7 +175,7 @@ class MonitorJenkinsJobTaskSpec extends Specification { then: IllegalStateException e = thrown IllegalStateException - e.message == 'expected properties file noexist.properties but one was not found or was empty' + e.message == 'Expected properties file noexist.properties but it was either missing, empty or contained invalid syntax' } def "marks 'unstable' results as successful if explicitly configured to do so"() { diff --git a/orca-kayenta/src/main/groovy/com/netflix/spinnaker/orca/kayenta/tasks/MonitorCanaryTask.groovy b/orca-kayenta/src/main/groovy/com/netflix/spinnaker/orca/kayenta/tasks/MonitorCanaryTask.groovy index c8ae4b4a33..1d00b0d76e 100644 --- a/orca-kayenta/src/main/groovy/com/netflix/spinnaker/orca/kayenta/tasks/MonitorCanaryTask.groovy +++ b/orca-kayenta/src/main/groovy/com/netflix/spinnaker/orca/kayenta/tasks/MonitorCanaryTask.groovy @@ -17,7 +17,7 @@ package com.netflix.spinnaker.orca.kayenta.tasks import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.kayenta.KayentaService import com.netflix.spinnaker.orca.pipeline.model.Pipeline @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit @Slf4j @Component -class MonitorCanaryTask implements RetryableTask { +class MonitorCanaryTask implements OverridableTimeoutRetryableTask { long backoffPeriod = 1000 long timeout = TimeUnit.HOURS.toMillis(12) diff --git a/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/PropertyAction.groovy b/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/PropertyAction.groovy index 4fb701baad..d240642550 100644 --- a/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/PropertyAction.groovy +++ b/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/PropertyAction.groovy @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.mahe import groovy.transform.CompileStatic diff --git a/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/cleanup/FastPropertyCleanupListener.groovy b/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/cleanup/FastPropertyCleanupListener.groovy index 6805f5de5f..cab8291b97 100644 --- a/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/cleanup/FastPropertyCleanupListener.groovy +++ b/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/cleanup/FastPropertyCleanupListener.groovy @@ -23,6 +23,7 @@ import com.netflix.spinnaker.orca.listeners.Persister import com.netflix.spinnaker.orca.mahe.MaheService import com.netflix.spinnaker.orca.mahe.PropertyAction import com.netflix.spinnaker.orca.pipeline.model.Execution +import com.netflix.spinnaker.orca.pipeline.model.Stage import groovy.util.logging.Slf4j import org.springframework.beans.factory.annotation.Autowired import org.springframework.stereotype.Component @@ -47,9 +48,11 @@ class FastPropertyCleanupListener implements ExecutionListener { ExecutionStatus executionStatus, boolean wasSuccessful) { + boolean rollbackBasedOnExecutionStatus = executionStatus in [ExecutionStatus.TERMINAL, ExecutionStatus.CANCELED] + List rollbacks = rollbackBasedOnExecutionStatus ? execution.stages : execution.stages.findAll { it.context.rollback } - if (executionStatus in [ExecutionStatus.TERMINAL, ExecutionStatus.CANCELED] || execution.context.rollback) { - execution.stages.each { stage -> + if (!rollbacks.empty) { + rollbacks.each { stage -> switch (stage.context.propertyAction) { case PropertyAction.CREATE.toString(): stage.context.propertyIdList.each { prop -> @@ -64,10 +67,10 @@ class FastPropertyCleanupListener implements ExecutionListener { Response response = mahe.upsertProperty(prop) resolveRollbackResponse(response, stage.context.propertyAction.toString(), prop.property) } - break; + break case PropertyAction.DELETE.toString(): stage.context.originalProperties.each { prop -> - if(prop.property.propertyId) { + if (prop.property.propertyId) { prop.property.remove('propertyId') } log.info("Rolling back the ${stage.context.propertyAction} of: ${prop.property.key}|${prop.property.value} on execution ${execution.id} by re-creating") diff --git a/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/tasks/CreatePropertiesTask.groovy b/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/tasks/CreatePropertiesTask.groovy index 5d75ae57fb..479547fad7 100644 --- a/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/tasks/CreatePropertiesTask.groovy +++ b/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/tasks/CreatePropertiesTask.groovy @@ -21,6 +21,7 @@ import com.netflix.spinnaker.orca.Task import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.mahe.MaheService import com.netflix.spinnaker.orca.mahe.PropertyAction +import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.model.Stage import groovy.util.logging.Slf4j import org.springframework.beans.factory.annotation.Autowired @@ -37,14 +38,19 @@ class CreatePropertiesTask implements Task { @Override TaskResult execute(Stage stage) { - List properties = assemblePersistedPropertyListFromContext(stage.context, stage.context.persistedProperties) - List originalProperties = assemblePersistedPropertyListFromContext(stage.context, stage.context.originalProperties) + Map context = stage.context + if (stage.execution instanceof Pipeline) { + List overrides = ((Pipeline) stage.execution).trigger.stageOverrides ?: [] + context = overrides.find { it.refId == stage.refId } ?: context + } + List properties = assemblePersistedPropertyListFromContext(context, context.persistedProperties) + List originalProperties = assemblePersistedPropertyListFromContext(context, context.originalProperties) List propertyIdList = [] PropertyAction propertyAction = PropertyAction.UNKNOWN properties.forEach { Map prop -> Response response - if (stage.context.delete) { + if (context.delete) { log.info("Deleting Property: ${prop.property.propertyId} on execution ${stage.execution.id}") response = maheService.deleteProperty(prop.property.propertyId, 'delete', prop.property.env) propertyAction = PropertyAction.DELETE @@ -63,7 +69,7 @@ class CreatePropertiesTask implements Task { } } - boolean rollback = stage.context.rollbackProperties + boolean rollback = context.rollbackProperties def outputs = [ propertyIdList: propertyIdList, diff --git a/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/tasks/MonitorPropertiesTask.groovy b/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/tasks/MonitorPropertiesTask.groovy index 405cad7de1..bd71021aa9 100644 --- a/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/tasks/MonitorPropertiesTask.groovy +++ b/orca-mahe/src/main/groovy/com/netflix/spinnaker/orca/mahe/tasks/MonitorPropertiesTask.groovy @@ -21,6 +21,7 @@ import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.Task import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.mahe.MaheService +import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.model.Stage import groovy.util.logging.Slf4j import org.apache.http.HttpStatus diff --git a/orca-mahe/src/test/groovy/com/netflix/spinnaker/orca/mahe/tasks/CreatePropertiesTaskSpec.groovy b/orca-mahe/src/test/groovy/com/netflix/spinnaker/orca/mahe/tasks/CreatePropertiesTaskSpec.groovy index 4d89cadd4b..36dd02dd26 100644 --- a/orca-mahe/src/test/groovy/com/netflix/spinnaker/orca/mahe/tasks/CreatePropertiesTaskSpec.groovy +++ b/orca-mahe/src/test/groovy/com/netflix/spinnaker/orca/mahe/tasks/CreatePropertiesTaskSpec.groovy @@ -21,6 +21,7 @@ import com.netflix.spinnaker.orca.mahe.MaheService import com.netflix.spinnaker.orca.mahe.pipeline.CreatePropertyStage import com.netflix.spinnaker.orca.mahe.pipeline.MonitorCreatePropertyStage import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.model.PipelineBuilder import com.netflix.spinnaker.orca.pipeline.model.Stage import retrofit.client.Response import retrofit.mime.TypedByteArray @@ -84,6 +85,37 @@ class CreatePropertiesTaskSpec extends Specification { } } + def "prefer a stage override if present for context"() { + given: + def trigger = [ stageOverrides: [] ] + def pipeline = new PipelineBuilder("foo").withTrigger(trigger).build() + def stageOverride = createPropertiesStage(pipeline, createScope(), createProperty("other"), null) + stageOverride.context.refId = "a" + def property = createProperty() + def createPropertiesStage = createPropertiesStage(pipeline, createScope(), property, null) + createPropertiesStage.refId = "a" + pipeline.stages.addAll([createPropertiesStage, createMonitorStage(pipeline)]) + + pipeline.trigger.stageOverrides << stageOverride.context + + + when: + def results = task.execute(createPropertiesStage) + + then: + 1 * maheService.upsertProperty(_) >> { Map res -> + def json = mapper.writeValueAsString([propertyId: 'other']) + new Response("http://mahe", 200, "OK", [], new TypedByteArray('application/json', json.bytes)) + } + + then: + + with(results.context) { + propertyIdList.size() == 1 + propertyIdList.contains(propertyId: 'other') + } + } + @Unroll("appIdList to appId: #appIdList -> #expectedAppId") def "assemblePersistedPropertyListFromContext with one application in scope list"() { given: diff --git a/orca-mahe/src/test/groovy/com/netflix/spinnaker/orca/mahe/tasks/PropertyChangeCleanupSpec.groovy b/orca-mahe/src/test/groovy/com/netflix/spinnaker/orca/mahe/tasks/PropertyChangeCleanupSpec.groovy index 38218f8b5e..e5d1b9383a 100644 --- a/orca-mahe/src/test/groovy/com/netflix/spinnaker/orca/mahe/tasks/PropertyChangeCleanupSpec.groovy +++ b/orca-mahe/src/test/groovy/com/netflix/spinnaker/orca/mahe/tasks/PropertyChangeCleanupSpec.groovy @@ -27,6 +27,8 @@ import retrofit.mime.TypedByteArray import spock.lang.Specification import spock.lang.Subject import spock.lang.Unroll + +import static com.netflix.spinnaker.orca.mahe.PropertyAction.CREATE import static com.netflix.spinnaker.orca.mahe.PropertyAction.DELETE import static com.netflix.spinnaker.orca.mahe.pipeline.CreatePropertyStage.PIPELINE_CONFIG_TYPE import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.pipeline @@ -53,7 +55,6 @@ class PropertyChangeCleanupSpec extends Specification { name = PIPELINE_CONFIG_TYPE context = propertyContext } - context = propertyContext } repository.retrievePipeline(pipeline.id) >> pipeline @@ -84,7 +85,6 @@ class PropertyChangeCleanupSpec extends Specification { name = PIPELINE_CONFIG_TYPE context = propertyContext } - context = propertyContext } repository.retrievePipeline(pipeline.id) >> pipeline @@ -97,8 +97,6 @@ class PropertyChangeCleanupSpec extends Specification { new Response("http://mahe", 500, "OK", [], null) } - pipeline.context.rollbackActions == null - IllegalStateException ex = thrown() assert ex.message.contains("Unable to rollback DELETE") @@ -117,7 +115,6 @@ class PropertyChangeCleanupSpec extends Specification { type = PIPELINE_CONFIG_TYPE name = PIPELINE_CONFIG_TYPE } - context = [propertyIdList: [[propertyId: propertyId]], propertyAction: PropertyAction.UPDATE.toString()] } repository.retrievePipeline(pipeline.id) >> pipeline @@ -135,6 +132,51 @@ class PropertyChangeCleanupSpec extends Specification { executionStatus << [ExecutionStatus.TERMINAL, ExecutionStatus.CANCELED] } + def "properties marked for rollback are rolled back on a successful execution"() { + def createStageContext = [rollback: true, propertyIdList: [[propertyId: propertyId]], originalProperties: [], propertyAction: PropertyAction.CREATE.toString()] + def deleteStageContext = [rollback: true, propertyIdList: [[propertyId: propertyId]], originalProperties: [[property: previous]], propertyAction: DELETE.toString()] + def retainedStageContext = [propertyIdList: [[propertyId: 'z' + propertyId]], originalProperties: [[property: previous]], propertyAction: CREATE.toString()] + def pipeline = pipeline { + stage { + type = PIPELINE_CONFIG_TYPE + name = PIPELINE_CONFIG_TYPE + context = createStageContext + } + stage { + type = PIPELINE_CONFIG_TYPE + name = PIPELINE_CONFIG_TYPE + context = deleteStageContext + } + stage { + type = PIPELINE_CONFIG_TYPE + name = PIPELINE_CONFIG_TYPE + context = retainedStageContext + } + } + + when: + listener.afterExecution(null, pipeline, ExecutionStatus.SUCCEEDED, true) + + then: + + 1 * mahe.deleteProperty(propertyId, 'spinnaker rollback', propertyEnv) >> { def res -> + def json = mapper.writeValueAsString([propertyId: propertyId]) + new Response("http://mahe", 200, "OK", [], new TypedByteArray('application/json', json.bytes)) + } + + 1 * mahe.upsertProperty(_) >> { Map res -> + String propId = "${res.property.key}|${res.property.value}" + def json = mapper.writeValueAsString([propertyId: propId]) + new Response("http://mahe", 200, "OK", [], new TypedByteArray('application/json', json.bytes)) + } + 0 * _ + + where: + propertyId = "test_rfletcher|mahe|test|us-west-1||||asg=mahe-test-v010|cluster=mahe-test" + propertyEnv = "test" + previous = createPropertyWithId(propertyId) + } + @Unroll() def "a newly created property should be deleted if the pipeline status is #executionStatus and has matching original property"() { given: @@ -145,7 +187,6 @@ class PropertyChangeCleanupSpec extends Specification { name = PIPELINE_CONFIG_TYPE context = propertyContext } - context = propertyContext } repository.retrievePipeline(pipeline.id) >> pipeline @@ -174,7 +215,6 @@ class PropertyChangeCleanupSpec extends Specification { name = PIPELINE_CONFIG_TYPE context = propertyContext } - context = propertyContext } repository.retrievePipeline(pipeline.id) >> pipeline @@ -187,8 +227,6 @@ class PropertyChangeCleanupSpec extends Specification { new Response("http://mahe", 500, "OK", [] , null) } - pipeline.context.rollbackActions == null - IllegalStateException ex = thrown() assert ex.message.contains("Unable to rollback CREATE") assert ex.message.contains(propertyId) @@ -209,7 +247,6 @@ class PropertyChangeCleanupSpec extends Specification { name = PIPELINE_CONFIG_TYPE context = propertyContext } - context = propertyContext } repository.retrievePipeline(pipeline.id) >> pipeline @@ -236,7 +273,6 @@ class PropertyChangeCleanupSpec extends Specification { name = PIPELINE_CONFIG_TYPE context = propertyContext } - context = propertyContext } repository.retrievePipeline(pipeline.id) >> pipeline @@ -263,7 +299,6 @@ class PropertyChangeCleanupSpec extends Specification { type = PIPELINE_CONFIG_TYPE name = PIPELINE_CONFIG_TYPE } - context = [propertyIdList: [propertyId], originalProperties: [previous], rollbackProperties: false, propertyAction: DELETE] } repository.retrievePipeline(pipeline.id) >> pipeline diff --git a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/CanaryStage.groovy b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/CanaryStage.groovy index 4e39c93cef..3a6c2e0d15 100644 --- a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/CanaryStage.groovy +++ b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/CanaryStage.groovy @@ -16,10 +16,15 @@ package com.netflix.spinnaker.orca.mine.pipeline +import com.netflix.spinnaker.orca.CancellableStage.Result +import com.netflix.spinnaker.orca.RetrySupport +import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.DestroyServerGroupTask +import com.netflix.spinnaker.orca.clouddriver.utils.OortHelper +import com.netflix.spinnaker.orca.clouddriver.KatoService + import java.util.concurrent.TimeUnit import com.netflix.frigga.autoscaling.AutoScalingGroupNameBuilder import com.netflix.spinnaker.orca.CancellableStage -import com.netflix.spinnaker.orca.clouddriver.tasks.cluster.ShrinkClusterTask import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -32,10 +37,13 @@ import org.springframework.stereotype.Component @Component class CanaryStage implements StageDefinitionBuilder, CancellableStage { public static final String PIPELINE_CONFIG_TYPE = "canary" - + public static final Integer DEFAULT_CLUSTER_DISABLE_WAIT_TIME = 180 @Autowired DeployCanaryStage deployCanaryStage @Autowired MonitorCanaryStage monitorCanaryStage - @Autowired ShrinkClusterTask shrinkClusterTask + @Autowired DestroyServerGroupTask destroyServerGroupTask + @Autowired OortHelper oortHelper + @Autowired KatoService katoService + @Autowired RetrySupport retrySupport @Override def > List> aroundStages(Stage stage) { @@ -55,13 +63,12 @@ class CanaryStage implements StageDefinitionBuilder, CancellableStage { } @Override - CancellableStage.Result cancel(Stage stage) { - log.info("Cancelling stage (stageId: ${stage.id}, executionId: ${stage.execution.id}, context: ${stage.context as Map})") + Result cancel(Stage stage) { + Collection> disableContexts = [] + Collection> destroyContexts = [] - // it's possible the server groups haven't been created yet, allow a grace period before cleanup - Thread.sleep(TimeUnit.MINUTES.toMillis(2)) + log.info("Cancelling stage (stageId: ${stage.id}, executionId: ${stage.execution.id}, context: ${stage.context as Map})") - Collection> shrinkContexts = [] stage.context.clusterPairs.each { Map clusterPair -> [clusterPair.baseline, clusterPair.canary].each { Map cluster -> @@ -70,28 +77,72 @@ class CanaryStage implements StageDefinitionBuilder, CancellableStage { builder.stack = cluster.stack builder.detail = cluster.freeFormDetails - String region = cluster.region ?: (cluster.availabilityZones as Map).keySet().first() + def cloudProvider = cluster.cloudProvider ?: 'aws' + // it's possible the server groups haven't been created yet, retry with backoff before cleanup + Map deployedCluster = [:] + retrySupport.retry({ + deployedCluster = oortHelper.getCluster(cluster.application, cluster.account, builder.buildGroupName(), cloudProvider).orElse([:]) + if (deployedCluster.serverGroups == null || deployedCluster.serverGroups?.size() == 0) { + throw new IllegalStateException("Expected serverGroup matching cluster {$cluster}") + } + }, 8, TimeUnit.SECONDS.toMillis(15), false) + Long start = stage.startTime + // add a small buffer to deal with latency between the cloud provider and Orca + Long createdTimeCutoff = (stage.endTime ?: System.currentTimeMillis()) + 5000 + + List serverGroups = deployedCluster.serverGroups ?: [] + + String clusterRegion = cluster.region ?: (cluster.availabilityZones as Map).keySet().first() + List matches = serverGroups.findAll { + it.region == clusterRegion && it.createdTime > start && it.createdTime < createdTimeCutoff + } ?: [] + + // really hope they're not running concurrent canaries in the same cluster + matches.each { + disableContexts << [ + disableServerGroup: [ + serverGroupName: it.name, + region : it.region, + credentials : cluster.account, + cloudProvider : cloudProvider, + remainingEnabledServerGroups: 0, + preferLargerOverNewer : false + ] + ] + destroyContexts << [ + serverGroupName: it.name, + region : it.region, + credentials : cluster.account, + cloudProvider : cloudProvider + ] + } + } + } - shrinkContexts << [ - cluster : builder.buildGroupName(), - region : region, - shrinkToSize : 0, - allowDeleteActive: true, - credentials : cluster.account, - cloudProvider : cluster.cloudProvider ?: 'aws' - ] + if (disableContexts) { + try { + katoService.requestOperations( + disableContexts.first().disableServerGroup.cloudProvider, + disableContexts + ).toBlocking().first() + Thread.sleep(TimeUnit.SECONDS.toMillis( + stage.context.clusterDisableWaitTime != null ? stage.context.clusterDisableWaitTime : DEFAULT_CLUSTER_DISABLE_WAIT_TIME) + ) + } catch (Exception e) { + log.error("Error disabling canary clusters in ${stage.id} with ${disableContexts}", e) } } - def shrinkResults = shrinkContexts.collect { - def shrinkStage = new Stage<>() - shrinkStage.context.putAll(it) - shrinkClusterTask.execute(shrinkStage) + def destroyResults = destroyContexts.collect { + def destroyStage = new Stage<>() + destroyStage.execution = stage.execution + destroyStage.context.putAll(it) + destroyServerGroupTask.execute(destroyStage) } - return new CancellableStage.Result(stage, [ - shrinkContexts: shrinkContexts, - shrinkResults : shrinkResults + return new Result(stage, [ + destroyContexts: destroyContexts, + destroyResults : destroyResults ]) } } diff --git a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/DeployCanaryStage.groovy b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/DeployCanaryStage.groovy index 0ff5e65efc..5c360bd043 100644 --- a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/DeployCanaryStage.groovy +++ b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/DeployCanaryStage.groovy @@ -65,13 +65,13 @@ class DeployCanaryStage extends ParallelDeployStage implements CloudProviderAwar } @Override - void postBranchGraph(Stage stage, TaskNode.Builder builder) { + > void taskGraph(Stage stage, TaskNode.Builder builder) { builder.withTask("completeDeployCanary", CompleteDeployCanaryTask) } @Override @CompileDynamic - > Collection> parallelContexts(Stage stage) { + protected > Collection> parallelContexts(Stage stage) { List baselineAmis = findBaselineAmis(stage) Map defaultStageContext = stage.context List canaryDeployments = defaultStageContext.clusterPairs diff --git a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/MonitorCanaryStage.groovy b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/MonitorCanaryStage.groovy index 7b726487b1..bd8fce7e62 100644 --- a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/MonitorCanaryStage.groovy +++ b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/pipeline/MonitorCanaryStage.groovy @@ -18,16 +18,18 @@ package com.netflix.spinnaker.orca.mine.pipeline import com.netflix.spinnaker.orca.CancellableStage import com.netflix.spinnaker.orca.clouddriver.tasks.MonitorKatoTask +import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.ServerGroupCacheForceRefreshTask import com.netflix.spinnaker.orca.mine.MineService import com.netflix.spinnaker.orca.mine.tasks.CleanupCanaryTask import com.netflix.spinnaker.orca.mine.tasks.CompleteCanaryTask +import com.netflix.spinnaker.orca.mine.tasks.DisableCanaryTask import com.netflix.spinnaker.orca.mine.tasks.MonitorCanaryTask import com.netflix.spinnaker.orca.mine.tasks.RegisterCanaryTask import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.TaskNode import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage -import com.netflix.spinnaker.orca.pipeline.util.StageNavigator +import com.netflix.spinnaker.orca.pipeline.tasks.WaitTask import groovy.util.logging.Slf4j import org.springframework.beans.factory.annotation.Autowired import org.springframework.stereotype.Component @@ -37,13 +39,20 @@ import org.springframework.stereotype.Component class MonitorCanaryStage implements StageDefinitionBuilder, CancellableStage { @Autowired MineService mineService - @Autowired StageNavigator stageNavigator + @Autowired CanaryStage canaryStage @Override > void taskGraph(Stage stage, TaskNode.Builder builder) { builder .withTask("registerCanary", RegisterCanaryTask) .withTask("monitorCanary", MonitorCanaryTask) + .withTask("disableCanaryCluster", DisableCanaryTask) + .withTask("monitorDisable", MonitorKatoTask) + .withTask("waitBeforeCleanup", WaitTask) + .withTask("disableBaselineCluster", DisableCanaryTask) + .withTask("monitorDisable", MonitorKatoTask) + .withTask("waitBeforeCleanup", WaitTask) + .withTask("forceCacheRefresh", ServerGroupCacheForceRefreshTask) .withTask("cleanupCanary", CleanupCanaryTask) .withTask("monitorCleanup", MonitorKatoTask) .withTask("completeCanary", CompleteCanaryTask) @@ -74,16 +83,15 @@ class MonitorCanaryStage implements StageDefinitionBuilder, CancellableStage { log.error("Unable to cancel canary '${canaryId}' in mine", e) } - def canaryStages = stageNavigator.ancestors(stage).findAll { - it.stageBuilder instanceof CanaryStage + Stage canaryStageInstance = stage.ancestors().find { + it.type == CanaryStage.PIPELINE_CONFIG_TYPE } - if (!canaryStages) { + if (!canaryStageInstance) { throw new IllegalStateException("No upstream canary stage found (stageId: ${stage.id}, executionId: ${stage.execution.id})") } - def canary = canaryStages.first() - def cancelResult = ((CancellableStage) canary.stageBuilder)?.cancel(canary.stage) + def cancelResult = canaryStage.cancel(canaryStageInstance) cancelResult.details.put("canary", cancelCanaryResults) return cancelResult diff --git a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/DeployedClustersUtil.groovy b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/DeployedClustersUtil.groovy index 29fe9e01e5..e4575ac3fd 100644 --- a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/DeployedClustersUtil.groovy +++ b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/DeployedClustersUtil.groovy @@ -28,4 +28,26 @@ class DeployedClustersUtil { } }.flatten() } + + static List toKatoAsgOperations(String asgOperationDescription, Map stageContext, String selector) { + stageContext.deployedClusterPairs.findAll { it.canaryStage == stageContext.canaryStageId }.collect { + [it."$selector"].collect { + [ + (asgOperationDescription): [ + serverGroupName: it.serverGroup, asgName: it.serverGroup, regions: [it.region], region: it.region, credentials: it.clusterAccount ?: it.accountName, cloudProvider: it.type ?: 'aws' + ] + ] + } + }.flatten() + } + + static Map> getDeployServerGroups(Map stageContext) { + def dSG = [:] + stageContext.deployedClusterPairs.findAll { it.canaryStage == stageContext.canaryStageId }.findAll { + [it.canaryCluster, it.baselineCluster].findAll { + dSG."$it.region" = dSG."$it.region" ? dSG."$it.region" << it.serverGroup : [it.serverGroup] + } + } + return dSG + } } diff --git a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/DisableCanaryTask.groovy b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/DisableCanaryTask.groovy new file mode 100644 index 0000000000..15900823da --- /dev/null +++ b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/DisableCanaryTask.groovy @@ -0,0 +1,55 @@ +package com.netflix.spinnaker.orca.mine.tasks + +import com.netflix.spinnaker.orca.ExecutionStatus +import com.netflix.spinnaker.orca.Task +import com.netflix.spinnaker.orca.TaskResult +import com.netflix.spinnaker.orca.clouddriver.KatoService +import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask +import com.netflix.spinnaker.orca.mine.MineService +import com.netflix.spinnaker.orca.pipeline.model.Stage +import groovy.util.logging.Slf4j +import org.springframework.beans.factory.annotation.Autowired +import org.springframework.stereotype.Component +import retrofit.RetrofitError +import static com.netflix.spinnaker.orca.mine.pipeline.CanaryStage.DEFAULT_CLUSTER_DISABLE_WAIT_TIME + +@Component +@Slf4j +class DisableCanaryTask extends AbstractCloudProviderAwareTask implements Task { + + @Autowired MineService mineService + @Autowired KatoService katoService + + @Override + TaskResult execute(Stage stage) { + + Integer waitTime = stage.context.clusterDisableWaitTime != null ? stage.context.clusterDisableWaitTime : DEFAULT_CLUSTER_DISABLE_WAIT_TIME + + try { + def canary = mineService.getCanary(stage.context.canary.id) + if (canary.health?.health == 'UNHEALTHY') { + // If unhealthy, already disabled in MonitorCanaryTask + return TaskResult.SUCCEEDED + } + } catch (RetrofitError e) { + log.error("Exception occurred while getting canary status with id {} from mine, continuing with disable", + stage.context.canary.id, e) + } + + def selector = stage.context.containsKey('disabledCluster') ? 'baselineCluster' : 'canaryCluster' + def ops = DeployedClustersUtil.toKatoAsgOperations('disableServerGroup', stage.context, selector) + def dSG = DeployedClustersUtil.getDeployServerGroups(stage.context) + + log.info "Disabling ${selector} in ${stage.id} with ${ops}" + String cloudProvider = ops && !ops.empty ? ops.first()?.values().first()?.cloudProvider : getCloudProvider(stage) ?: 'aws' + def taskId = katoService.requestOperations(cloudProvider, ops).toBlocking().first() + + stage.context.remove('waitTaskState') + return new TaskResult(ExecutionStatus.SUCCEEDED, [ + 'kato.last.task.id' : taskId, + 'deploy.server.groups' : dSG, + disabledCluster : selector, + waitTime : waitTime + ]) + } +} diff --git a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/MonitorAcaTaskTask.groovy b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/MonitorAcaTaskTask.groovy index b202429653..e47195cfcd 100644 --- a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/MonitorAcaTaskTask.groovy +++ b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/MonitorAcaTaskTask.groovy @@ -18,7 +18,7 @@ package com.netflix.spinnaker.orca.mine.tasks import java.util.concurrent.TimeUnit import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask import com.netflix.spinnaker.orca.mine.MineService @@ -30,7 +30,7 @@ import retrofit.RetrofitError @Component @Slf4j -class MonitorAcaTaskTask extends AbstractCloudProviderAwareTask implements RetryableTask { +class MonitorAcaTaskTask extends AbstractCloudProviderAwareTask implements OverridableTimeoutRetryableTask { long backoffPeriod = 10000 long timeout = TimeUnit.DAYS.toMillis(2) diff --git a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/MonitorCanaryTask.groovy b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/MonitorCanaryTask.groovy index 1947ed7829..b5d65751b0 100644 --- a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/MonitorCanaryTask.groovy +++ b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/MonitorCanaryTask.groovy @@ -18,7 +18,7 @@ package com.netflix.spinnaker.orca.mine.tasks import java.util.concurrent.TimeUnit import com.netflix.spinnaker.orca.ExecutionStatus -import com.netflix.spinnaker.orca.RetryableTask +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.clouddriver.KatoService import com.netflix.spinnaker.orca.clouddriver.tasks.AbstractCloudProviderAwareTask @@ -31,7 +31,7 @@ import retrofit.RetrofitError @Component @Slf4j -class MonitorCanaryTask extends AbstractCloudProviderAwareTask implements RetryableTask { +class MonitorCanaryTask extends AbstractCloudProviderAwareTask implements OverridableTimeoutRetryableTask { long backoffPeriod = TimeUnit.MINUTES.toMillis(1) long timeout = TimeUnit.DAYS.toMillis(2) diff --git a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/RegisterCanaryTask.groovy b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/RegisterCanaryTask.groovy index 575e190b87..a3406f90d2 100644 --- a/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/RegisterCanaryTask.groovy +++ b/orca-mine/src/main/groovy/com/netflix/spinnaker/orca/mine/tasks/RegisterCanaryTask.groovy @@ -58,8 +58,14 @@ class RegisterCanaryTask implements Task { def outputs = [ canary : canary, stageTimeoutMs : getMonitorTimeout(canary), - deployedClusterPairs: deployStage.context.deployedClusterPairs + deployedClusterPairs: deployStage.context.deployedClusterPairs, + application : c.application ] + + if (deployStage.context.deployedClusterPairs[0]?.canaryCluster?.accountName) { + outputs.account = deployStage.context.deployedClusterPairs[0].canaryCluster.accountName + } + return new TaskResult(ExecutionStatus.SUCCEEDED, outputs) } diff --git a/orca-mine/src/test/groovy/com/netflix/spinnaker/orca/mine/pipeline/CanaryStageSpec.groovy b/orca-mine/src/test/groovy/com/netflix/spinnaker/orca/mine/pipeline/CanaryStageSpec.groovy new file mode 100644 index 0000000000..8d0d413fa0 --- /dev/null +++ b/orca-mine/src/test/groovy/com/netflix/spinnaker/orca/mine/pipeline/CanaryStageSpec.groovy @@ -0,0 +1,108 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.mine.pipeline + +import com.netflix.spinnaker.orca.CancellableStage.Result +import com.netflix.spinnaker.orca.RetrySupport +import com.netflix.spinnaker.orca.clouddriver.model.TaskId +import com.netflix.spinnaker.orca.clouddriver.tasks.servergroup.DestroyServerGroupTask +import com.netflix.spinnaker.orca.clouddriver.KatoService +import com.netflix.spinnaker.orca.clouddriver.utils.OortHelper +import com.netflix.spinnaker.orca.pipeline.model.Stage +import rx.Observable +import spock.lang.Specification +import spock.lang.Unroll + +import static com.netflix.spinnaker.orca.test.model.ExecutionBuilder.stage + +class CanaryStageSpec extends Specification { + + @Unroll + void "cancel destroys canary/baseline if found and were deployed during canary stage"() { + given: + Map stageContext = [ + clusterDisableWaitTime: 0, + clusterPairs: [ + [ + baseline: [application: "app", stack: "stack1", freeFormDetails: "baseline", region: "us-east-1", account: "test"], + canary: [application: "app", stack: "stack1", freeFormDetails: "canary", region: "us-east-1", account: "test"] + ] + ] + ] + + def disableOperation = [ + [disableServerGroup: + [ + serverGroupName: "app-stack1-baseline-v003", region: "us-east-1", credentials: "test", + cloudProvider : "aws", remainingEnabledServerGroups: 0, preferLargerOverNewer: false + ] + ], + [disableServerGroup: + [ + serverGroupName: "app-stack1-canary-v003", region: "us-east-1", credentials: "test", + cloudProvider : "aws", remainingEnabledServerGroups: 0, preferLargerOverNewer: false + ] + ] + ] + + TaskId taskId = new TaskId(UUID.randomUUID().toString()) + + Stage canceledStage = stage { + context = stageContext + startTime = 5 + endTime = 10 + } + + OortHelper oortHelper = Mock(OortHelper) + KatoService katoService = Mock(KatoService) + DestroyServerGroupTask destroyServerGroupTask = Mock(DestroyServerGroupTask) + + CanaryStage canaryStage = new CanaryStage( + oortHelper: oortHelper, + katoService: katoService, + destroyServerGroupTask: destroyServerGroupTask, + retrySupport: Spy(RetrySupport) { + _ * sleep(_) >> { /* do nothing */ } + } + ) + + when: + Result result = canaryStage.cancel(canceledStage) + + then: + result.details.destroyContexts.size() == destroyedServerGroups + 1 * oortHelper.getCluster("app", "test", "app-stack1-baseline", "aws") >> [ + serverGroups: [[region: "us-east-1", createdTime: createdTime, name: "app-stack1-baseline-v003"]] + ] + 1 * oortHelper.getCluster("app", "test", "app-stack1-canary", "aws") >> [ + serverGroups: [[region: "us-east-1", createdTime: createdTime, name: "app-stack1-canary-v003"]] + ] + + disableOps * katoService.requestOperations("aws", disableOperation) >> { Observable.from(taskId) } + + where: + createdTime | disableOps | destroyedServerGroups + 4 | 0 | 0 + 5 | 0 | 0 + 6 | 1 | 2 + 10 | 1 | 2 + 5010 | 0 | 0 + 5011 | 0 | 0 + + } + +} diff --git a/orca-mine/src/test/groovy/com/netflix/spinnaker/orca/mine/pipeline/MonitorCanaryStageSpec.groovy b/orca-mine/src/test/groovy/com/netflix/spinnaker/orca/mine/pipeline/MonitorCanaryStageSpec.groovy index 595ff08bb6..e20765dfed 100644 --- a/orca-mine/src/test/groovy/com/netflix/spinnaker/orca/mine/pipeline/MonitorCanaryStageSpec.groovy +++ b/orca-mine/src/test/groovy/com/netflix/spinnaker/orca/mine/pipeline/MonitorCanaryStageSpec.groovy @@ -20,20 +20,16 @@ package com.netflix.spinnaker.orca.mine.pipeline import com.netflix.spinnaker.orca.CancellableStage import com.netflix.spinnaker.orca.mine.MineService import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.model.PipelineBuilder import com.netflix.spinnaker.orca.pipeline.model.Stage -import com.netflix.spinnaker.orca.pipeline.util.StageNavigator import spock.lang.Specification -import spock.lang.Subject class MonitorCanaryStageSpec extends Specification { def mineService = Mock(MineService) - def stageNavigator = Stub(StageNavigator) - - @Subject - def monitorCanaryStage = new MonitorCanaryStage(mineService: mineService, stageNavigator: stageNavigator) def "should short-circuit if canary registered but execution not explicitly canceled"() { given: + def monitorCanaryStage = new MonitorCanaryStage(mineService: mineService) def stage = new Stage<>(new Pipeline("orca"), "pipelineStage", [ canary: [id: "canaryId"] ]) @@ -49,14 +45,15 @@ class MonitorCanaryStageSpec extends Specification { def "should propagate cancel upstream if canary registered and execution explicitly canceled"() { given: - def canaryStage = new Stage() + def pipeline = new PipelineBuilder("orca").withStage(CanaryStage.PIPELINE_CONFIG_TYPE).build() + def canaryStage = pipeline.namedStage(CanaryStage.PIPELINE_CONFIG_TYPE) + canaryStage.setRefId("1") def canaryStageBuilder = Mock(CanaryStage) - def stage = new Stage(new Pipeline("orca"), "pipelineStage", [ + def monitorCanaryStage = new MonitorCanaryStage(mineService: mineService, canaryStage: canaryStageBuilder) + def stage = new Stage(pipeline, "pipelineStage", [ canary: [id: "canaryId"] ]) - - and: - stageNavigator.ancestors(_) >> [new StageNavigator.Result(canaryStage, canaryStageBuilder)] + stage.setRequisiteStageRefIds(["1"]) when: stage.execution.canceled = true @@ -71,6 +68,7 @@ class MonitorCanaryStageSpec extends Specification { } def "should raise exception if no upstream canary stage found"() { + def monitorCanaryStage = new MonitorCanaryStage(mineService: mineService) def stage = new Stage<>(new Pipeline("orca"), "pipelineStage", [ canary: [id: "canaryId"] ]) diff --git a/orca-pipelinetemplate/orca-pipelinetemplate.gradle b/orca-pipelinetemplate/orca-pipelinetemplate.gradle index ddcd574b23..38f6a2365f 100644 --- a/orca-pipelinetemplate/orca-pipelinetemplate.gradle +++ b/orca-pipelinetemplate/orca-pipelinetemplate.gradle @@ -1,3 +1,5 @@ +apply from: "$rootDir/gradle/kotlin.gradle" + tasks.compileGroovy.enabled = false dependencies { @@ -8,6 +10,7 @@ dependencies { compile('com.hubspot.jinjava:jinjava:2.2.3') compile "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:${spinnaker.version('jackson')}" + compile "com.fasterxml.jackson.module:jackson-module-kotlin:${spinnaker.version("jackson")}" compile('com.jayway.jsonpath:json-path:2.2.0') compile spinnaker.dependency("slf4jApi") @@ -20,4 +23,5 @@ dependencies { testCompile spinnaker.dependency("slf4jSimple") testCompile 'org.spockframework:spock-unitils:1.1-groovy-2.4-rc-2' + testCompile 'org.codehaus.groovy:groovy-json:2.4.11' } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/config/PipelineTemplateConfiguration.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/config/PipelineTemplateConfiguration.java index 2f0256aad5..96a771ec3d 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/config/PipelineTemplateConfiguration.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/config/PipelineTemplateConfiguration.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.module.kotlin.KotlinModule; import com.hubspot.jinjava.lib.tag.Tag; import com.netflix.spinnaker.orca.front50.Front50Service; import com.netflix.spinnaker.orca.front50.PipelineModelMutator; @@ -40,7 +41,12 @@ @ConditionalOnExpression("${pipelineTemplates.enabled:true}") @ComponentScan( basePackageClasses = PipelineTemplateModule.class, - basePackages = {"com.netflix.spinnaker.orca.pipelinetemplate.tasks", "com.netflix.spinnaker.orca.pipelinetemplate.pipeline"} + basePackages = { + "com.netflix.spinnaker.orca.pipelinetemplate.tasks", + "com.netflix.spinnaker.orca.pipelinetemplate.pipeline", + "com.netflix.spinnaker.orca.pipelinetemplate.handler", + "com.netflix.spinnaker.orca.pipelinetemplate.v1schema.handler" + } ) public class PipelineTemplateConfiguration { @@ -51,7 +57,8 @@ public class PipelineTemplateConfiguration { ObjectMapper pipelineTemplateObjectMapper() { return new ObjectMapper() .enable(SerializationFeature.INDENT_OUTPUT) - .disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + .disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) + .registerModule(new KotlinModule()); } @Bean diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePipelinePreprocessor.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePipelinePreprocessor.java deleted file mode 100644 index 5c3cf8489d..0000000000 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePipelinePreprocessor.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Copyright 2017 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License") - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.netflix.spinnaker.orca.pipelinetemplate; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.netflix.spectator.api.Registry; -import com.netflix.spinnaker.orca.extensionpoint.pipeline.PipelinePreprocessor; -import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.IllegalTemplateConfigurationException; -import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.TemplateLoaderException; -import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.TemplateRenderException; -import com.netflix.spinnaker.orca.pipelinetemplate.generator.ExecutionGenerator; -import com.netflix.spinnaker.orca.pipelinetemplate.loader.TemplateLoader; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.TemplateMerge; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.V1SchemaExecutionGenerator; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.graph.GraphMutator; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.DefaultRenderContext; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.RenderContext; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.RenderUtil; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.Renderer; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.validator.V1TemplateConfigurationSchemaValidator; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.validator.V1TemplateSchemaValidator; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.validator.V1TemplateSchemaValidator.SchemaValidatorContext; -import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors; -import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors.Error; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * highlevel lifecycle - * - * 1. Find all pipeline templates from configuration source. - * 2. Merge templates together - * 3. Render all renderable fields in both template and configuration. - */ -@Component -public class PipelineTemplatePipelinePreprocessor implements PipelinePreprocessor { - - private final ObjectMapper pipelineTemplateObjectMapper; - private final TemplateLoader templateLoader; - private final Renderer renderer; - private final Registry registry; - - @Autowired - public PipelineTemplatePipelinePreprocessor(ObjectMapper pipelineTemplateObjectMapper, - TemplateLoader templateLoader, - Renderer renderer, - Registry registry) { - this.pipelineTemplateObjectMapper = pipelineTemplateObjectMapper; - this.templateLoader = templateLoader; - this.renderer = renderer; - this.registry = registry; - } - - @Override - public Map process(Map pipeline) { - Errors errors; - try { - return processInternal(pipeline); - } catch (TemplateLoaderException e) { - errors = new Errors().add(new Error().withMessage("failed loading template").withCause(e.getMessage())); - } catch (TemplateRenderException e) { - errors = e.getErrors(); - if (!errors.hasErrors(true)) { - errors.add(new Error().withMessage("failed rendering template expression").withCause(e.getMessage())); - } - } catch (IllegalTemplateConfigurationException e) { - errors = new Errors().add( - e.getError() != null ? e.getError() : new Error().withMessage("malformed template configuration").withCause(e.getMessage()) - ); - } - return errors.toResponse(); - } - - private Map processInternal(Map pipeline) { - TemplatedPipelineRequest request = pipelineTemplateObjectMapper.convertValue(pipeline, TemplatedPipelineRequest.class); - if (!request.isTemplatedPipelineRequest()) { - return pipeline; - } - - Errors validationErrors = new Errors(); - - TemplateConfiguration templateConfiguration = request.getConfig(); - new V1TemplateConfigurationSchemaValidator().validate(templateConfiguration, validationErrors); - if (validationErrors.hasErrors(request.plan)) { - return validationErrors.toResponse(); - } - - PipelineTemplate template = getPipelineTemplate(request, templateConfiguration); - - new V1TemplateSchemaValidator().validate(template, validationErrors, new SchemaValidatorContext(!templateConfiguration.getStages().isEmpty())); - if (validationErrors.hasErrors(request.plan)) { - return validationErrors.toResponse(); - } - - Map trigger = (HashMap) pipeline.get("trigger"); - GraphMutator graphMutator = new GraphMutator(templateConfiguration, renderer, registry, trigger); - graphMutator.mutate(template); - - ExecutionGenerator executionGenerator = new V1SchemaExecutionGenerator(); - Map generatedPipeline = executionGenerator.generate(template, templateConfiguration, request); - - return generatedPipeline; - } - - private PipelineTemplate getPipelineTemplate(TemplatedPipelineRequest request, TemplateConfiguration templateConfiguration) { - if (request.plan && request.template != null) { - // Allow template inlining to perform plans without first publishing the template somewhere. - return request.template; - } - - if (request.getConfig().getPipeline().getTemplate() == null) { - throw new IllegalTemplateConfigurationException(new Error().withMessage("configuration is missing a template")); - } - - setTemplateSourceWithJinja(request); - List templates = templateLoader.load(templateConfiguration.getPipeline().getTemplate()); - - PipelineTemplate pipelineTemplate = TemplateMerge.merge(templates); - - // ensure that any expressions contained with template variables are rendered - RenderContext context = RenderUtil.createDefaultRenderContext( - pipelineTemplate, templateConfiguration, request.getTrigger() - ); - renderTemplateVariables(context, pipelineTemplate); - - return pipelineTemplate; - } - - private void renderTemplateVariables(RenderContext renderContext, PipelineTemplate pipelineTemplate) { - if (pipelineTemplate.getVariables() == null) { - return; - } - - pipelineTemplate.getVariables().forEach(v -> { - Object value = v.getDefaultValue(); - if (value != null && value instanceof String) { - v.setDefaultValue(renderer.renderGraph(value.toString(), renderContext)); - } - }); - } - - private void setTemplateSourceWithJinja(TemplatedPipelineRequest request) { - RenderContext context = new DefaultRenderContext(request.getConfig().getPipeline().getApplication(), null, request.getTrigger()); - request.getConfig().getPipeline().getTemplate().setSource(renderer.render(request.getConfig().getPipeline().getTemplate().getSource(), context )); - } -} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePreprocessor.kt b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePreprocessor.kt new file mode 100644 index 0000000000..93b2869561 --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePreprocessor.kt @@ -0,0 +1,95 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate + +import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spectator.api.BasicTag +import com.netflix.spectator.api.Registry +import com.netflix.spinnaker.orca.extensionpoint.pipeline.PipelinePreprocessor +import com.netflix.spinnaker.orca.pipelinetemplate.handler.* +import org.slf4j.LoggerFactory +import org.springframework.beans.factory.annotation.Autowired +import org.springframework.stereotype.Component +import javax.annotation.PostConstruct + +@Component("pipelineTemplatePreprocessor") +open class PipelineTemplatePreprocessor +@Autowired constructor( + private val pipelineTemplateObjectMapper: ObjectMapper, + private val schemaVersionHandler: SchemaVersionHandler, + private val errorHandler: PipelineTemplateErrorHandler, + private val registry: Registry +) : PipelinePreprocessor { + + private val log = LoggerFactory.getLogger(javaClass) + private val requestsId = registry.createId("mpt.requests") + + @PostConstruct fun confirmUsage() = log.info("Using ${javaClass.simpleName}") + + override fun process(pipeline: MutableMap?): MutableMap { + if (pipeline == null) { + return mutableMapOf() + } + + val request = pipelineTemplateObjectMapper.convertValue(pipeline, TemplatedPipelineRequest::class.java) + if (!request.isTemplatedPipelineRequest) { + return pipeline + } + + log.debug("Starting handler chain") + + val chain = DefaultHandlerChain() + val context = GlobalPipelineTemplateContext(chain, request) + + chain.add(schemaVersionHandler) + + while (!chain.isEmpty()) { + val handler = chain.removeFirst() + try { + handler.handle(chain, context) + } catch (t: Throwable) { + if (handler is PipelineTemplateErrorHandler) { + recordRequest(context, false) + throw IrrecoverableConditionException(t) + } + + log.error("Unexpected error occurred while processing template: ", context.getRequest().getId(), t) + context.getCaughtThrowables().add(t) + chain.clear() + } + + // Ensure the error handler is always the last thing we run + if (chain.isEmpty() && handler !is PipelineTemplateErrorHandler) { + chain.add(errorHandler) + } + } + + recordRequest(context, context.getErrors().hasErrors(false)) + + log.debug("Handler chain complete") + return context.getProcessedOutput() + } + + private fun recordRequest(context: PipelineTemplateContext, success: Boolean) { + registry.counter(requestsId.withTags(listOf( + BasicTag("status", if (success) "success" else "failure"), + BasicTag("schema", context.getRequest().schema ?: "unknown"), + BasicTag("plan", context.getRequest().plan.toString()) + ))).increment() + } +} + +private class IrrecoverableConditionException(t: Throwable) : RuntimeException("Could not recover from an error condition", t) diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/TemplatedPipelineRequest.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/TemplatedPipelineRequest.java index 1c7e4b9a87..ef5a9add2b 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/TemplatedPipelineRequest.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/TemplatedPipelineRequest.java @@ -15,21 +15,31 @@ */ package com.netflix.spinnaker.orca.pipelinetemplate; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.HashMap; import java.util.Map; public class TemplatedPipelineRequest { String id; + String schema; String type; - Map trigger; - TemplateConfiguration config; - PipelineTemplate template; + Map trigger = new HashMap<>(); + Map config; + Map template; Boolean plan = false; boolean limitConcurrent = true; boolean keepWaitingPipelines = false; + @JsonProperty("config") + private void unpackConfig(Map config) { + if (config == null) { + return; + } + this.config = config; + schema = (String) config.get("schema"); + } + public String getId() { return id; } @@ -38,8 +48,12 @@ public void setId(String id) { this.id = id; } - public boolean isTemplatedPipelineRequest() { - return "templatedPipeline".equals(type); + public String getSchema() { + return schema; + } + + public void setSchema(String schema) { + this.schema = schema; } public String getType() { @@ -50,12 +64,8 @@ public void setType(String type) { this.type = type; } - public TemplateConfiguration getConfig() { - return config; - } - - public void setConfig(TemplateConfiguration config) { - this.config = config; + public boolean isTemplatedPipelineRequest() { + return "templatedPipeline".equals(type); } public Map getTrigger() { @@ -66,11 +76,19 @@ public void setTrigger(Map trigger) { this.trigger = trigger; } - public PipelineTemplate getTemplate() { + public Map getConfig() { + return config; + } + + public void setConfig(Map config) { + this.config = config; + } + + public Map getTemplate() { return template; } - public void setTemplate(PipelineTemplate template) { + public void setTemplate(Map template) { this.template = template; } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/IllegalTemplateConfigurationException.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/IllegalTemplateConfigurationException.java index 4479ad4f2a..24ba6d7369 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/IllegalTemplateConfigurationException.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/IllegalTemplateConfigurationException.java @@ -15,9 +15,10 @@ */ package com.netflix.spinnaker.orca.pipelinetemplate.exceptions; +import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors; import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors.Error; -public class IllegalTemplateConfigurationException extends IllegalStateException { +public class IllegalTemplateConfigurationException extends IllegalStateException implements PipelineTemplateException { private Error error; @@ -33,4 +34,9 @@ public IllegalTemplateConfigurationException(Error error) { public Error getError() { return error; } + + @Override + public Errors getErrors() { + return new Errors().add(error); + } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/PipelineTemplateException.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/PipelineTemplateException.java new file mode 100644 index 0000000000..2fb30d3106 --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/PipelineTemplateException.java @@ -0,0 +1,23 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.exceptions; + +import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors; + +public interface PipelineTemplateException { + + Errors getErrors(); +} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/TemplateLoaderException.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/TemplateLoaderException.java index 75f6c89215..ece753d301 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/TemplateLoaderException.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/TemplateLoaderException.java @@ -17,8 +17,9 @@ package com.netflix.spinnaker.orca.pipelinetemplate.exceptions; import com.netflix.spinnaker.kork.web.exceptions.InvalidRequestException; +import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors; -public class TemplateLoaderException extends InvalidRequestException { +public class TemplateLoaderException extends InvalidRequestException implements PipelineTemplateException { public TemplateLoaderException(String message) { super(message); } @@ -30,4 +31,9 @@ public TemplateLoaderException(String message, Throwable cause) { public TemplateLoaderException(Throwable cause) { super(cause); } + + @Override + public Errors getErrors() { + return new Errors(); + } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/TemplateRenderException.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/TemplateRenderException.java index 6bc05fd833..f5b42bb289 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/TemplateRenderException.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/exceptions/TemplateRenderException.java @@ -18,7 +18,7 @@ import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors; import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors.Error; -public class TemplateRenderException extends RuntimeException { +public class TemplateRenderException extends RuntimeException implements PipelineTemplateException { public static TemplateRenderException fromError(Error error) { return new TemplateRenderException(error.getMessage(), null, error); diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/Handler.kt b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/Handler.kt new file mode 100644 index 0000000000..45ee3f19ef --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/Handler.kt @@ -0,0 +1,53 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.handler + +interface Handler { + fun handle(chain: HandlerChain, context: PipelineTemplateContext) +} + +interface HandlerGroup { + fun getHandlers(): List +} + +interface HandlerChain { + fun add(handler: Handler) + fun add(handlerChain: HandlerGroup) + fun removeFirst(): Handler + fun isEmpty(): Boolean + fun clear() +} + +class DefaultHandlerChain : HandlerChain { + + private val chain = mutableListOf() + + override fun add(handler: Handler) { + chain.add(handler) + } + + override fun add(handlerChain: HandlerGroup) { + chain.addAll(handlerChain.getHandlers()) + } + + override fun removeFirst() = chain.removeAt(0) + + override fun isEmpty() = chain.isEmpty() + + override fun clear() { + chain.clear() + } +} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/PipelineTemplateContext.kt b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/PipelineTemplateContext.kt new file mode 100644 index 0000000000..52f8d9d45d --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/PipelineTemplateContext.kt @@ -0,0 +1,63 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.handler + +import com.netflix.spinnaker.orca.pipelinetemplate.TemplatedPipelineRequest +import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors + +interface PipelineTemplateContext { + fun getChain(): HandlerChain + fun getErrors(): Errors + fun getProcessedOutput(): MutableMap + fun getRequest(): TemplatedPipelineRequest + fun getCaughtThrowables(): MutableList + fun setSchemaContext(c: T) + fun getSchemaContext(): T +} + +interface PipelineTemplateSchemaContext + +class GlobalPipelineTemplateContext( + private val chain: HandlerChain, + private val request: TemplatedPipelineRequest +) : PipelineTemplateContext { + + private val errors = Errors() + private val processedOutput = mutableMapOf() + private val throwables = mutableListOf() + + private var schemaContext: PipelineTemplateSchemaContext? = null + override fun getChain() = chain + override fun getErrors() = errors + override fun getProcessedOutput() = processedOutput + override fun getRequest() = request + override fun getCaughtThrowables() = throwables + + override fun setSchemaContext(c: T) { + if (schemaContext != null) { + throw IllegalArgumentException("schema context already set") + } + schemaContext = c + } + + @Suppress("UNCHECKED_CAST") + override fun getSchemaContext(): T { + if (schemaContext == null) { + throw IllegalStateException("schema context has not been set yet") + } + return schemaContext as T + } +} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/PipelineTemplateErrorHandler.kt b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/PipelineTemplateErrorHandler.kt new file mode 100644 index 0000000000..f400e44a44 --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/PipelineTemplateErrorHandler.kt @@ -0,0 +1,59 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.handler + +import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.IllegalTemplateConfigurationException +import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.TemplateLoaderException +import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.TemplateRenderException +import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors +import org.springframework.stereotype.Component + +@Component +class PipelineTemplateErrorHandler : Handler { + + override fun handle(chain: HandlerChain, context: PipelineTemplateContext) { + context.getCaughtThrowables().map { generateErrors(it) }.forEach { context.getErrors().addAll(it) } + + if (context.getErrors().hasErrors(context.getRequest().plan)) { + context.getProcessedOutput().putAll(context.getErrors().toResponse()) + } + } + + // Gross backwards compat with old error handler logic + private fun generateErrors(t: Throwable): Errors { + val e = Errors() + if (t is TemplateLoaderException) { + if (t.errors.hasErrors(true)) { + e.addAll(t.errors) + } else { + e.add(Errors.Error().withMessage("failed loading template").withCause(t.message)) + } + } else if (t is TemplateRenderException) { + if (!t.errors.hasErrors(true)) { + e.add(Errors.Error().withMessage("failed rendering template expression").withCause(t.message)) + } else { + e.addAll(t.errors) + } + } else if (t is IllegalTemplateConfigurationException) { + e.add( + if (t.error != null) t.error else Errors.Error().withMessage("malformed template configuration").withCause(t.message) + ) + } else { + e.add(Errors.Error().withMessage("unexpected error").withCause(t.toString())) + } + return e + } +} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/SchemaVersionHandler.kt b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/SchemaVersionHandler.kt new file mode 100644 index 0000000000..a8b89b05a2 --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/handler/SchemaVersionHandler.kt @@ -0,0 +1,36 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.handler + +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.handler.V1SchemaHandlerGroup +import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors +import org.springframework.beans.factory.annotation.Autowired +import org.springframework.stereotype.Component + +@Component +class SchemaVersionHandler +@Autowired constructor( + private val v1SchemaHandlerGroup: V1SchemaHandlerGroup +): Handler { + override fun handle(chain: HandlerChain, context: PipelineTemplateContext) { + when(context.getRequest().schema) { + "1" -> chain.add(v1SchemaHandlerGroup) + else -> context.getErrors().add( + Errors.Error().withMessage("unexpected schema version '${context.getRequest().schema}'") + ) + } + } +} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/loader/TemplateLoader.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/loader/TemplateLoader.java index f74d1780da..6dd0f43f5a 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/loader/TemplateLoader.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/loader/TemplateLoader.java @@ -45,9 +45,13 @@ public TemplateLoader(Collection schemeLoaders) { * @return a LIFO list of pipeline templates */ public List load(TemplateConfiguration.TemplateSource template) { + PipelineTemplate pipelineTemplate = load(template.getSource()); + return load(pipelineTemplate); + } + + public List load(PipelineTemplate pipelineTemplate) { List pipelineTemplates = new ArrayList<>(); - PipelineTemplate pipelineTemplate = load(template.getSource()); pipelineTemplates.add(0, pipelineTemplate); Set seenTemplateSources = new HashSet<>(); diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/pipeline/UpdatePipelineTemplateStage.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/pipeline/UpdatePipelineTemplateStage.java index c39858ee81..bf44ecb0a6 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/pipeline/UpdatePipelineTemplateStage.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/pipeline/UpdatePipelineTemplateStage.java @@ -15,17 +15,42 @@ */ package com.netflix.spinnaker.orca.pipelinetemplate.pipeline; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.netflix.spinnaker.orca.front50.Front50Service; +import com.netflix.spinnaker.orca.front50.pipeline.UpdatePipelineStage; import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder; import com.netflix.spinnaker.orca.pipeline.TaskNode.Builder; import com.netflix.spinnaker.orca.pipeline.model.Execution; import com.netflix.spinnaker.orca.pipeline.model.Stage; +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner; import com.netflix.spinnaker.orca.pipelinetemplate.tasks.PlanTemplateDependentsTask; import com.netflix.spinnaker.orca.pipelinetemplate.tasks.UpdatePipelineTemplateTask; +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate; +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; +import javax.annotation.Nonnull; +import java.util.Base64; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + @Component public class UpdatePipelineTemplateStage implements StageDefinitionBuilder { + @Autowired(required = false) + private Front50Service front50Service; + + @Autowired + private ObjectMapper pipelineTemplateObjectMapper; + + @Autowired + private UpdatePipelineStage updatePipelineStage; + @Override public > void taskGraph(Stage stage, Builder builder) { if (!Boolean.valueOf(stage.getContext().getOrDefault("skipPlanDependents", "false").toString())) { @@ -35,4 +60,56 @@ public > void taskGraph(Stage stage, Builder builder) builder .withTask("updatePipelineTemplate", UpdatePipelineTemplateTask.class); } + + @Nonnull + @Override + public > List> aroundStages(@Nonnull Stage stage) { + if (front50Service == null) { + return Collections.emptyList(); + } + + if (!stage.getContext().containsKey("pipelineTemplate")) { + throw new IllegalArgumentException("Missing required task parameter (pipelineTemplate)"); + } + + if (!(stage.getContext().get("pipelineTemplate") instanceof String)) { + throw new IllegalArgumentException("'pipelineTemplate' context key must be a base64-encoded string: Ensure you're on the most recent version of gate"); + } + + PipelineTemplate pipelineTemplate = stage.decodeBase64( + "/pipelineTemplate", + PipelineTemplate.class, + pipelineTemplateObjectMapper + ); + + List> dependentPipelines = front50Service.getPipelineTemplateDependents(pipelineTemplate.getId(), true); + + return dependentPipelines.stream() + .filter(pipeline -> { + // We only need to re-save pipelines that actually inherit configurations. + TemplateConfiguration config = pipelineTemplateObjectMapper.convertValue(pipeline.get("config"), TemplateConfiguration.class); + return !config.getConfiguration().getInherit().isEmpty(); + }) + .map(pipeline -> configureSavePipelineStage(stage, pipeline)) + .collect(Collectors.toList()); + } + + private > Stage configureSavePipelineStage(Stage stage, Map pipeline) { + Map context = new HashMap<>(); + + try { + context.put("pipeline", Base64.getEncoder().encodeToString(pipelineTemplateObjectMapper.writeValueAsBytes(pipeline))); + } catch (JsonProcessingException e) { + throw new RuntimeException(String.format("Failed converting pipeline to JSON: %s", pipeline.get("id")), e); + } + + return StageDefinitionBuilder.newStage( + stage.getExecution(), + updatePipelineStage.getType(), + "updateDependentPipeline", + context, + stage, + SyntheticStageOwner.STAGE_AFTER + ); + } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/tasks/PlanTemplateDependentsTask.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/tasks/PlanTemplateDependentsTask.java index c7f265eaf8..43b1102245 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/tasks/PlanTemplateDependentsTask.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/tasks/PlanTemplateDependentsTask.java @@ -19,9 +19,9 @@ import com.netflix.spinnaker.orca.ExecutionStatus; import com.netflix.spinnaker.orca.RetryableTask; import com.netflix.spinnaker.orca.TaskResult; +import com.netflix.spinnaker.orca.extensionpoint.pipeline.PipelinePreprocessor; import com.netflix.spinnaker.orca.front50.Front50Service; import com.netflix.spinnaker.orca.pipeline.model.Stage; -import com.netflix.spinnaker.orca.pipelinetemplate.PipelineTemplatePipelinePreprocessor; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @@ -44,7 +44,7 @@ public class PlanTemplateDependentsTask implements RetryableTask { private ObjectMapper pipelineTemplateObjectMapper; @Autowired - private PipelineTemplatePipelinePreprocessor pipelineTemplatePipelinePreprocessor; + private PipelinePreprocessor pipelineTemplatePreprocessor; @Nonnull @Override @@ -78,7 +78,7 @@ public TaskResult execute(@Nonnull Stage stage) { request.put("template", pipelineTemplate); request.put("plan", true); - Map response = pipelineTemplatePipelinePreprocessor.process(request); + Map response = pipelineTemplatePreprocessor.process(request); if (response.containsKey("errors")) { errorResponses.put((String) dependentPipeline.get("id"), response.get("errors")); } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplateMerge.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplateMerge.java index 24ccb32bf3..9f47814114 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplateMerge.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplateMerge.java @@ -89,7 +89,13 @@ public static List mergeNamedContent(List a, List boolean updated = false; for (int i = 0; i < merged.size(); i++) { if (merged.get(i).getName().equals(bNode.getName())) { - merged.set(i, bNode); + if (bNode.isRemove()) { + merged.remove(i); + } else if (bNode.isMerge()) { + merged.set(i, (T) merged.get(i).merge(bNode)); + } else { + merged.set(i, bNode); + } updated = true; break; } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplatedPipelineModelMutator.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplatedPipelineModelMutator.java index f633da05a2..5dc418360a 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplatedPipelineModelMutator.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplatedPipelineModelMutator.java @@ -75,22 +75,29 @@ public void mutate(Map pipeline) { } } - mapPipelineConfigId(pipeline, configuration); - - pipeline.computeIfAbsent("application", k -> configuration.getPipeline().getApplication()); - pipeline.computeIfAbsent("name", k -> configuration.getPipeline().getName()); + mapRootPipelineConfigs(pipeline, configuration); applyConfigurations(configuration.getConfiguration(), pipeline); renderConfigurations(pipeline, RenderUtil.createDefaultRenderContext(template, configuration, null)); } @SuppressWarnings("unchecked") - private void mapPipelineConfigId(Map pipeline, TemplateConfiguration configuration) { + private void mapRootPipelineConfigs(Map pipeline, TemplateConfiguration configuration) { if (pipeline.containsKey("id") && pipeline.get("id") != configuration.getPipeline().getPipelineConfigId()) { Map config = (Map) pipeline.get("config"); Map p = (Map) config.get("pipeline"); p.put("pipelineConfigId", pipeline.get("id")); } + if (pipeline.containsKey("name") && pipeline.get("name") != configuration.getPipeline().getName()) { + Map config = (Map) pipeline.get("config"); + Map p = (Map) config.get("pipeline"); + p.put("name", pipeline.get("name")); + } + if (pipeline.containsKey("application") && pipeline.get("application") != configuration.getPipeline().getApplication()) { + Map config = (Map) pipeline.get("config"); + Map p = (Map) config.get("pipeline"); + p.put("application", pipeline.get("application")); + } } private void applyConfigurationsFromTemplate(PipelineConfiguration configuration, Configuration templateConfiguration, Map pipeline) { @@ -159,7 +166,7 @@ private void renderConfigurations(Map pipeline, RenderContext re if (pipeline.containsKey("triggers")) { pipeline.put("triggers", renderList((List) pipeline.get("triggers"), renderContext)); } - if (pipeline.containsKey("parameters")) { + if (pipeline.containsKey("parameterConfig")) { pipeline.put("parameterConfig", renderList((List) pipeline.get("parameterConfig"), renderContext)); } if (pipeline.containsKey("notifications")) { diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/V1SchemaExecutionGenerator.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/V1SchemaExecutionGenerator.java index 47f0155ce7..37750b9424 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/V1SchemaExecutionGenerator.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/V1SchemaExecutionGenerator.java @@ -51,6 +51,7 @@ public Map generate(PipelineTemplate template, TemplateConfigura } addNotifications(pipeline, template, configuration); + addParameters(pipeline, template, configuration); pipeline.put("stages", template.getStages() .stream() @@ -91,4 +92,21 @@ private void addNotifications(Map pipeline, PipelineTemplate tem ); } } + + private void addParameters(Map pipeline, PipelineTemplate template, TemplateConfiguration configuration) { + if (configuration.getConfiguration().getInherit().contains("parameters")) { + pipeline.put( + "parameterConfig", + TemplateMerge.mergeNamedContent( + template.getConfiguration().getParameters(), + configuration.getConfiguration().getParameters() + ) + ); + } else { + pipeline.put( + "parameterConfig", + Optional.ofNullable(configuration.getConfiguration().getParameters()).orElse(Collections.emptyList()) + ); + } + } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/converter/PipelineTemplateConverter.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/converter/PipelineTemplateConverter.java new file mode 100644 index 0000000000..f57fea682d --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/converter/PipelineTemplateConverter.java @@ -0,0 +1,171 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.converter; + +import com.google.common.base.Charsets; +import com.google.common.io.Files; +import com.google.common.io.Resources; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yaml.snakeyaml.DumperOptions; +import org.yaml.snakeyaml.Yaml; +import org.yaml.snakeyaml.representer.Representer; + +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +// Who needs type-checking anyway? +@SuppressWarnings("unchecked") +public class PipelineTemplateConverter { + + private final static Logger log = LoggerFactory.getLogger(PipelineTemplateConverter.class); + + public String convertToPipelineTemplate(Map pipeline) { + Map p = new LinkedHashMap<>(); + p.put("schema", "1"); + p.put("id", String.format("%s-%s", pipeline.getOrDefault("application", "spinnaker"), ((String) pipeline.getOrDefault("name", "generatedTemplate")).replaceAll("\\W", ""))); + p.put("metadata", generateMetadata(pipeline)); + p.put("protect", false); + p.put("configuration", generateConfiguration(pipeline)); + p.put("variables", new ArrayList<>()); + p.put("stages", convertStages((List) pipeline.get("stages"))); + + Representer representer = new Representer(); + DumperOptions options = new DumperOptions(); + options.setIndent(2); + options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK); + Yaml yaml = new Yaml(representer, options); + + String output = yaml.dump(p); + + return String.format("%s%s", loadTemplateHeader(), output); + } + + private Map generateMetadata(Map pipeline) { + Map m = new LinkedHashMap<>(); + m.put("name", pipeline.getOrDefault("name", "GIVE ME A NAME")); + m.put("description", pipeline.getOrDefault("description", "GIVE ME A DESCRIPTION")); + m.put("owner", pipeline.get("lastModifiedBy")); + m.put("scopes", (pipeline.get("application") == null) ? new ArrayList<>() : Collections.singletonList(pipeline.get("application"))); + return m; + } + + private Map generateConfiguration(Map pipeline) { + Map m = new LinkedHashMap<>(); + Map cm = new LinkedHashMap<>(); + cm.put("limitConcurrent", true); + m.put("concurrentExecutions", cm); + m.put("triggers", convertTriggers((List) pipeline.get("triggers"))); + m.put("parameters", pipeline.getOrDefault("parameterConfig", new ArrayList<>())); + m.put("notifications", convertNotifications((List) pipeline.get("notifications"))); + return m; + } + + private List> convertStages(List> stages) { + return stages.stream() + .map(s -> { + List dependsOn = new ArrayList<>(); + if (s.containsKey("requisiteStageRefIds") && !((List) s.get("requisiteStageRefIds")).isEmpty()) { + dependsOn = buildStageRefIds(stages, (List) s.get("requisiteStageRefIds")); + } + + Map stage = new LinkedHashMap<>(); + stage.put("id", getStageId((String) s.get("type"), (String) s.get("refId"))); + stage.put("type", s.get("type")); + stage.put("dependsOn", dependsOn); + stage.put("name", s.get("name")); + stage.put("config", scrubStageConfig(s)); + return stage; + }) + .collect(Collectors.toList()); + } + + private static Map scrubStageConfig(Map config) { + Map working = new LinkedHashMap<>(config); + working.remove("type"); + working.remove("name"); + working.remove("refId"); + working.remove("requisiteStageRefIds"); + return working; + } + + private static List buildStageRefIds(List> stages, List requisiteStageRefIds) { + List refIds = new ArrayList<>(); + for (String refId : requisiteStageRefIds) { + Optional stage = stages.stream() + .filter(s -> refId.equals(s.get("refId"))) + .map(s -> getStageId((String) s.get("type"), (String) s.get("refId"))) + .findFirst(); + stage.ifPresent(refIds::add); + } + return refIds; + } + + private static String getStageId(String type, String refId) { + return String.format("%s%s", type, refId); + } + + private List> convertTriggers(List> triggers) { + if (triggers == null) { + return Collections.emptyList(); + } + + List> ret = new ArrayList<>(triggers.size()); + + int i = 0; + for (Map trigger : triggers) { + trigger.put("name", String.format("unnamed%d", i)); + i++; + ret.add(trigger); + } + + return ret; + } + + private List> convertNotifications(List> notifications) { + if (notifications == null) { + return Collections.emptyList(); + } + + List> ret = new ArrayList<>(notifications.size()); + + int i = 0; + for (Map notification : notifications) { + notification.put("name", String.format("%s%d", notification.get("type"), i)); + i++; + ret.add(notification); + } + + return ret; + } + + private String loadTemplateHeader() { + try { + return Files.toString(new File(Resources.getResource("pipelineTemplateHeader.txt").toURI()), Charsets.UTF_8); + } catch (IOException | URISyntaxException e) { + log.error("Could not load pipeline template header resource", e); + return "GENERATED BY spinnaker"; + } + } +} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/GraphMutator.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/GraphMutator.java index 3d1106fc1c..a695a64936 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/GraphMutator.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/GraphMutator.java @@ -46,7 +46,7 @@ public GraphMutator(TemplateConfiguration configuration, Renderer renderer, Regi visitors.add(new RenderTransform(configuration, renderer, registry, trigger)); visitors.add(new ConfigStageInjectionTransform(configuration)); visitors.add(new StageInheritanceControlTransform()); - visitors.add(new ConditionalStanzaTransform(configuration, renderer, trigger)); + visitors.add(new ConditionalStanzaTransform(configuration, trigger)); visitors.add(new TrimConditionalsTransform()); } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConditionalStanzaTransform.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConditionalStanzaTransform.java index 84cd1d7f34..5032c02cac 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConditionalStanzaTransform.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConditionalStanzaTransform.java @@ -21,7 +21,6 @@ import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.DefaultRenderContext; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.RenderContext; -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.Renderer; import java.util.List; import java.util.Map; @@ -31,12 +30,10 @@ public class ConditionalStanzaTransform implements PipelineTemplateVisitor { private TemplateConfiguration templateConfiguration; - private Renderer renderer; private Map trigger; - public ConditionalStanzaTransform(TemplateConfiguration templateConfiguration, Renderer renderer, Map trigger) { + public ConditionalStanzaTransform(TemplateConfiguration templateConfiguration, Map trigger) { this.templateConfiguration = templateConfiguration; - this.renderer = renderer; this.trigger = trigger; } @@ -53,9 +50,9 @@ private void trimConditionals(List stages, PipelineTe .forEach(stage -> { RenderContext context = new DefaultRenderContext(templateConfiguration.getPipeline().getApplication(), template, trigger); context.getVariables().putAll(templateConfiguration.getPipeline().getVariables()); + // Conditionals have already been rendered for (String conditional : stage.getWhen()) { - String rendered = renderer.render(conditional, context); - if (!Boolean.parseBoolean(rendered)) { + if (!Boolean.parseBoolean(conditional)) { stage.setRemove(); } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConfigStageInjectionTransform.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConfigStageInjectionTransform.java index b6bae25a72..ee00b20eb6 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConfigStageInjectionTransform.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConfigStageInjectionTransform.java @@ -193,7 +193,7 @@ private static List createGraph(List stages) { private static void injectStages(List stages, List templateStages) { // Using a stream here can cause a ConcurrentModificationException. - for (StageDefinition s : stages) { + for (StageDefinition s : new ArrayList<>(stages)) { if (s.getInject() == null) { continue; } @@ -205,22 +205,22 @@ private static void injectStages(List stages, List targetIds, } stage.getRequisiteStageRefIds().addAll(targetEdges); - allStages.add(targetIndexes.last() + 1, stage); + allStages.add(targetIndexes.last(), stage); Map graph = allStages .stream() diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/DefaultVariableAssignmentTransform.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/DefaultVariableAssignmentTransform.java index d7aa5261f2..f258f2d588 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/DefaultVariableAssignmentTransform.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/DefaultVariableAssignmentTransform.java @@ -22,6 +22,8 @@ import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration; import org.apache.commons.lang3.StringUtils; +import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -36,18 +38,21 @@ public DefaultVariableAssignmentTransform(TemplateConfiguration templateConfigur @Override public void visitPipelineTemplate(PipelineTemplate pipelineTemplate) { - if (pipelineTemplate.getVariables() == null || pipelineTemplate.getVariables().isEmpty()) { + List pipelineTemplateVariables = pipelineTemplate.getVariables(); + if (pipelineTemplateVariables == null || pipelineTemplateVariables.isEmpty()) { return; } - Map configVars = templateConfiguration.getPipeline().getVariables(); + Map configVars = templateConfiguration.getPipeline().getVariables() != null + ? templateConfiguration.getPipeline().getVariables() + : new HashMap<>(); // if the config is missing vars and the template defines a default value, assign those values from the config - pipelineTemplate.getVariables().stream() + pipelineTemplateVariables.stream() .filter(templateVar -> !configVars.containsKey(templateVar.getName()) && templateVar.hasDefaultValue()) .forEach(templateVar -> configVars.put(templateVar.getName(), templateVar.getDefaultValue())); - List missingVariables = pipelineTemplate.getVariables().stream() + List missingVariables = pipelineTemplateVariables.stream() .filter(templateVar -> !configVars.containsKey(templateVar.getName())) .map(Variable::getName) .collect(Collectors.toList()); @@ -56,6 +61,31 @@ public void visitPipelineTemplate(PipelineTemplate pipelineTemplate) { throw new IllegalTemplateConfigurationException("Missing variable values for: " + StringUtils.join(missingVariables, ", ")); } - // TODO rz - validate variable values match the defined variable type + // collect variables where value type doesn't match the required type + List wrongTypeErrorMessages = pipelineTemplateVariables.stream() + .filter(templateVar -> { + String expectedType = templateVar.getType(); + if (expectedType.equalsIgnoreCase("object")) { + return false; // not invalid, all classes are objects + } + + Class actualType = configVars.get(templateVar.getName()).getClass(); + Object actualVar = configVars.get(templateVar.getName()); + + return !( + (expectedType.equalsIgnoreCase("int") && (actualVar instanceof Integer)) || + (expectedType.equalsIgnoreCase("bool") && actualVar instanceof Boolean) || + (expectedType.equalsIgnoreCase("list") && actualVar instanceof Collection) || + (expectedType.equalsIgnoreCase("string") && actualVar instanceof CharSequence) || + (expectedType.equalsIgnoreCase("float") && actualVar instanceof Float) || + (expectedType.equalsIgnoreCase(actualType.getSimpleName())) + ); + }) + .map(var -> var.getName() + " (expected type '" + var.getType() + "' found type '" + configVars.get(var.getName()).getClass().getSimpleName() + "')") + .collect(Collectors.toList()); + + if (!wrongTypeErrorMessages.isEmpty()) { + throw new IllegalTemplateConfigurationException("Incorrectly defined variable(s): " + StringUtils.join(wrongTypeErrorMessages, ", ")); + } } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/RenderTransform.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/RenderTransform.java index 99b0eb5205..7aa66ad655 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/RenderTransform.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/RenderTransform.java @@ -20,6 +20,7 @@ import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.IllegalTemplateConfigurationException; import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.TemplateRenderException; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.PipelineTemplateVisitor; +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.NamedHashMap; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PartialDefinition; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.StageDefinition; @@ -74,6 +75,15 @@ private void render(PipelineTemplate template) { // We don't care about configuration partials, they were already merged into the template at this point renderPartials(template.getPartials(), filterStages(template.getStages(), true), context); + + renderConfigurations(template.getConfiguration().getParameters(), context, "template:configuration.parameters"); + renderConfigurations(templateConfiguration.getConfiguration().getParameters(), context, "configuration:configuration.parameters"); + + renderConfigurations(template.getConfiguration().getTriggers(), context, "template:configuration.triggers"); + renderConfigurations(templateConfiguration.getConfiguration().getTriggers(), context, "configuration:configuration.triggers"); + + renderConfigurations(template.getConfiguration().getNotifications(), context, "template:configuration.notifications"); + renderConfigurations(templateConfiguration.getConfiguration().getNotifications(), context, "configuration:configuration.notifications"); } private void renderStages(List stages, RenderContext context, String locationNamespace) { @@ -117,11 +127,18 @@ private void renderStage(StageDefinition stage, RenderContext context, String lo stage.setName(renderStageProperty(stage.getName(), context, getStagePropertyLocation(locationNamespace, stage.getId(), "name"))); stage.setComments(renderStageProperty(stage.getComments(), context, getStagePropertyLocation(locationNamespace, stage.getId(), "comments"))); + stage.setWhen( + stage.getWhen() + .stream() + .map(w -> renderStageProperty(w, context, getStagePropertyLocation(locationNamespace, stage.getId(), "when"))) + .collect(Collectors.toList()) + ); } private String renderStageProperty(String input, RenderContext context, String location) { + Object result; try { - return (String) RenderUtil.deepRender(renderer, input, context); + result = RenderUtil.deepRender(renderer, input, context); } catch (TemplateRenderException e) { throw TemplateRenderException.fromError( new Error() @@ -130,6 +147,7 @@ private String renderStageProperty(String input, RenderContext context, String l e ); } + return (result == null) ? null : result.toString(); } private static String getStagePropertyLocation(String namespace, String stageId, String propertyName) { @@ -159,6 +177,8 @@ private void renderPartials(List partials, List partials, List configurations, RenderContext context, String location) { + if (configurations == null) { + return; + } + + for (Map config : configurations) { + for (Map.Entry pair : config.entrySet()) { + try { + pair.setValue(RenderUtil.deepRender(renderer, pair.getValue(), context)); + } catch (TemplateRenderException e) { + throw TemplateRenderException.fromError( + new Error() + .withMessage("Failed rendering configuration property") + .withLocation(location), + e + ); + } + } + } + } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/TrimConditionalsTransform.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/TrimConditionalsTransform.java index 6e29a532e4..889d945e07 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/TrimConditionalsTransform.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/TrimConditionalsTransform.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.graph.transform; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.PipelineTemplateVisitor; diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1Handlers.kt b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1Handlers.kt new file mode 100644 index 0000000000..28e3e5acc2 --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1Handlers.kt @@ -0,0 +1,111 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.handler + +import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spectator.api.Registry +import com.netflix.spinnaker.orca.pipelinetemplate.handler.Handler +import com.netflix.spinnaker.orca.pipelinetemplate.handler.HandlerChain +import com.netflix.spinnaker.orca.pipelinetemplate.handler.HandlerGroup +import com.netflix.spinnaker.orca.pipelinetemplate.handler.PipelineTemplateContext +import com.netflix.spinnaker.orca.pipelinetemplate.loader.TemplateLoader +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.V1SchemaExecutionGenerator +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.graph.GraphMutator +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.Renderer +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.validator.V1TemplateConfigurationSchemaValidator +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.validator.V1TemplateSchemaValidator +import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors +import org.springframework.beans.factory.annotation.Autowired +import org.springframework.stereotype.Component +import java.util.stream.Collectors + +@Component +class V1SchemaHandlerGroup +@Autowired constructor( + private val templateLoader: TemplateLoader, + private val renderer: Renderer, + private val objectMapper: ObjectMapper, + private val registry: Registry +): HandlerGroup { + + override fun getHandlers(): List + = listOf( + V1TemplateLoaderHandler(templateLoader, renderer, objectMapper), + V1ConfigurationValidationHandler(), + V1TemplateValidationHandler(), + V1GraphMutatorHandler(renderer, registry), + V1PipelineGenerator() + ) +} + +class V1ConfigurationValidationHandler : Handler { + override fun handle(chain: HandlerChain, context: PipelineTemplateContext) { + val errors = Errors() + + val ctx = context.getSchemaContext() + V1TemplateConfigurationSchemaValidator().validate( + ctx.configuration, + errors, + V1TemplateConfigurationSchemaValidator.SchemaValidatorContext( + ctx.template.stages.stream().map { it.id }.collect(Collectors.toList()) + ) + ) + if (errors.hasErrors(context.getRequest().plan)) { + context.getErrors().addAll(errors) + chain.clear() + } + } +} + +class V1TemplateValidationHandler : Handler { + override fun handle(chain: HandlerChain, context: PipelineTemplateContext) { + val errors = Errors() + V1TemplateSchemaValidator().validate( + context.getSchemaContext().template, + errors, + V1TemplateSchemaValidator.SchemaValidatorContext( + !context.getSchemaContext().configuration.stages.isEmpty() + ) + ) + if (errors.hasErrors(context.getRequest().plan)) { + context.getErrors().addAll(errors) + chain.clear() + } + } +} + +class V1GraphMutatorHandler( + private val renderer: Renderer, + private val registry: Registry +) : Handler { + + override fun handle(chain: HandlerChain, context: PipelineTemplateContext) { + val trigger = context.getRequest().trigger as MutableMap + val ctx = context.getSchemaContext() + + val mutator = GraphMutator(ctx.configuration, renderer, registry, trigger) + mutator.mutate(ctx.template) + } +} + +class V1PipelineGenerator : Handler { + + override fun handle(chain: HandlerChain, context: PipelineTemplateContext) { + val ctx = context.getSchemaContext() + val generator = V1SchemaExecutionGenerator() + context.getProcessedOutput().putAll(generator.generate(ctx.template, ctx.configuration, context.getRequest())) + } +} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1PipelineTemplateContext.kt b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1PipelineTemplateContext.kt new file mode 100644 index 0000000000..5757fd97db --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1PipelineTemplateContext.kt @@ -0,0 +1,25 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.handler + +import com.netflix.spinnaker.orca.pipelinetemplate.handler.PipelineTemplateSchemaContext +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration + +data class V1PipelineTemplateContext( + val configuration: TemplateConfiguration, + val template: PipelineTemplate +) : PipelineTemplateSchemaContext diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1TemplateLoaderHandler.kt b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1TemplateLoaderHandler.kt new file mode 100644 index 0000000000..70e78f214f --- /dev/null +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1TemplateLoaderHandler.kt @@ -0,0 +1,98 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.handler + +import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spinnaker.orca.pipelinetemplate.handler.Handler +import com.netflix.spinnaker.orca.pipelinetemplate.handler.HandlerChain +import com.netflix.spinnaker.orca.pipelinetemplate.handler.PipelineTemplateContext +import com.netflix.spinnaker.orca.pipelinetemplate.loader.TemplateLoader +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.TemplateMerge +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.DefaultRenderContext +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.RenderContext +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.Renderer +import java.util.stream.Collectors + +class V1TemplateLoaderHandler( + private val templateLoader: TemplateLoader, + private val renderer: Renderer, + private val objectMapper: ObjectMapper +) : Handler { + + override fun handle(chain: HandlerChain, context: PipelineTemplateContext) { + val config = objectMapper.convertValue(context.getRequest().config, TemplateConfiguration::class.java) + + // Allow template inlining to perform plans without publishing the template + if (context.getRequest().plan && context.getRequest().template != null) { + val template = objectMapper.convertValue(context.getRequest().template, PipelineTemplate::class.java) + val templates = templateLoader.load(template) + context.setSchemaContext(V1PipelineTemplateContext(config, TemplateMerge.merge(templates))) + return + } + + val trigger = context.getRequest().trigger as MutableMap? + setTemplateSourceWithJinja(config, trigger) + + // If a template source isn't provided by the configuration, we're assuming that the configuration is fully-formed. + val template: PipelineTemplate + if (config.pipeline.template == null) { + template = PipelineTemplate().apply { + variables = config.pipeline.variables.entries.stream() + .map { PipelineTemplate.Variable().apply { + name = it.key + defaultValue = it.value + }} + .collect(Collectors.toList()) + } + } else { + val templates = templateLoader.load(config.pipeline.template) + template = TemplateMerge.merge(templates) + } + + // ensure that any expressions contained with template variables are rendered + val renderContext = DefaultRenderContext(config.pipeline.application, template, trigger) + renderTemplateVariables(renderContext, template) + + context.setSchemaContext(V1PipelineTemplateContext( + config, + template + )) + } + + + private fun renderTemplateVariables(renderContext: RenderContext, pipelineTemplate: PipelineTemplate) { + if (pipelineTemplate.variables == null) { + return + } + + pipelineTemplate.variables.forEach { v -> + val value = v.defaultValue + if (value != null && value is String) { + v.defaultValue = renderer.renderGraph(value.toString(), renderContext) + } + } + } + + private fun setTemplateSourceWithJinja(tc: TemplateConfiguration, trigger: MutableMap?) { + if (trigger == null || tc.pipeline.template == null) { + return + } + val context = DefaultRenderContext(tc.pipeline.application, null, trigger) + tc.pipeline.template.source = renderer.render(tc.pipeline.template.source, context) + } +} diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/NamedContent.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/NamedContent.java index dbe11216da..da1ffa6dcc 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/NamedContent.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/NamedContent.java @@ -15,7 +15,10 @@ */ package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model; -public interface NamedContent { +public interface NamedContent { String getName(); + boolean isRemove(); + boolean isMerge(); + T merge(T overlay); } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/NamedHashMap.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/NamedHashMap.java index 7c38a97e3c..9dcdfb2d6c 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/NamedHashMap.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/NamedHashMap.java @@ -17,9 +17,34 @@ import java.util.HashMap; -public class NamedHashMap extends HashMap implements NamedContent { +public class NamedHashMap extends HashMap implements NamedContent { + + public NamedHashMap() { + } + + private NamedHashMap(NamedHashMap m) { + super(m); + } public String getName() { return String.valueOf(get("name")); } + + @Override + public boolean isRemove() { + return Boolean.parseBoolean(String.valueOf(getOrDefault("remove", "false"))); + } + + @Override + public boolean isMerge() { + return Boolean.parseBoolean(String.valueOf(getOrDefault("merge", "false"))); + } + + @Override + public NamedHashMap merge(NamedHashMap overlay) { + NamedHashMap m = new NamedHashMap(this); + m.putAll(overlay); + m.remove("merge"); + return m; + } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/PipelineTemplate.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/PipelineTemplate.java index 928bad630c..6f7dc934f5 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/PipelineTemplate.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/PipelineTemplate.java @@ -33,7 +33,7 @@ public class PipelineTemplate implements VersionedSchema { private String source; private Metadata metadata = new Metadata(); private Boolean protect = false; - private List variables; + private List variables = new ArrayList<>(); private Configuration configuration; private List stages; private List modules; @@ -78,13 +78,15 @@ public void setScopes(List scopes) { } } - public static class Variable implements NamedContent { + public static class Variable implements NamedContent, Cloneable { private String name; - private String group = "General"; + private String group; private String description; - private String type = "string"; + private String type; private Object defaultValue; private String example; + private boolean merge = false; + private boolean remove = false; @Override public String getName() { @@ -96,7 +98,7 @@ public void setName(String name) { } public String getGroup() { - return group; + return Optional.ofNullable(group).orElse("General"); } public void setGroup(String group) { @@ -112,7 +114,7 @@ public void setDescription(String description) { } public String getType() { - return type; + return Optional.ofNullable(type).orElse("object"); } public void setType(String type) { @@ -138,6 +140,43 @@ public String getExample() { public void setExample(String example) { this.example = example; } + + public boolean isMerge() { + return merge; + } + + public void setMerge(boolean merge) { + this.merge = merge; + } + + public boolean isRemove() { + return remove; + } + + public void setRemove(boolean remove) { + this.remove = remove; + } + + @Override + public Variable merge(Variable overlay) { + Variable v; + try { + v = (Variable) this.clone(); + } catch (CloneNotSupportedException e) { + throw new RuntimeException("Could not clone Variable", e); + } + if (overlay.group != null) { v.group = overlay.group; } + if (overlay.description != null) { v.description = overlay.description; } + if (overlay.type != null) { v.type = overlay.type; } + if (overlay.defaultValue != null) { v.defaultValue = overlay.defaultValue; } + if (overlay.example != null) { v.example = overlay.example; } + return v; + } + + @Override + protected Object clone() throws CloneNotSupportedException { + return super.clone(); + } } public static class Configuration { diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/TemplateConfiguration.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/TemplateConfiguration.java index 4dfa5b1df5..d85245cc29 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/TemplateConfiguration.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/TemplateConfiguration.java @@ -30,7 +30,7 @@ public class TemplateConfiguration implements VersionedSchema { private String schema; - private PipelineDefinition pipeline; + private PipelineDefinition pipeline = new PipelineDefinition(); private PipelineConfiguration configuration = new PipelineConfiguration(); private List stages; private List modules; diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/TemplateModule.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/TemplateModule.java index eaad0e6248..0073ec7564 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/TemplateModule.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/model/TemplateModule.java @@ -15,13 +15,14 @@ */ package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model; +import java.util.ArrayList; import java.util.List; public class TemplateModule implements Identifiable { private String id; private String usage; - private List variables; + private List variables = new ArrayList<>(); private Object definition; @Override diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/JinjaRenderer.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/JinjaRenderer.java index 8fef37541d..7db92bb9cf 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/JinjaRenderer.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/JinjaRenderer.java @@ -39,9 +39,6 @@ import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors; import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors.Error; import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors.Severity; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.yaml.snakeyaml.parser.ParserException; @@ -49,8 +46,11 @@ import java.io.IOException; import java.nio.charset.Charset; import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; public class JinjaRenderer implements Renderer { diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/YamlRenderedValueConverter.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/YamlRenderedValueConverter.java index bd966ce047..59f5b737b4 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/YamlRenderedValueConverter.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/YamlRenderedValueConverter.java @@ -39,9 +39,12 @@ public YamlRenderedValueConverter(Yaml yaml) { @Override public Object convertRenderedValue(String renderedValue) { - if (containsEL(renderedValue) || isYamlKeyword(renderedValue)) { + if (containsEL(renderedValue) || isYamlKeyword(renderedValue) || containsYamlParsingExceptions(renderedValue)) { return renderedValue; } + if (containsNoExpandMarker(renderedValue)) { + return trimNoExpandMarker(renderedValue); + } try { Object converted = yaml.load(renderedValue); @@ -67,4 +70,17 @@ private static boolean containsEL(String renderedValue) { private static boolean isYamlKeyword(String renderedValue) { return YAML_KEYWORDS.contains(renderedValue.toLowerCase()); } + + private static boolean containsYamlParsingExceptions(String renderedValue) { + return renderedValue != null && + renderedValue.startsWith("* "); // A markdown list: YAML will parse this as an alias and fail. + } + + private static boolean containsNoExpandMarker(String renderedValue) { + return renderedValue.startsWith("noexpand:"); + } + + private static String trimNoExpandMarker(String renderedValue) { + return renderedValue.substring("noexpand:".length(), renderedValue.length()); + } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/ModuleTag.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/ModuleTag.java index 3ceba35f03..0b7f3bd4cb 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/ModuleTag.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/ModuleTag.java @@ -46,7 +46,6 @@ public class ModuleTag implements Tag { private Renderer renderer; - private ObjectMapper objectMapper; public ModuleTag(Renderer renderer, ObjectMapper pipelineTemplateObjectMapper) { @@ -144,8 +143,12 @@ public String interpret(TagNode tagNode, JinjavaInterpreter interpreter) { ); } + if (rendered instanceof CharSequence) { + return (String) rendered; + } + try { - return new String(objectMapper.writeValueAsBytes(rendered)); + return objectMapper.writeValueAsString(rendered); } catch (JsonProcessingException e) { throw TemplateRenderException.fromError( new Error() diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/PipelineIdTag.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/PipelineIdTag.java index 17227a8511..35d2b9be2c 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/PipelineIdTag.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/PipelineIdTag.java @@ -59,27 +59,38 @@ public String interpret(TagNode tagNode, JinjavaInterpreter interpreter) { }); Context context = interpreter.getContext(); + String application = paramPairs.getOrDefault(APPLICATION, (String) context.get(APPLICATION)).replaceAll("^[\"\']|[\"\']$", ""); - String name = paramPairs.get(NAME).replaceAll("^[\"\']|[\"\']$", ""); + application = checkContext(application, context); - if (name == null || application == null) { - throw new TemplateSyntaxException(tagNode.getMaster().getImage(), "Tag 'pipelineId' is missing required fields: " + helper, tagNode.getLineNumber()); - } + String name = paramPairs.get(NAME).replaceAll("^[\"\']|[\"\']$", ""); + name = checkContext(name, context); List> pipelines = Optional.ofNullable(front50Service.getPipelines(application, false)).orElse(Collections.emptyList()); + Map result = findPipeline(pipelines, application, name); + return (String) result.get("id"); + } - Map result = pipelines - .stream() - .filter(p -> p.get(NAME).equals(name)) - .findFirst() - .orElseThrow( - () -> TemplateRenderException.fromError( - new Error() - .withMessage(String.format("Failed to find pipeline ID with name '%s' in application '%s'", name, application) - )) - ); + private String checkContext(String param, Context context) { + Object var = context.get(param); - return (String) result.get("id"); + if (var != null) { + return (String) var; + } + + return param; + } + + private Map findPipeline(List> pipelines, String application, String pipelineName) { + return pipelines + .stream() + .filter(p -> p.get(NAME).equals(pipelineName)) + .findFirst() + .orElseThrow( + () -> TemplateRenderException.fromError( + new Error() + .withMessage(String.format("Failed to find pipeline ID with name '%s' in application '%s'", pipelineName, application) + ))); } @Override diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateConfigurationSchemaValidator.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateConfigurationSchemaValidator.java index e53e73326a..2538091f2b 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateConfigurationSchemaValidator.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateConfigurationSchemaValidator.java @@ -15,9 +15,9 @@ */ package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.validator; +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.StageDefinition; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration; import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration.PipelineDefinition; -import com.netflix.spinnaker.orca.pipelinetemplate.validator.EmptyValidatorContext; import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors; import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors.Error; import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors.Severity; @@ -25,16 +25,14 @@ import com.netflix.spinnaker.orca.pipelinetemplate.validator.ValidatorContext; import com.netflix.spinnaker.orca.pipelinetemplate.validator.VersionedSchema; -public class V1TemplateConfigurationSchemaValidator implements SchemaValidator { +import java.util.List; - private static final String SUPPORTED_VERSION = "1"; +public class V1TemplateConfigurationSchemaValidator implements SchemaValidator { - public void validate(VersionedSchema configuration, Errors errors) { - validate(configuration, errors, new EmptyValidatorContext()); - } + private static final String SUPPORTED_VERSION = "1"; @Override - public void validate(VersionedSchema configuration, Errors errors, ValidatorContext context) { + public void validate(VersionedSchema configuration, Errors errors, SchemaValidatorContext context) { if (!(configuration instanceof TemplateConfiguration)) { throw new IllegalArgumentException("Expected TemplateConfiguration"); } @@ -63,18 +61,31 @@ public void validate(VersionedSchema configuration, Errors errors, ValidatorCont V1SchemaValidationHelper.validateStageDefinitions(config.getStages(), errors, V1TemplateConfigurationSchemaValidator::location); config.getStages().forEach(s -> { - if ((s.getDependsOn() == null || s.getDependsOn().isEmpty()) && (s.getInject() == null || !s.getInject().hasAny())) { + if (shouldRequireDagRules(s, config, context.stageIds)) { errors.add(new Error() .withMessage("A configuration-defined stage should have either dependsOn or an inject rule defined") .withLocation(location(String.format("stages.%s", s.getId()))) .withSeverity(Severity.WARN)); } }); + } - // TODO rz - validate required variables are set and of the correct type + private static boolean shouldRequireDagRules(StageDefinition s, TemplateConfiguration config, List stageIds) { + return config.getPipeline().getTemplate() != null && + !stageIds.contains(s.getId()) && + (s.getDependsOn() == null || s.getDependsOn().isEmpty()) && + (s.getInject() == null || !s.getInject().hasAny()); } private static String location(String location) { return "configuration:" + location; } + + public static class SchemaValidatorContext implements ValidatorContext { + List stageIds; + + public SchemaValidatorContext(List stageIds) { + this.stageIds = stageIds; + } + } } diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateSchemaValidator.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateSchemaValidator.java index 6964f9be4a..19d4d6cb18 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateSchemaValidator.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateSchemaValidator.java @@ -47,8 +47,6 @@ public void validate(VersionedSchema pipelineTemplate, Errors errors, SchemaVali } V1SchemaValidationHelper.validateStageDefinitions(template.getStages(), errors, V1TemplateSchemaValidator::location); - - // TODO rz - validate variable type & defaultValue combinations } private static String location(String location) { diff --git a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/validator/Errors.java b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/validator/Errors.java index 6c67250756..a151be8c10 100644 --- a/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/validator/Errors.java +++ b/orca-pipelinetemplate/src/main/java/com/netflix/spinnaker/orca/pipelinetemplate/validator/Errors.java @@ -140,7 +140,7 @@ public Errors getNestedErrors() { public Map toResponse() { HashMap err = new HashMap<>(); - err.put("severity", severity); + err.put("severity", severity.toString()); if (message != null) err.put("message", message); if (location != null) err.put("location", location); if (cause != null) err.put("cause", cause); diff --git a/orca-pipelinetemplate/src/main/resources/pipelineTemplateHeader.txt b/orca-pipelinetemplate/src/main/resources/pipelineTemplateHeader.txt new file mode 100644 index 0000000000..e50eb2eb54 --- /dev/null +++ b/orca-pipelinetemplate/src/main/resources/pipelineTemplateHeader.txt @@ -0,0 +1,18 @@ +# GENERATED BY spinnaker +# +# The template generated below should be used as a base for further modifications. +# It does not make assumptions as to what things can be made into variables, +# modules, partials or Jinja expressions. This is your responsibility as the owner +# of the template. +# +# Some recommendations to massage the initial output: +# +# * Give your pipeline template a unique ID. It's best to namespace it by your +# application or team name, so that it does not conflict with other teams, +# e.g. "myteam-myTemplate". +# * Rename the pipeline stage IDs, notifications and trigger names to be more +# meaningful. Enumerated stage IDs are ultimately a detriment for long-term +# maintenance of your template. +# * Best intentions are made to order configuration, but the list of stages +# themselves are not ordered: Rearrange the stages so that they're roughly +# chronological. diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePipelinePreprocessorSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePipelinePreprocessorSpec.groovy index 88ddffc738..6b218a0ef0 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePipelinePreprocessorSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/PipelineTemplatePipelinePreprocessorSpec.groovy @@ -17,16 +17,18 @@ package com.netflix.spinnaker.orca.pipelinetemplate import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spectator.api.Clock +import com.netflix.spectator.api.Counter +import com.netflix.spectator.api.Id import com.netflix.spectator.api.Registry import com.netflix.spectator.api.Timer import com.netflix.spinnaker.orca.front50.Front50Service +import com.netflix.spinnaker.orca.pipelinetemplate.handler.PipelineTemplateErrorHandler +import com.netflix.spinnaker.orca.pipelinetemplate.handler.SchemaVersionHandler import com.netflix.spinnaker.orca.pipelinetemplate.loader.FileTemplateSchemeLoader import com.netflix.spinnaker.orca.pipelinetemplate.loader.TemplateLoader -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.handler.V1SchemaHandlerGroup import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.StageDefinition -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.JinjaRenderer -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.RenderUtil import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.Renderer import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.YamlRenderedValueConverter import org.unitils.reflectionassert.ReflectionComparatorMode @@ -52,13 +54,20 @@ class PipelineTemplatePipelinePreprocessorSpec extends Specification { monotonicTime() >> 0L } timer(_) >> Mock(Timer) + counter(_) >> Mock(Counter) + createId(_) >> Mock(Id) } @Subject - PipelineTemplatePipelinePreprocessor subject = new PipelineTemplatePipelinePreprocessor( + PipelineTemplatePreprocessor subject = new PipelineTemplatePreprocessor( objectMapper, - templateLoader, - renderer, + new SchemaVersionHandler(new V1SchemaHandlerGroup( + templateLoader, + renderer, + objectMapper, + registry + )), + new PipelineTemplateErrorHandler(), registry ) @@ -152,7 +161,8 @@ class PipelineTemplatePipelinePreprocessorSpec extends Specification { requisiteStageRefIds: ['tagImage'], waitTime: 5 ] - ] + ], + parameterConfig: [] ] assertReflectionEquals(expected, result, ReflectionComparatorMode.IGNORE_DEFAULTS) } @@ -198,71 +208,6 @@ class PipelineTemplatePipelinePreprocessorSpec extends Specification { false || ['wait'] || ['wait', 'childOfConditionalStage'] } - @Unroll - def 'should be able to set source using jinja'() { - when: - def result = subject.process(createInjectedTemplateRequest(template)) - - then: - result.stages*.name == expectedStageNames - - where: - template || expectedStageNames - 'jinja-001.yml' || ['jinja1'] - 'jinja-002.yml' || ['jinja2'] - } - - def 'should allow inlined templates during plan'() { - when: - def result = subject.process(createInlinedTemplateRequest(true)) - - then: - noExceptionThrown() - 0 * templateLoader.load(_) - result.stages*.name == ['wait'] - - when: - result = subject.process(createInlinedTemplateRequest(false)) - - then: - result.errors != null - } - - @Unroll - def 'should render jinja expressions contained within template variables'() { - given: - def pipelineTemplate = new PipelineTemplate(variables: templateVariables.collect { - new PipelineTemplate.Variable(name: it.key, defaultValue: it.value) - }) - - def templateConfig = new TemplateConfiguration( - pipeline: new TemplateConfiguration.PipelineDefinition(variables: configVariables) - ) - - def renderContext = RenderUtil.createDefaultRenderContext( - pipelineTemplate, templateConfig, [ - parameters: [ - "list" : "us-west-2,us-east-1", - "boolean": "true", - "string" : "this is a string" - ] - ]) - - when: - subject.renderTemplateVariables(renderContext, pipelineTemplate) - - then: - pipelineTemplate.variables*.defaultValue == expectedDefaultValues - - where: - templateVariables | configVariables || expectedDefaultValues - [key1: "string1", key2: "string2"] | [:] || ["string1", "string2"] - [key1: "{{ trigger.parameters.string }}", key2: "string2"] | [:] || ["this is a string", "string2"] - [key1: "{{ trigger.parameters.list | split(',') }}", key2: "string2"] | [:] || [["us-west-2", "us-east-1"], "string2"] - [key1: "string1", key2: "{{ key1 }}"] | [:] || ["string1", "string1"] - [key2: "{{ key1 }}"] | [key1: "string1"] || ["string1"] - } - def "should include group for partials-generated stages"() { def pipeline = [ type: 'templatedPipeline', @@ -319,6 +264,15 @@ class PipelineTemplatePipelinePreprocessorSpec extends Specification { result.stages*.group == ['my group of stages: wowow waiting', 'my group of stages: wowow waiting'] } + def 'should render stage config variables used in partials'() { + when: + def template = createTemplateRequest('conditional-partials.yml', [includeWait: false]) + def result = subject.process(template) + + then: + result.stages*.shouldBeSet == ["isSet"] + } + @Unroll def 'should not render falsy conditional stages inside partials'() { when: @@ -384,51 +338,4 @@ class PipelineTemplatePipelinePreprocessorSpec extends Specification { plan: plan ] } - - Map createInjectedTemplateRequest(String templatePath) { - return [ - type: 'templatedPipeline', - trigger: [ - parameters: [ - template: getClass().getResource("/templates/${templatePath}").toURI() - ] - ], - config: [ - schema: '1', - pipeline: [ - application: 'myapp', - template: [ - source: '{{trigger.parameters.template}}' - ], - ], - ], - plan: false - ] - } - - Map createInlinedTemplateRequest(boolean plan) { - return [ - type: 'templatedPipeline', - config: [ - schema: '1', - pipeline: [ - application: 'myapp' - ] - ], - template: [ - schema: '1', - id: 'myTemplate', - stages: [ - [ - id: 'wait', - type: 'wait', - config: [ - waitTime: 5 - ] - ] - ] - ], - plan: plan - ] - } } diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/TemplatedPipelineRequestSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/TemplatedPipelineRequestSpec.groovy new file mode 100644 index 0000000000..48bca16814 --- /dev/null +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/TemplatedPipelineRequestSpec.groovy @@ -0,0 +1,77 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate + +import com.fasterxml.jackson.databind.DeserializationFeature +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.databind.SerializationFeature +import com.fasterxml.jackson.module.kotlin.KotlinModule +import spock.lang.Specification +import spock.lang.Unroll + +class TemplatedPipelineRequestSpec extends Specification { + + def objectMapper = new ObjectMapper() + .enable(SerializationFeature.INDENT_OUTPUT) + .disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) + .registerModule(new KotlinModule()) + + @Unroll + def 'should deserialize config'() { + when: + objectMapper.convertValue(pipeline, TemplatedPipelineRequest.class) + + then: + noExceptionThrown() + + where: + pipeline << [ + [ + application: 'spindemo', + name: 'test', + stages: [ + type: 'wait', + waitTime: 5 + ], + limitConcurrent: true, + appConfig: [:] + ], + [ + application: 'spindemo', + name: 'test', + stages: [ + type: 'wait', + waitTime: 5 + ], + limitConcurrent: true, + appConfig: [:], + config: null + ], + [ + application: 'spindemo', + name: 'test', + stages: [ + type: 'wait', + waitTime: 5 + ], + limitConcurrent: true, + appConfig: [:], + config: [:] + ] + ] + } + +} diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/pipeline/UpdatePipelineTemplateStageSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/pipeline/UpdatePipelineTemplateStageSpec.groovy new file mode 100644 index 0000000000..d324d303c6 --- /dev/null +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/pipeline/UpdatePipelineTemplateStageSpec.groovy @@ -0,0 +1,101 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.pipeline + +import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spinnaker.orca.front50.Front50Service +import com.netflix.spinnaker.orca.front50.pipeline.UpdatePipelineStage +import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.model.Stage +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate.Configuration +import spock.lang.Specification + +class UpdatePipelineTemplateStageSpec extends Specification { + + def front50Service = Mock(Front50Service) + def stageBuilder = new UpdatePipelineTemplateStage() + + def setup() { + stageBuilder.updatePipelineStage = new UpdatePipelineStage() + stageBuilder.pipelineTemplateObjectMapper = new ObjectMapper() + stageBuilder.front50Service = front50Service + } + + def "should create synthetic save pipeline stages for each dependent pipeline"() { + setup: + def pipelineTemplate = new PipelineTemplate( + schema: "1", + id: 'myTemplate', + metadata: [ + name: 'myTemplate' + ], + configuration: new Configuration( + triggers: [ + [ + name: 'myTrigger', + type: 'jenkins' + ] + ] + ), + variables: [] + ) + + def pipeline1 = [ + id: 'one', + config: [ + configuration: [ + inherit: ['triggers'] + ] + ] + ] + def pipeline2 = [ + id: 'two', + config: [ + configuration: [ + inherit: ['triggers'] + ] + ] + ] + def pipeline3 = [ + id: 'three', + config: [ + configuration: [ + inherit: [] + ] + ] + ] + + and: + def config = [pipelineTemplate: Base64.encoder.encodeToString(new ObjectMapper().writeValueAsString(pipelineTemplate).bytes)] + def stage = new Stage<>(new Pipeline("orca"), "updatePipelineTemplate", config) + + when: + def syntheticStages = stageBuilder.aroundStages(stage) + def beforeStages = syntheticStages.findAll { it.syntheticStageOwner == SyntheticStageOwner.STAGE_BEFORE } + def afterStages = syntheticStages.findAll { it.syntheticStageOwner == SyntheticStageOwner.STAGE_AFTER } + + then: + 1 * front50Service.getPipelineTemplateDependents("myTemplate", true) >> { + [pipeline1, pipeline2, pipeline3] + } + + afterStages.size() == 2 + afterStages*.name == ['updateDependentPipeline', 'updateDependentPipeline'] + beforeStages.size() == 0 + } +} diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/tasks/PlanTemplateDependentsTaskSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/tasks/PlanTemplateDependentsTaskSpec.groovy index abfbefe17f..6e2e9e2943 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/tasks/PlanTemplateDependentsTaskSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/tasks/PlanTemplateDependentsTaskSpec.groovy @@ -20,7 +20,7 @@ import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.front50.Front50Service import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.model.Stage -import com.netflix.spinnaker.orca.pipelinetemplate.PipelineTemplatePipelinePreprocessor +import com.netflix.spinnaker.orca.pipelinetemplate.PipelineTemplatePreprocessor import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate import spock.lang.Specification import spock.lang.Subject @@ -29,7 +29,7 @@ class PlanTemplateDependentsTaskSpec extends Specification { Front50Service front50Service = Mock() - PipelineTemplatePipelinePreprocessor pipelinePreprocessor = Mock() + PipelineTemplatePreprocessor pipelinePreprocessor = Mock() ObjectMapper objectMapper = new ObjectMapper() @@ -37,7 +37,7 @@ class PlanTemplateDependentsTaskSpec extends Specification { def task = new PlanTemplateDependentsTask( front50Service: front50Service, pipelineTemplateObjectMapper: objectMapper, - pipelineTemplatePipelinePreprocessor: pipelinePreprocessor + pipelineTemplatePreprocessor: pipelinePreprocessor ) def 'should aggregate all failed pipeline plan errors'() { diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplateMergeSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplateMergeSpec.groovy index 1510aed1fc..31b0a9dc86 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplateMergeSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplateMergeSpec.groovy @@ -44,6 +44,12 @@ class TemplateMergeSpec extends Specification { parameters: [ new NamedHashMap().with { put('name', 'parameter1') + put('description', 'blah blah') + put('defaultValue', 'default') + return it + }, + new NamedHashMap().with { + put('name', 'parameter2') return it } ], @@ -74,6 +80,19 @@ class TemplateMergeSpec extends Specification { put('name', 'trigger2') return it } + ], + parameters: [ + new NamedHashMap().with { + put('name', 'parameter1') + put('merge', true) + put('defaultValue', 'overridden') + return it + }, + new NamedHashMap().with { + put('name', 'parameter2') + put('remove', true) + return it + } ] ) stages = [ @@ -95,6 +114,8 @@ class TemplateMergeSpec extends Specification { result.variables.find { it.name == 'foo' }.defaultValue == 'overridden value' result.configuration.triggers*.name == ['trigger1', 'trigger2'] result.configuration.parameters*.name == ['parameter1'] + result.configuration.parameters*.description == ['blah blah'] + result.configuration.parameters*.defaultValue == ['overridden'] result.configuration.notifications*.name == ['notification1'] result.stages*.id == ['s1', 's2', 's3'] result.modules == null diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplatedPipelineModelMutatorSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplatedPipelineModelMutatorSpec.groovy index 1daa0386ef..8b714628a7 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplatedPipelineModelMutatorSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/TemplatedPipelineModelMutatorSpec.groovy @@ -137,4 +137,31 @@ class TemplatedPipelineModelMutatorSpec extends Specification { )] } pipeline.id == pipeline.config.pipeline.pipelineConfigId } + + def "should map root-level configurations to mpt format if undefined"() { + given: + def pipeline = [ + id: 'id', + name: 'my pipeline', + application: 'orca', + config: [ + schema: '1', + pipeline: [ + template: [ + source: 'static-template' + ] + ] + ] + ] + + when: + subject.mutate(pipeline) + + then: + 1 * templateLoader.load(_) >> { [new PipelineTemplate( + schema: '1' + )] } + pipeline.name == pipeline.config.pipeline.name + pipeline.application == pipeline.config.pipeline.application + } } diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/V1SchemaIntegrationSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/V1SchemaIntegrationSpec.groovy index 48fdc6a3a6..b5a6c27066 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/V1SchemaIntegrationSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/V1SchemaIntegrationSpec.groovy @@ -17,12 +17,17 @@ package com.netflix.spinnaker.orca.pipelinetemplate.v1schema import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spectator.api.Clock +import com.netflix.spectator.api.Counter +import com.netflix.spectator.api.Id import com.netflix.spectator.api.Registry import com.netflix.spectator.api.Timer import com.netflix.spinnaker.orca.front50.Front50Service -import com.netflix.spinnaker.orca.pipelinetemplate.PipelineTemplatePipelinePreprocessor +import com.netflix.spinnaker.orca.pipelinetemplate.PipelineTemplatePreprocessor +import com.netflix.spinnaker.orca.pipelinetemplate.handler.PipelineTemplateErrorHandler +import com.netflix.spinnaker.orca.pipelinetemplate.handler.SchemaVersionHandler import com.netflix.spinnaker.orca.pipelinetemplate.loader.ResourceSchemeLoader import com.netflix.spinnaker.orca.pipelinetemplate.loader.TemplateLoader +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.handler.V1SchemaHandlerGroup import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.JinjaRenderer @@ -33,7 +38,6 @@ import org.springframework.core.io.support.PathMatchingResourcePatternResolver import org.unitils.reflectionassert.ReflectionComparatorMode import org.yaml.snakeyaml.Yaml import spock.lang.Specification -import spock.lang.Subject import spock.lang.Unroll import static org.unitils.reflectionassert.ReflectionAssert.assertReflectionEquals @@ -53,20 +57,27 @@ class V1SchemaIntegrationSpec extends Specification { monotonicTime() >> 0L } timer(_) >> Mock(Timer) + createId(_) >> Mock(Id) + counter(_) >> Mock(Counter) } - @Subject - PipelineTemplatePipelinePreprocessor subject = new PipelineTemplatePipelinePreprocessor( - objectMapper, - templateLoader, - renderer, - registry - ) - @Unroll - def 'test "#integration.name"'() { - expect: - assertReflectionEquals(integration.expected, subject.process(integration.toRequest()), ReflectionComparatorMode.IGNORE_DEFAULTS) + def 'test handler strategy "#integration.name"'() { + given: + PipelineTemplatePreprocessor subject = new PipelineTemplatePreprocessor( + objectMapper, + new SchemaVersionHandler(new V1SchemaHandlerGroup(templateLoader, renderer, objectMapper, registry)), + new PipelineTemplateErrorHandler(), + registry + ) + + def expected = integration.expected + + when: + def result = subject.process(integration.toRequest()) + + then: + assertReflectionEquals(expected, result, ReflectionComparatorMode.IGNORE_DEFAULTS) where: integration << new IntegrationTestDataProvider().provide() @@ -126,6 +137,7 @@ class V1SchemaIntegrationSpec extends Specification { } def req = [ + schema: '1', type: 'templatedPipeline', trigger: [:], config: configuration, diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/converter/PipelineTemplateConverterSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/converter/PipelineTemplateConverterSpec.groovy new file mode 100644 index 0000000000..1c83922425 --- /dev/null +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/converter/PipelineTemplateConverterSpec.groovy @@ -0,0 +1,36 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.converter + +import groovy.json.JsonSlurper +import spock.lang.Specification + +class PipelineTemplateConverterSpec extends Specification { + + def "should convert a pipeline to an ordered pipeline template yaml document"() { + given: + def pipeline = new JsonSlurper().parse(new File("src/test/resources/convertedPipelineTemplateSource.json")) + + and: + String expected = new File("src/test/resources/convertedPipelineTemplate.yml").text + + when: + String result = new PipelineTemplateConverter().convertToPipelineTemplate(pipeline) + + then: + expected == result + } +} diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConditionalStanzaTransformSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConditionalStanzaTransformSpec.groovy index f9d97d5792..0b25a563e9 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConditionalStanzaTransformSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/ConditionalStanzaTransformSpec.groovy @@ -19,7 +19,6 @@ import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTempla import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.StageDefinition import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration.PipelineDefinition -import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.Renderer import spock.lang.Specification import spock.lang.Subject @@ -29,13 +28,7 @@ class ConditionalStanzaTransformSpec extends Specification { pipeline: new PipelineDefinition(variables: [:]) ) - Renderer renderer = Mock() { - render(_, _) >> { conditional, context -> - return (conditional == 'true').toString() - } - } - - @Subject ConditionalStanzaTransform subject = new ConditionalStanzaTransform(configuration, renderer, [:]) + @Subject ConditionalStanzaTransform subject = new ConditionalStanzaTransform(configuration, [:]) def 'should remove falsy conditional stages'() { given: diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/RenderTransformSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/RenderTransformSpec.groovy index 159699de1d..c004ab0d4b 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/RenderTransformSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/graph/transform/RenderTransformSpec.groovy @@ -91,6 +91,9 @@ class RenderTransformSpec extends Specification { channel: '{{slackChannel}}', when: ['awaiting'] ] + ], + when: [ + "{{ application == 'blah' }}" ] ), new StageDefinition( @@ -153,6 +156,7 @@ class RenderTransformSpec extends Specification { ] ] findStage(template, 'deploy').name == 'Deploy to Env' + findStage(template, 'manualjudgment').when == ['false'] } def 'should render partials'() { diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1TemplateLoaderHandlerSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1TemplateLoaderHandlerSpec.groovy new file mode 100644 index 0000000000..abf6ea45dc --- /dev/null +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/handler/V1TemplateLoaderHandlerSpec.groovy @@ -0,0 +1,237 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.handler + +import com.fasterxml.jackson.databind.ObjectMapper +import com.netflix.spinnaker.orca.front50.Front50Service +import com.netflix.spinnaker.orca.pipelinetemplate.TemplatedPipelineRequest +import com.netflix.spinnaker.orca.pipelinetemplate.handler.DefaultHandlerChain +import com.netflix.spinnaker.orca.pipelinetemplate.handler.GlobalPipelineTemplateContext +import com.netflix.spinnaker.orca.pipelinetemplate.loader.FileTemplateSchemeLoader +import com.netflix.spinnaker.orca.pipelinetemplate.loader.TemplateLoader +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.JinjaRenderer +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.RenderUtil +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.Renderer +import spock.lang.Specification +import spock.lang.Subject +import spock.lang.Unroll + +class V1TemplateLoaderHandlerSpec extends Specification { + + ObjectMapper objectMapper = new ObjectMapper() + + Renderer renderer = new JinjaRenderer(objectMapper, Mock(Front50Service), []) + + TemplateLoader templateLoader = new TemplateLoader([new FileTemplateSchemeLoader(objectMapper)]) + + @Subject + def subject = new V1TemplateLoaderHandler(templateLoader, renderer, objectMapper) + + def 'should create stub template when no template is provided'() { + given: + def configuration = [ + schema: '1', + pipeline: [ + application: 'orca', + name: 'My Template', + variables: [ + foo: 'bar' + ] + ], + stages: [ + [ + id: 'wait', + type: 'wait', + config: [ + waitTime: 5 + ] + ] + ] + ] + + and: + def chain = new DefaultHandlerChain() + def context = new GlobalPipelineTemplateContext(chain, new TemplatedPipelineRequest( + config: configuration + )) + + when: + subject.handle(chain, context) + + then: + noExceptionThrown() + context.schemaContext != null + (context.schemaContext as V1PipelineTemplateContext).template.variables*.name == ['foo'] + (context.schemaContext as V1PipelineTemplateContext).template.variables*.defaultValue == ['bar'] + } + + @Unroll + def 'should render jinja expressions contained within template variables'() { + given: + def pipelineTemplate = new PipelineTemplate(variables: templateVariables.collect { + new PipelineTemplate.Variable(name: it.key, defaultValue: it.value) + }) + + def templateConfig = new TemplateConfiguration( + pipeline: new TemplateConfiguration.PipelineDefinition(variables: configVariables) + ) + + def renderContext = RenderUtil.createDefaultRenderContext( + pipelineTemplate, templateConfig, [ + parameters: [ + "list" : "us-west-2,us-east-1", + "boolean": "true", + "string" : "this is a string" + ] + ]) + + when: + subject.renderTemplateVariables(renderContext, pipelineTemplate) + + then: + pipelineTemplate.variables*.defaultValue == expectedDefaultValues + + where: + templateVariables | configVariables || expectedDefaultValues + [key1: "string1", key2: "string2"] | [:] || ["string1", "string2"] + [key1: "{{ trigger.parameters.string }}", key2: "string2"] | [:] || ["this is a string", "string2"] + [key1: "string1", key2: "{{ key1 }}"] | [:] || ["string1", "string1"] + [key2: "{{ key1 }}"] | [key1: "string1"] || ["string1"] + } + + @Unroll + def 'should be able to set source using jinja'() { + given: + def chain = new DefaultHandlerChain() + def context = new GlobalPipelineTemplateContext(chain, createInjectedTemplateRequest(template)) + + when: + subject.handle(chain, context) + + then: + ((V1PipelineTemplateContext) context.getSchemaContext()).template.stages*.name == expectedStageNames + + where: + template || expectedStageNames + 'jinja-001.yml' || ['jinja1'] + 'jinja-002.yml' || ['jinja2'] + } + + def 'should allow inlined templates during plan'() { + given: + def chain = new DefaultHandlerChain() + def context = new GlobalPipelineTemplateContext(chain, createInlinedTemplateRequest(true)) + + when: + subject.handle(chain, context) + + then: + noExceptionThrown() + ((V1PipelineTemplateContext) context.getSchemaContext()).template.stages*.name == ['wait'] + } + + def 'should load parent templates of inlined template during plan'() { + given: + def chain = new DefaultHandlerChain() + def context = new GlobalPipelineTemplateContext(chain, createInlinedTemplateRequestWithParent(true, 'jinja-001.yml')) + + when: + subject.handle(chain, context) + + then: + noExceptionThrown() + ((V1PipelineTemplateContext) context.getSchemaContext()).template.stages*.name == ['jinja1', 'childTemplateWait'] + } + + TemplatedPipelineRequest createInjectedTemplateRequest(String templatePath) { + return new TemplatedPipelineRequest( + type: 'templatedPipeline', + trigger: [ + parameters: [ + template: getClass().getResource("/templates/${templatePath}").toURI() + ] + ], + config: [ + schema: '1', + pipeline: [ + application: 'myapp', + template: [ + source: '{{trigger.parameters.template}}' + ], + ], + ], + plan: false + ) + } + + TemplatedPipelineRequest createInlinedTemplateRequest(boolean plan) { + return new TemplatedPipelineRequest( + type: 'templatedPipeline', + config: [ + schema: '1', + pipeline: [ + application: 'myapp' + ] + ], + template: [ + schema: '1', + id: 'myTemplate', + stages: [ + [ + id: 'wait', + type: 'wait', + config: [ + waitTime: 5 + ] + ] + ] + ], + plan: plan + ) + } + + TemplatedPipelineRequest createInlinedTemplateRequestWithParent(boolean plan, String templatePath) { + return new TemplatedPipelineRequest( + type: 'templatedPipeline', + config: [ + schema: '1', + pipeline: [ + application: 'myapp' + ] + ], + template: [ + schema: '1', + id: 'myTemplate', + stages: [ + [ + id: 'childTemplateWait', + type: 'wait', + config: [ + waitTime: 5 + ], + inject: [ + last: true + ] + ] + ], + source: getClass().getResource("/templates/${templatePath}").toURI() + ], + plan: plan + ) + } +} diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/JinjaRendererSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/JinjaRendererSpec.groovy index 804427c7ce..5246c9639c 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/JinjaRendererSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/JinjaRendererSpec.groovy @@ -87,6 +87,8 @@ class JinjaRendererSpec extends Specification { '${ #stage("First Wait")["status"].toString() == "SUCCESS" }' || String | '${ #stage("First Wait")["status"].toString() == "SUCCESS" }' '${ parameters.CONFIG_FOLDER ?: \'\' }' || String | '${ parameters.CONFIG_FOLDER ?: \'\' }' '' || String | null + '* markdown list' || String | '* markdown list' + 'noexpand:{"t": "deployment"}' || String | '{"t": "deployment"}' } diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/ModuleTagSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/ModuleTagSpec.groovy index 405fa0b0ff..634a8afe93 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/ModuleTagSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/ModuleTagSpec.groovy @@ -58,7 +58,6 @@ class ModuleTagSpec extends Specification { def result = renderer.render('{% module myModule myOtherVar=world, subject=testerName, job=trigger.job %}', context) then: - // The ModuleTag outputs JSON - result == '"hello world, Mr. Tester Testington. You triggered myJob"' + result == 'hello world, Mr. Tester Testington. You triggered myJob' } } diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/PipelineIdTagSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/PipelineIdTagSpec.groovy index 841a76c6cb..6e96669b4c 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/PipelineIdTagSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/render/tags/PipelineIdTagSpec.groovy @@ -21,6 +21,7 @@ import com.netflix.spinnaker.orca.front50.Front50Service import com.netflix.spinnaker.orca.pipelinetemplate.exceptions.TemplateRenderException import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.DefaultRenderContext import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.JinjaRenderer +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.RenderContext import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.render.Renderer import spock.lang.Specification import spock.lang.Subject @@ -77,7 +78,26 @@ class PipelineIdTagSpec extends Specification { '{% pipelineId name="Rob\'s great pipeline" %}' || '1685429e-beb1-4d35-963c-123456789012' } - def 'should handle missing input params'() { + def 'should render pipeline id using variables defined in context'() { + given: + front50Service.getPipelines('myApp', false) >> [ + [ + name: 'Bake and Tag', + application: 'myApp', + id: '9595429f-afa0-4c34-852b-01a9a01967f9', + stages: [] + ] + ] + + RenderContext context = new DefaultRenderContext('myApp', null, [:]) + context.variables.put("pipelineName", "Bake and Tag") + context.variables.put("applicationName", "myApp") + + expect: + renderer.render('{% pipelineId application=applicationName name=pipelineName %}', context) == '9595429f-afa0-4c34-852b-01a9a01967f9' + } + + def 'should handle missing input params'() { given: 'a pipelineId tag with no app defined' def applicationInContext = 'myApp' def context = new DefaultRenderContext(applicationInContext,null, [:]) diff --git a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateConfigurationSchemaValidatorSpec.groovy b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateConfigurationSchemaValidatorSpec.groovy index d9437155c8..e4634b79ba 100644 --- a/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateConfigurationSchemaValidatorSpec.groovy +++ b/orca-pipelinetemplate/src/test/groovy/com/netflix/spinnaker/orca/pipelinetemplate/v1schema/validator/V1TemplateConfigurationSchemaValidatorSpec.groovy @@ -18,6 +18,7 @@ package com.netflix.spinnaker.orca.pipelinetemplate.v1schema.validator import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.StageDefinition import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration.PipelineDefinition +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration.TemplateSource import com.netflix.spinnaker.orca.pipelinetemplate.validator.Errors import spock.lang.Specification import spock.lang.Subject @@ -34,7 +35,7 @@ class V1TemplateConfigurationSchemaValidatorSpec extends Specification { def templateConfiguration = new TemplateConfiguration(schema: schema) when: - subject.validate(templateConfiguration, errors) + subject.validate(templateConfiguration, errors, new V1TemplateConfigurationSchemaValidator.SchemaValidatorContext([])) then: if (hasErrors) { @@ -56,7 +57,7 @@ class V1TemplateConfigurationSchemaValidatorSpec extends Specification { def templateConfiguration = new TemplateConfiguration(schema: "1", pipeline: new PipelineDefinition(application: application)) when: - subject.validate(templateConfiguration, errors) + subject.validate(templateConfiguration, errors, new V1TemplateConfigurationSchemaValidator.SchemaValidatorContext([])) then: if (hasErrors) { @@ -78,7 +79,10 @@ class V1TemplateConfigurationSchemaValidatorSpec extends Specification { def errors = new Errors() def templateConfiguration = new TemplateConfiguration( schema: "1", - pipeline: new PipelineDefinition(application: 'myapp'), + pipeline: new PipelineDefinition( + application: 'myapp', + template: new TemplateSource() + ), stages: [ new StageDefinition( id: 'foo', @@ -89,10 +93,11 @@ class V1TemplateConfigurationSchemaValidatorSpec extends Specification { ) when: - subject.validate(templateConfiguration, errors) + subject.validate(templateConfiguration, errors, new V1TemplateConfigurationSchemaValidator.SchemaValidatorContext(templateStageIds)) then: if (hasErrors) { + errors.hasErrors(true) errors.errors[0].message == "A configuration-defined stage should have either dependsOn or an inject rule defined" errors.errors[0].location == 'configuration:stages.foo' } else { @@ -100,9 +105,10 @@ class V1TemplateConfigurationSchemaValidatorSpec extends Specification { } where: - dependsOn | injectFirst | hasErrors - null | true | false - ['bar'] | false | false - null | null | true + dependsOn | injectFirst | templateStageIds | hasErrors + null | true | [] | false + ['bar'] | false | [] | false + null | null | [] | true + null | null | ["foo"] | false } } diff --git a/orca-pipelinetemplate/src/test/resources/convertedPipelineTemplate.yml b/orca-pipelinetemplate/src/test/resources/convertedPipelineTemplate.yml new file mode 100644 index 0000000000..0c33bfa1f5 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/convertedPipelineTemplate.yml @@ -0,0 +1,199 @@ +# GENERATED BY spinnaker +# +# The template generated below should be used as a base for further modifications. +# It does not make assumptions as to what things can be made into variables, +# modules, partials or Jinja expressions. This is your responsibility as the owner +# of the template. +# +# Some recommendations to massage the initial output: +# +# * Give your pipeline template a unique ID. It's best to namespace it by your +# application or team name, so that it does not conflict with other teams, +# e.g. "myteam-myTemplate". +# * Rename the pipeline stage IDs, notifications and trigger names to be more +# meaningful. Enumerated stage IDs are ultimately a detriment for long-term +# maintenance of your template. +# * Best intentions are made to order configuration, but the list of stages +# themselves are not ordered: Rearrange the stages so that they're roughly +# chronological. +schema: '1' +id: spinnaker-generatedTemplate +metadata: + name: GIVE ME A NAME + description: This is my favorite pipeline! + owner: example@example.com + scopes: [] +protect: false +configuration: + concurrentExecutions: + limitConcurrent: true + triggers: + - enabled: false + job: ZZ-demo + master: myMaster + name: unnamed0 + type: jenkins + - cronExpression: 0 0/12 * * * ? + enabled: true + id: b2ba0819-dbe5-42bd-a8b0-0499c131711f + name: unnamed1 + type: cron + parameters: [] + notifications: + - address: example@example.com + level: pipeline + name: email0 + type: email + when: + - pipeline.failed +variables: [] +stages: +- id: bake1 + type: bake + dependsOn: [] + name: Bake + config: + baseLabel: release + baseOs: trusty + cloudProviderType: aws + enhancedNetworking: false + extendedAttributes: {} + overrideTimeout: true + package: orca + regions: + - us-east-1 + - us-west-1 + - us-west-2 + - eu-west-1 + sendNotifications: true + showAdvancedOptions: true + stageTimeoutMs: 900000 + storeType: ebs + user: example@example.com + vmType: hvm +- id: deploy2 + type: deploy + dependsOn: + - bake1 + name: Deploy + config: + clusters: + - account: test + application: spindemo + availabilityZones: + us-west-1: + - us-west-1a + - us-west-1c + capacity: + desired: 1 + max: 1 + min: 1 + cloudProvider: aws + cooldown: 10 + copySourceCustomBlockDeviceMappings: true + ebsOptimized: false + enabledMetrics: [] + freeFormDetails: demo + healthCheckGracePeriod: 600 + healthCheckType: EC2 + iamRole: myIAMRole + instanceMonitoring: false + instanceType: m3.large + interestingHealthProviderNames: + - Amazon + keyPair: keypair + loadBalancers: + - spindemo-demo-frontend + maxRemainingAsgs: 2 + preferSourceCapacity: true + provider: aws + scaleDown: true + securityGroups: [] + stack: test + strategy: redblack + subnetType: mySubnet + suspendedProcesses: [] + tags: {} + targetGroups: [] + targetHealthyDeployPercentage: 100 + terminationPolicies: + - Default + useAmiBlockDeviceMappings: false + useSourceCapacity: true + - account: test + application: spindemo + availabilityZones: + us-east-1: + - us-east-1c + - us-east-1d + - us-east-1e + capacity: + desired: 0 + max: 0 + min: 0 + cloudProvider: aws + cooldown: 10 + ebsOptimized: false + freeFormDetails: demo + healthCheckGracePeriod: 600 + healthCheckType: EC2 + iamRole: myIAMRole + instanceMonitoring: false + instanceType: m3.large + interestingHealthProviderNames: + - Amazon + keyPair: keypair + provider: aws + securityGroups: [] + stack: test + strategy: highlander + subnetType: mySubnet + suspendedProcesses: [] + tags: {} + targetHealthyDeployPercentage: 100 + terminationPolicies: + - Default + useSourceCapacity: false +- id: checkPreconditions6 + type: checkPreconditions + dependsOn: + - wait8 + name: Check Preconditions (us-west-1) + config: + preconditions: + - context: + cluster: spindemo-test-demo + comparison: <= + credentials: test + expected: 2 + regions: + - us-west-1 + failPipeline: true + type: clusterSize +- id: checkPreconditions7 + type: checkPreconditions + dependsOn: + - wait8 + name: Check Preconditions (us-east-1) + config: + completeOtherBranchesThenFail: false + continuePipeline: false + failPipeline: true + preconditions: + - context: + cluster: spindemo-test-demo + comparison: == + credentials: test + expected: 1 + regions: + - us-east-1 + failPipeline: true + type: clusterSize +- id: wait8 + type: wait + dependsOn: + - deploy2 + name: Wait + config: + comments: Wait 2 min for the clusters to normalize before the precondition asg size checks + waitTime: 120 diff --git a/orca-pipelinetemplate/src/test/resources/convertedPipelineTemplateSource.json b/orca-pipelinetemplate/src/test/resources/convertedPipelineTemplateSource.json new file mode 100644 index 0000000000..57fd7197d6 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/convertedPipelineTemplateSource.json @@ -0,0 +1,222 @@ +{ + "appConfig": {}, + "description": "This is my favorite pipeline!", + "executionEngine": "v3", + "lastModifiedBy": "example@example.com", + "limitConcurrent": true, + "notifications": [ + { + "address": "example@example.com", + "level": "pipeline", + "type": "email", + "when": [ + "pipeline.failed" + ] + } + ], + "parallel": true, + "parameterConfig": [], + "stageCounter": 7, + "stages": [ + { + "baseLabel": "release", + "baseOs": "trusty", + "cloudProviderType": "aws", + "enhancedNetworking": false, + "extendedAttributes": {}, + "name": "Bake", + "overrideTimeout": true, + "package": "orca", + "refId": "1", + "regions": [ + "us-east-1", + "us-west-1", + "us-west-2", + "eu-west-1" + ], + "requisiteStageRefIds": [], + "sendNotifications": true, + "showAdvancedOptions": true, + "stageTimeoutMs": 900000, + "storeType": "ebs", + "type": "bake", + "user": "example@example.com", + "vmType": "hvm" + }, + { + "clusters": [ + { + "account": "test", + "application": "spindemo", + "availabilityZones": { + "us-west-1": [ + "us-west-1a", + "us-west-1c" + ] + }, + "capacity": { + "desired": 1, + "max": 1, + "min": 1 + }, + "cloudProvider": "aws", + "cooldown": 10, + "copySourceCustomBlockDeviceMappings": true, + "ebsOptimized": false, + "enabledMetrics": [], + "freeFormDetails": "demo", + "healthCheckGracePeriod": 600, + "healthCheckType": "EC2", + "iamRole": "myIAMRole", + "instanceMonitoring": false, + "instanceType": "m3.large", + "interestingHealthProviderNames": [ + "Amazon" + ], + "keyPair": "keypair", + "loadBalancers": [ + "spindemo-demo-frontend" + ], + "maxRemainingAsgs": 2, + "preferSourceCapacity": true, + "provider": "aws", + "scaleDown": true, + "securityGroups": [], + "stack": "test", + "strategy": "redblack", + "subnetType": "mySubnet", + "suspendedProcesses": [], + "tags": {}, + "targetGroups": [], + "targetHealthyDeployPercentage": 100, + "terminationPolicies": [ + "Default" + ], + "useAmiBlockDeviceMappings": false, + "useSourceCapacity": true + }, + { + "account": "test", + "application": "spindemo", + "availabilityZones": { + "us-east-1": [ + "us-east-1c", + "us-east-1d", + "us-east-1e" + ] + }, + "capacity": { + "desired": 0, + "max": 0, + "min": 0 + }, + "cloudProvider": "aws", + "cooldown": 10, + "ebsOptimized": false, + "freeFormDetails": "demo", + "healthCheckGracePeriod": 600, + "healthCheckType": "EC2", + "iamRole": "myIAMRole", + "instanceMonitoring": false, + "instanceType": "m3.large", + "interestingHealthProviderNames": [ + "Amazon" + ], + "keyPair": "keypair", + "provider": "aws", + "securityGroups": [], + "stack": "test", + "strategy": "highlander", + "subnetType": "mySubnet", + "suspendedProcesses": [], + "tags": {}, + "targetHealthyDeployPercentage": 100, + "terminationPolicies": [ + "Default" + ], + "useSourceCapacity": false + } + ], + "name": "Deploy", + "refId": "2", + "requisiteStageRefIds": [ + "1" + ], + "type": "deploy" + }, + { + "name": "Check Preconditions (us-west-1)", + "preconditions": [ + { + "context": { + "cluster": "spindemo-test-demo", + "comparison": "<=", + "credentials": "test", + "expected": 2, + "regions": [ + "us-west-1" + ] + }, + "failPipeline": true, + "type": "clusterSize" + } + ], + "refId": "6", + "requisiteStageRefIds": [ + "8" + ], + "type": "checkPreconditions" + }, + { + "completeOtherBranchesThenFail": false, + "continuePipeline": false, + "failPipeline": true, + "name": "Check Preconditions (us-east-1)", + "preconditions": [ + { + "context": { + "cluster": "spindemo-test-demo", + "comparison": "==", + "credentials": "test", + "expected": 1, + "regions": [ + "us-east-1" + ] + }, + "failPipeline": true, + "type": "clusterSize" + } + ], + "refId": "7", + "requisiteStageRefIds": [ + "8" + ], + "type": "checkPreconditions" + }, + { + "comments": "Wait 2 min for the clusters to normalize before the precondition asg size checks", + "name": "Wait", + "refId": "8", + "requisiteStageRefIds": [ + "2" + ], + "type": "wait", + "waitTime": 120 + } + ], + "triggers": [ + { + "enabled": false, + "job": "ZZ-demo", + "master": "myMaster", + "type": "jenkins" + }, + { + "cronExpression": "0 0/12 * * * ?", + "enabled": true, + "id": "b2ba0819-dbe5-42bd-a8b0-0499c131711f", + "type": "cron" + } + ], + "updateTs": "1504243528000" +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-config.yml new file mode 100644 index 0000000000..08eb90a3da --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-config.yml @@ -0,0 +1,7 @@ +--- +schema: "1" +pipeline: + application: orca + variables: + includeStage: false +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-expected.json new file mode 100644 index 0000000000..c87852b78c --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-expected.json @@ -0,0 +1,10 @@ +{ + "id": "unknown", + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "Unnamed Execution", + "stages": [], + "notifications": [], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-template.yml new file mode 100644 index 0000000000..a95086f5f8 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/conditionalStage-template.yml @@ -0,0 +1,15 @@ +--- +schema: "1" +id: conditionalStage +metadata: + name: Conditional stage test + description: Tests conditional expressions +variables: +- name: includeStage +stages: +- id: wait + type: wait + config: + waitTime: 5 + when: + - "{{ includeStage }}" diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-child.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-child.yml new file mode 100644 index 0000000000..cdf8823e29 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-child.yml @@ -0,0 +1,13 @@ +schema: "1" +id: child-template +source: example-root.yml # Indicates that this template inherits from the root-template +metadata: + name: Child template + description: A child template +stages: +- id: waitChild1 + type: wait + dependsOn: + - wait1 # Depending on a stage from the root-template + config: + waitTime: "{{ waitTime }}" # Using a variable from the root-template diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-config.yml new file mode 100644 index 0000000000..43b139efc3 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-config.yml @@ -0,0 +1,27 @@ +schema: "1" +pipeline: + application: myApp + name: My super awesome pipeline + template: + source: example-template.yml + variables: + waitTime: 20 + childWaitTime: 15 +configuration: + notifications: + - address: example@example.com + level: pipeline + name: email0 + type: email + when: + - pipeline.failed + triggers: [] +stages: +- id: finalWait + type: wait + dependsOn: + - waitChild1 + - waitChild2 + config: + waitTime: 10 + diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-expected.json new file mode 100644 index 0000000000..24fa5c105c --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-expected.json @@ -0,0 +1,51 @@ +{ + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "myApp", + "name": "My super awesome pipeline", + "stages": [ + { + "requisiteStageRefIds": [], + "name": "wait1", + "id": null, + "refId": "wait1", + "type": "wait", + "waitTime": 20 + }, + { + "requisiteStageRefIds": ["wait1"], + "id": null, + "name": "waitChild2", + "refId": "waitChild2", + "type": "wait", + "waitTime": 15 + }, + { + "requisiteStageRefIds": ["wait1"], + "id": null, + "name": "waitChild1", + "refId": "waitChild1", + "type": "wait", + "waitTime": 20 + }, + { + "requisiteStageRefIds": ["waitChild2", "waitChild1"], + "id": null, + "name": "finalWait", + "refId": "finalWait", + "type": "wait", + "waitTime": 10 + } + ], + "id": "unknown", + "notifications": [ + { + "address": "example@example.com", + "level": "pipeline", + "name": "email0", + "when": ["pipeline.failed"], + "type": "email" + } + ], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-root.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-root.yml new file mode 100644 index 0000000000..eb3dce161c --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-root.yml @@ -0,0 +1,14 @@ +schema: "1" # Schema must match for all +id: root-template +metadata: + name: Simple wait template + description: Extendable root template +variables: # Variables available to all that inherit +- name: waitTime + description: The time a wait stage should pause + type: int +stages: # Stages available to all that inherit +- id: wait1 + type: wait + config: + waitTime: "{{ waitTime }}" # Variables can be used anywhere diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-template.yml new file mode 100644 index 0000000000..907dbffdf1 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/example-template.yml @@ -0,0 +1,16 @@ +schema: "1" +id: child-2-template +source: example-child.yml +variables: +- name: childWaitTime + description: pause time for another wait +metadata: + name: A Second Child template + description: A second child template +stages: +- id: waitChild2 + type: wait + dependsOn: + - wait1 + config: + waitTime: "{{ childWaitTime }}" diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-config.yml new file mode 100644 index 0000000000..e2c87d799a --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-config.yml @@ -0,0 +1,20 @@ +schema: "1" +pipeline: + application: myApp + name: My super awesome pipeline + template: + source: exampleCombined-template.yml + variables: + waitTime: 20 + childWaitTime: 15 +configuration: + notifications: + - address: example@example.com + level: pipeline + name: email0 + type: email + when: + - pipeline.failed + triggers: [] + + diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-expected.json new file mode 100644 index 0000000000..24fa5c105c --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-expected.json @@ -0,0 +1,51 @@ +{ + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "myApp", + "name": "My super awesome pipeline", + "stages": [ + { + "requisiteStageRefIds": [], + "name": "wait1", + "id": null, + "refId": "wait1", + "type": "wait", + "waitTime": 20 + }, + { + "requisiteStageRefIds": ["wait1"], + "id": null, + "name": "waitChild2", + "refId": "waitChild2", + "type": "wait", + "waitTime": 15 + }, + { + "requisiteStageRefIds": ["wait1"], + "id": null, + "name": "waitChild1", + "refId": "waitChild1", + "type": "wait", + "waitTime": 20 + }, + { + "requisiteStageRefIds": ["waitChild2", "waitChild1"], + "id": null, + "name": "finalWait", + "refId": "finalWait", + "type": "wait", + "waitTime": 10 + } + ], + "id": "unknown", + "notifications": [ + { + "address": "example@example.com", + "level": "pipeline", + "name": "email0", + "when": ["pipeline.failed"], + "type": "email" + } + ], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-template.yml new file mode 100644 index 0000000000..ee575968f5 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/exampleCombined-template.yml @@ -0,0 +1,35 @@ +schema: "1" +id: combined-template +metadata: + name: Less simple wait template + description: A template with many waits +variables: +- name: waitTime + description: The time a wait stage should pause + type: int +- name: childWaitTime + description: pause time for another wait +stages: +- id: wait1 + type: wait + config: + waitTime: "{{ waitTime }}" +- id: waitChild1 + type: wait + dependsOn: + - wait1 + config: + waitTime: "{{ waitTime }}" +- id: waitChild2 + type: wait + dependsOn: + - wait1 + config: + waitTime: "{{ childWaitTime }}" +- id: finalWait + type: wait + dependsOn: + - waitChild1 + - waitChild2 + config: + waitTime: 10 diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/inheritance-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/inheritance-expected.json index 333195cb1f..528c521220 100644 --- a/orca-pipelinetemplate/src/test/resources/integration/v1schema/inheritance-expected.json +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/inheritance-expected.json @@ -30,5 +30,6 @@ } ], "id": "unknown", - "notifications": [] + "notifications": [], + "parameterConfig": [] } diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-config.yml new file mode 100644 index 0000000000..dcad0248bc --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-config.yml @@ -0,0 +1,6 @@ +schema: "1" +pipeline: + application: orca + name: Invalid Stage Configs Test + variables: + waitTime: 5 diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-expected.json new file mode 100644 index 0000000000..0edb772ab3 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-expected.json @@ -0,0 +1,29 @@ +{ + "errors": [ + { + "severity": "FATAL", + "message": "Stage ID is unset", + "location": "template:stages" + }, + { + "severity": "FATAL", + "message": "Stage is missing type", + "location": "template:stages.wait1" + }, + { + "severity": "FATAL", + "message": "Stage configuration is unset", + "location": "template:stages.wait2" + }, + { + "severity": "FATAL", + "message": "A stage cannot have both dependsOn and an inject rule defined simultaneously", + "location": "template:stages.wait3" + }, + { + "severity": "FATAL", + "message": "A stage cannot have multiple inject rules defined", + "location": "template:stages.wait4" + } + ] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-template.yml new file mode 100644 index 0000000000..099cfde60b --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/invalidStageFormats-template.yml @@ -0,0 +1,38 @@ +--- +schema: "1" +id: simple +metadata: + name: Wait + description: Simple waiting template. +variables: +- name: waitTime + description: The time a wait stage should pause + type: int +stages: +- id: goodwait + type: wait + config: + waitTime: "{{ waitTime }}" +- type: wait + config: + waitTime: "{{ waitTime }}" +- id: wait1 + config: + waitTime: 10 +- id: wait2 + type: wait +- id: wait3 + type: wait + config: + waitTime: 10 + dependsOn: + - goodWait + inject: + before: [wait2] +- id: wait4 + type: wait + config: + waitTime: 10 + inject: + after: [wait2] + before: [wait2] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-config.yml new file mode 100644 index 0000000000..12f11993f1 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-config.yml @@ -0,0 +1,13 @@ +--- +schema: "1" +pipeline: + application: orca + variables: + regions: + - us-east-1 + - us-west-2 +stages: [] +modules: +- id: wait + definition: + foo: 10 diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-expected.json new file mode 100644 index 0000000000..5ad1ed2be0 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-expected.json @@ -0,0 +1,40 @@ +{ + "id": "unknown", + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "Unnamed Execution", + "stages": [ + { + "requisiteStageRefIds": [], + "name": "wait", + "id": null, + "refId": "wait", + "type": "wait", + "someConfig": { + "foo": 10 + } + }, + { + "requisiteStageRefIds": ["wait"], + "name": "deploy", + "id": null, + "refId": "deploy", + "type": "deploy", + "clusters": [ + { + "provider": "aws", + "account": "myAccount", + "region": "us-east-1" + }, + { + "provider": "aws", + "account": "myAccount", + "region": "us-west-2" + } + ] + } + ], + "notifications": [], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-template.yml new file mode 100644 index 0000000000..dcb6c6a7a0 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/modules-template.yml @@ -0,0 +1,38 @@ +--- +schema: "1" +id: modules +metadata: + name: Modules test + description: Tests modules functionality +variables: +- name: regions + type: list +stages: +- id: wait + type: wait + config: + someConfig: "{% module wait %}" +- id: deploy + type: deploy + dependsOn: + - wait + config: + clusters: | + {% for region in regions %} + - {% module deployClusterAws region=region %} + {% endfor %} + +modules: +- id: wait + usage: Defines config for a wait stage + definition: + foo: 5 +- id: deployClusterAws + usage: Defines a deploy stage cluster using the AWS cloud provider + variables: + - name: region + description: The AWS region to deploy into + definition: + provider: aws + account: myAccount + region: "{{ region }}" diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-config.yml new file mode 100644 index 0000000000..eaf622325f --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-config.yml @@ -0,0 +1,5 @@ +--- +schema: "1" +pipeline: + application: orca +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-expected.json new file mode 100644 index 0000000000..43066f8efe --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-expected.json @@ -0,0 +1,20 @@ +{ + "id": "unknown", + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "Unnamed Execution", + "stages": [ + { + "refId": "foo.wait", + "requisiteStageRefIds": [], + "type": "wait", + "name": "wait", + "waitTime": 5, + "id": null, + "group": "myPartial: foo" + } + ], + "notifications": [], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-template.yml new file mode 100644 index 0000000000..9da4eb2e51 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/nestedConditionalStage-template.yml @@ -0,0 +1,31 @@ +--- +schema: "1" +id: nestedConditionalStage +metadata: + name: Nested partials conditional test + description: Asserts that stages inside of a partial can be conditionally excluded +stages: +- id: foo + type: partial.myPartial + config: + includeStage: false + +partials: +- id: myPartial + usage: It has stages. + variables: + - name: includeStage + description: Whether the nested stage should be included + stages: + - id: wait + type: wait + config: + waitTime: 5 + - id: wait2 + type: wait + dependsOn: + - wait + config: + waitTime: 5 + when: + - "{{ includeStage }}" diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-config.yml new file mode 100644 index 0000000000..4a38693a9b --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-config.yml @@ -0,0 +1,13 @@ +--- +schema: "1" +pipeline: + application: orca +configuration: + inherit: ['parameters'] + parameters: + - name: instances + default: 9000 + merge: true + - name: toBeRemoved + remove: true +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-expected.json new file mode 100644 index 0000000000..6d1cb9d9bd --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-expected.json @@ -0,0 +1,16 @@ +{ + "id": "unknown", + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "Unnamed Execution", + "stages": [], + "notifications": [], + "parameterConfig": [ + { + "name": "instances", + "description": "Number of instances to start per region", + "default": 9000 + } + ] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-template.yml new file mode 100644 index 0000000000..d66812e66f --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/parameterInheritance-template.yml @@ -0,0 +1,14 @@ +--- +schema: "1" +id: parameterInheritance +metadata: + name: Granular parameter & config inheritance + description: Asserts that configs can modify inherited parameters, as well as remove unneeded ones. +configuration: + parameters: + - name: instances + description: Number of instances to start per region + default: 3 + - name: toBeRemoved + description: This parameter will be removed by the config +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-config.yml new file mode 100644 index 0000000000..eaf622325f --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-config.yml @@ -0,0 +1,5 @@ +--- +schema: "1" +pipeline: + application: orca +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-expected.json new file mode 100644 index 0000000000..23a2ab5fd6 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-expected.json @@ -0,0 +1,45 @@ +{ + "id": "unknown", + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "Unnamed Execution", + "stages": [ + { + "requisiteStageRefIds": [], + "name": "firstWait", + "id": null, + "refId": "firstWait", + "type": "wait", + "waitTime": 5 + }, + { + "requisiteStageRefIds": ["firstWait"], + "name": "Build chrome", + "id": null, + "refId": "buildChrome.buildTarget", + "type": "jenkins", + "group": "buildBrowser: buildChrome", + "foo": "bar" + }, + { + "requisiteStageRefIds": ["buildChrome.buildTarget"], + "name": "Publish chrome", + "id": null, + "refId": "buildChrome.publishTarget", + "type": "jenkins", + "group": "buildBrowser: buildChrome", + "baz": "bang" + }, + { + "requisiteStageRefIds": ["buildChrome.publishTarget"], + "name": "finalWait", + "id": null, + "refId": "finalWait", + "type": "wait", + "waitTime": 5 + } + ], + "notifications": [], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-template.yml new file mode 100644 index 0000000000..a1204f23e5 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partials-template.yml @@ -0,0 +1,43 @@ +--- +schema: "1" +id: partials +metadata: + name: Partials test + description: Tests partials spec +stages: +- id: firstWait + type: wait + config: + waitTime: 5 +- id: buildChrome + type: partial.buildBrowser + dependsOn: + - firstWait + config: + target: chrome +- id: finalWait + type: wait + dependsOn: + - buildChrome + config: + waitTime: 5 + +partials: +- id: buildBrowser + usage: Builds the pipeline artifact targeting the a specified browser. + variables: + - name: target + description: The target browser to build for + stages: + - id: buildTarget + type: jenkins + name: Build {{ target }} + config: + foo: bar + - id: publishTarget + type: jenkins + name: Publish {{ target }} + dependsOn: + - buildTarget + config: + baz: bang diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-config.yml new file mode 100644 index 0000000000..eaf622325f --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-config.yml @@ -0,0 +1,5 @@ +--- +schema: "1" +pipeline: + application: orca +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-expected.json new file mode 100644 index 0000000000..278d45b8c7 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-expected.json @@ -0,0 +1,36 @@ +{ + "id": "unknown", + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "Unnamed Execution", + "stages": [ + { + "requisiteStageRefIds": [], + "name": "firstWait", + "id": null, + "refId": "firstWait", + "type": "wait", + "waitTime": 5 + }, + { + "requisiteStageRefIds": ["firstWait"], + "name": "Build chrome", + "id": null, + "refId": "buildChrome.buildTarget", + "type": "jenkins", + "group": "buildBrowser: buildChrome", + "foo": "We're building what? We're building chrome" + }, + { + "requisiteStageRefIds": ["buildChrome.buildTarget"], + "name": "finalWait", + "id": null, + "refId": "finalWait", + "type": "wait", + "waitTime": 5 + } + ], + "notifications": [], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-template.yml new file mode 100644 index 0000000000..79a6939acf --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/partialsAndModules-template.yml @@ -0,0 +1,42 @@ +--- +schema: "1" +id: partials +metadata: + name: Partials test + description: Tests partials spec +stages: +- id: firstWait + type: wait + config: + waitTime: 5 +- id: buildChrome + type: partial.buildBrowser + dependsOn: + - firstWait + config: + target: chrome +- id: finalWait + type: wait + dependsOn: + - buildChrome + config: + waitTime: 5 + +modules: +- id: randomConfig + variables: + - name: target + definition: We're building {{ target }} + +partials: +- id: buildBrowser + usage: Builds the pipeline artifact targeting the a specified browser. + variables: + - name: target + description: The target browser to build for + stages: + - id: buildTarget + type: jenkins + name: Build {{ target }} + config: + foo: We're building what? {% module randomConfig target=target %} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-config.yml new file mode 100644 index 0000000000..10b9ace27f --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-config.yml @@ -0,0 +1,11 @@ +--- +schema: "1" +pipeline: + application: orca +stages: +- id: wait2 + type: wait + config: + waitTime: "{{ waitTime }}" + dependsOn: + - wait1 diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-expected.json new file mode 100644 index 0000000000..a0c85afb6a --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-expected.json @@ -0,0 +1,9 @@ +{ + "errors": [ + { + "severity": "FATAL", + "cause": "The template being used has marked itself as protected", + "message": "Modification of the stage graph (adding, removing, editing) is disallowed" + } + ] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-template.yml new file mode 100644 index 0000000000..f63a7ee686 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/protect-template.yml @@ -0,0 +1,13 @@ +--- +schema: "1" +id: simple +protect: true +metadata: + name: Barebones + description: The simplest template possible. +stages: +- id: wait1 + type: wait + config: + waitTime: "{{ waitTime }}" + diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/simple-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/simple-expected.json index 2785b08553..c87852b78c 100644 --- a/orca-pipelinetemplate/src/test/resources/integration/v1schema/simple-expected.json +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/simple-expected.json @@ -5,5 +5,6 @@ "application": "orca", "name": "Unnamed Execution", "stages": [], - "notifications": [] + "notifications": [], + "parameterConfig": [] } diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-config.yml new file mode 100644 index 0000000000..f6500ea0aa --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-config.yml @@ -0,0 +1,31 @@ +schema: "1" +pipeline: + application: orca + name: Stage Injection Test + variables: + waitTime: 5 +stages: +- id: first + type: wait + inject: + first: true + config: + waitTime: "{{ waitTime }}" +- id: last + type: wait + inject: + last: true + config: + waitTime: "{{ waitTime }}" +- id: afterWait2 + type: wait + inject: + after: [wait2] + config: + waitTime: "{{ waitTime }}" +- id: beforeWait2 + type: wait + inject: + before: [wait2] + config: + waitTime: "{{ waitTime }}" diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-expected.json new file mode 100644 index 0000000000..daba0123e4 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-expected.json @@ -0,0 +1,67 @@ +{ + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "Stage Injection Test", + "stages": [ + { + "requisiteStageRefIds": [], + "name": "first", + "id": null, + "refId": "first", + "type": "wait", + "waitTime": 5 + }, + { + "requisiteStageRefIds": ["first"], + "name": "wait", + "id": null, + "refId": "wait", + "type": "wait", + "waitTime": 5 + }, + { + "requisiteStageRefIds": ["wait"], + "name": "wait3", + "id": null, + "refId": "wait3", + "type": "wait", + "waitTime": 5 + }, + { + "requisiteStageRefIds": ["wait"], + "name": "beforeWait2", + "id": null, + "refId": "beforeWait2", + "type": "wait", + "waitTime": 5 + }, + { + "requisiteStageRefIds": ["beforeWait2"], + "name": "wait2", + "id": null, + "refId": "wait2", + "type": "wait", + "waitTime": 5 + }, + { + "requisiteStageRefIds": ["wait2"], + "name": "afterWait2", + "id": null, + "refId": "afterWait2", + "type": "wait", + "waitTime": 5 + }, + { + "requisiteStageRefIds": ["wait3", "afterWait2"], + "name": "last", + "id": null, + "refId": "last", + "type": "wait", + "waitTime": 5 + } + ], + "id": "unknown", + "notifications": [], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-template.yml new file mode 100644 index 0000000000..914cbf1f37 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageInjection-template.yml @@ -0,0 +1,24 @@ +schema: "1" +id: stageInjection +metadata: + name: Stage injection + description: Stage injection test +variables: +- name: waitTime + description: The time a wait stage should pause + type: int +stages: +- id: wait + type: wait + config: + waitTime: "{{ waitTime }}" +- id: wait2 + type: wait + dependsOn: [wait] + config: + waitTime: "{{ waitTime }}" +- id: wait3 + type: wait + dependsOn: [wait] + config: + waitTime: "{{ waitTime }}" diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-config.yml new file mode 100644 index 0000000000..6449df7839 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-config.yml @@ -0,0 +1,24 @@ +--- +schema: "1" +pipeline: + application: orca + name: MPT Stage Replacement Test + template: + source: stageReplacement-template.yml + variables: + myCustomFirstStageName: "HELLO ROB" +stages: +- id: bake1 + type: findImageFromTags + name: "{{ myCustomFirstStageName }}" + config: + cloudProvider: aws + cloudProviderType: aws + packageName: "{{ application }}" + regions: + - us-east-1 + - us-west-1 + - us-west-2 + - eu-west-1 + tags: + stack: test diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-expected.json new file mode 100644 index 0000000000..ac4c6ab1d7 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-expected.json @@ -0,0 +1,132 @@ +{ + "id": "unknown", + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "MPT Stage Replacement Test", + "notifications": [], + "stages": [ + { + "cloudProvider": "aws", + "cloudProviderType": "aws", + "name": "HELLO ROB", + "packageName": "orca", + "refId": "bake1", + "regions": [ + "us-east-1", + "us-west-1", + "us-west-2", + "eu-west-1" + ], + "tags": { + "stack": "test" + }, + "requisiteStageRefIds": [], + "type": "findImageFromTags", + "id": null + }, + { + "id": null, + "clusters": [ + { + "account": "test", + "application": "spindemo", + "availabilityZones": { + "us-west-1": [ + "us-west-1a", + "us-west-1c" + ] + }, + "capacity": { + "desired": 1, + "max": 1, + "min": 1 + }, + "cloudProvider": "aws", + "cooldown": 10, + "copySourceCustomBlockDeviceMappings": true, + "ebsOptimized": false, + "enabledMetrics": [], + "freeFormDetails": "demo", + "healthCheckGracePeriod": 600, + "healthCheckType": "EC2", + "iamRole": "myIAMRole", + "instanceMonitoring": false, + "instanceType": "m3.large", + "interestingHealthProviderNames": [ + "Amazon" + ], + "keyPair": "keypair", + "loadBalancers": [ + "spindemo-demo-frontend" + ], + "maxRemainingAsgs": 2, + "preferSourceCapacity": true, + "provider": "aws", + "scaleDown": true, + "securityGroups": [], + "stack": "test", + "strategy": "redblack", + "subnetType": "mySubnet", + "suspendedProcesses": [], + "tags": {}, + "targetGroups": [], + "targetHealthyDeployPercentage": 100, + "terminationPolicies": [ + "Default" + ], + "useAmiBlockDeviceMappings": false, + "useSourceCapacity": true + }, + { + "account": "test", + "application": "spindemo", + "availabilityZones": { + "us-east-1": [ + "us-east-1c", + "us-east-1d", + "us-east-1e" + ] + }, + "capacity": { + "desired": 0, + "max": 0, + "min": 0 + }, + "cloudProvider": "aws", + "cooldown": 10, + "ebsOptimized": false, + "freeFormDetails": "demo", + "healthCheckGracePeriod": 600, + "healthCheckType": "EC2", + "iamRole": "myIAMRole", + "instanceMonitoring": false, + "instanceType": "m3.large", + "interestingHealthProviderNames": [ + "Amazon" + ], + "keyPair": "keypair", + "provider": "aws", + "securityGroups": [], + "stack": "test", + "strategy": "highlander", + "subnetType": "mySubnet", + "suspendedProcesses": [], + "tags": {}, + "targetHealthyDeployPercentage": 100, + "terminationPolicies": [ + "Default" + ], + "useSourceCapacity": false + } + ], + "name": "Deploy", + "refId": "deploy2", + "requisiteStageRefIds": [ + "bake1" + ], + "type": "deploy" + } + ], + "parameterConfig": [] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-template.yml new file mode 100644 index 0000000000..febc8b2382 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/stageReplacement-template.yml @@ -0,0 +1,114 @@ +--- +schema: "1" +id: stageReplacement +metadata: + name: Stage Replacement + description: Example for replacing a stage +variables: +- name: myCustomFirstStageName +stages: +- id: bake1 + type: bake + name: "{{ myCustomFirstStageName }}" + config: + baseLabel: release + baseOs: trusty + cloudProviderType: aws + enhancedNetworking: false + extendedAttributes: {} + overrideTimeout: true + package: orca + regions: + - us-east-1 + - us-west-1 + - us-west-2 + - eu-west-1 + sendNotifications: true + showAdvancedOptions: true + stageTimeoutMs: 900000 + storeType: ebs + user: example@example.com + vmType: hvm +- id: deploy2 + type: deploy + dependsOn: + - bake1 + name: Deploy + config: + clusters: + - account: test + application: spindemo + availabilityZones: + us-west-1: + - us-west-1a + - us-west-1c + capacity: + desired: 1 + max: 1 + min: 1 + cloudProvider: aws + cooldown: 10 + copySourceCustomBlockDeviceMappings: true + ebsOptimized: false + enabledMetrics: [] + freeFormDetails: demo + healthCheckGracePeriod: 600 + healthCheckType: EC2 + iamRole: myIAMRole + instanceMonitoring: false + instanceType: m3.large + interestingHealthProviderNames: + - Amazon + keyPair: keypair + loadBalancers: + - spindemo-demo-frontend + maxRemainingAsgs: 2 + preferSourceCapacity: true + provider: aws + scaleDown: true + securityGroups: [] + stack: test + strategy: redblack + subnetType: mySubnet + suspendedProcesses: [] + tags: {} + targetGroups: [] + targetHealthyDeployPercentage: 100 + terminationPolicies: + - Default + useAmiBlockDeviceMappings: false + useSourceCapacity: true + - account: test + application: spindemo + availabilityZones: + us-east-1: + - us-east-1c + - us-east-1d + - us-east-1e + capacity: + desired: 0 + max: 0 + min: 0 + cloudProvider: aws + cooldown: 10 + ebsOptimized: false + freeFormDetails: demo + healthCheckGracePeriod: 600 + healthCheckType: EC2 + iamRole: myIAMRole + instanceMonitoring: false + instanceType: m3.large + interestingHealthProviderNames: + - Amazon + keyPair: keypair + provider: aws + securityGroups: [] + stack: test + strategy: highlander + subnetType: mySubnet + suspendedProcesses: [] + tags: {} + targetHealthyDeployPercentage: 100 + terminationPolicies: + - Default + useSourceCapacity: false diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-config.yml new file mode 100644 index 0000000000..367c21c7b7 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-config.yml @@ -0,0 +1,5 @@ +--- +schema: "0.1" +pipeline: + application: orca +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-expected.json new file mode 100644 index 0000000000..83c9dcb8a4 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-expected.json @@ -0,0 +1,8 @@ +{ + "errors": [ + { + "severity": "FATAL", + "message": "unexpected schema version '0.1'" + } + ] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-template.yml new file mode 100644 index 0000000000..7243254d59 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/unsupportedVersion-template.yml @@ -0,0 +1,7 @@ +--- +schema: "0.1" +id: badversion +metadata: + name: Bad Version + description: I don't follow directions. +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-config.yml new file mode 100644 index 0000000000..1742710540 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-config.yml @@ -0,0 +1,7 @@ +--- +schema: "1" +pipeline: + application: orca +configuration: + inherit: ['parameters'] +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-expected.json new file mode 100644 index 0000000000..9cce9c0ec9 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-expected.json @@ -0,0 +1,16 @@ +{ + "id": "unknown", + "keepWaitingPipelines": false, + "limitConcurrent": true, + "application": "orca", + "name": "Unnamed Execution", + "stages": [], + "notifications": [], + "parameterConfig": [ + { + "name": "instances", + "description": "Number of instances to start per region", + "default": 3 + } + ] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-template.yml new file mode 100644 index 0000000000..cdba61c4ea --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableParameters-template.yml @@ -0,0 +1,16 @@ +--- +schema: "1" +id: variableParameters +metadata: + name: Variable Parameters + description: Asserts that variables can be used in parameters. +configuration: + parameters: + - name: instances + description: Number of instances to start per region + default: "{{ instances }}" +variables: +- name: instances + description: The number of instances to start per region + defaultValue: 3 +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-config.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-config.yml new file mode 100644 index 0000000000..cd722700cc --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-config.yml @@ -0,0 +1,8 @@ +--- +schema: "1" +pipeline: + application: orca + name: WrongVariableTypeTest + variables: + waitTime: "HELLO_WORLD" +stages: [] diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-expected.json b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-expected.json new file mode 100644 index 0000000000..b4049201d1 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-expected.json @@ -0,0 +1,9 @@ +{ + "errors": [ + { + "severity": "FATAL", + "message": "malformed template configuration", + "cause": "Incorrectly defined variable(s): waitTime (expected type 'int' found type 'String')" + } + ] +} diff --git a/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-template.yml b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-template.yml new file mode 100644 index 0000000000..254b7c52b4 --- /dev/null +++ b/orca-pipelinetemplate/src/test/resources/integration/v1schema/variableWrongType-template.yml @@ -0,0 +1,16 @@ +--- +schema: "1" +id: varwrongtype +metadata: + name: Waiting Template + description: Simple waiting template. +variables: +- name: waitTime + description: The time a wait stage should pause + type: int +stages: +- id: sleepingbeauty + type: wait + config: + waitTime: "{{ waitTime }}" + diff --git a/orca-pipelinetemplate/src/test/resources/templates/conditional-partials.yml b/orca-pipelinetemplate/src/test/resources/templates/conditional-partials.yml index 172b798b67..b705860b21 100644 --- a/orca-pipelinetemplate/src/test/resources/templates/conditional-partials.yml +++ b/orca-pipelinetemplate/src/test/resources/templates/conditional-partials.yml @@ -3,16 +3,20 @@ id: simpleTemplate variables: - name: includeWait type: boolean +- name: someVariable + defaultValue: someValue stages: - id: stageWithPartialsAndConditional type: partial.partialWithConditional dependsOn: [] - config: {} + config: + anotherVariable: "{{ someVariable }}" partials: - id: partialWithConditional usage: Partial that conditionally adds a step - variables: [] + variables: + - name: anotherVariable stages: - id: conditionalWaitOnPartial type: wait @@ -24,3 +28,4 @@ partials: type: wait config: waitTime: 5 + shouldBeSet: "{{ 'isSet' if anotherVariable == someVariable else 'notSet' }}" diff --git a/orca-queue-redis/orca-queue-redis.gradle b/orca-queue-redis/orca-queue-redis.gradle index cb6ce778d7..9fce050fe9 100644 --- a/orca-queue-redis/orca-queue-redis.gradle +++ b/orca-queue-redis/orca-queue-redis.gradle @@ -25,5 +25,5 @@ dependencies { compile project(":orca-queue") compile "com.fasterxml.jackson.module:jackson-module-kotlin:${spinnaker.version("jackson")}" - testCompile project(path: ":orca-queue", configuration: "testOutput") + testCompile project(path: ":orca-queue", configuration: "testArtifacts") } diff --git a/orca-queue-redis/src/main/kotlin/com/netflix/spinnaker/config/RedisQueueShovelConfiguration.kt b/orca-queue-redis/src/main/kotlin/com/netflix/spinnaker/config/RedisQueueShovelConfiguration.kt new file mode 100644 index 0000000000..197d43851a --- /dev/null +++ b/orca-queue-redis/src/main/kotlin/com/netflix/spinnaker/config/RedisQueueShovelConfiguration.kt @@ -0,0 +1,86 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.config + +import com.netflix.spectator.api.Registry +import com.netflix.spinnaker.orca.config.RedisConfiguration +import com.netflix.spinnaker.orca.q.redis.RedisDeadMessageHandler +import com.netflix.spinnaker.orca.q.redis.RedisQueue +import com.netflix.spinnaker.orca.q.QueueShovel +import org.apache.commons.pool2.impl.GenericObjectPoolConfig +import org.springframework.beans.factory.BeanInitializationException +import org.springframework.beans.factory.annotation.Qualifier +import org.springframework.beans.factory.annotation.Value +import org.springframework.boot.autoconfigure.condition.ConditionalOnBean +import org.springframework.boot.autoconfigure.condition.ConditionalOnExpression +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty +import org.springframework.context.ApplicationEventPublisher +import org.springframework.context.annotation.Bean +import org.springframework.context.annotation.Configuration +import redis.clients.jedis.Jedis +import redis.clients.util.Pool +import java.time.Clock +import java.time.Duration + +@Configuration +@ConditionalOnExpression("\${queue.redis.enabled:true}") +open class RedisQueueShovelConfiguration { + + @Bean + @ConditionalOnProperty("redis.connectionPrevious") + open fun previousQueueJedisPool( + @Value("\${redis.connection:redis://localhost:6379}") mainConnection: String, + @Value("\${redis.connectionPrevious:#{null}}") previousConnection: String?, + @Value("\${redis.timeout:2000}") timeout: Int, + redisPoolConfig: GenericObjectPoolConfig, + registry: Registry): Pool { + if (mainConnection == previousConnection) { + throw BeanInitializationException("previous Redis connection must not be the same as current connection") + } + + return RedisConfiguration.createPool(redisPoolConfig, previousConnection, timeout, registry, "previousQueueJedisPool") + } + + @Bean(name = arrayOf("previousQueueImpl")) + @ConditionalOnBean(name = arrayOf("previousQueueJedisPool")) open fun previousRedisQueue( + @Qualifier("previousQueueJedisPool") redisPool: Pool, + redisQueueProperties: RedisQueueProperties, + clock: Clock, + deadMessageHandler: RedisDeadMessageHandler, + publisher: ApplicationEventPublisher + ) = + RedisQueue( + queueName = redisQueueProperties.queueName, + pool = redisPool, + clock = clock, + deadMessageHandler = deadMessageHandler::handle, + publisher = publisher, + ackTimeout = Duration.ofSeconds(redisQueueProperties.ackTimeoutSeconds.toLong()) + ) + + + @Bean + @ConditionalOnBean(name = arrayOf("previousQueueJedisPool")) open fun redisQueueShovel( + @Qualifier("queueImpl") queueImpl: RedisQueue, + @Qualifier("previousQueueImpl") previousQueueImpl: RedisQueue, + registry: Registry + ) = + QueueShovel( + queue = queueImpl, + previousQueue = previousQueueImpl, + registry = registry + ) +} diff --git a/orca-queue-redis/src/main/kotlin/com/netflix/spinnaker/orca/q/redis/RedisQueue.kt b/orca-queue-redis/src/main/kotlin/com/netflix/spinnaker/orca/q/redis/RedisQueue.kt index 68d70fb5aa..7a186d99c5 100644 --- a/orca-queue-redis/src/main/kotlin/com/netflix/spinnaker/orca/q/redis/RedisQueue.kt +++ b/orca-queue-redis/src/main/kotlin/com/netflix/spinnaker/orca/q/redis/RedisQueue.kt @@ -19,7 +19,6 @@ package com.netflix.spinnaker.orca.q.redis import com.fasterxml.jackson.databind.DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.kotlin.KotlinModule -import com.fasterxml.jackson.module.kotlin.readValue import com.google.common.hash.Hashing import com.netflix.spinnaker.orca.q.AttemptsAttribute import com.netflix.spinnaker.orca.q.MaxAttemptsAttribute @@ -34,6 +33,7 @@ import org.springframework.scheduling.annotation.Scheduled import redis.clients.jedis.Jedis import redis.clients.jedis.JedisCommands import redis.clients.jedis.Transaction +import redis.clients.jedis.params.sortedset.ZAddParams import redis.clients.util.Pool import java.io.IOException import java.nio.charset.Charset @@ -41,7 +41,6 @@ import java.time.Clock import java.time.Duration import java.time.Duration.ZERO import java.time.temporal.TemporalAmount -import java.util.UUID.randomUUID class RedisQueue( private val queueName: String, @@ -62,38 +61,32 @@ class RedisQueue( private val queueKey = "$queueName.queue" private val unackedKey = "$queueName.unacked" private val messagesKey = "$queueName.messages" - private val attemptsKey = "$queueName.attempts" private val locksKey = "$queueName.locks" - private val hashKey = "$queueName.hash" - private val hashesKey = "$queueName.hashes" - companion object { - fun convertToMessage(json: String, mapper: ObjectMapper): Message { - val messageMap = mapper.readValue>(json) + // TODO: use AttemptsAttribute instead + private val attemptsKey = "$queueName.attempts" - return if (messageMap.containsKey("payload")) { - mapper.convertValue(messageMap.get("payload"), Message::class.java) - } else { - mapper.readValue(json) - } - } - } + // TODO: legacy id support + private val hashKey = "$queueName.hash" + private val hashesKey = "$queueName.hashes" override fun poll(callback: (Message, () -> Unit) -> Unit) { pool.resource.use { redis -> redis.zrangeByScore(queueKey, 0.0, score(), 0, 1) .firstOrNull() - ?.takeIf { id -> redis.acquireLock(id) } - ?.also { id -> - val ack = this::ackMessage.partially1(id) - redis.readMessage(id) { message -> + ?.takeIf { fingerprint -> + redis.acquireLock(fingerprint) + } + ?.also { fingerprint -> + val ack = this::ackMessage.partially1(fingerprint) + redis.readMessage(fingerprint) { message -> val attempts = message.getAttribute()?.attempts ?: 0 val maxAttempts = message.getAttribute()?.maxAttempts ?: 0 if (maxAttempts > 0 && attempts > maxAttempts) { - log.warn("Message $id with payload $message exceeded $maxAttempts retries") + log.warn("Message $fingerprint with payload $message exceeded $maxAttempts retries") handleDeadMessage(message) - redis.removeMessage(id) + redis.removeMessage(fingerprint) fire() } else { callback(message, ack) @@ -106,9 +99,10 @@ class RedisQueue( override fun push(message: Message, delay: TemporalAmount) { pool.resource.use { redis -> - val messageHash = message.hash() - if (redis.sismember(hashesKey, messageHash)) { - log.warn("Ignoring message as an identical one is already on the queue: $messageHash, message: $message") + val fingerprint = message.hash() + if (redis.zismember(queueKey, fingerprint)) { + log.warn("Re-prioritizing message as an identical one is already on the queue: $fingerprint, message: $message") + redis.zadd(queueKey, score(delay), fingerprint) fire(message) } else { redis.queueMessage(message, delay) @@ -117,38 +111,60 @@ class RedisQueue( } } + override fun reschedule(message: Message, delay: TemporalAmount) { + pool.resource.use { redis -> + val fingerprint = message.hash() + log.debug("Re-scheduling message: $message, fingerprint: $fingerprint to deliver in $delay") + val status: Long = redis.zadd(queueKey, score(delay), fingerprint, ZAddParams.zAddParams().xx()) + if (status.toInt() == 1){ + fire(message) + } else { + fire(message) + } + } + } + @Scheduled(fixedDelayString = "\${queue.retry.frequency.ms:10000}") override fun retry() { pool.resource.use { redis -> redis .zrangeByScore(unackedKey, 0.0, score()) - .let { ids -> - if (ids.size > 0) { - ids + .let { fingerprints -> + if (fingerprints.size > 0) { + fingerprints .map { "$locksKey:$it" } .let { redis.del(*it.toTypedArray()) } } - ids.forEach { id -> - val attempts = redis.hgetInt(attemptsKey, id) + fingerprints.forEach { fingerprint -> + val attempts = redis.hgetInt(attemptsKey, fingerprint) if (attempts >= Queue.maxRetries) { - redis.readMessage(id) { message -> - log.warn("Message $id with payload $message exceeded max retries") + redis.readMessage(fingerprint) { message -> + log.warn("Message $fingerprint with payload $message exceeded max retries") handleDeadMessage(message) - redis.removeMessage(id) + redis.removeMessage(fingerprint) } fire() } else { - if (redis.sismember(hashesKey, redis.hget(hashKey, id))) { - // we only need to read the message for metrics purposes - redis.readMessage(id) { message -> - log.warn("Not retrying message $id because an identical message is already on the queue") - redis.removeMessage(id) - fire(message) - } + if (redis.zismember(queueKey, fingerprint)) { + redis + .multi { + zrem(unackedKey, fingerprint) + zadd(queueKey, score(), fingerprint) + // we only need to read the message for metrics purposes + hget(messagesKey, fingerprint) + } + .let { (_, _, json) -> + mapper + .readValue(json as String) + .let { message -> + log.warn("Not retrying message $fingerprint because an identical message is already on the queue") + fire(message) + } + } } else { - log.warn("Retrying message $id after $attempts attempts") - redis.requeueMessage(id) + log.warn("Retrying message $fingerprint after $attempts attempts") + redis.requeueMessage(fingerprint) fire() } } @@ -167,115 +183,133 @@ class RedisQueue( zcount(queueKey, 0.0, score()) zcard(unackedKey) hlen(messagesKey) - hlen(hashKey) - scard(hashesKey) } .map { (it as Long).toInt() } - .let { (queued, ready, processing, messages, hashCount, dedupeHashes) -> + .let { (queued, ready, processing, messages) -> return QueueState( depth = queued, ready = ready, unacked = processing, - orphaned = messages - (queued + processing), - hashDrift = hashCount - (processing + dedupeHashes) + orphaned = messages - (queued + processing) ) } } - private operator fun List.component6(): E = get(5) - override fun toString() = "RedisQueue[$queueName]" - private fun ackMessage(id: String) { + private fun ackMessage(fingerprint: String) { pool.resource.use { redis -> - redis.removeMessage(id) + if (redis.zismember(queueKey, fingerprint)) { + // only remove this message from the unacked queue as a matching one has + // been put on the main queue + redis.multi { + zrem(unackedKey, fingerprint) + del("$locksKey:$fingerprint") + } + } else { + redis.removeMessage(fingerprint) + } fire() } } private fun Jedis.queueMessage(message: Message, delay: TemporalAmount = ZERO) { - val id = randomUUID().toString() - val hash = message.hash() + val fingerprint = message.hash() + // ensure the message has the attempts tracking attribute message.setAttribute( - // ensure the message has the attempts tracking attribute - message.getAttribute(AttemptsAttribute()) + message.getAttribute(AttemptsAttribute()) ) multi { - hset(messagesKey, id, mapper.writeValueAsString(message)) - zadd(queueKey, score(delay), id) - hset(hashKey, id, hash) - sadd(hashesKey, hash) + hset(messagesKey, fingerprint, mapper.writeValueAsString(message)) + zadd(queueKey, score(delay), fingerprint) + + // TODO: legacy id compatibility + hset(hashKey, fingerprint, fingerprint) + sadd(hashesKey, fingerprint) } } - private fun Jedis.requeueMessage(id: String) { - val hash = hget(hashKey, id) + private fun Jedis.requeueMessage(fingerprint: String) { + val hash = hget(hashKey, fingerprint) multi { - zrem(unackedKey, id) - zadd(queueKey, score(), id) + zrem(unackedKey, fingerprint) + zadd(queueKey, score(), fingerprint) + + // TODO: legacy id compatibility if (hash != null) { sadd(hashesKey, hash) } } } - private fun Jedis.removeMessage(id: String) { + private fun Jedis.removeMessage(fingerprint: String) { multi { - zrem(queueKey, id) - zrem(unackedKey, id) - hdel(messagesKey, id) - hdel(attemptsKey, id) - hdel(hashKey, id) + zrem(queueKey, fingerprint) + zrem(unackedKey, fingerprint) + hdel(messagesKey, fingerprint) + del("$locksKey:$fingerprint") + + // TODO: use AttemptAttribute instead + hdel(attemptsKey, fingerprint) + + // TODO: legacy id compatibility + hdel(hashKey, fingerprint) } } /** - * Tries to read the message with the specified [id] passing it to [block]. - * If it's not accessible for whatever reason any references are cleaned up. + * Tries to read the message with the specified [fingerprint] passing it to + * [block]. If it's not accessible for whatever reason any references are + * cleaned up. */ - private fun Jedis.readMessage(id: String, block: (Message) -> Unit) { - val hash = hget(hashKey, id) + private fun Jedis.readMessage(fingerprint: String, block: (Message) -> Unit) { + val hash = hget(hashKey, fingerprint) multi { - hget(messagesKey, id) - zrem(queueKey, id) - zadd(unackedKey, score(ackTimeout), id) + hget(messagesKey, fingerprint) + zrem(queueKey, fingerprint) + zadd(unackedKey, score(ackTimeout), fingerprint) + + // TODO: legacy id compatibility + srem(hashesKey, fingerprint) if (hash != null) { srem(hashesKey, hash) } - hincrBy(attemptsKey, id, 1) + + // TODO: use AttemptsAttribute instead + hincrBy(attemptsKey, fingerprint, 1) }.let { val json = it[0] as String? if (json == null) { - log.error("Payload for message $id is missing") + log.error("Payload for message $fingerprint is missing") // clean up what is essentially an unrecoverable message - removeMessage(id) + removeMessage(fingerprint) } else { try { - val message = convertToMessage(json, mapper) + val message = mapper.readValue(json) // TODO: AttemptsAttribute could replace `attemptsKey` message.setAttribute( - message.getAttribute(AttemptsAttribute()) + message.getAttribute(AttemptsAttribute()) ).increment() - hset(messagesKey, id, mapper.writeValueAsString(message)) + hset(messagesKey, fingerprint, mapper.writeValueAsString(message)) block.invoke(message) - } catch(e: IOException) { - log.error("Failed to read message $id, requeuing...", e) - requeueMessage(id) + } catch (e: IOException) { + log.error("Failed to read message $fingerprint, requeuing...", e) + requeueMessage(fingerprint) } } } } - private fun handleDeadMessage(it: Message) { - deadMessageHandler.invoke(this, it) + private fun handleDeadMessage(message: Message) { + deadMessageHandler.invoke(this, message) } - private fun Jedis.acquireLock(id: String) = - (set("$locksKey:$id", "\uD83D\uDD12", "NX", "EX", lockTtlSeconds) == "OK") + private fun Jedis.acquireLock(fingerprint: String) = + (set("$locksKey:$fingerprint", "\uD83D\uDD12", "NX", "EX", lockTtlSeconds) == "OK") .also { if (!it) { fire() @@ -301,6 +335,9 @@ class RedisQueue( private fun JedisCommands.hgetInt(key: String, field: String, default: Int = 0) = hget(key, field)?.toInt() ?: default + private fun JedisCommands.zismember(key: String, member: String) = + zrank(key, member) != null + private fun Message.hash() = Hashing .murmur3_128() diff --git a/orca-queue-redis/src/test/kotlin/com/netflix/spinnaker/orca/q/redis/RedisQueueTest.kt b/orca-queue-redis/src/test/kotlin/com/netflix/spinnaker/orca/q/redis/RedisQueueTest.kt index 57ea66060f..17c60fcc6e 100644 --- a/orca-queue-redis/src/test/kotlin/com/netflix/spinnaker/orca/q/redis/RedisQueueTest.kt +++ b/orca-queue-redis/src/test/kotlin/com/netflix/spinnaker/orca/q/redis/RedisQueueTest.kt @@ -16,20 +16,11 @@ package com.netflix.spinnaker.orca.q.redis -import com.fasterxml.jackson.databind.ObjectMapper -import com.fasterxml.jackson.module.kotlin.KotlinModule import com.netflix.spinnaker.kork.jedis.EmbeddedRedis -import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.q.DeadMessageCallback -import com.netflix.spinnaker.orca.q.Message import com.netflix.spinnaker.orca.q.QueueTest -import com.netflix.spinnaker.orca.q.StartExecution import com.netflix.spinnaker.orca.q.metrics.MonitorableQueueTest import org.funktionale.partials.invoke -import org.jetbrains.spek.api.Spek -import org.jetbrains.spek.api.dsl.describe -import org.jetbrains.spek.api.dsl.it -import org.junit.Assert import org.springframework.context.ApplicationEvent import org.springframework.context.ApplicationEventPublisher import java.time.Clock @@ -38,7 +29,6 @@ object RedisQueueTest : QueueTest(createQueue(p3 = null), ::shutdown object RedisMonitorableQueueTest : MonitorableQueueTest( createQueue, - RedisQueue::retry, ::shutdownCallback ) @@ -63,29 +53,3 @@ private val createQueue = { clock: Clock, private fun shutdownCallback() { redis?.destroy() } - -class ConvertToMessageSpec : Spek({ - describe("should support deserializing a nested message") { - val objectMapper = ObjectMapper().apply { - registerModule(KotlinModule()) - } - - val message = StartExecution(Pipeline::class.java, "1", "foo") - - it("is not nested") { - Assert.assertEquals( - message, - RedisQueue.convertToMessage(objectMapper.writeValueAsString(message), objectMapper) - ) - } - - it("is nested") { - Assert.assertEquals( - message, - RedisQueue.convertToMessage(objectMapper.writeValueAsString(Envelope(message)), objectMapper) - ) - } - } -}) - -private data class Envelope(val payload: Message) diff --git a/orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/config/SqsQueueConfiguration.kt b/orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/config/SqsQueueConfiguration.kt deleted file mode 100644 index ae45a3d9e9..0000000000 --- a/orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/config/SqsQueueConfiguration.kt +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Copyright 2017 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.netflix.spinnaker.config - -import com.amazonaws.auth.AWSCredentialsProvider -import com.amazonaws.services.sqs.AmazonSQS -import com.amazonaws.services.sqs.AmazonSQSClient -import com.netflix.spinnaker.clouddriver.aws.bastion.BastionConfig -import com.netflix.spinnaker.orca.q.amazon.SqsQueue -import com.netflix.spinnaker.orca.q.handler.DeadMessageHandler -import org.springframework.boot.autoconfigure.condition.ConditionalOnExpression -import org.springframework.boot.context.properties.EnableConfigurationProperties -import org.springframework.context.annotation.Bean -import org.springframework.context.annotation.ComponentScan -import org.springframework.context.annotation.Configuration -import org.springframework.context.annotation.Import - -@Configuration -@ConditionalOnExpression("\${queue.sqs.enabled:false}") -@Import(BastionConfig::class) -@ComponentScan(basePackages = arrayOf("com.netflix.spinnaker.orca.q.amazon")) -@EnableConfigurationProperties(SqsProperties::class) -open class SqsQueueConfiguration { - - @Bean open fun amazonSqsClient(awsCredentialsProvider: AWSCredentialsProvider, sqsProperties: SqsProperties) = - AmazonSQSClient - .builder() - .withCredentials(awsCredentialsProvider) - .withRegion(sqsProperties.region) - .build() - - @Bean(name = arrayOf("queueImpl")) - open fun sqsQueue(amazonSqs: AmazonSQS, sqsProperties: SqsProperties, deadMessageHandler: DeadMessageHandler) = - SqsQueue(amazonSqs, sqsProperties, deadMessageHandler = deadMessageHandler::handle) -} diff --git a/orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/orca/q/amazon/SqsQueue.kt b/orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/orca/q/amazon/SqsQueue.kt deleted file mode 100644 index ffa7e14ac5..0000000000 --- a/orca-queue-sqs/src/main/kotlin/com/netflix/spinnaker/orca/q/amazon/SqsQueue.kt +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright 2017 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.netflix.spinnaker.orca.q.amazon - -import com.amazonaws.services.sqs.AmazonSQS -import com.amazonaws.services.sqs.model.ReceiveMessageRequest -import com.amazonaws.services.sqs.model.SendMessageRequest -import com.fasterxml.jackson.databind.ObjectMapper -import com.fasterxml.jackson.module.kotlin.KotlinModule -import com.netflix.spinnaker.config.SqsProperties -import com.netflix.spinnaker.orca.q.Message -import com.netflix.spinnaker.orca.q.Queue -import com.netflix.spinnaker.orca.q.QueueCallback -import java.time.Duration -import java.time.temporal.ChronoUnit.SECONDS -import java.time.temporal.TemporalAmount - -class SqsQueue( - private val amazonSqs: AmazonSQS, - sqsProperties: SqsProperties, - override val ackTimeout: Duration = Duration.ofMinutes(1), - override val deadMessageHandler: (Queue, Message) -> Unit -) : Queue { - - private val objectMapper = ObjectMapper().apply { - registerModule(KotlinModule()) - } - - private val queueUrl = amazonSqs.createQueue(sqsProperties.queueName).queueUrl - private val messageReceiptHandles = mutableSetOf() - - override fun poll(callback: QueueCallback) { - val req = ReceiveMessageRequest(queueUrl) - .withMaxNumberOfMessages(1) - .withWaitTimeSeconds(10) - .withVisibilityTimeout(ackTimeout.seconds.toInt()) - .withAttributeNames("ApproximateFirstReceiveTimestamp", "ApproximateReceiveCount", "SentTimestamp") - val result = amazonSqs.receiveMessage(req) - - if (result.messages.isNotEmpty()) { - val sqsMessage = result.messages.first() - val message = objectMapper.readValue(sqsMessage.body, Message::class.java) - messageReceiptHandles.add(sqsMessage.receiptHandle) - - callback.invoke(message) { - ack(sqsMessage.receiptHandle) - } - } - } - - override fun push(message: Message) { - amazonSqs.sendMessage(queueUrl, objectMapper.writeValueAsString(message)) - } - - override fun push(message: Message, delay: TemporalAmount) { - amazonSqs.sendMessage( - SendMessageRequest(queueUrl, objectMapper.writeValueAsString(message)) - .withDelaySeconds(delay.get(SECONDS).toInt()) - ) - } - - private fun ack(receiptHandle: String) { - if (messageReceiptHandles.remove(receiptHandle)) { - amazonSqs.deleteMessage(queueUrl, receiptHandle) - } - } - -} diff --git a/orca-queue/orca-queue.gradle b/orca-queue/orca-queue.gradle index efc886b679..8dd7ca7409 100644 --- a/orca-queue/orca-queue.gradle +++ b/orca-queue/orca-queue.gradle @@ -32,14 +32,14 @@ dependencies { // export tests so the Queue TCK is available to other modules configurations { - testOutput.extendsFrom(testCompile) + testArtifacts.extendsFrom testRuntime } -task jarTest(type: Jar, dependsOn: testClasses) { - from sourceSets.test.output +task testJar(type: Jar) { classifier = "test" + from sourceSets.test.output } artifacts { - testOutput jarTest + testArtifacts testJar } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/config/QueueConfiguration.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/config/QueueConfiguration.kt index c512a96fd0..891b1d56cf 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/config/QueueConfiguration.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/config/QueueConfiguration.kt @@ -17,7 +17,6 @@ package com.netflix.spinnaker.config import com.netflix.spectator.api.Registry -import com.netflix.spinnaker.orca.config.OrcaConfiguration.applyThreadPoolMetrics import com.netflix.spinnaker.orca.log.BlackholeExecutionLogRepository import com.netflix.spinnaker.orca.log.ExecutionLogRepository import com.netflix.spinnaker.orca.q.Queue @@ -34,16 +33,17 @@ import org.springframework.context.event.SimpleApplicationEventMulticaster import org.springframework.scheduling.annotation.EnableScheduling import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor import java.time.Clock -import java.util.concurrent.ThreadPoolExecutor @Configuration @ComponentScan(basePackages = arrayOf("com.netflix.spinnaker.orca.q", "com.netflix.spinnaker.orca.log", "com.netflix.spinnaker.orca.q.trafficshaping")) @EnableScheduling open class QueueConfiguration { - @Bean @ConditionalOnMissingBean(Clock::class) + @Bean + @ConditionalOnMissingBean(Clock::class) open fun systemClock(): Clock = Clock.systemDefaultZone() - @Bean(name = arrayOf("queueImpl")) @ConditionalOnMissingBean(Queue::class) + @Bean(name = arrayOf("queueImpl")) + @ConditionalOnMissingBean(Queue::class) open fun inMemoryQueue(clock: Clock, deadMessageHandler: DeadMessageHandler, publisher: ApplicationEventPublisher) = InMemoryQueue( clock = clock, @@ -51,20 +51,17 @@ open class QueueConfiguration { publisher = publisher ) - @Bean @ConditionalOnMissingBean(ExecutionLogRepository::class) + @Bean + @ConditionalOnMissingBean(ExecutionLogRepository::class) open fun executionLogRepository(): ExecutionLogRepository = BlackholeExecutionLogRepository() @Bean open fun messageHandlerPool(registry: Registry): ThreadPoolTaskExecutor = - applyThreadPoolMetrics( - registry, - ThreadPoolTaskExecutor().apply { - corePoolSize = 20 - maxPoolSize = 20 - setQueueCapacity(0) - }, - "messageHandler" - ) + ThreadPoolTaskExecutor().apply { + corePoolSize = 20 + maxPoolSize = 20 + setQueueCapacity(0) + } /** * This overrides Spring's default application event multicaster as we need @@ -81,12 +78,8 @@ open class QueueConfiguration { } @Bean open fun applicationEventTaskExecutor(registry: Registry): ThreadPoolTaskExecutor = - applyThreadPoolMetrics( - registry, - ThreadPoolTaskExecutor().apply { - corePoolSize = 20 - maxPoolSize = 20 - }, - "applicationEventMulticaster" - ) + ThreadPoolTaskExecutor().apply { + corePoolSize = 20 + maxPoolSize = 20 + } } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Executions.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Executions.kt index 1c13d7c3ee..4ba8758121 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Executions.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Executions.kt @@ -94,8 +94,11 @@ fun Stage<*>.allUpstreamStagesComplete(): Boolean = fun Stage<*>.anyUpstreamStagesFailed(): Boolean = upstreamStages().any { it.getStatus() in listOf(TERMINAL, STOPPED, CANCELED) || it.getStatus() == NOT_STARTED && it.anyUpstreamStagesFailed() } +fun Stage<*>.syntheticStages(): List> = + getExecution().getStages().filter { it.getParentStageId() == getId() } + fun Stage<*>.beforeStages(): List> = - getExecution().getStages().filter { it.getParentStageId() == getId() && it.getSyntheticStageOwner() == STAGE_BEFORE } + syntheticStages().filter { it.getSyntheticStageOwner() == STAGE_BEFORE } fun Stage<*>.allBeforeStagesComplete(): Boolean = beforeStages().all { it.getStatus() in listOf(SUCCEEDED, FAILED_CONTINUE, SKIPPED) } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Message.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Message.kt index 85612eac22..c1036871a1 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Message.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Message.kt @@ -24,18 +24,15 @@ import com.netflix.spinnaker.orca.Task import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository -import java.util.* /** * Messages used internally by the queueing system. */ @JsonTypeInfo(use = MINIMAL_CLASS, include = PROPERTY, property = "@class") -interface Attribute { -} +interface Attribute -data class MaxAttemptsAttribute(val maxAttempts: Int = -1) : Attribute { -} +data class MaxAttemptsAttribute(val maxAttempts: Int = -1) : Attribute data class TotalThrottleTimeAttribute(var totalThrottleTimeMs: Long = 0) : Attribute { fun add(throttleTimeMs: Long) { @@ -53,14 +50,14 @@ data class AttemptsAttribute(var attempts: Int = 0) : Attribute { sealed class Message { val attributes: MutableList = mutableListOf() - fun setAttribute(attribute: A) : A { + fun setAttribute(attribute: A): A { removeAttribute(attribute) attributes.add(attribute) return attribute } - fun removeAttribute(attribute: A) { + fun removeAttribute(attribute: A) { attributes.removeIf { it.javaClass == attribute.javaClass } } @@ -74,7 +71,10 @@ sealed class Message { } } - inline fun getAttribute(defaultValue : A): A { + inline fun hasAttribute() = + attributes.any { it is A } + + inline fun getAttribute(defaultValue: A): A { return getAttribute() ?: defaultValue } } @@ -163,6 +163,9 @@ data class RunTask( constructor(message: TaskLevel, taskType: Class) : this(message.executionType, message.executionId, message.application, message.stageId, message.taskId, taskType) + + constructor(source: ExecutionLevel, stageId: String, taskId: String, taskType: Class) : + this(source.executionType, source.executionId, source.application, stageId, taskId, taskType) } data class StartStage( @@ -195,17 +198,42 @@ data class CompleteStage( override val executionType: Class>, override val executionId: String, override val application: String, - override val stageId: String, - val status: ExecutionStatus + override val stageId: String ) : Message(), StageLevel { - constructor(source: ExecutionLevel, stageId: String, status: ExecutionStatus) : - this(source.executionType, source.executionId, source.application, stageId, status) + constructor(source: ExecutionLevel, stageId: String) : + this(source.executionType, source.executionId, source.application, stageId) - constructor(source: StageLevel, status: ExecutionStatus) : - this(source, source.stageId, status) + constructor(source: StageLevel) : + this(source.executionType, source.executionId, source.application, source.stageId) - constructor(source: Stage<*>, status: ExecutionStatus) : - this(source.getExecution().javaClass, source.getExecution().getId(), source.getExecution().getApplication(), source.getId(), status) + constructor(source: Stage<*>) : + this(source.getExecution().javaClass, source.getExecution().getId(), source.getExecution().getApplication(), source.getId()) +} + +data class SkipStage( + override val executionType: Class>, + override val executionId: String, + override val application: String, + override val stageId: String +) : Message(), StageLevel { + constructor(source: StageLevel) : + this(source.executionType, source.executionId, source.application, source.stageId) + + constructor(source: Stage<*>) : + this(source.getExecution().javaClass, source.getExecution().getId(), source.getExecution().getApplication(), source.getId()) +} + +data class AbortStage( + override val executionType: Class>, + override val executionId: String, + override val application: String, + override val stageId: String +) : Message(), StageLevel { + constructor(source: StageLevel) : + this(source.executionType, source.executionId, source.application, source.stageId) + + constructor(source: Stage<*>) : + this(source.getExecution().javaClass, source.getExecution().getId(), source.getExecution().getApplication(), source.getId()) } data class PauseStage( @@ -267,6 +295,15 @@ data class StartExecution( this(source.javaClass, source.getId(), source.getApplication()) } +data class RescheduleExecution( + override val executionType: Class>, + override val executionId: String, + override val application: String +) : Message(), ExecutionLevel { + constructor(source: Execution<*>) : + this(source.javaClass, source.getId(), source.getApplication()) +} + data class CompleteExecution( override val executionType: Class>, override val executionId: String, diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/MessageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/MessageHandler.kt index f1daa0d8be..21fafe4b3e 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/MessageHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/MessageHandler.kt @@ -83,4 +83,23 @@ interface MessageHandler : (Message) -> Unit { } catch(e: ExecutionNotFoundException) { queue.push(InvalidExecutionId(this)) } + + fun Stage<*>.startNext() { + getExecution().let { execution -> + val downstreamStages = downstreamStages() + if (downstreamStages.isNotEmpty()) { + downstreamStages.forEach { + queue.push(StartStage(it)) + } + } else if (getSyntheticStageOwner() == SyntheticStageOwner.STAGE_BEFORE) { + queue.push(ContinueParentStage(parent())) + } else if (getSyntheticStageOwner() == SyntheticStageOwner.STAGE_AFTER) { + parent().let { parent -> + queue.push(CompleteStage(parent)) + } + } else { + queue.push(CompleteExecution(execution)) + } + } + } } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Queue.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Queue.kt index c1bd1fa8c1..90951258ee 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Queue.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/Queue.kt @@ -49,6 +49,16 @@ interface Queue { */ fun push(message: Message, delay: TemporalAmount): Unit + /** + * Update [message] if it exists for immediate delivery. + */ + fun reschedule(message: Message): Unit = reschedule(message, ZERO) + + /** + * Update [mesasge] if it exists for delivery after [delay]. + */ + fun reschedule(message: Message, delay: TemporalAmount): Unit + /** * Check for any un-acknowledged messages that are overdue and move them back * onto the queue. diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueExecutionRunner.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueExecutionRunner.kt index eff0f68b9c..c2ee70df26 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueExecutionRunner.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueExecutionRunner.kt @@ -29,6 +29,10 @@ class QueueExecutionRunner( override fun > start(execution: T) = queue.push(StartExecution(execution)) + override fun > reschedule(execution: T) { + queue.push(RescheduleExecution(execution)) + } + override fun > restart(execution: T, stageId: String) { queue.push(RestartStage(execution, stageId, AuthenticatedRequest.getSpinnakerUser().orElse(null))) } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueExecutor.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueExecutor.kt index db5f5ad968..813dba8c1a 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueExecutor.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueExecutor.kt @@ -18,7 +18,8 @@ package com.netflix.spinnaker.orca.q import java.util.concurrent.Executor -interface QueueExecutor { - val executor: Executor - fun hasCapacity(): Boolean +abstract class QueueExecutor( + protected val executor: T +) : Executor by executor { + abstract fun hasCapacity(): Boolean } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueProcessor.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueProcessor.kt index 918fbc973b..02c84477c6 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueProcessor.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueProcessor.kt @@ -20,16 +20,18 @@ import com.netflix.spectator.api.Registry import com.netflix.spinnaker.orca.discovery.DiscoveryActivated import org.slf4j.Logger import org.slf4j.LoggerFactory.getLogger +import org.slf4j.MDC import org.springframework.scheduling.annotation.Scheduled import org.springframework.stereotype.Component import java.util.concurrent.RejectedExecutionException import java.util.concurrent.atomic.AtomicBoolean import javax.annotation.PostConstruct +import com.netflix.spinnaker.security.AuthenticatedRequest.SPINNAKER_EXECUTION_ID @Component class QueueProcessor( private val queue: Queue, - private val queueExecutor: QueueExecutor, + private val queueExecutor: QueueExecutor<*>, private val registry: Registry, private val handlers: Collection> ) : DiscoveryActivated { @@ -54,9 +56,16 @@ class QueueProcessor( val handler = handlerFor(message) if (handler != null) { try { - queueExecutor.executor.execute { - handler.invoke(message) - ack.invoke() + queueExecutor.execute { + try { + if (message is ExecutionLevel) { + MDC.put(SPINNAKER_EXECUTION_ID, message.executionId) + } + handler.invoke(message) + ack.invoke() + } finally { + MDC.remove(SPINNAKER_EXECUTION_ID) + } } } catch (e: RejectedExecutionException) { log.warn("Executor at capacity, immediately re-queuing message", e) diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueShovel.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueShovel.kt new file mode 100644 index 0000000000..0837e35575 --- /dev/null +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/QueueShovel.kt @@ -0,0 +1,68 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.q + +import com.netflix.spectator.api.Registry +import com.netflix.spinnaker.orca.discovery.DiscoveryActivated +import org.slf4j.Logger +import org.slf4j.LoggerFactory +import org.springframework.scheduling.annotation.Scheduled +import java.util.concurrent.atomic.AtomicBoolean +import javax.annotation.PostConstruct + + +/** + * The QueueShovel can be used to migrate from one queue implementation to another without an + * operator needing to perform any substantial external work. + * + * In the case of a RedisQueue, when a previous Redis connection is configured, this shovel + * would be wired up to move messages off the old Redis server and onto the new one as the + * messages become available for processing. + */ +class QueueShovel( + private val queue: Queue, + private val previousQueue: Queue, + private val registry: Registry +) : DiscoveryActivated { + + override val log: Logger = LoggerFactory.getLogger(javaClass) + override val enabled = AtomicBoolean(false) + + private val pollOpsRateId = registry.createId("orca.nu.shovel.pollOpsRate") + private val shoveledMessageId = registry.createId("orca.nu.shovel.pushedMessageRate") + private val shovelErrorId = registry.createId("orca.nu.shovel.pushedMessageErrorRate") + + @Scheduled(fixedDelayString = "\${queue.shovel.pollFrequency.ms:500}") + fun migrateOne() { + ifEnabled { + registry.counter(pollOpsRateId).increment() + previousQueue.poll { message, ack -> + try { + queue.push(message) + ack.invoke() + registry.counter(shoveledMessageId).increment() + } catch (e: Throwable) { + log.error("Failed shoveling message from previous queue to active (message: {})", message, e) + registry.counter(shovelErrorId).increment() + } + } + } + } + + @PostConstruct + fun confirmShovelUsage() = + log.info("Running ${javaClass.simpleName} migrator") +} diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/StageDefinitionBuilders.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/StageDefinitionBuilders.kt index bc05a6bd85..5ca5a40b4c 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/StageDefinitionBuilders.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/StageDefinitionBuilders.kt @@ -16,7 +16,6 @@ package com.netflix.spinnaker.orca.q -import com.netflix.spinnaker.orca.pipeline.BranchingStageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.RestrictExecutionDuringTimeWindow import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder.newStage @@ -31,13 +30,7 @@ import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFOR * Build and append the tasks for [stage]. */ fun StageDefinitionBuilder.buildTasks(stage: Stage<*>) { - val taskGraph = - if (this is BranchingStageDefinitionBuilder && (stage.getParentStageId() == null || stage.parent().getType() != stage.getType())) { - buildPostGraph(stage) - } else { - buildTaskGraph(stage) - } - taskGraph + buildTaskGraph(stage) .listIterator() .forEachWithMetadata { processTaskNode(stage, it) } } @@ -80,7 +73,7 @@ private fun processTaskNode( fun StageDefinitionBuilder.buildSyntheticStages( stage: Stage>, callback: (Stage<*>) -> Unit = {} -): Unit { +) { val executionWindow = stage.buildExecutionWindow() syntheticStages(stage).apply { buildBeforeStages(stage, executionWindow, callback) @@ -90,10 +83,10 @@ fun StageDefinitionBuilder.buildSyntheticStages( } @Suppress("UNCHECKED_CAST") -private fun BranchingStageDefinitionBuilder.parallelContexts(stage: Stage<*>) = +private fun StageDefinitionBuilder.parallelStages(stage: Stage<*>) = when (stage.getExecution()) { - is Pipeline -> parallelContexts(stage as Stage) - is Orchestration -> parallelContexts(stage as Stage) + is Pipeline -> parallelStages(stage as Stage) + is Orchestration -> parallelStages(stage as Stage) else -> throw IllegalStateException() } @@ -115,6 +108,7 @@ private fun SyntheticStages.buildBeforeStages(stage: Stage>, ex listOf(executionWindow) + this[STAGE_BEFORE].orEmpty() } beforeStages.forEachIndexed { i, it -> + it.sanitizeContext() it.setRefId("${stage.getRefId()}<${i + 1}") if (i > 0) { it.setRequisiteStageRefIds(setOf("${stage.getRefId()}<$i")) @@ -131,6 +125,7 @@ private fun SyntheticStages.buildBeforeStages(stage: Stage>, ex private fun SyntheticStages.buildAfterStages(stage: Stage>, callback: (Stage<*>) -> Unit) { val afterStages = this[STAGE_AFTER].orEmpty() afterStages.forEachIndexed { i, it -> + it.sanitizeContext() it.setRefId("${stage.getRefId()}>${i + 1}") if (i > 0) { it.setRequisiteStageRefIds(setOf("${stage.getRefId()}>$i")) @@ -148,30 +143,16 @@ private fun SyntheticStages.buildAfterStages(stage: Stage>, cal } private fun StageDefinitionBuilder.buildParallelStages(stage: Stage>, executionWindow: Stage>?, callback: (Stage<*>) -> Unit) { - if (this is BranchingStageDefinitionBuilder && (stage.getParentStageId() == null || stage.parent().getType() != stage.getType())) { - val parallelContexts = parallelContexts(stage) - parallelContexts - .map { context -> - val execution = stage.getExecution() - val stageType = context.getOrDefault("type", stage.getType()).toString() - val stageName = context.getOrDefault("name", stage.getName()).toString() - @Suppress("UNCHECKED_CAST") - when (execution) { - is Pipeline -> newStage(execution, stageType, stageName, context.filterKeys { it != "restrictExecutionDuringTimeWindow" }, stage as Stage, STAGE_BEFORE) - is Orchestration -> newStage(execution, stageType, stageName, context.filterKeys { it != "restrictExecutionDuringTimeWindow" }, stage as Stage, STAGE_BEFORE) - else -> throw IllegalStateException() - } - } - .forEachIndexed { i, it -> - it.setRefId("${stage.getRefId()}=${i + 1}") - it.setRequisiteStageRefIds(if (executionWindow == null) emptySet() else setOf(executionWindow.getRefId())) - stage.getExecution().apply { - injectStage(getStages().indexOf(stage), it) - callback.invoke(it) - } + parallelStages(stage) + .forEachIndexed { i, it -> + it.sanitizeContext() + it.setRefId("${stage.getRefId()}=${i + 1}") + it.setRequisiteStageRefIds(if (executionWindow == null) emptySet() else setOf(executionWindow.getRefId())) + stage.getExecution().apply { + injectStage(getStages().indexOf(stage), it) + callback.invoke(it) } - stage.setName(parallelStageName(stage, parallelContexts.size > 1)) - } + } } private fun Stage>.buildExecutionWindow(): Stage<*>? { @@ -204,9 +185,18 @@ private fun Stage>.buildExecutionWindow(): Stage<*>? { } @Suppress("UNCHECKED_CAST") -private fun Execution<*>.injectStage(index: Int, it: Stage<*>) { +private fun Execution<*>.injectStage(index: Int, stage: Stage<*>) { when (this) { - is Pipeline -> stages.add(index, it as Stage) - is Orchestration -> stages.add(index, it as Stage) + is Pipeline -> stages.add(index, stage as Stage) + is Orchestration -> stages.add(index, stage as Stage) + } +} + +private fun Stage<*>.sanitizeContext() { + if (getType() != RestrictExecutionDuringTimeWindow.TYPE) { + getContext().apply { + remove("restrictExecutionDuringTimeWindow") + remove("restrictedExecutionWindow") + } } } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/ThreadPoolQueueExecutor.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/ThreadPoolQueueExecutor.kt index 135bd9c586..421567e753 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/ThreadPoolQueueExecutor.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/ThreadPoolQueueExecutor.kt @@ -22,7 +22,10 @@ import org.springframework.stereotype.Component @Component class ThreadPoolQueueExecutor( - @Qualifier("messageHandlerPool") override val executor: ThreadPoolTaskExecutor) : QueueExecutor { - override fun hasCapacity(): Boolean = executor.threadPoolExecutor.activeCount < executor.threadPoolExecutor.maximumPoolSize + @Qualifier("messageHandlerPool") executor: ThreadPoolTaskExecutor +) : QueueExecutor(executor) { + override fun hasCapacity() = + executor.threadPoolExecutor.run { + activeCount < maximumPoolSize + } } - diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/AbortStageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/AbortStageHandler.kt new file mode 100644 index 0000000000..12445c87cc --- /dev/null +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/AbortStageHandler.kt @@ -0,0 +1,53 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.q.handler + +import com.netflix.spinnaker.orca.ExecutionStatus.* +import com.netflix.spinnaker.orca.events.StageComplete +import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository +import com.netflix.spinnaker.orca.q.* +import org.springframework.context.ApplicationEventPublisher +import org.springframework.stereotype.Component +import java.time.Clock + +@Component +class AbortStageHandler( + override val queue: Queue, + override val repository: ExecutionRepository, + private val publisher: ApplicationEventPublisher, + private val clock: Clock +) : MessageHandler { + + override fun handle(message: AbortStage) { + message.withStage { stage -> + if (stage.getStatus() in setOf(RUNNING, NOT_STARTED)) { + stage.setStatus(TERMINAL) + stage.setEndTime(clock.millis()) + repository.storeStage(stage) + queue.push(CancelStage(message)) + if (stage.getParentStageId() == null) { + queue.push(CompleteExecution(message)) + } else { + queue.push(CompleteStage(stage.parent())) + } + publisher.publishEvent(StageComplete(this, stage)) + } + } + } + + override val messageType = AbortStage::class.java +} diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/AuthenticationAware.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/AuthenticationAware.kt index 6f2256c53d..011384deb3 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/AuthenticationAware.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/AuthenticationAware.kt @@ -43,6 +43,7 @@ interface AuthenticationAware { try { ExecutionContext.set(ExecutionContext( getExecution().getApplication(), + currentUser.username, getExecution().javaClass.simpleName.toLowerCase(), getExecution().getId(), getExecution().getOrigin() diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CancelExecutionHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CancelExecutionHandler.kt index 77795c0c94..da7158de11 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CancelExecutionHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CancelExecutionHandler.kt @@ -18,10 +18,7 @@ package com.netflix.spinnaker.orca.q.handler import com.netflix.spinnaker.orca.ExecutionStatus.PAUSED import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository -import com.netflix.spinnaker.orca.q.CancelExecution -import com.netflix.spinnaker.orca.q.MessageHandler -import com.netflix.spinnaker.orca.q.Queue -import com.netflix.spinnaker.orca.q.ResumeStage +import com.netflix.spinnaker.orca.q.* import org.springframework.stereotype.Component @Component @@ -43,6 +40,9 @@ class CancelExecutionHandler( .forEach { stage -> queue.push(ResumeStage(stage)) } + + // then, make sure those runTask messages get run right away + queue.push(RescheduleExecution(execution)) } } } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CancelStageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CancelStageHandler.kt index af09d6271b..fd0a6cd141 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CancelStageHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CancelStageHandler.kt @@ -17,7 +17,7 @@ package com.netflix.spinnaker.orca.q.handler import com.netflix.spinnaker.orca.CancellableStage -import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilderFactory import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository import com.netflix.spinnaker.orca.q.CancelStage import com.netflix.spinnaker.orca.q.MessageHandler @@ -30,7 +30,7 @@ import java.util.concurrent.Executor class CancelStageHandler( override val queue: Queue, override val repository: ExecutionRepository, - override val stageDefinitionBuilders: Collection, + override val stageDefinitionBuilderFactory: StageDefinitionBuilderFactory, @Qualifier("messageHandlerPool") private val executor: Executor ) : MessageHandler, StageBuilderAware { diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteStageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteStageHandler.kt index 250d6a26cf..997f0b4db5 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteStageHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteStageHandler.kt @@ -18,9 +18,6 @@ package com.netflix.spinnaker.orca.q.handler import com.netflix.spinnaker.orca.ExecutionStatus.* import com.netflix.spinnaker.orca.events.StageComplete -import com.netflix.spinnaker.orca.pipeline.model.Stage -import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_AFTER -import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFORE import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository import com.netflix.spinnaker.orca.pipeline.util.ContextParameterProcessor import com.netflix.spinnaker.orca.q.* @@ -40,12 +37,13 @@ class CompleteStageHandler( override fun handle(message: CompleteStage) { message.withStage { stage -> if (stage.getStatus() in setOf(RUNNING, NOT_STARTED)) { - stage.setStatus(message.status) + val status = stage.determineStatus() + stage.setStatus(status) stage.setEndTime(clock.millis()) stage.includeExpressionEvaluationSummary() repository.storeStage(stage) - if (message.status in listOf(SUCCEEDED, FAILED_CONTINUE, SKIPPED)) { + if (status in listOf(SUCCEEDED, FAILED_CONTINUE)) { stage.startNext() } else { queue.push(CancelStage(message)) @@ -63,24 +61,4 @@ class CompleteStageHandler( } override val messageType = CompleteStage::class.java - - private fun Stage<*>.startNext() { - getExecution().let { execution -> - val downstreamStages = downstreamStages() - if (downstreamStages.isNotEmpty()) { - downstreamStages.forEach { - queue.push(StartStage(it)) - } - } else if (getSyntheticStageOwner() == STAGE_BEFORE) { - queue.push(ContinueParentStage(parent())) - } else if (getSyntheticStageOwner() == STAGE_AFTER) { - parent().let { parent -> - queue.push(CompleteStage(parent, SUCCEEDED)) - } - } else { - log.debug("No stages waiting to start, completing execution (executionId: ${getExecution().getId()}, stageId: ${getId()})") - queue.push(CompleteExecution(execution)) - } - } - } } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteTaskHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteTaskHandler.kt index 979f4cd7c7..f03fd1aaf3 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteTaskHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteTaskHandler.kt @@ -44,11 +44,11 @@ class CompleteTaskHandler( repository.storeStage(stage) if (message.status != SUCCEEDED) { - queue.push(CompleteStage(message, message.status)) + queue.push(CompleteStage(message)) } else if (task.isStageEnd) { stage.firstAfterStages().let { afterStages -> if (afterStages.isEmpty()) { - queue.push(CompleteStage(message, message.status)) + queue.push(CompleteStage(message)) } else { afterStages.forEach { queue.push(StartStage(message, it.getId())) diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/ContinueParentStageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/ContinueParentStageHandler.kt index 93b6a0dd2d..28485bc433 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/ContinueParentStageHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/ContinueParentStageHandler.kt @@ -17,7 +17,6 @@ package com.netflix.spinnaker.orca.q.handler import com.netflix.spinnaker.orca.ExecutionStatus.NOT_STARTED -import com.netflix.spinnaker.orca.ExecutionStatus.SUCCEEDED import com.netflix.spinnaker.orca.pipeline.model.Stage import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository import com.netflix.spinnaker.orca.q.* @@ -40,12 +39,10 @@ class ContinueParentStageHandler( override fun handle(message: ContinueParentStage) { message.withStage { stage -> if (stage.allBeforeStagesComplete()) { - if (stage.hasTasks()) { - stage.runFirstTask() - } else if (stage.hasAfterStages()) { - stage.runAfterStages() - } else { - queue.push(CompleteStage(stage, SUCCEEDED)) + when { + stage.hasTasks() -> stage.runFirstTask() + stage.hasAfterStages() -> stage.runAfterStages() + else -> queue.push(CompleteStage(stage)) } } else if (!stage.anyBeforeStagesFailed()) { log.warn("Re-queuing $message as other BEFORE stages are still running") diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/DeadMessageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/DeadMessageHandler.kt index 9c74b24fc7..a42b881e4f 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/DeadMessageHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/DeadMessageHandler.kt @@ -18,6 +18,7 @@ package com.netflix.spinnaker.orca.q.handler import com.netflix.spinnaker.orca.ExecutionStatus.TERMINAL import com.netflix.spinnaker.orca.q.* +import com.netflix.spinnaker.orca.q.Message.* import org.slf4j.LoggerFactory import org.springframework.stereotype.Component @@ -26,11 +27,28 @@ import org.springframework.stereotype.Component open fun handle(queue: Queue, message: Message) { log.error("Dead message: $message") - when (message) { - is TaskLevel -> queue.push(CompleteTask(message, TERMINAL)) - is StageLevel -> queue.push(CompleteStage(message, TERMINAL)) - is ExecutionLevel -> queue.push(CompleteExecution(message)) - else -> log.error("Unhandled message type ${message.javaClass}") + terminationMessageFor(message) + ?.let { + it.setAttribute(DeadMessageAttribute) + queue.push(it) + } + } + + private fun terminationMessageFor(message: Message): Message? { + if (message.hasAttribute()) { + log.warn("Already sent $message to DLQ") + return null + } + return when (message) { + is TaskLevel -> CompleteTask(message, TERMINAL) + is StageLevel -> AbortStage(message) + is ExecutionLevel -> CompleteExecution(message) + else -> { + log.error("Unhandled message type ${message.javaClass}") + null + } } } } + +internal object DeadMessageAttribute : Attribute diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RescheduleExecutionHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RescheduleExecutionHandler.kt new file mode 100644 index 0000000000..2040a5c300 --- /dev/null +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RescheduleExecutionHandler.kt @@ -0,0 +1,52 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.q.handler + +import com.netflix.spinnaker.orca.ExecutionStatus +import com.netflix.spinnaker.orca.Task +import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository +import com.netflix.spinnaker.orca.q.* +import org.springframework.stereotype.Component + +@Component +class RescheduleExecutionHandler( + override val queue: Queue, + override val repository: ExecutionRepository +) : MessageHandler { + + override val messageType = RescheduleExecution::class.java + + @Suppress("UNCHECKED_CAST") + override fun handle(message: RescheduleExecution) { + message.withExecution { execution -> + execution + .getStages() + .filter { it.getStatus() == ExecutionStatus.RUNNING } + .forEach { stage -> + stage.getTasks() + .filter { it.status == ExecutionStatus.RUNNING } + .forEach { + queue.reschedule(RunTask(message, + stage.getId(), + it.id, + Class.forName(it.implementingClass) as Class + )) + } + } + } + } +} diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RestartStageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RestartStageHandler.kt index 316c71ef19..31506d15a6 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RestartStageHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RestartStageHandler.kt @@ -18,7 +18,7 @@ package com.netflix.spinnaker.orca.q.handler import com.netflix.spinnaker.orca.ExecutionStatus.NOT_STARTED import com.netflix.spinnaker.orca.ExecutionStatus.RUNNING -import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilderFactory import com.netflix.spinnaker.orca.pipeline.model.Stage import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository import com.netflix.spinnaker.orca.q.MessageHandler @@ -32,7 +32,7 @@ import java.time.Clock class RestartStageHandler( override val queue: Queue, override val repository: ExecutionRepository, - override val stageDefinitionBuilders: Collection, + override val stageDefinitionBuilderFactory: StageDefinitionBuilderFactory, private val clock: Clock ) : MessageHandler, StageBuilderAware { diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RunTaskHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RunTaskHandler.kt index 4c5da4a065..2109ea5ba6 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RunTaskHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/RunTaskHandler.kt @@ -18,11 +18,8 @@ package com.netflix.spinnaker.orca.q.handler import com.netflix.spectator.api.Registry import com.netflix.spectator.api.histogram.BucketCounter -import com.netflix.spinnaker.orca.ExecutionStatus +import com.netflix.spinnaker.orca.* import com.netflix.spinnaker.orca.ExecutionStatus.* -import com.netflix.spinnaker.orca.RetryableTask -import com.netflix.spinnaker.orca.Task -import com.netflix.spinnaker.orca.TaskResult import com.netflix.spinnaker.orca.exceptions.ExceptionHandler import com.netflix.spinnaker.orca.exceptions.TimeoutException import com.netflix.spinnaker.orca.pipeline.model.Execution @@ -164,22 +161,34 @@ class RunTaskHandler( else -> Duration.ofSeconds(1) } - private fun Task.formatTimeout(timeout: Long): String { + private fun formatTimeout(timeout: Long): String { return DurationFormatUtils.formatDurationWords(timeout, true, true) } private fun Task.checkForTimeout(stage: Stage<*>, taskModel: com.netflix.spinnaker.orca.pipeline.model.Task, message: Message) { + checkForStageTimeout(stage) + checkForTaskTimeout(taskModel, stage, message) + } + + private fun Task.checkForTaskTimeout(taskModel: com.netflix.spinnaker.orca.pipeline.model.Task, stage: Stage<*>, message: Message) { if (this is RetryableTask) { val startTime = taskModel.startTime.toInstant() val pausedDuration = stage.getExecution().pausedDurationRelativeTo(startTime) - val throttleTime = message.getAttribute()?.totalThrottleTimeMs ?: 0 val elapsedTime = Duration.between(startTime, clock.instant()) - if (elapsedTime.minus(pausedDuration).minusMillis(throttleTime) > timeoutDuration(stage)) { + val throttleTime = message.getAttribute()?.totalThrottleTimeMs ?: 0 + val actualTimeout = ( + if (this is OverridableTimeoutRetryableTask && stage.getTopLevelTimeout().isPresent) + stage.getTopLevelTimeout().get().toDuration() + else + timeout.toDuration() + ) + if (elapsedTime.minus(pausedDuration).minusMillis(throttleTime) > actualTimeout) { val durationString = formatTimeout(elapsedTime.toMillis()) val msg = StringBuilder("${javaClass.simpleName} of stage ${stage.getName()} timed out after $durationString. ") msg.append("pausedDuration: ${formatTimeout(pausedDuration.toMillis())}, ") msg.append("throttleTime: ${formatTimeout(throttleTime)}, ") - msg.append("elapsedTime: ${formatTimeout(elapsedTime.toMillis())}") + msg.append("elapsedTime: ${formatTimeout(elapsedTime.toMillis())},") + msg.append("timeoutValue: ${formatTimeout(actualTimeout.toMillis())}") log.warn(msg.toString()) throw TimeoutException(msg.toString()) @@ -187,8 +196,16 @@ class RunTaskHandler( } } - private fun RetryableTask.timeoutDuration(stage: Stage<*>): Duration - = stage.getTopLevelTimeout().orElse(timeout).toDuration() + private fun checkForStageTimeout(stage: Stage<*>) { + stage.getTopLevelTimeout().map(Duration::ofMillis).ifPresent({ + val startTime = stage.getTopLevelStage().getStartTime().toInstant() + val elapsedTime = Duration.between(startTime, clock.instant()) + val pausedDuration = stage.getExecution().pausedDurationRelativeTo(startTime) + if (elapsedTime.minus(pausedDuration) > it) { + throw TimeoutException("Stage ${stage.getName()} timed out after ${formatTimeout(elapsedTime.toMillis())}") + } + }) + } private fun Execution<*>.pausedDurationRelativeTo(instant: Instant?): Duration { diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/SkipStageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/SkipStageHandler.kt new file mode 100644 index 0000000000..1a8a8a20b3 --- /dev/null +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/SkipStageHandler.kt @@ -0,0 +1,49 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.q.handler + +import com.netflix.spinnaker.orca.ExecutionStatus.* +import com.netflix.spinnaker.orca.events.StageComplete +import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository +import com.netflix.spinnaker.orca.q.MessageHandler +import com.netflix.spinnaker.orca.q.Queue +import com.netflix.spinnaker.orca.q.SkipStage +import org.springframework.context.ApplicationEventPublisher +import org.springframework.stereotype.Component +import java.time.Clock + +@Component +class SkipStageHandler( + override val queue: Queue, + override val repository: ExecutionRepository, + private val publisher: ApplicationEventPublisher, + private val clock: Clock +) : MessageHandler { + override fun handle(message: SkipStage) { + message.withStage { stage -> + if (stage.getStatus() in setOf(RUNNING, NOT_STARTED)) { + stage.setStatus(SKIPPED) + stage.setEndTime(clock.millis()) + repository.storeStage(stage) + stage.startNext() + publisher.publishEvent(StageComplete(this, stage)) + } + } + } + + override val messageType = SkipStage::class.java +} diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/StageBuilderAware.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/StageBuilderAware.kt index 10b11a18d7..a9dba6abe3 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/StageBuilderAware.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/StageBuilderAware.kt @@ -16,15 +16,14 @@ package com.netflix.spinnaker.orca.q.handler -import com.netflix.spinnaker.orca.pipeline.ExecutionRunner.NoSuchStageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilderFactory import com.netflix.spinnaker.orca.pipeline.model.Stage interface StageBuilderAware { - val stageDefinitionBuilders: Collection + val stageDefinitionBuilderFactory: StageDefinitionBuilderFactory fun Stage<*>.builder(): StageDefinitionBuilder = - stageDefinitionBuilders.find { it.type == getType() || getContext().get("alias") == it.type } - ?: throw NoSuchStageDefinitionBuilder(getType()) + stageDefinitionBuilderFactory.builderFor(this) } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/StartStageHandler.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/StartStageHandler.kt index 5490da1c29..415569634e 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/StartStageHandler.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/StartStageHandler.kt @@ -17,10 +17,11 @@ package com.netflix.spinnaker.orca.q.handler import com.fasterxml.jackson.databind.ObjectMapper -import com.netflix.spinnaker.orca.ExecutionStatus.* +import com.netflix.spinnaker.orca.ExecutionStatus.NOT_STARTED +import com.netflix.spinnaker.orca.ExecutionStatus.RUNNING import com.netflix.spinnaker.orca.events.StageStarted import com.netflix.spinnaker.orca.exceptions.ExceptionHandler -import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilderFactory import com.netflix.spinnaker.orca.pipeline.expressions.PipelineExpressionEvaluator import com.netflix.spinnaker.orca.pipeline.model.OptionalStageSupport import com.netflix.spinnaker.orca.pipeline.model.Pipeline @@ -41,7 +42,7 @@ class StartStageHandler( override val queue: Queue, override val repository: ExecutionRepository, override val stageNavigator: StageNavigator, - override val stageDefinitionBuilders: Collection, + override val stageDefinitionBuilderFactory: StageDefinitionBuilderFactory, override val contextParameterProcessor: ContextParameterProcessor, private val publisher: ApplicationEventPublisher, private val exceptionHandlers: List, @@ -62,7 +63,7 @@ class StartStageHandler( if (stage.getStatus() != NOT_STARTED) { log.warn("Ignoring $message as stage is already ${stage.getStatus()}") } else if (stage.shouldSkip()) { - queue.push(CompleteStage(message, SKIPPED)) + queue.push(SkipStage(message)) } else { try { stage.withAuth { @@ -88,7 +89,7 @@ class StartStageHandler( log.error("Error running ${stage.getType()} stage for ${message.executionType.simpleName}[${message.executionId}]", e) stage.getContext()["exception"] = exceptionDetails repository.storeStage(stage) - queue.push(CompleteStage(message, stage.failureStatus())) + queue.push(CompleteStage(message)) } } } @@ -117,7 +118,7 @@ class StartStageHandler( if (task == null) { val afterStages = firstAfterStages() if (afterStages.isEmpty()) { - queue.push(CompleteStage(this, SUCCEEDED)) + queue.push(CompleteStage(this)) } else { afterStages.forEach { queue.push(StartStage(it)) diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/stages.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/stages.kt index 388a49839e..3fc542266a 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/stages.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/handler/stages.kt @@ -19,14 +19,28 @@ package com.netflix.spinnaker.orca.q.handler import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.ExecutionStatus.* import com.netflix.spinnaker.orca.pipeline.model.Stage +import com.netflix.spinnaker.orca.pipeline.model.Task +import com.netflix.spinnaker.orca.q.syntheticStages + +fun Stage<*>.determineStatus(): ExecutionStatus { + val syntheticStatuses = syntheticStages().map(Stage<*>::getStatus) + val taskStatuses = getTasks().map(Task::getStatus) + val allStatuses = syntheticStatuses + taskStatuses + return when { + allStatuses.contains(TERMINAL) -> TERMINAL + allStatuses.contains(STOPPED) -> STOPPED + allStatuses.contains(CANCELED) -> CANCELED + allStatuses.contains(FAILED_CONTINUE) -> FAILED_CONTINUE + allStatuses.all { it == SUCCEEDED } -> SUCCEEDED + else -> TODO("handle this case") + } +} fun Stage<*>.failureStatus(default: ExecutionStatus = TERMINAL) = - if (shouldContinueOnFailure()) { - FAILED_CONTINUE - } else if (shouldFailPipeline()) { - default - } else { - STOPPED + when { + shouldContinueOnFailure() -> FAILED_CONTINUE + shouldFailPipeline() -> default + else -> STOPPED } private fun Stage<*>.shouldFailPipeline() = diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/memory/InMemoryQueue.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/memory/InMemoryQueue.kt index 1dba8e6607..ab1ba37757 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/memory/InMemoryQueue.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/memory/InMemoryQueue.kt @@ -62,11 +62,19 @@ class InMemoryQueue( } override fun push(message: Message, delay: TemporalAmount) { - if (queue.none { it.payload == message }) { - queue.put(Envelope(message, clock.instant().plus(delay), clock)) - fire(message) - } else { + val existed = queue.removeIf { it.payload == message } + queue.put(Envelope(message, clock.instant().plus(delay), clock)) + if (existed) { fire(message) + } else { + fire(message) + } + } + + override fun reschedule(message: Message, delay: TemporalAmount) { + val existed = queue.removeIf { it.payload == message } + if (existed) { + queue.put(Envelope(message, clock.instant().plus(delay), clock)) } } @@ -79,12 +87,13 @@ class InMemoryQueue( deadMessageHandler.invoke(this, message.payload) fire() } else { - if (queue.none { it.payload == message.payload }) { - log.warn("redelivering unacked message ${message.payload}") - queue.put(message.copy(scheduledTime = now, count = message.count + 1)) - fire() - } else { + val existed = queue.removeIf { it.payload == message.payload } + log.warn("redelivering unacked message ${message.payload}") + queue.put(message.copy(scheduledTime = now, count = message.count + 1)) + if (existed) { fire(message.payload) + } else { + fire() } } } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/AtlasQueueMonitor.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/AtlasQueueMonitor.kt index 76b41e13a8..d260f32869 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/AtlasQueueMonitor.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/AtlasQueueMonitor.kt @@ -22,6 +22,7 @@ import com.netflix.spinnaker.orca.q.ApplicationAware import com.netflix.spinnaker.orca.q.Queue import org.slf4j.LoggerFactory import org.springframework.beans.factory.annotation.Autowired +import org.springframework.beans.factory.annotation.Qualifier import org.springframework.boot.autoconfigure.condition.ConditionalOnBean import org.springframework.context.ApplicationListener import org.springframework.scheduling.annotation.Scheduled @@ -39,7 +40,7 @@ import javax.annotation.PostConstruct @ConditionalOnBean(MonitorableQueue::class) open class AtlasQueueMonitor @Autowired constructor( - private val queue: MonitorableQueue, + @Qualifier("queueImpl") private val queue: MonitorableQueue, private val registry: Registry, private val clock: Clock ) : ApplicationListener { @@ -56,6 +57,8 @@ open class AtlasQueueMonitor is MessageDead -> event.counter.increment() is MessageDuplicate -> event.counter.increment() is LockFailed -> event.counter.increment() + is MessageRescheduled -> event.counter.increment() + is MessageNotFound -> event.counter.increment() } } @@ -79,9 +82,6 @@ open class AtlasQueueMonitor registry.gauge("queue.orphaned.messages", this, { it.lastState.orphaned.toDouble() }) - registry.gauge("queue.hash.drift", this, { - it.lastState.hashDrift.toDouble() - }) registry.gauge("queue.last.poll.age", this, { Duration .between(it.lastQueuePoll, clock.instant()) @@ -157,4 +157,18 @@ open class AtlasQueueMonitor */ private val LockFailed.counter: Counter get() = registry.counter("queue.lock.failed") + + /** + * Count of attempted message rescheduling that succeeded (in other words, + * that message existed on the queue). + */ + private val MessageRescheduled.counter: Counter + get() = registry.counter("queue.reschedule.succeeded") + + /** + * Count of attempted message rescheduling that failed (in other words, + * that message did not exist on the queue). + */ + private val MessageNotFound.counter: Counter + get() = registry.counter("queue.message.notfound") } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/MonitorableQueue.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/MonitorableQueue.kt index d6350c1c19..90fc601025 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/MonitorableQueue.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/MonitorableQueue.kt @@ -44,6 +44,8 @@ inline fun MonitorableQueue.fire(message: Message? = nu LockFailed::class -> LockFailed(this) MessagePushed::class -> MessagePushed(this, message!!) MessageDuplicate::class -> MessageDuplicate(this, message!!) + MessageRescheduled::class -> MessageRescheduled(this, message!!) + MessageNotFound::class -> MessageNotFound(this, message!!) else -> throw IllegalArgumentException("Unknown event type ${E::class}") } publisher.publishEvent(event) @@ -68,13 +70,5 @@ data class QueueState( * Some implementations may not have any way to implement this metric. It is * only intended for alerting leaks. */ - val orphaned: Int = 0, - /** - * Difference between number of known message hashes and number of de-dupe - * hashes plus in-process messages. - * - * Some implementations may not have any way to implement this metric. It is - * only intended for alerting leaks. - */ - val hashDrift: Int = 0 + val orphaned: Int = 0 ) diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/QueueEvent.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/QueueEvent.kt index 7590dad4d0..38a6c6992f 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/QueueEvent.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/metrics/QueueEvent.kt @@ -38,4 +38,5 @@ class MessageRetried(source: MonitorableQueue) : QueueEvent(source) class MessageDead(source: MonitorableQueue) : QueueEvent(source) class MessageDuplicate(source: MonitorableQueue, payload: Message) : PayloadQueueEvent(source, payload) class LockFailed(source: MonitorableQueue) : QueueEvent(source) - +class MessageRescheduled(source: MonitorableQueue, payload: Message) : PayloadQueueEvent(source, payload) +class MessageNotFound(source: MonitorableQueue, payload: Message) : PayloadQueueEvent(source, payload) diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/trafficshaping/TrafficShapingQueue.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/trafficshaping/TrafficShapingQueue.kt index 86e6565920..a48f5150e1 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/trafficshaping/TrafficShapingQueue.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/q/trafficshaping/TrafficShapingQueue.kt @@ -91,6 +91,8 @@ class TrafficShapingQueue( override fun push(message: Message, delay: TemporalAmount) = queueImpl.push(message, delay) + override fun reschedule(message: Message, delay: TemporalAmount) = queueImpl.reschedule(message, delay) + override fun retry() { queueImpl.retry() } diff --git a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/time/time.kt b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/time/time.kt index aac5e31f06..67fb5a12a0 100644 --- a/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/time/time.kt +++ b/orca-queue/src/main/kotlin/com/netflix/spinnaker/orca/time/time.kt @@ -1,3 +1,18 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.spinnaker.orca.time import java.time.Duration diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/PipelineBuilder.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/PipelineBuilder.kt index 1039489644..7687a7534e 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/PipelineBuilder.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/PipelineBuilder.kt @@ -19,10 +19,14 @@ package com.netflix.spinnaker.orca.q import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.model.Stage +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFORE import com.netflix.spinnaker.orca.pipeline.model.Task import java.lang.System.currentTimeMillis import kotlin.reflect.jvm.jvmName +/** + * Build a pipeline. + */ fun pipeline(init: Pipeline.() -> Unit = {}): Pipeline { val pipeline = Pipeline("covfefe") pipeline.buildTime = currentTimeMillis() @@ -30,6 +34,11 @@ fun pipeline(init: Pipeline.() -> Unit = {}): Pipeline { return pipeline } +/** + * Build a top-level stage. Use in the context of [#pipeline]. + * + * Automatically hooks up execution. + */ fun > T.stage(init: Stage.() -> Unit): Stage { val stage = Stage() stage.execution = this @@ -40,6 +49,26 @@ fun > T.stage(init: Stage.() -> Unit): Stage { return stage } +/** + * Build a synthetic stage. Use in the context of [#stage]. + * + * Automatically hooks up execution and parent stage. + */ +fun > Stage.stage(init: Stage.() -> Unit): Stage { + val stage = Stage() + stage.execution = execution + stage.type = "test" + stage.refId = "$refId<1" + stage.parentStageId = id + stage.syntheticStageOwner = STAGE_BEFORE + execution.stages.add(stage) + stage.init() + return stage +} + +/** + * Build a task. Use in the context of [#stage]. + */ fun > Stage.task(init: Task.() -> Unit): Task { val task = Task() task.implementingClass = DummyTask::class.jvmName diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueIntegrationTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueIntegrationTest.kt index 5c636cecdb..e9630b9f06 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueIntegrationTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueIntegrationTest.kt @@ -31,14 +31,17 @@ import com.netflix.spinnaker.orca.config.OrcaConfiguration import com.netflix.spinnaker.orca.exceptions.DefaultExceptionHandler import com.netflix.spinnaker.orca.pipeline.RestrictExecutionDuringTimeWindow import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder +import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder.newStage import com.netflix.spinnaker.orca.pipeline.TaskNode.Builder import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Stage +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFORE import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository import com.netflix.spinnaker.orca.pipeline.persistence.jedis.JedisExecutionRepository import com.netflix.spinnaker.orca.pipeline.util.ContextParameterProcessor import com.netflix.spinnaker.orca.pipeline.util.StageNavigator import com.netflix.spinnaker.orca.test.redis.EmbeddedRedisConfiguration +import com.netflix.spinnaker.spek.shouldAllEqual import com.netflix.spinnaker.spek.shouldEqual import com.nhaarman.mockito_kotlin.* import org.junit.After @@ -59,10 +62,10 @@ import java.time.Instant.now import java.time.ZoneId @SpringBootTest(classes = arrayOf(TestConfig::class), properties = arrayOf( - "queue.retry.delay.ms=10", - "logging.level.root=ERROR", - "logging.level.org.springframework.test=ERROR", - "logging.level.com.netflix.spinnaker=FATAL" + "queue.retry.delay.ms=10"//, +// "logging.level.root=ERROR", +// "logging.level.org.springframework.test=ERROR", +// "logging.level.com.netflix.spinnaker=FATAL" )) @RunWith(SpringRunner::class) open class QueueIntegrationTest { @@ -422,6 +425,44 @@ open class QueueIntegrationTest { } } + @Test + fun `parallel stages do not duplicate execution windows`() { + val pipeline = pipeline { + application = "spinnaker" + stage { + refId = "1" + type = "parallel" + context = mapOf( + "restrictExecutionDuringTimeWindow" to true, + "restrictedExecutionWindow" to mapOf( + "days" to (1..7).toList(), + "whitelist" to listOf(mapOf( + "startHour" to now().atZone(timeZone).hour, + "startMin" to 0, + "endHour" to now().atZone(timeZone).hour + 1, + "endMin" to 0 + )) + ) + ) + } + } + repository.store(pipeline) + + whenever(dummyTask.timeout) doReturn 2000L + whenever(dummyTask.execute(any())) doReturn TaskResult.SUCCEEDED + + context.runToCompletion(pipeline, runner::start, repository) + + repository.retrievePipeline(pipeline.id).apply { + status shouldEqual SUCCEEDED + stages.size shouldEqual 5 + stages.first().type shouldEqual RestrictExecutionDuringTimeWindow.TYPE + stages[1..3].map { it.type } shouldAllEqual "dummy" + stages.last().type shouldEqual "parallel" + stages.map { it.status } shouldMatch allElements(equalTo(SUCCEEDED)) + } + } + // TODO: this test is verifying a bunch of things at once, it would make sense to break it up @Test fun `can resolve expressions in stage contexts`() { val pipeline = pipeline { @@ -454,7 +495,6 @@ open class QueueIntegrationTest { repository.retrievePipeline(pipeline.id).apply { status shouldEqual SUCCEEDED - println(stages.first().context) // resolved expressions should be persisted stages.first().context["expr"] shouldEqual true (stages.first().context["key"] as Map)["expr"] shouldEqual true @@ -615,7 +655,7 @@ open class TestConfig { on { timeout } doReturn Duration.ofMinutes(2).toMillis() } - @Bean open fun dummyStage(): StageDefinitionBuilder = object : StageDefinitionBuilder { + @Bean open fun dummyStage() = object : StageDefinitionBuilder { override fun > taskGraph(stage: Stage, builder: Builder) { builder.withTask("dummy", DummyTask::class.java) } @@ -623,6 +663,15 @@ open class TestConfig { override fun getType() = "dummy" } + @Bean open fun parallelStage() = object : StageDefinitionBuilder { + override fun > parallelStages(stage: Stage) = + listOf("us-east-1", "us-west-2", "eu-west-1").map { region -> + newStage(stage.execution, "dummy", "dummy $region", stage.context + mapOf("region" to region), stage, STAGE_BEFORE) + } + + override fun getType() = "parallel" + } + @Bean open fun currentInstanceId() = "localhost" @Bean open fun contextParameterProcessor() = ContextParameterProcessor() diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueProcessorTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueProcessorTest.kt index 2cea1ca815..bf71b332ec 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueProcessorTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueProcessorTest.kt @@ -226,7 +226,6 @@ private class BlockingThreadExecutor : Executor { } } -class BlockingQueueExecutor : QueueExecutor { - override val executor: Executor = BlockingThreadExecutor() - override fun hasCapacity(): Boolean = true +class BlockingQueueExecutor : QueueExecutor(BlockingThreadExecutor()) { + override fun hasCapacity() = true } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueShovelTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueShovelTest.kt new file mode 100644 index 0000000000..3f25a4c509 --- /dev/null +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueShovelTest.kt @@ -0,0 +1,55 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.spinnaker.orca.q + +import com.netflix.spectator.api.NoopRegistry +import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.nhaarman.mockito_kotlin.* +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.jetbrains.spek.api.lifecycle.CachingMode +import org.jetbrains.spek.subject.SubjectSpek + +class QueueShovelTest : SubjectSpek({ + + val queue: Queue = mock() + val previousQueue: Queue = mock() + val registry = NoopRegistry() + + subject(CachingMode.GROUP) { + QueueShovel(queue, previousQueue, registry) + } + + describe("polling the previous queue") { + val message = StartExecution(Pipeline::class.java, "1", "spinnaker") + + beforeGroup { + subject.enabled.set(true) + whenever(previousQueue.poll(any())) doAnswer { + it.getArgument(0)(message, {}) + } + } + + on("the shovel poll method is invoked") { + subject.migrateOne() + } + + it("pushes the message onto the current queue") { + verify(queue).push(message) + } + } +}) diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueTest.kt index b1b4ca1991..24eaa41ad4 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/QueueTest.kt @@ -26,9 +26,11 @@ import org.jetbrains.spek.api.dsl.describe import org.jetbrains.spek.api.dsl.given import org.jetbrains.spek.api.dsl.it import org.jetbrains.spek.api.dsl.on +import org.threeten.extra.Hours import java.io.Closeable import java.time.Clock import java.time.Duration +import java.time.Duration.ZERO abstract class QueueTest( createQueue: (Clock, DeadMessageCallback) -> Q, @@ -105,7 +107,7 @@ abstract class QueueTest( afterGroup(::resetMocks) on("polling the queue twice") { - queue!!.apply { + with(queue!!) { poll(callback) poll(callback) } @@ -169,7 +171,7 @@ abstract class QueueTest( beforeGroup { queue = createQueue(clock, deadLetterCallback) - queue!!.apply { + with(queue!!) { push(message) poll { _, ack -> ack() @@ -181,7 +183,7 @@ abstract class QueueTest( afterGroup(::resetMocks) on("polling the queue after the message acknowledgment has timed out") { - queue!!.apply { + with(queue!!) { clock.incrementBy(ackTimeout) retry() poll(callback) @@ -198,7 +200,7 @@ abstract class QueueTest( beforeGroup { queue = createQueue(clock, deadLetterCallback) - queue!!.apply { + with(queue!!) { push(message) poll { _, _ -> } } @@ -208,7 +210,7 @@ abstract class QueueTest( afterGroup(::resetMocks) on("polling the queue after the message acknowledgment has timed out") { - queue!!.apply { + with(queue!!) { clock.incrementBy(ackTimeout) retry() poll(callback) @@ -225,7 +227,7 @@ abstract class QueueTest( beforeGroup { queue = createQueue(clock, deadLetterCallback) - queue!!.apply { + with(queue!!) { push(message) repeat(2) { poll { _, _ -> } @@ -239,7 +241,7 @@ abstract class QueueTest( afterGroup(::resetMocks) on("polling the queue again") { - queue!!.apply { + with(queue!!) { poll(callback) } } @@ -254,7 +256,7 @@ abstract class QueueTest( beforeGroup { queue = createQueue(clock, deadLetterCallback) - queue!!.apply { + with(queue!!) { push(message) repeat(maxRetries) { poll { _, _ -> } @@ -268,7 +270,7 @@ abstract class QueueTest( afterGroup(::resetMocks) on("polling the queue again") { - queue!!.apply { + with(queue!!) { poll(callback) } } @@ -283,7 +285,7 @@ abstract class QueueTest( and("the message has been dead-lettered") { on("the next time retry checks happen") { - queue!!.apply { + with(queue!!) { retry() poll(callback) } @@ -304,6 +306,81 @@ abstract class QueueTest( given("a message was pushed") { val message = StartExecution(Pipeline::class.java, "1", "foo") + and("a duplicate is pushed with a newer delivery time") { + val delay = Hours.of(1) + + beforeGroup { + queue = createQueue(clock, deadLetterCallback).apply { + push(message, delay) + push(message.copy()) + } + } + + afterGroup(::stopQueue) + afterGroup(::resetMocks) + + on("polling the queue") { + queue!!.poll(callback) + } + + it("delivers the message immediately and only once") { + verify(callback).invoke(eq(message), any()) + } + + it("does not hold on to the first message") { + clock.incrementBy(delay) + queue!!.poll(callback) + verifyNoMoreInteractions(callback) + } + } + + and("the message delivery time is updated") { + val delay = Hours.of(1) + + beforeGroup { + queue = createQueue(clock, deadLetterCallback).apply { + push(message, delay) + reschedule(message, ZERO) + } + } + + afterGroup(::stopQueue) + afterGroup(::resetMocks) + + on("polling the queue") { + queue!!.poll(callback) + } + + it("delivers the message immediately and only once") { + verify(callback).invoke(eq(message), any()) + } + + it("does not deliver again"){ + verifyNoMoreInteractions(callback) + } + } + + and("the delivery time for a message that isn't on the queue isn't updated") { + val message2 = StartExecution(Pipeline::class.java, "2", "bar") + + beforeGroup { + queue = createQueue(clock, deadLetterCallback).apply { + reschedule(message2, ZERO) + } + } + + afterGroup(::stopQueue) + afterGroup(::resetMocks) + + on("polling the queue") { + queue!!.poll(callback) + } + + it("there are no messages on the queue"){ + verifyNoMoreInteractions(callback) + } + } + and("a different message is pushed before acknowledging the first") { val newMessage = message.copy(executionId = "2") @@ -374,6 +451,40 @@ abstract class QueueTest( } } + and("another identical message is pushed with a delay and the first is never acknowledged") { + val delay = Hours.of(1) + + beforeGroup { + queue = createQueue(clock, deadLetterCallback).apply { + push(message) + poll { _, ack -> + push(message.copy(), delay) + } + } + } + + afterGroup(::stopQueue) + afterGroup(::resetMocks) + + on("polling the queue again after the first message times out") { + with(queue!!) { + clock.incrementBy(ackTimeout) + retry() + poll(callback) + } + } + + it("re-queued the message for immediate delivery") { + verify(callback).invoke(eq(message), any()) + } + + it("discarded the delayed message") { + clock.incrementBy(delay) + queue!!.poll(callback) + verifyNoMoreInteractions(callback) + } + } + and("another identical message is pushed after acknowledging the first") { beforeGroup { queue = createQueue(clock, deadLetterCallback).apply { diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/Stages.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/Stages.kt index bcc83fcba5..5b897f1ed6 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/Stages.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/Stages.kt @@ -16,7 +16,6 @@ package com.netflix.spinnaker.orca.q -import com.netflix.spinnaker.orca.pipeline.BranchingStageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder.newStage import com.netflix.spinnaker.orca.pipeline.TaskNode.Builder @@ -54,8 +53,8 @@ val stageWithSyntheticBefore = object : StageDefinitionBuilder { } override fun > aroundStages(stage: Stage) = listOf( - newStage(stage.execution, singleTaskStage.type, "pre1", mutableMapOf(), stage, STAGE_BEFORE), - newStage(stage.execution, singleTaskStage.type, "pre2", mutableMapOf(), stage, STAGE_BEFORE) + newStage(stage.execution, singleTaskStage.type, "pre1", stage.context, stage, STAGE_BEFORE), + newStage(stage.execution, singleTaskStage.type, "pre2", stage.context, stage, STAGE_BEFORE) ) } @@ -63,7 +62,7 @@ val stageWithSyntheticBeforeAndNoTasks = object : StageDefinitionBuilder { override fun getType() = "stageWithSyntheticBeforeAndNoTasks" override fun > aroundStages(stage: Stage) = listOf( - newStage(stage.execution, singleTaskStage.type, "pre", mutableMapOf(), stage, STAGE_BEFORE) + newStage(stage.execution, singleTaskStage.type, "pre", stage.context, stage, STAGE_BEFORE) ) } @@ -71,8 +70,8 @@ val stageWithSyntheticBeforeAndAfterAndNoTasks = object : StageDefinitionBuilder override fun getType() = "stageWithSyntheticBeforeAndAfterAndNoTasks" override fun > aroundStages(stage: Stage) = listOf( - newStage(stage.execution, singleTaskStage.type, "pre", mutableMapOf(), stage, STAGE_BEFORE), - newStage(stage.execution, singleTaskStage.type, "post", mutableMapOf(), stage, STAGE_AFTER) + newStage(stage.execution, singleTaskStage.type, "pre", stage.context, stage, STAGE_BEFORE), + newStage(stage.execution, singleTaskStage.type, "post", stage.context, stage, STAGE_AFTER) ) } @@ -83,8 +82,8 @@ val stageWithSyntheticAfter = object : StageDefinitionBuilder { } override fun > aroundStages(stage: Stage) = listOf( - newStage(stage.execution, singleTaskStage.type, "post1", mutableMapOf(), stage, STAGE_AFTER), - newStage(stage.execution, singleTaskStage.type, "post2", mutableMapOf(), stage, STAGE_AFTER) + newStage(stage.execution, singleTaskStage.type, "post1", stage.context, stage, STAGE_AFTER), + newStage(stage.execution, singleTaskStage.type, "post2", stage.context, stage, STAGE_AFTER) ) } @@ -92,7 +91,7 @@ val stageWithSyntheticAfterAndNoTasks = object : StageDefinitionBuilder { override fun getType() = "stageWithSyntheticAfterAndNoTasks" override fun > aroundStages(stage: Stage) = listOf( - newStage(stage.execution, singleTaskStage.type, "post", mutableMapOf(), stage, STAGE_AFTER) + newStage(stage.execution, singleTaskStage.type, "post", stage.context, stage, STAGE_AFTER) ) } @@ -100,30 +99,18 @@ val stageWithNestedSynthetics = object : StageDefinitionBuilder { override fun getType() = "stageWithNestedSynthetics" override fun > aroundStages(stage: Stage) = listOf( - newStage(stage.execution, stageWithSyntheticBefore.type, "post", mutableMapOf(), stage, STAGE_AFTER) + newStage(stage.execution, stageWithSyntheticBefore.type, "post", stage.context, stage, STAGE_AFTER) ) } -val stageWithParallelBranches = object : BranchingStageDefinitionBuilder { - override fun > parallelContexts(stage: Stage): Collection> = - listOf( - mapOf("region" to "us-east-1", "name" to "run in us-east-1"), - mapOf("region" to "us-west-2", "name" to "run in us-west-2"), - mapOf("region" to "eu-west-1", "name" to "run in eu-west-1") - ) - - override fun parallelStageName(stage: Stage<*>, hasParallelFlows: Boolean) = - if (hasParallelFlows) "is parallel" else "is not parallel" - - override fun preBranchGraph(stage: Stage<*>, builder: Builder) { - builder.withTask("pre-branch", DummyTask::class.java) - } +val stageWithParallelBranches = object : StageDefinitionBuilder { + override fun > parallelStages(stage: Stage) = + listOf("us-east-1", "us-west-2", "eu-west-1") + .map { region -> + newStage(stage.execution, singleTaskStage.type, "run in $region", stage.context + mapOf("region" to region), stage, STAGE_BEFORE) + } override fun > taskGraph(stage: Stage, builder: Builder) { - builder.withTask("in-branch", DummyTask::class.java) - } - - override fun postBranchGraph(stage: Stage<*>, builder: Builder) { builder.withTask("post-branch", DummyTask::class.java) } } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/Tasks.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/Tasks.kt index e8b77de3b3..f79e213c15 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/Tasks.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/Tasks.kt @@ -16,8 +16,10 @@ package com.netflix.spinnaker.orca.q +import com.netflix.spinnaker.orca.OverridableTimeoutRetryableTask import com.netflix.spinnaker.orca.RetryableTask import com.netflix.spinnaker.orca.Task interface DummyTask : RetryableTask interface InvalidTask : Task +interface DummyTimeoutOverrideTask : OverridableTimeoutRetryableTask diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/AbortStageHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/AbortStageHandlerTest.kt new file mode 100644 index 0000000000..74e2c55e9d --- /dev/null +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/AbortStageHandlerTest.kt @@ -0,0 +1,175 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.q.handler + +import com.netflix.spinnaker.orca.ExecutionStatus.* +import com.netflix.spinnaker.orca.events.StageComplete +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFORE +import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository +import com.netflix.spinnaker.orca.q.* +import com.netflix.spinnaker.orca.time.fixedClock +import com.netflix.spinnaker.orca.time.toInstant +import com.netflix.spinnaker.spek.shouldEqual +import com.nhaarman.mockito_kotlin.* +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.jetbrains.spek.api.lifecycle.CachingMode.GROUP +import org.jetbrains.spek.subject.SubjectSpek +import org.springframework.context.ApplicationEventPublisher + +object AbortStageHandlerTest : SubjectSpek({ + + val queue: Queue = mock() + val repository: ExecutionRepository = mock() + val publisher: ApplicationEventPublisher = mock() + val clock = fixedClock() + + subject(GROUP) { + AbortStageHandler(queue, repository, publisher, clock) + } + + fun resetMocks() { + reset(queue, repository, publisher) + } + + describe("aborting a stage") { + given("a stage that already completed") { + val pipeline = pipeline { + application = "whatever" + stage { + refId = "1" + type = "whatever" + status = SUCCEEDED + } + } + + val message = AbortStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } + + afterGroup(::resetMocks) + + on("receiving a message") { + subject.handle(message) + } + + it("does nothing at all") { + verifyZeroInteractions(queue) + verifyZeroInteractions(publisher) + verify(repository, never()).storeStage(any()) + } + } + + given("a top level stage") { + val pipeline = pipeline { + application = "whatever" + stage { + refId = "1" + type = "whatever" + status = RUNNING + } + } + + val message = AbortStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } + + afterGroup(::resetMocks) + + on("receiving a message") { + subject.handle(message) + } + + it("marks the stage as TERMINAL") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual TERMINAL + it.getEndTime().toInstant() shouldEqual clock.instant() + }) + } + + it("cancels the stage") { + verify(queue).push(CancelStage(message)) + } + + it("completes the execution") { + verify(queue).push(CompleteExecution(message)) + } + + it("emits an event") { + verify(publisher).publishEvent(check { + it.status shouldEqual TERMINAL + }) + } + } + + given("a synthetic level stage") { + val pipeline = pipeline { + application = "whatever" + stage { + refId = "1" + type = "whatever" + status = RUNNING + stage { + refId = "1<1" + type = "whatever" + status = RUNNING + syntheticStageOwner = STAGE_BEFORE + } + } + } + + val message = AbortStage(pipeline.stageByRef("1<1")) + + beforeGroup { + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } + + afterGroup(::resetMocks) + + on("receiving a message") { + subject.handle(message) + } + + it("marks the stage as TERMINAL") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual TERMINAL + it.getEndTime().toInstant() shouldEqual clock.instant() + }) + } + + it("cancels the stage") { + verify(queue).push(CancelStage(message)) + } + + it("completes the parent stage") { + verify(queue).push(CompleteStage(pipeline.stageByRef("1"))) + } + + it("emits an event") { + verify(publisher).publishEvent(check { + it.status shouldEqual TERMINAL + }) + } + } + } +}) diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CancelExecutionHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CancelExecutionHandlerTest.kt index d046398bd8..b0f9ec8c40 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CancelExecutionHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CancelExecutionHandlerTest.kt @@ -65,6 +65,10 @@ object CancelExecutionHandlerTest : SubjectSpek({ verify(repository).cancel(pipeline.id, "fzlem@netflix.com", "because") } + it("it triggers a reevaluate") { + verify(queue).push(RescheduleExecution(pipeline)) + } + it("does not send any further messages") { verifyZeroInteractions(queue) } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CancelStageHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CancelStageHandlerTest.kt index bef45e1ae9..131a0dc446 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CancelStageHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CancelStageHandlerTest.kt @@ -19,6 +19,7 @@ package com.netflix.spinnaker.orca.q.handler import com.google.common.util.concurrent.MoreExecutors import com.netflix.spinnaker.orca.CancellableStage import com.netflix.spinnaker.orca.ExecutionStatus.* +import com.netflix.spinnaker.orca.pipeline.DefaultStageDefinitionBuilderFactory import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository @@ -40,7 +41,7 @@ object CancelStageHandlerTest : SubjectSpek({ CancelStageHandler( queue, repository, - listOf(singleTaskStage, cancellableStage), + DefaultStageDefinitionBuilderFactory(singleTaskStage, cancellableStage), executor ) } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteStageHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteStageHandlerTest.kt index 3216ce6450..3f3782b1e5 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteStageHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteStageHandlerTest.kt @@ -20,6 +20,8 @@ import com.netflix.spinnaker.orca.ExecutionStatus.* import com.netflix.spinnaker.orca.events.StageComplete import com.netflix.spinnaker.orca.pipeline.expressions.PipelineExpressionEvaluator import com.netflix.spinnaker.orca.pipeline.model.Stage +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_AFTER +import com.netflix.spinnaker.orca.pipeline.model.SyntheticStageOwner.STAGE_BEFORE import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository import com.netflix.spinnaker.orca.pipeline.util.ContextParameterProcessor import com.netflix.spinnaker.orca.q.* @@ -27,10 +29,7 @@ import com.netflix.spinnaker.orca.time.fixedClock import com.netflix.spinnaker.spek.and import com.netflix.spinnaker.spek.shouldEqual import com.nhaarman.mockito_kotlin.* -import org.jetbrains.spek.api.dsl.context -import org.jetbrains.spek.api.dsl.describe -import org.jetbrains.spek.api.dsl.it -import org.jetbrains.spek.api.dsl.on +import org.jetbrains.spek.api.dsl.* import org.jetbrains.spek.api.lifecycle.CachingMode.GROUP import org.jetbrains.spek.subject.SubjectSpek import org.springframework.context.ApplicationEventPublisher @@ -49,186 +48,270 @@ object CompleteStageHandlerTest : SubjectSpek({ fun resetMocks() = reset(queue, repository, publisher) - setOf(SUCCEEDED, FAILED_CONTINUE, SKIPPED).forEach { stageStatus -> - describe("when a stage completes with $stageStatus status") { - and("it is already complete") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = singleTaskStage.type - status = stageStatus - endTime = clock.instant().minusSeconds(2).toEpochMilli() + describe("completing top level stages") { + setOf(SUCCEEDED, FAILED_CONTINUE).forEach { taskStatus -> + describe("when a stage's tasks complete with $taskStatus status") { + and("it is already complete") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = multiTaskStage.type + multiTaskStage.plan(this) + tasks[0].status = SUCCEEDED + tasks[1].status = taskStatus + tasks[2].status = SUCCEEDED + status = taskStatus + endTime = clock.instant().minusSeconds(2).toEpochMilli() + } } - } - val message = CompleteStage(pipeline.stageByRef("1"), stageStatus) + val message = CompleteStage(pipeline.stageByRef("1")) - beforeGroup { - whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline - } + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } - afterGroup(::resetMocks) + afterGroup(::resetMocks) - on("receiving $message") { - subject.handle(message) - } + on("receiving $message") { + subject.handle(message) + } - it("ignores the message") { - verify(repository, never()).storeStage(any()) - verifyZeroInteractions(queue) - verifyZeroInteractions(publisher) + it("ignores the message") { + verify(repository, never()).storeStage(any()) + verifyZeroInteractions(queue) + verifyZeroInteractions(publisher) + } } - } - and("it is the last stage") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = singleTaskStage.type - status = RUNNING + and("it is the last stage") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = singleTaskStage.type + singleTaskStage.plan(this) + tasks.first().status = taskStatus + status = RUNNING + } } - } - val message = CompleteStage(pipeline.stageByRef("1"), stageStatus) + val message = CompleteStage(pipeline.stageByRef("1")) - beforeGroup { - whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline - } + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } - afterGroup(::resetMocks) + afterGroup(::resetMocks) - action("the handler receives a message") { - subject.handle(message) - } + action("the handler receives a message") { + subject.handle(message) + } - it("updates the stage state") { - verify(repository).storeStage(check { - it.getStatus() shouldEqual stageStatus - it.getEndTime() shouldEqual clock.millis() - }) - } + it("updates the stage state") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual taskStatus + it.getEndTime() shouldEqual clock.millis() + }) + } - it("completes the execution") { - verify(queue).push(CompleteExecution( - pipeline // execution is SUCCEEDED even if stage was FAILED_CONTINUE or SKIPPED - )) - } + it("completes the execution") { + verify(queue).push(CompleteExecution(pipeline)) + } - it("does not emit any commands") { - verify(queue, never()).push(any()) - } + it("does not emit any commands") { + verify(queue, never()).push(any()) + } - it("publishes an event") { - verify(publisher).publishEvent(check { - it.executionType shouldEqual pipeline.javaClass - it.executionId shouldEqual pipeline.id - it.stageId shouldEqual message.stageId - it.status shouldEqual stageStatus - }) + it("publishes an event") { + verify(publisher).publishEvent(check { + it.executionType shouldEqual pipeline.javaClass + it.executionId shouldEqual pipeline.id + it.stageId shouldEqual message.stageId + it.status shouldEqual taskStatus + }) + } } - } - and("there is a single downstream stage") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = singleTaskStage.type - status = RUNNING + and("there is a single downstream stage") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = singleTaskStage.type + singleTaskStage.plan(this) + tasks.first().status = taskStatus + status = RUNNING + } + stage { + refId = "2" + requisiteStageRefIds = setOf("1") + type = singleTaskStage.type + } } - stage { - refId = "2" - requisiteStageRefIds = setOf("1") - type = singleTaskStage.type + val message = CompleteStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline } - } - val message = CompleteStage(pipeline.stageByRef("1"), stageStatus) - beforeGroup { - whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline - } + afterGroup(::resetMocks) - afterGroup(::resetMocks) + action("the handler receives a message") { + subject.handle(message) + } - action("the handler receives a message") { - subject.handle(message) - } + it("updates the stage state") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual taskStatus + it.getEndTime() shouldEqual clock.millis() + }) + } - it("updates the stage state") { - verify(repository).storeStage(check { - it.getStatus() shouldEqual stageStatus - it.getEndTime() shouldEqual clock.millis() - }) - } + it("runs the next stage") { + verify(queue).push(StartStage( + message.executionType, + message.executionId, + "foo", + pipeline.stages.last().id + )) + } - it("runs the next stage") { - verify(queue).push(StartStage( - message.executionType, - message.executionId, - "foo", - pipeline.stages.last().id - )) + it("does not run any tasks") { + verify(queue, never()).push(any()) + } } - it("does not run any tasks") { - verify(queue, never()).push(any()) - } - } + and("there are multiple downstream stages") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = singleTaskStage.type + singleTaskStage.plan(this) + tasks.first().status = taskStatus + status = RUNNING + } + stage { + refId = "2" + requisiteStageRefIds = setOf("1") + type = singleTaskStage.type + } + stage { + refId = "3" + requisiteStageRefIds = setOf("1") + type = singleTaskStage.type + } + } + val message = CompleteStage(pipeline.stageByRef("1")) - and("there are multiple downstream stages") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = singleTaskStage.type - status = RUNNING + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline } - stage { - refId = "2" - requisiteStageRefIds = setOf("1") - type = singleTaskStage.type + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) } - stage { - refId = "3" - requisiteStageRefIds = setOf("1") - type = singleTaskStage.type + + it("runs the next stages") { + argumentCaptor().apply { + verify(queue, times(2)).push(capture()) + allValues.map { it.stageId }.toSet() shouldEqual pipeline.stages[1..2].map { it.id }.toSet() + } } } - val message = CompleteStage(pipeline.stageByRef("1"), stageStatus) - beforeGroup { - whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline - } + and("there are parallel stages still running") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = singleTaskStage.type + singleTaskStage.plan(this) + tasks.first().status = taskStatus + status = RUNNING + } + stage { + refId = "2" + type = singleTaskStage.type + status = RUNNING + } + } + val message = CompleteStage(pipeline.stageByRef("1")) - afterGroup(::resetMocks) + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } - action("the handler receives a message") { - subject.handle(message) + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + it("still signals completion of the execution") { + verify(queue).push(CompleteExecution(pipeline)) + } } - it("runs the next stages") { - argumentCaptor().apply { - verify(queue, times(2)).push(capture()) - allValues.map { it.stageId }.toSet() shouldEqual pipeline.stages[1..2].map { it.id }.toSet() + setOf(CANCELED, TERMINAL, STOPPED).forEach { failureStatus -> + and("there are parallel stages that failed") { + val pipeline = pipeline { + application = "covfefe" + stage { + refId = "1" + type = singleTaskStage.type + singleTaskStage.plan(this) + tasks.first().status = SUCCEEDED + status = RUNNING + } + stage { + refId = "2" + type = singleTaskStage.type + status = failureStatus + } + } + val message = CompleteStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } + + afterGroup(::resetMocks) + + on("receiving $message") { + subject.handle(message) + } + + it("still signals completion of the execution") { + verify(queue).push(CompleteExecution(pipeline)) + } } } } + } - and("there are parallel stages still running") { + setOf(TERMINAL, CANCELED).forEach { taskStatus -> + describe("when a stage's task fails with $taskStatus status") { val pipeline = pipeline { application = "foo" stage { refId = "1" - type = singleTaskStage.type + type = multiTaskStage.type + multiTaskStage.plan(this) + tasks[0].status = SUCCEEDED + tasks[1].status = taskStatus + tasks[2].status = NOT_STARTED status = RUNNING } stage { refId = "2" + requisiteStageRefIds = listOf("1") type = singleTaskStage.type - status = RUNNING } } - val message = CompleteStage(pipeline.stageByRef("1"), stageStatus) + val message = CompleteStage(pipeline.stageByRef("1")) beforeGroup { whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline @@ -236,247 +319,283 @@ object CompleteStageHandlerTest : SubjectSpek({ afterGroup(::resetMocks) - action("the handler receives a message") { + on("receiving the message") { subject.handle(message) } - it("still signals completion of the execution") { - verify(queue).push(CompleteExecution(pipeline)) + it("updates the stage state") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual taskStatus + it.getEndTime() shouldEqual clock.millis() + }) + } + + it("does not run any downstream stages") { + verify(queue, never()).push(isA()) + } + + it("fails the execution") { + verify(queue).push(CompleteExecution( + message.executionType, + message.executionId, + "foo" + )) + } + + it("runs the stage's cancellation routine") { + verify(queue).push(CancelStage(message)) + } + + it("publishes an event") { + verify(publisher).publishEvent(check { + it.executionType shouldEqual pipeline.javaClass + it.executionId shouldEqual pipeline.id + it.stageId shouldEqual message.stageId + it.status shouldEqual taskStatus + }) } } + } - setOf(CANCELED, TERMINAL, STOPPED).forEach { failureStatus -> - and("there are parallel stages that failed") { + mapOf(STAGE_BEFORE to stageWithSyntheticBefore, STAGE_AFTER to stageWithSyntheticAfter).forEach { syntheticType, stageBuilder -> + setOf(TERMINAL, CANCELED, STOPPED).forEach { failureStatus -> + describe("when a $syntheticType synthetic stage completed with $failureStatus") { val pipeline = pipeline { - application = "covfefe" + application = "foo" stage { refId = "1" - type = singleTaskStage.type status = RUNNING - } - stage { - refId = "2" - type = singleTaskStage.type - status = failureStatus + type = stageBuilder.type + stageBuilder.buildSyntheticStages(this) + stageBuilder.plan(this) } } - val message = CompleteStage(pipeline.stageByRef("1"), stageStatus) + val message = CompleteStage(pipeline.stageByRef("1")) beforeGroup { + pipeline + .stages + .first { it.syntheticStageOwner == syntheticType } + .status = failureStatus whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline } afterGroup(::resetMocks) - on("receiving $message") { + on("receiving the message") { subject.handle(message) } - it("still signals completion of the execution") { - verify(queue).push(CompleteExecution(pipeline)) + it("updates the stage state") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual failureStatus + it.getEndTime() shouldEqual clock.millis() + }) } } } - } - } - setOf(TERMINAL, CANCELED).forEach { stageStatus -> - describe("when a stage fails with $stageStatus status") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = singleTaskStage.type - status = RUNNING - } - stage { - refId = "2" - requisiteStageRefIds = listOf("1") - type = singleTaskStage.type + describe("when any $syntheticType synthetic stage completed with FAILED_CONTINUE") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + status = RUNNING + type = stageBuilder.type + stageBuilder.buildSyntheticStages(this) + stageBuilder.plan(this) + } } - } - val message = CompleteStage(pipeline.stageByRef("1"), stageStatus) - - beforeGroup { - whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline - } + val message = CompleteStage(pipeline.stageByRef("1")) - afterGroup(::resetMocks) - - action("the handler receives a message") { - subject.handle(message) - } - - it("updates the stage state") { - verify(repository).storeStage(check { - it.getStatus() shouldEqual stageStatus - it.getEndTime() shouldEqual clock.millis() - }) - } - - it("does not run any downstream stages") { - verify(queue, never()).push(isA()) - } + beforeGroup { + pipeline + .stages + .first { it.syntheticStageOwner == syntheticType } + .status = FAILED_CONTINUE + pipeline.stageById(message.stageId).tasks.forEach { it.status = SUCCEEDED } + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } - it("fails the execution") { - verify(queue).push(CompleteExecution( - message.executionType, - message.executionId, - "foo" - )) - } + afterGroup(::resetMocks) - it("runs the stage's cancellation routine") { - verify(queue).push(CancelStage(message)) - } + on("receiving the message") { + subject.handle(message) + } - it("publishes an event") { - verify(publisher).publishEvent(check { - it.executionType shouldEqual pipeline.javaClass - it.executionId shouldEqual pipeline.id - it.stageId shouldEqual message.stageId - it.status shouldEqual stageStatus - }) + it("updates the stage state") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual FAILED_CONTINUE + it.getEndTime() shouldEqual clock.millis() + }) + } } } } - describe("synthetic stages") { - context("when a synthetic stage completes successfully") { - context("before a main stage") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = stageWithSyntheticAfter.type - stageWithSyntheticBefore.buildSyntheticStages(this) - stageWithSyntheticBefore.buildTasks(this) + describe("completing synthetic stages") { + listOf(SUCCEEDED, FAILED_CONTINUE).forEach { taskStatus -> + given("a synthetic stage's task completes with $taskStatus") { + and("it comes before its parent stage") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = stageWithSyntheticBefore.type + stageWithSyntheticBefore.buildSyntheticStages(this) + stageWithSyntheticBefore.buildTasks(this) + } } - } - context("there are more after stages") { - val message = CompleteStage(pipeline.stageByRef("1<1"), SUCCEEDED) + and("there are more before stages") { + val message = CompleteStage(pipeline.stageByRef("1<1")) - beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } + beforeGroup { + pipeline.stageById(message.stageId).apply { + status = RUNNING + singleTaskStage.plan(this) + tasks.first().status = taskStatus + } - afterGroup(::resetMocks) + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } - action("the handler receives a message") { - subject.handle(message) - } + afterGroup(::resetMocks) - it("runs the next synthetic stage") { - verify(queue).push(StartStage( - pipeline.stageByRef("1<2") - )) + on("receiving the message") { + subject.handle(message) + } + + it("runs the next synthetic stage") { + verify(queue).push(StartStage( + pipeline.stageByRef("1<2") + )) + } } - } - context("it is the last after stage") { - val message = CompleteStage(pipeline.stageByRef("1<2"), SUCCEEDED) + and("it is the last before stage") { + val message = CompleteStage(pipeline.stageByRef("1<2")) - beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } + beforeGroup { + pipeline.stageById(message.stageId).apply { + status = RUNNING + singleTaskStage.plan(this) + tasks.first().status = taskStatus + } - afterGroup(::resetMocks) + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } - action("the handler receives a message") { - subject.handle(message) - } + afterGroup(::resetMocks) - it("signals the parent stage to run") { - verify(queue).push(ContinueParentStage( - pipeline.stageByRef("1") - )) + on("receiving the message") { + subject.handle(message) + } + + it("signals the parent stage to run") { + verify(queue).push(ContinueParentStage( + pipeline.stageByRef("1") + )) + } } } - } - context("after the main stage") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = stageWithSyntheticAfter.type - stageWithSyntheticAfter.buildSyntheticStages(this) - stageWithSyntheticAfter.buildTasks(this) + and("it comes after its parent stage") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = stageWithSyntheticAfter.type + stageWithSyntheticAfter.buildSyntheticStages(this) + stageWithSyntheticAfter.buildTasks(this) + } } - } - context("there are more after stages") { - val message = CompleteStage(pipeline.stageByRef("1>1"), SUCCEEDED) + and("there are more after stages") { + val message = CompleteStage(pipeline.stageByRef("1>1")) - beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } + beforeGroup { + pipeline.stageById(message.stageId).apply { + status = RUNNING + singleTaskStage.plan(this) + tasks.first().status = taskStatus + } - afterGroup(::resetMocks) + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } - action("the handler receives a message") { - subject.handle(message) - } + afterGroup(::resetMocks) - it("runs the next synthetic stage") { - verify(queue).push(StartStage( - message.executionType, - message.executionId, - "foo", - pipeline.stages.last().id - )) + on("receiving the message") { + subject.handle(message) + } + + it("runs the next synthetic stage") { + verify(queue).push(StartStage( + message.executionType, + message.executionId, + "foo", + pipeline.stages.last().id + )) + } } - } - context("it is the last after stage") { - val message = CompleteStage(pipeline.stageByRef("1>2"), SUCCEEDED) + and("it is the last after stage") { + val message = CompleteStage(pipeline.stageByRef("1>2")) - beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } + beforeGroup { + pipeline.stageById(message.stageId).apply { + status = RUNNING + singleTaskStage.plan(this) + tasks.first().status = taskStatus + } - afterGroup(::resetMocks) + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } - action("the handler receives a message") { - subject.handle(message) - } + afterGroup(::resetMocks) - it("signals the completion of the parent stage") { - verify(queue).push(CompleteStage( - message.executionType, - message.executionId, - "foo", - pipeline.stages.first().id, - SUCCEEDED - )) + on("receiving the message") { + subject.handle(message) + } + + it("signals the completion of the parent stage") { + verify(queue).push(CompleteStage( + message.executionType, + message.executionId, + "foo", + pipeline.stages.first().id + )) + } } } } } - setOf(TERMINAL, CANCELED).forEach { status -> - context("when a synthetic stage ends with $status status") { + setOf(TERMINAL, CANCELED).forEach { taskStatus -> + given("a synthetic stage's task ends with $taskStatus status") { val pipeline = pipeline { application = "foo" stage { refId = "1" type = stageWithSyntheticBefore.type stageWithSyntheticBefore.buildSyntheticStages(this) + stageWithSyntheticBefore.plan(this) } } - val message = CompleteStage(pipeline.stageByRef("1<1"), status) + val message = CompleteStage(pipeline.stageByRef("1<1")) beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING + pipeline.stageById(message.stageId).apply { + status = RUNNING + singleTaskStage.plan(this) + tasks.first().status = taskStatus + } + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline } - action("the handler receives a message") { + on("receiving the message") { subject.handle(message) } @@ -494,64 +613,66 @@ object CompleteStageHandlerTest : SubjectSpek({ } describe("branching stages") { - context("when one branch completes") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - name = "parallel" - type = stageWithParallelBranches.type - stageWithParallelBranches.buildSyntheticStages(this) - stageWithParallelBranches.buildTasks(this) + listOf(SUCCEEDED, FAILED_CONTINUE).forEach { status -> + context("when one branch completes with $status") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + name = "parallel" + type = stageWithParallelBranches.type + stageWithParallelBranches.buildSyntheticStages(this) + stageWithParallelBranches.buildTasks(this) + } } - } - val message = CompleteStage(pipeline.stageByRef("1=1"), SUCCEEDED) + val message = CompleteStage(pipeline.stageByRef("1=1")) - beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } + beforeGroup { + pipeline.stageById(message.stageId).status = RUNNING + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } - afterGroup(::resetMocks) + afterGroup(::resetMocks) - action("the handler receives a message") { - subject.handle(message) - } + on("receiving the message") { + subject.handle(message) + } - it("signals the parent stage to try to run") { - verify(queue).push(ContinueParentStage(pipeline.stageByRef("1"))) + it("signals the parent stage to try to run") { + verify(queue).push(ContinueParentStage(pipeline.stageByRef("1"))) + } } - } - context("when all branches are complete") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - name = "parallel" - type = stageWithParallelBranches.type - stageWithParallelBranches.buildSyntheticStages(this) - stageWithParallelBranches.buildTasks(this) + context("when all branches are complete") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + name = "parallel" + type = stageWithParallelBranches.type + stageWithParallelBranches.buildSyntheticStages(this) + stageWithParallelBranches.buildTasks(this) + } } - } - val message = CompleteStage(pipeline.stageByRef("1=1"), SUCCEEDED) + val message = CompleteStage(pipeline.stageByRef("1=1")) - beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING - pipeline.stageByRef("1=2").status = SUCCEEDED - pipeline.stageByRef("1=3").status = SUCCEEDED + beforeGroup { + pipeline.stageById(message.stageId).status = RUNNING + pipeline.stageByRef("1=2").status = SUCCEEDED + pipeline.stageByRef("1=3").status = SUCCEEDED - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } - afterGroup(::resetMocks) + afterGroup(::resetMocks) - action("the handler receives a message") { - subject.handle(message) - } + on("receiving the message") { + subject.handle(message) + } - it("signals the parent stage to try to run") { - verify(queue).push(ContinueParentStage(pipeline.stageByRef("1"))) + it("signals the parent stage to try to run") { + verify(queue).push(ContinueParentStage(pipeline.stageByRef("1"))) + } } } } @@ -562,7 +683,7 @@ object CompleteStageHandlerTest : SubjectSpek({ ((it.getContext()["exception"] as Map<*, *>)["details"] as Map<*, *>)["errors"] as List<*> } - context("exception in stage context") { + given("an exception in the stage context") { val expressionError = "Expression foo failed for field bar" val existingException = "Existing error" val pipeline = pipeline { @@ -574,19 +695,22 @@ object CompleteStageHandlerTest : SubjectSpek({ "exception" to mapOf("details" to mapOf("errors" to mutableListOf(existingException))), PipelineExpressionEvaluator.SUMMARY to mapOf("failedExpression" to listOf(mapOf("description" to expressionError, "level" to "ERROR"))) ) + status = RUNNING + type = singleTaskStage.type + singleTaskStage.plan(this) + tasks.first().status = SUCCEEDED } } - val message = CompleteStage(pipeline.stageByRef("1"), SUCCEEDED) + val message = CompleteStage(pipeline.stageByRef("1")) beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline } afterGroup(::resetMocks) - action("the handler receives a message") { + on("receiving the message") { subject.handle(message) } @@ -598,7 +722,7 @@ object CompleteStageHandlerTest : SubjectSpek({ } } - context("no other exception errors in stage context") { + given("no other exception errors in the stage context") { val expressionError = "Expression foo failed for field bar" val pipeline = pipeline { application = "foo" @@ -608,19 +732,22 @@ object CompleteStageHandlerTest : SubjectSpek({ context = mutableMapOf( PipelineExpressionEvaluator.SUMMARY to mapOf("failedExpression" to listOf(mapOf("description" to expressionError, "level" to "ERROR"))) ) as Map + status = RUNNING + type = singleTaskStage.type + singleTaskStage.plan(this) + tasks.first().status = SUCCEEDED } } - val message = CompleteStage(pipeline.stageByRef("1"), SUCCEEDED) + val message = CompleteStage(pipeline.stageByRef("1")) beforeGroup { - pipeline.stageById(message.stageId).status = RUNNING whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline } afterGroup(::resetMocks) - action("the handler receives a message") { + on("receiving the message") { subject.handle(message) } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteTaskHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteTaskHandlerTest.kt index 9f06dde7e0..ce4a1b7948 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteTaskHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/CompleteTaskHandlerTest.kt @@ -136,8 +136,7 @@ object CompleteTaskHandlerTest : SubjectSpek({ message.executionType, message.executionId, "foo", - message.stageId, - SUCCEEDED + message.stageId )) } } @@ -267,8 +266,7 @@ object CompleteTaskHandlerTest : SubjectSpek({ message.executionType, message.executionId, "foo", - message.stageId, - status + message.stageId )) } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/ContinueParentStageHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/ContinueParentStageHandlerTest.kt index d1faba0eb1..792528ea6c 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/ContinueParentStageHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/ContinueParentStageHandlerTest.kt @@ -41,86 +41,54 @@ object ContinueParentStageHandlerTest : SubjectSpek( fun resetMocks() = reset(queue, repository) - describe("running a parent stage after its pre-stages complete") { - given("other pre-stages are not yet complete") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = stageWithSyntheticBefore.type - stageWithSyntheticBefore.buildSyntheticStages(this) - stageWithSyntheticBefore.buildTasks(this) + listOf(SUCCEEDED, FAILED_CONTINUE).forEach { status -> + describe("running a parent stage after its pre-stages complete with $status") { + given("other pre-stages are not yet complete") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = stageWithSyntheticBefore.type + stageWithSyntheticBefore.buildSyntheticStages(this) + stageWithSyntheticBefore.buildTasks(this) + } } - } - - val message = ContinueParentStage(pipeline.stageByRef("1")) - - beforeGroup { - pipeline.stageByRef("1<1").status = SUCCEEDED - pipeline.stageByRef("1<2").status = RUNNING - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } - afterGroup(::resetMocks) + val message = ContinueParentStage(pipeline.stageByRef("1")) - on("receiving $message") { - subject.handle(message) - } - - it("re-queues the message for later evaluation") { - verify(queue).push(message, retryDelay) - } - } - - given("another pre-stage failed") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = stageWithSyntheticBefore.type - stageWithSyntheticBefore.buildSyntheticStages(this) - stageWithSyntheticBefore.buildTasks(this) + beforeGroup { + pipeline.stageByRef("1<1").status = status + pipeline.stageByRef("1<2").status = RUNNING + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline } - } - - val message = ContinueParentStage(pipeline.stageByRef("1")) - - beforeGroup { - pipeline.stageByRef("1<1").status = SUCCEEDED - pipeline.stageByRef("1<2").status = TERMINAL - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } - - afterGroup(::resetMocks) - on("receiving $message") { - subject.handle(message) - } + afterGroup(::resetMocks) - it("does not re-queue the message") { - verifyZeroInteractions(queue) - } - } + on("receiving $message") { + subject.handle(message) + } - given("the parent stage has tasks") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = stageWithSyntheticBefore.type - stageWithSyntheticBefore.buildSyntheticStages(this) - stageWithSyntheticBefore.buildTasks(this) + it("re-queues the message for later evaluation") { + verify(queue).push(message, retryDelay) } } - val message = ContinueParentStage(pipeline.stageByRef("1")) + given("another pre-stage failed") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = stageWithSyntheticBefore.type + stageWithSyntheticBefore.buildSyntheticStages(this) + stageWithSyntheticBefore.buildTasks(this) + } + } - beforeGroup { - pipeline.stageByRef("1").beforeStages().forEach { it.setStatus(SUCCEEDED) } - } + val message = ContinueParentStage(pipeline.stageByRef("1")) - and("they have not started yet") { beforeGroup { + pipeline.stageByRef("1<1").status = status + pipeline.stageByRef("1<2").status = TERMINAL whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline } @@ -130,55 +98,89 @@ object ContinueParentStageHandlerTest : SubjectSpek( subject.handle(message) } - it("runs the parent stage's first task") { - verify(queue).push(StartTask(pipeline.stageByRef("1"), "1")) + it("does not re-queue the message") { + verifyZeroInteractions(queue) } } - and("they have already started") { + given("the parent stage has tasks") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = stageWithSyntheticBefore.type + stageWithSyntheticBefore.buildSyntheticStages(this) + stageWithSyntheticBefore.buildTasks(this) + } + } + + val message = ContinueParentStage(pipeline.stageByRef("1")) + beforeGroup { - pipeline.stageByRef("1").tasks.first().status = RUNNING - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + pipeline.stageByRef("1").beforeStages().forEach { it.setStatus(status) } } - afterGroup(::resetMocks) + and("they have not started yet") { + beforeGroup { + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } - on("receiving $message") { - subject.handle(message) + afterGroup(::resetMocks) + + on("receiving $message") { + subject.handle(message) + } + + it("runs the parent stage's first task") { + verify(queue).push(StartTask(pipeline.stageByRef("1"), "1")) + } } - it("ignores the message") { - verifyZeroInteractions(queue) + and("they have already started") { + beforeGroup { + pipeline.stageByRef("1").tasks.first().status = RUNNING + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } + + afterGroup(::resetMocks) + + on("receiving $message") { + subject.handle(message) + } + + it("ignores the message") { + verifyZeroInteractions(queue) + } } } - } - given("the parent stage has no tasks") { - val pipeline = pipeline { - application = "foo" - stage { - refId = "1" - type = stageWithSyntheticBeforeAndNoTasks.type - stageWithSyntheticBeforeAndNoTasks.buildSyntheticStages(this) - stageWithSyntheticBeforeAndNoTasks.buildTasks(this) + given("the parent stage has no tasks") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = stageWithSyntheticBeforeAndNoTasks.type + stageWithSyntheticBeforeAndNoTasks.buildSyntheticStages(this) + stageWithSyntheticBeforeAndNoTasks.buildTasks(this) + } } - } - val message = ContinueParentStage(pipeline.stageByRef("1")) + val message = ContinueParentStage(pipeline.stageByRef("1")) - beforeGroup { - pipeline.stageByRef("1").beforeStages().forEach { it.setStatus(SUCCEEDED) } - whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline - } + beforeGroup { + pipeline.stageByRef("1").beforeStages().forEach { it.setStatus(status) } + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } - afterGroup(::resetMocks) + afterGroup(::resetMocks) - on("receiving $message") { - subject.handle(message) - } + on("receiving $message") { + subject.handle(message) + } - it("completes the stage") { - verify(queue).push(CompleteStage(pipeline.stageByRef("1"), SUCCEEDED)) + it("completes the stage with $status") { + verify(queue).push(CompleteStage(pipeline.stageByRef("1"))) + } } } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/DeadMessageHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/DeadMessageHandlerTest.kt index 9a9c93f7ce..33cb5bec26 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/DeadMessageHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/DeadMessageHandlerTest.kt @@ -22,8 +22,10 @@ import com.netflix.spinnaker.orca.q.* import com.nhaarman.mockito_kotlin.mock import com.nhaarman.mockito_kotlin.reset import com.nhaarman.mockito_kotlin.verify +import com.nhaarman.mockito_kotlin.verifyZeroInteractions import org.jetbrains.spek.api.dsl.describe import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on import org.jetbrains.spek.api.lifecycle.CachingMode.GROUP import org.jetbrains.spek.subject.SubjectSpek @@ -42,7 +44,7 @@ object DeadMessageHandlerTest : SubjectSpek({ afterGroup(::resetMocks) - action("the handler receives a message") { + on("receiving a message") { subject.handle(queue, message) } @@ -56,12 +58,12 @@ object DeadMessageHandlerTest : SubjectSpek({ afterGroup(::resetMocks) - action("the handler receives a message") { + on("receiving a message") { subject.handle(queue, message) } - it("terminates the stage") { - verify(queue).push(CompleteStage(message, TERMINAL)) + it("aborts the stage") { + verify(queue).push(AbortStage(message)) } } @@ -70,7 +72,7 @@ object DeadMessageHandlerTest : SubjectSpek({ afterGroup(::resetMocks) - action("the handler receives a message") { + on("receiving a message") { subject.handle(queue, message) } @@ -78,4 +80,20 @@ object DeadMessageHandlerTest : SubjectSpek({ verify(queue).push(CompleteTask(message, TERMINAL)) } } + + describe("handling a message that was previously dead-lettered") { + val message = CompleteExecution(Pipeline::class.java, "1", "spinnaker").apply { + setAttribute(DeadMessageAttribute) + } + + afterGroup(::resetMocks) + + on("receiving a message") { + subject.handle(queue, message) + } + + it("does nothing") { + verifyZeroInteractions(queue) + } + } }) diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RescheduleExecutionHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RescheduleExecutionHandlerTest.kt new file mode 100644 index 0000000000..ef7b7d43cd --- /dev/null +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RescheduleExecutionHandlerTest.kt @@ -0,0 +1,97 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.q.handler + +import com.netflix.spinnaker.orca.ExecutionStatus +import com.netflix.spinnaker.orca.Task +import com.netflix.spinnaker.orca.pipeline.model.Pipeline +import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository +import com.netflix.spinnaker.orca.q.* +import com.nhaarman.mockito_kotlin.* +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.lifecycle.CachingMode +import org.jetbrains.spek.subject.SubjectSpek + +object RescheduleExecutionHandlerTest : SubjectSpek({ + + val queue: Queue = mock() + val repository: ExecutionRepository = mock() + + subject(CachingMode.GROUP) { + RescheduleExecutionHandler(queue, repository) + } + + fun resetMocks() = reset(queue, repository) + + describe("reschedule an execution") { + val pipeline = pipeline { + application = "spinnaker" + status = ExecutionStatus.RUNNING + stage { + refId = "1" + status = ExecutionStatus.SUCCEEDED + } + stage { + refId = "2a" + requisiteStageRefIds = listOf("1") + status = ExecutionStatus.RUNNING + task { + id = "4" + status = ExecutionStatus.RUNNING + } + } + stage { + refId = "2b" + requisiteStageRefIds = listOf("1") + status = ExecutionStatus.RUNNING + task { + id = "5" + status = ExecutionStatus.RUNNING + } + } + stage { + refId = "3" + requisiteStageRefIds = listOf("2a", "2b") + status = ExecutionStatus.NOT_STARTED + } + } + val message = RescheduleExecution(Pipeline::class.java, pipeline.id, pipeline.application) + + beforeGroup { + whenever(repository.retrievePipeline(pipeline.id)) doReturn pipeline + } + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + @Suppress("UNCHECKED_CAST") + it("it updates the time for each running task") { + val stage2a = pipeline.stageByRef("2a") + val stage2b = pipeline.stageByRef("2b") + val task4 = stage2a.taskById("4") + val task5 = stage2b.taskById("5") + + verify(queue).reschedule(RunTask(message, stage2a.id, task4.id, Class.forName(task4.implementingClass) as Class)) + verify(queue).reschedule(RunTask(message, stage2b.id, task5.id, Class.forName(task5.implementingClass) as Class)) + verifyNoMoreInteractions(queue) + } + } +}) diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RestartStageHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RestartStageHandlerTest.kt index a4e545516f..785471802f 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RestartStageHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RestartStageHandlerTest.kt @@ -21,6 +21,7 @@ import com.natpryce.hamkrest.should.shouldMatch import com.natpryce.hamkrest.should.shouldNotMatch import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.ExecutionStatus.* +import com.netflix.spinnaker.orca.pipeline.DefaultStageDefinitionBuilderFactory import com.netflix.spinnaker.orca.pipeline.StageDefinitionBuilder import com.netflix.spinnaker.orca.pipeline.model.Execution import com.netflix.spinnaker.orca.pipeline.model.Pipeline @@ -49,7 +50,7 @@ object RestartStageHandlerTest : SubjectSpek({ RestartStageHandler( queue, repository, - listOf( + DefaultStageDefinitionBuilderFactory( singleTaskStage, stageWithSyntheticBefore, stageWithNestedSynthetics diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RunTaskHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RunTaskHandlerTest.kt index 748b586103..e6c5fd203c 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RunTaskHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/RunTaskHandlerTest.kt @@ -49,6 +49,7 @@ object RunTaskHandlerTest : SubjectSpek({ val repository: ExecutionRepository = mock() val stageNavigator: StageNavigator = mock() val task: DummyTask = mock() + val timeoutOverrideTask: DummyTimeoutOverrideTask = mock() val exceptionHandler: ExceptionHandler = mock() val clock = fixedClock() val contextParameterProcessor = ContextParameterProcessor() @@ -59,14 +60,14 @@ object RunTaskHandlerTest : SubjectSpek({ repository, stageNavigator, contextParameterProcessor, - listOf(task), + listOf(task, timeoutOverrideTask), clock, listOf(exceptionHandler), NoopRegistry() ) } - fun resetMocks() = reset(queue, repository, task, exceptionHandler) + fun resetMocks() = reset(queue, repository, task, timeoutOverrideTask, exceptionHandler) describe("running a task") { @@ -742,20 +743,22 @@ object RunTaskHandlerTest : SubjectSpek({ val timeoutOverride = Duration.ofMinutes(10) timeoutOverride.toMillis().let { listOf(it.toInt(), it, it.toDouble()) }.forEach { stageTimeoutMs -> - and("the override is a ${stageTimeoutMs.javaClass.simpleName}") {and("the task is between the default and overridden duration") { - val pipeline = pipeline { - stage { - type = "whatever" - context["stageTimeoutMs"] = stageTimeoutMs - task { - id = "1" - - status = RUNNING - startTime = clock.instant().minusMillis(timeout.toMillis() + 1).toEpochMilli() + and("the override is a ${stageTimeoutMs.javaClass.simpleName}") { + and("the stage is between the default and overridden duration") { + val pipeline = pipeline { + stage { + type = "whatever" + context["stageTimeoutMs"] = stageTimeoutMs + startTime = clock.instant().minusMillis(timeout.toMillis() + 1).toEpochMilli() + task { + id = "1" + + status = RUNNING + startTime = clock.instant().minusMillis(timeout.toMillis() - 1).toEpochMilli() + } + } } - } - } - val message = RunTask(Pipeline::class.java, pipeline.id, "foo", pipeline.stages.first().id, "1", DummyTask::class.java) + val message = RunTask(Pipeline::class.java, pipeline.id, "foo", pipeline.stages.first().id, "1", DummyTask::class.java) beforeGroup { whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline @@ -773,24 +776,176 @@ object RunTaskHandlerTest : SubjectSpek({ } } - and("the timeout override has been exceeded") { - val pipeline = pipeline { - stage { - type = "whatever" - context["stageTimeoutMs"] = stageTimeoutMs - task { - id = "1" + and("the timeout override has been exceeded by stage") { + and("the stage has never been paused") { + val pipeline = pipeline { + stage { + type = "whatever" + context["stageTimeoutMs"] = stageTimeoutMs + startTime = clock.instant().minusMillis(timeoutOverride.toMillis() + 1).toEpochMilli() + task { + id = "1" + + status = RUNNING + startTime = clock.instant().minusMillis(timeout.toMillis() - 1).toEpochMilli() + } + } + } + val message = RunTask(Pipeline::class.java, pipeline.id, "foo", pipeline.stages.first().id, "1", DummyTask::class.java) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + whenever(task.timeout) doReturn timeout.toMillis() + } + + afterGroup(::resetMocks) + + on("receiving $message") { + subject.handle(message) + } + + it("fails the task") { + verify(queue).push(CompleteTask(message, TERMINAL)) + } + + it("does not execute the task") { + verify(task, never()).execute(any()) + } + } - status = RUNNING - startTime = clock.instant().minusMillis(timeoutOverride.toMillis() + 1).toEpochMilli() + and("the execution had been paused") { + val pipeline = pipeline { + paused = PausedDetails().apply { + pauseTime = clock.instant().minus(Minutes.of(3)).toEpochMilli() + resumeTime = clock.instant().minus(Minutes.of(2)).toEpochMilli() + } + stage { + type = "whatever" + context["stageTimeoutMs"] = stageTimeoutMs + startTime = clock.instant().minusMillis(timeoutOverride.toMillis() + 1).toEpochMilli() + task { + id = "1" + status = RUNNING + startTime = clock.instant().minusMillis(timeout.toMillis() - 1).toEpochMilli() + } + } + } + val message = RunTask(Pipeline::class.java, pipeline.id, "foo", pipeline.stages.first().id, "1", DummyTask::class.java) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + whenever(task.timeout) doReturn timeout.toMillis() + } + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + it("executes the task") { + verify(task).execute(any()) + } + } + + and("the execution had been paused but is timed out anyway") { + val pipeline = pipeline { + paused = PausedDetails().apply { + pauseTime = clock.instant().minus(Minutes.of(3)).toEpochMilli() + resumeTime = clock.instant().minus(Minutes.of(2)).toEpochMilli() + } + stage { + type = "whatever" + context["stageTimeoutMs"] = stageTimeoutMs + startTime = clock.instant().minusMillis(timeoutOverride.plusMinutes(1).toMillis() + 1).toEpochMilli() + task { + id = "1" + status = RUNNING + startTime = clock.instant().minusMillis(timeout.toMillis() - 1).toEpochMilli() + } + } + } + val message = RunTask(Pipeline::class.java, pipeline.id, "foo", pipeline.stages.first().id, "1", DummyTask::class.java) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + whenever(task.timeout) doReturn timeout.toMillis() + } + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + it("fails the task") { + verify(queue).push(CompleteTask(message, TERMINAL)) + } + + it("does not execute the task") { + verify(task, never()).execute(any()) + } + } + + and("the execution had been paused but only before this stage started running") { + val pipeline = pipeline { + paused = PausedDetails().apply { + pauseTime = clock.instant().minus(Minutes.of(15)).toEpochMilli() + resumeTime = clock.instant().minus(Minutes.of(14)).toEpochMilli() + } + stage { + type = "whatever" + context["stageTimeoutMs"] = stageTimeoutMs + startTime = clock.instant().minusMillis(timeoutOverride.toMillis() + 1).toEpochMilli() + task { + id = "1" + status = RUNNING + startTime = clock.instant().minusMillis(timeout.toMillis() - 1).toEpochMilli() + } + } + } + val message = RunTask(Pipeline::class.java, pipeline.id, "foo", pipeline.stages.first().id, "1", DummyTask::class.java) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + whenever(task.timeout) doReturn timeout.toMillis() + } + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + it("fails the task") { + verify(queue).push(CompleteTask(message, TERMINAL)) + } + + it("does not execute the task") { + verify(task, never()).execute(any()) + } } } - } - val message = RunTask(Pipeline::class.java, pipeline.id, "foo", pipeline.stages.first().id, "1", DummyTask::class.java) + + and("the task is an overridabletimeout task that shouldn't time out") { + val pipeline = pipeline { + stage { + type = "whatever" + context["stageTimeoutMs"] = stageTimeoutMs + startTime = clock.instant().minusMillis(timeout.toMillis() + 1).toEpochMilli() //started 5.1 minutes ago + task { + id = "1" + implementingClass = DummyTimeoutOverrideTask::class.jvmName + status = RUNNING + startTime = clock.instant().minusMillis(timeout.toMillis() + 1).toEpochMilli() //started 5.1 minutes ago + } + } + } + val message = RunTask(Pipeline::class.java, pipeline.id, "foo", pipeline.stages.first().id, "1", DummyTimeoutOverrideTask::class.java) beforeGroup { whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline - whenever(task.timeout) doReturn timeout.toMillis() + whenever(timeoutOverrideTask.timeout) doReturn timeout.toMillis() } afterGroup(::resetMocks) @@ -799,12 +954,8 @@ object RunTaskHandlerTest : SubjectSpek({ subject.handle(message) } - it("fails the task") { - verify(queue).push(CompleteTask(message, TERMINAL)) - } - - it("does not execute the task") { - verify(task, never()).execute(any()) + it("executes the task") { + verify(timeoutOverrideTask).execute(any()) } } } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/SkipStageHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/SkipStageHandlerTest.kt new file mode 100644 index 0000000000..3f1e742502 --- /dev/null +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/SkipStageHandlerTest.kt @@ -0,0 +1,240 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License") + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.spinnaker.orca.q.handler + +import com.netflix.spinnaker.orca.ExecutionStatus.* +import com.netflix.spinnaker.orca.events.StageComplete +import com.netflix.spinnaker.orca.pipeline.persistence.ExecutionRepository +import com.netflix.spinnaker.orca.q.* +import com.netflix.spinnaker.orca.time.fixedClock +import com.netflix.spinnaker.spek.shouldEqual +import com.nhaarman.mockito_kotlin.* +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.jetbrains.spek.api.lifecycle.CachingMode.GROUP +import org.jetbrains.spek.subject.SubjectSpek +import org.springframework.context.ApplicationEventPublisher + +object SkipStageHandlerTest : SubjectSpek({ + + val queue: Queue = mock() + val repository: ExecutionRepository = mock() + val publisher: ApplicationEventPublisher = mock() + val clock = fixedClock() + + subject(GROUP) { + SkipStageHandler(queue, repository, publisher, clock) + } + + fun resetMocks() = reset(queue, repository, publisher) + + describe("skipping a stage") { + given("it is already complete") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = "whatever" + status = SUCCEEDED + endTime = clock.instant().minusSeconds(2).toEpochMilli() + } + } + val message = SkipStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } + + afterGroup(::resetMocks) + + on("receiving $message") { + subject.handle(message) + } + + it("ignores the message") { + verify(repository, never()).storeStage(any()) + verifyZeroInteractions(queue) + verifyZeroInteractions(publisher) + } + } + + given("it is the last stage") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = "whatever" + status = RUNNING + } + } + val message = SkipStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + it("updates the stage state") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual SKIPPED + it.getEndTime() shouldEqual clock.millis() + }) + } + + it("completes the execution") { + verify(queue).push(CompleteExecution(pipeline)) + } + + it("does not emit any commands") { + verify(queue, never()).push(any()) + } + + it("publishes an event") { + verify(publisher).publishEvent(check { + it.executionType shouldEqual pipeline.javaClass + it.executionId shouldEqual pipeline.id + it.stageId shouldEqual message.stageId + it.status shouldEqual SKIPPED + }) + } + } + + given("there is a single downstream stage") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = "whatever" + status = RUNNING + } + stage { + refId = "2" + requisiteStageRefIds = setOf("1") + type = "whatever" + } + } + val message = SkipStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + it("updates the stage state") { + verify(repository).storeStage(check { + it.getStatus() shouldEqual SKIPPED + it.getEndTime() shouldEqual clock.millis() + }) + } + + it("runs the next stage") { + verify(queue).push(StartStage( + message.executionType, + message.executionId, + "foo", + pipeline.stages.last().id + )) + } + + it("does not run any tasks") { + verify(queue, never()).push(any()) + } + } + + given("there are multiple downstream stages") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = "whatever" + status = RUNNING + } + stage { + refId = "2" + requisiteStageRefIds = setOf("1") + type = "whatever" + } + stage { + refId = "3" + requisiteStageRefIds = setOf("1") + type = "whatever" + } + } + val message = SkipStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + it("runs the next stages") { + argumentCaptor().apply { + verify(queue, times(2)).push(capture()) + allValues.map { it.stageId }.toSet() shouldEqual pipeline.stages[1..2].map { it.id }.toSet() + } + } + } + + given("there are parallel stages still running") { + val pipeline = pipeline { + application = "foo" + stage { + refId = "1" + type = "whatever" + status = RUNNING + } + stage { + refId = "2" + type = "whatever" + status = RUNNING + } + } + val message = SkipStage(pipeline.stageByRef("1")) + + beforeGroup { + whenever(repository.retrievePipeline(message.executionId)) doReturn pipeline + } + + afterGroup(::resetMocks) + + action("the handler receives a message") { + subject.handle(message) + } + + it("still signals completion of the execution") { + verify(queue).push(CompleteExecution(pipeline)) + } + } + } +}) diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/StartStageHandlerTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/StartStageHandlerTest.kt index 3254db0d8e..31a3d07cab 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/StartStageHandlerTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/handler/StartStageHandlerTest.kt @@ -17,7 +17,6 @@ package com.netflix.spinnaker.orca.q.handler import com.fasterxml.jackson.databind.ObjectMapper -import com.natpryce.hamkrest.allElements import com.natpryce.hamkrest.assertion.assertThat import com.natpryce.hamkrest.equalTo import com.natpryce.hamkrest.hasElement @@ -26,6 +25,7 @@ import com.natpryce.hamkrest.should.shouldMatch import com.netflix.spinnaker.orca.ExecutionStatus.* import com.netflix.spinnaker.orca.events.StageStarted import com.netflix.spinnaker.orca.exceptions.ExceptionHandler +import com.netflix.spinnaker.orca.pipeline.DefaultStageDefinitionBuilderFactory import com.netflix.spinnaker.orca.pipeline.RestrictExecutionDuringTimeWindow import com.netflix.spinnaker.orca.pipeline.model.Pipeline import com.netflix.spinnaker.orca.pipeline.model.Stage @@ -39,6 +39,7 @@ import com.netflix.spinnaker.orca.pipeline.util.StageNavigator import com.netflix.spinnaker.orca.q.* import com.netflix.spinnaker.orca.time.fixedClock import com.netflix.spinnaker.spek.and +import com.netflix.spinnaker.spek.shouldAllEqual import com.netflix.spinnaker.spek.shouldEqual import com.nhaarman.mockito_kotlin.* import org.jetbrains.spek.api.dsl.* @@ -64,7 +65,7 @@ object StartStageHandlerTest : SubjectSpek({ queue, repository, stageNavigator, - listOf( + DefaultStageDefinitionBuilderFactory( singleTaskStage, multiTaskStage, stageWithSyntheticBefore, @@ -169,7 +170,7 @@ object StartStageHandlerTest : SubjectSpek({ } it("immediately completes the stage") { - verify(queue).push(CompleteStage(message, SUCCEEDED)) + verify(queue).push(CompleteStage(message)) verifyNoMoreInteractions(queue) } @@ -544,6 +545,7 @@ object StartStageHandlerTest : SubjectSpek({ allValues[1..3].forEach { it.requisiteStageRefIds shouldEqual setOf(firstValue.refId) } + allValues[1..3].map { it.type } shouldAllEqual singleTaskStage.type } } @@ -629,10 +631,8 @@ object StartStageHandlerTest : SubjectSpek({ subject.handle(message) } - it("marks the stage as terminal") { - verify(queue).push(check { - it.status shouldEqual TERMINAL - }) + it("completes the stage") { + verify(queue).push(isA()) } it("attaches the exception to the stage context") { @@ -641,6 +641,7 @@ object StartStageHandlerTest : SubjectSpek({ }) } } + and("only the branch should fail") { beforeGroup { pipeline.stageByRef("1").apply { @@ -659,10 +660,8 @@ object StartStageHandlerTest : SubjectSpek({ subject.handle(message) } - it("marks the stage as stopped") { - verify(queue).push(check { - it.status shouldEqual STOPPED - }) + it("completes the stage") { + verify(queue).push(isA()) } it("attaches the exception to the stage context") { @@ -671,6 +670,7 @@ object StartStageHandlerTest : SubjectSpek({ }) } } + and("the branch should be allowed to continue") { beforeGroup { pipeline.stageByRef("1").apply { @@ -689,10 +689,8 @@ object StartStageHandlerTest : SubjectSpek({ subject.handle(message) } - it("marks the stage as FAILED_CONTINUE") { - verify(queue).push(check { - it.status shouldEqual FAILED_CONTINUE - }) + it("completes the stage") { + verify(queue).push(isA()) } it("attaches the exception to the stage context") { @@ -761,13 +759,16 @@ object StartStageHandlerTest : SubjectSpek({ pipeline.stages.size shouldEqual 4 assertThat( pipeline.stages.map { it.type }, - allElements(equalTo(stageWithParallelBranches.type)) + equalTo(listOf(singleTaskStage.type, singleTaskStage.type, singleTaskStage.type, stageWithParallelBranches.type)) ) - // TODO: contexts, etc. } - it("renames the primary branch") { - pipeline.stageByRef("1").name shouldEqual "is parallel" + it("builds stages that will run in parallel") { + assertThat( + pipeline.stages.flatMap { it.requisiteStageRefIds }, + isEmpty + ) + // TODO: contexts, etc. } it("renames each parallel branch") { @@ -808,7 +809,7 @@ object StartStageHandlerTest : SubjectSpek({ it("builds tasks for the branch") { val stage = pipeline.stageById(message.stageId) assertThat(stage.tasks, !isEmpty) - stage.tasks.map(Task::getName) shouldEqual listOf("in-branch") + stage.tasks.map(Task::getName) shouldEqual listOf("dummy") } it("does not build more synthetic stages") { @@ -919,9 +920,7 @@ object StartStageHandlerTest : SubjectSpek({ } it("skips the stage") { - verify(queue).push(check { - it.status shouldEqual SKIPPED - }) + verify(queue).push(isA()) } it("doesn't build any tasks") { @@ -998,9 +997,7 @@ object StartStageHandlerTest : SubjectSpek({ } it("skips the stage") { - verify(queue).push(check { - it.status shouldEqual SKIPPED - }) + verify(queue).push(isA()) } } } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/memory/InMemoryQueueTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/memory/InMemoryQueueTest.kt index 11296ba75f..01c2c5bebb 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/memory/InMemoryQueueTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/memory/InMemoryQueueTest.kt @@ -27,8 +27,7 @@ import java.time.Clock object InMemoryQueueTest : QueueTest(createQueue(p3 = null)) object InMemoryMonitorableQueueTest : MonitorableQueueTest( - createQueue, - InMemoryQueue::retry + createQueue ) private val createQueue = { diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/metrics/MonitorableQueueTest.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/metrics/MonitorableQueueTest.kt index 8682fcaec2..d482d3c4d0 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/metrics/MonitorableQueueTest.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/orca/q/metrics/MonitorableQueueTest.kt @@ -35,7 +35,6 @@ import java.time.Duration abstract class MonitorableQueueTest( createQueue: (Clock, DeadMessageCallback, ApplicationEventPublisher?) -> Q, - triggerRedeliveryCheck: Q.() -> Unit, shutdownCallback: (() -> Unit)? = null ) : Spek({ @@ -216,8 +215,10 @@ abstract class MonitorableQueueTest( } on("checking for unacknowledged messages") { - clock.incrementBy(queue!!.ackTimeout) - triggerRedeliveryCheck.invoke(queue!!) + with(queue!!) { + clock.incrementBy(ackTimeout) + retry() + } } it("fires an event") { @@ -236,8 +237,10 @@ abstract class MonitorableQueueTest( } on("checking for unacknowledged messages") { - clock.incrementBy(queue!!.ackTimeout) - triggerRedeliveryCheck.invoke(queue!!) + with(queue!!) { + clock.incrementBy(ackTimeout) + retry() + } } it("fires an event indicating the message is being retried") { @@ -269,8 +272,10 @@ abstract class MonitorableQueueTest( } on("checking for unacknowledged messages") { - clock.incrementBy(queue!!.ackTimeout) - triggerRedeliveryCheck.invoke(queue!!) + with(queue!!) { + clock.incrementBy(ackTimeout) + retry() + } } it("fires an event indicating the message is a duplicate") { @@ -297,9 +302,11 @@ abstract class MonitorableQueueTest( on("failing to acknowledge the message ${Queue.maxRetries} times") { (1..Queue.maxRetries).forEach { - queue!!.poll { _, _ -> } - clock.incrementBy(queue!!.ackTimeout) - triggerRedeliveryCheck.invoke(queue!!) + with(queue!!) { + poll { _, _ -> } + clock.incrementBy(ackTimeout) + retry() + } } } diff --git a/orca-queue/src/test/kotlin/com/netflix/spinnaker/spek/should.kt b/orca-queue/src/test/kotlin/com/netflix/spinnaker/spek/should.kt index b25615bbe1..3793d523ab 100644 --- a/orca-queue/src/test/kotlin/com/netflix/spinnaker/spek/should.kt +++ b/orca-queue/src/test/kotlin/com/netflix/spinnaker/spek/should.kt @@ -16,9 +16,14 @@ package com.netflix.spinnaker.spek +import com.natpryce.hamkrest.allElements import com.natpryce.hamkrest.equalTo import com.natpryce.hamkrest.should.shouldMatch infix fun T.shouldEqual(expected: T) { this shouldMatch equalTo(expected) } + +infix fun > T.shouldAllEqual(expected: E) { + this shouldMatch allElements(equalTo(expected)) +} diff --git a/orca-web/orca-web.gradle b/orca-web/orca-web.gradle index 5b364dc031..040eb4403c 100644 --- a/orca-web/orca-web.gradle +++ b/orca-web/orca-web.gradle @@ -38,6 +38,7 @@ dependencies { compile project(":orca-front50") compile project(":orca-applications") compile project(":orca-flex") + compile project(":orca-dry-run") compile project(":orca-echo") compile project(":orca-kayenta") compile project(":orca-mine") @@ -52,6 +53,7 @@ dependencies { compile spinnaker.dependency('korkExceptions') compile spinnaker.dependency("korkStackdriver") compile spinnaker.dependency("korkWeb") + compile spinnaker.dependency("logstashEncoder") spinnaker.group('test') testCompile project(":orca-test") diff --git a/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/OperationsController.groovy b/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/OperationsController.groovy index 9789ce7fab..b61bb378dc 100644 --- a/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/OperationsController.groovy +++ b/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/OperationsController.groovy @@ -16,6 +16,8 @@ package com.netflix.spinnaker.orca.controllers +import com.netflix.spinnaker.orca.pipeline.util.ArtifactResolver + import javax.servlet.http.HttpServletResponse import com.fasterxml.jackson.databind.ObjectMapper import com.netflix.spinnaker.kork.web.exceptions.InvalidRequestException @@ -36,6 +38,7 @@ import org.springframework.web.bind.annotation.RequestBody import org.springframework.web.bind.annotation.RequestMapping import org.springframework.web.bind.annotation.RequestMethod import org.springframework.web.bind.annotation.RestController +import static net.logstash.logback.argument.StructuredArguments.value @RestController @Slf4j @@ -80,7 +83,7 @@ class OperationsController { pipeline.trigger = trigger def json = objectMapper.writeValueAsString(pipeline) - log.info('received pipeline {}:{}', pipeline.id, json) + log.info('received pipeline {}:{}', value("pipelineId", pipeline.id), json) if (pipeline.disabled) { throw new InvalidRequestException("Pipeline is disabled and cannot be started.") @@ -92,7 +95,7 @@ class OperationsController { } if (plan) { - log.info('not starting pipeline (plan: true): {}', pipeline.id) + log.info('not starting pipeline (plan: true): {}', value("pipelineId", pipeline.id)) if (pipeline.errors != null) { throw new ValidationException("Pipeline template is invalid", pipeline.errors as List>) } @@ -145,6 +148,8 @@ class OperationsController { pipeline.trigger.parameters[it.name] = pipeline.trigger.parameters.containsKey(it.name) ? pipeline.trigger.parameters[it.name] : it.default } } + + ArtifactResolver.resolveArtifacts(pipeline) } private void getBuildInfo(Map trigger) { diff --git a/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/PipelineTemplateController.groovy b/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/PipelineTemplateController.groovy index 9e7c55ed2e..c45e4b3f10 100644 --- a/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/PipelineTemplateController.groovy +++ b/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/PipelineTemplateController.groovy @@ -17,11 +17,13 @@ package com.netflix.spinnaker.orca.controllers import com.netflix.spinnaker.kork.web.exceptions.InvalidRequestException import com.netflix.spinnaker.orca.pipelinetemplate.PipelineTemplateService +import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.converter.PipelineTemplateConverter import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.PipelineTemplate import com.netflix.spinnaker.orca.pipelinetemplate.v1schema.model.TemplateConfiguration.TemplateSource import groovy.util.logging.Slf4j import org.springframework.beans.factory.annotation.Autowired import org.springframework.boot.autoconfigure.condition.ConditionalOnExpression +import org.springframework.web.bind.annotation.RequestBody import org.springframework.web.bind.annotation.RequestMapping import org.springframework.web.bind.annotation.RequestMethod import org.springframework.web.bind.annotation.RequestParam @@ -43,4 +45,9 @@ class PipelineTemplateController { pipelineTemplateService.resolveTemplate(new TemplateSource(source: source)) } + + @RequestMapping(value = "/convertPipelineToTemplate", method = RequestMethod.POST, produces = 'text/x-yaml') + String convertPipelineToPipelineTemplate(@RequestBody Map pipeline) { + new PipelineTemplateConverter().convertToPipelineTemplate(pipeline) + } } diff --git a/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/TaskController.groovy b/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/TaskController.groovy index 85086ec965..711ba73f98 100644 --- a/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/TaskController.groovy +++ b/orca-web/src/main/groovy/com/netflix/spinnaker/orca/controllers/TaskController.groovy @@ -16,6 +16,9 @@ package com.netflix.spinnaker.orca.controllers +import com.netflix.spinnaker.orca.pipeline.OrchestrationLauncher +import com.netflix.spinnaker.orca.pipeline.model.Task + import java.time.Clock import com.netflix.spinnaker.orca.ExecutionStatus import com.netflix.spinnaker.orca.front50.Front50Service @@ -92,7 +95,7 @@ class TaskController { .toInstant() .toEpochMilli() - executionRepository + def orchestrations = executionRepository .retrieveOrchestrationsForApplication(application, executionCriteria) .filter({ Orchestration orchestration -> !orchestration.startTime || (orchestration.startTime > startTimeCutoff) }) .map({ Orchestration orchestration -> convert(orchestration) }) @@ -101,6 +104,8 @@ class TaskController { .toBlocking() .single() .sort(startTimeOrId) + + orchestrations.subList(0, Math.min(orchestrations.size(), limit)) } @PreAuthorize("@fiatPermissionEvaluator.storeWholePermission()") @@ -169,7 +174,7 @@ class TaskController { executionRepository.retrievePipelinesForPipelineConfigId(it, executionCriteria) }).subscribeOn(Schedulers.io()).toList().toBlocking().single().sort(startTimeOrId) - return filterPipelinesByHistoryCutoff(allPipelines) + return filterPipelinesByHistoryCutoff(allPipelines, limit) } @PostAuthorize("hasPermission(returnObject.application, 'APPLICATION', 'READ')") @@ -213,6 +218,8 @@ class TaskController { @ResponseStatus(HttpStatus.ACCEPTED) void pause(@PathVariable String id) { executionRepository.pause(id, AuthenticatedRequest.getSpinnakerUser().orElse("anonymous")) + def pipeline = executionRepository.retrievePipeline(id) + executionRunner.reschedule(pipeline) } @PreAuthorize("hasPermission(this.getPipeline(#id)?.application, 'APPLICATION', 'WRITE')") @@ -258,6 +265,8 @@ class TaskController { stage.context["lastModifiedBy"] = AuthenticatedRequest.getSpinnakerUser().orElse("anonymous") executionRepository.storeStage(stage) + + executionRunner.reschedule(pipeline) } pipeline } @@ -322,10 +331,10 @@ class TaskController { executionRepository.retrievePipelinesForPipelineConfigId(it, executionCriteria) }).subscribeOn(Schedulers.io()).toList().toBlocking().single().sort(startTimeOrId) - return filterPipelinesByHistoryCutoff(allPipelines) + return filterPipelinesByHistoryCutoff(allPipelines, limit) } - private List filterPipelinesByHistoryCutoff(List pipelines) { + private List filterPipelinesByHistoryCutoff(List pipelines, int limit) { // TODO-AJ The eventual goal is to return `allPipelines` without the need to group + filter below (WIP) def cutoffTime = (new Date(clock.millis()) - daysOfExecutionHistory).time @@ -343,7 +352,7 @@ class TaskController { recentPipelines = sortedPipelinesGroup[0..upperBounds] } - pipelinesSatisfyingCutoff.addAll(recentPipelines) + pipelinesSatisfyingCutoff.addAll(recentPipelines.subList(0, Math.min(recentPipelines.size(), limit))) } return pipelinesSatisfyingCutoff.sort(startTimeOrId) diff --git a/orca-web/src/test/groovy/com/netflix/spinnaker/orca/controllers/TaskControllerSpec.groovy b/orca-web/src/test/groovy/com/netflix/spinnaker/orca/controllers/TaskControllerSpec.groovy index 6a12ffff59..6a817f3186 100644 --- a/orca-web/src/test/groovy/com/netflix/spinnaker/orca/controllers/TaskControllerSpec.groovy +++ b/orca-web/src/test/groovy/com/netflix/spinnaker/orca/controllers/TaskControllerSpec.groovy @@ -16,6 +16,8 @@ package com.netflix.spinnaker.orca.controllers +import com.netflix.spinnaker.orca.pipeline.ExecutionRunner + import java.time.Clock import java.time.Instant import com.fasterxml.jackson.databind.ObjectMapper @@ -45,6 +47,7 @@ class TaskControllerSpec extends Specification { def executionRepository = Mock(ExecutionRepository) def front50Service = Mock(Front50Service) def startTracker = Mock(PipelineStartTracker) + def executionRunner = Mock(ExecutionRunner) def clock = Clock.fixed(Instant.now(), UTC) int daysOfExecutionHistory = 14 @@ -57,6 +60,7 @@ class TaskControllerSpec extends Specification { new TaskController( front50Service: front50Service, executionRepository: executionRepository, + executionRunner: executionRunner, daysOfExecutionHistory: daysOfExecutionHistory, numberOfOldPipelineExecutionsToInclude: numberOfOldPipelineExecutionsToInclude, startTracker: startTracker, @@ -268,7 +272,7 @@ class TaskControllerSpec extends Specification { List results = new ObjectMapper().readValue(response.contentAsString, List) then: - results.id == ['not-started-2', 'not-started-1', 'newer-2', 'newer-1'] + results.id == ['newer-2', 'newer-1'] } void 'should update existing stage context'() { @@ -293,6 +297,7 @@ class TaskControllerSpec extends Specification { judgmentStatus: "stop", value: "1", lastModifiedBy: "anonymous" ] } as Stage) + 1 * executionRunner.reschedule(pipeline) 0 * _ and: diff --git a/settings.gradle b/settings.gradle index 1219f24d3b..d660a91e5b 100644 --- a/settings.gradle +++ b/settings.gradle @@ -19,6 +19,7 @@ include "orca-extensionpoint", "orca-retrofit", "orca-front50", "orca-bakery", + "orca-dry-run", "orca-echo", "orca-eureka", "orca-igor", @@ -33,7 +34,6 @@ include "orca-extensionpoint", "orca-applications", "orca-queue", "orca-queue-redis", - "orca-queue-sqs", "orca-pipelinetemplate", "orca-validation"