Skip to content

Commit

Permalink
refactor(model): Unify execution subtypes
Browse files Browse the repository at this point in the history
* refactor(model): move properties from pipeline & orchestration up

* refactor(model): make Execution non-abstract

* refactor(model): introduce execution type enum and use instead of instanceof

* refactor(model): unify retrieve & delete methods in repository

* refactor(model): unify retrieve observable methods in repository

* refactor(model): convert all refs to Pipeline / Orchestration to Execution

* refactor(model): clean up Kotlin property syntax

* refactor(model): fix one class/test I missed

* refactor(model): migrate queue messages to new `executionType` format

* refactor(model): change enums to uppercase

* refactor(model): change enums to uppercase

* refactor(model): rebase hell

* refactor(model): typo
  • Loading branch information
robfletcher authored Nov 8, 2017
1 parent b37d3e4 commit 23802ef
Show file tree
Hide file tree
Showing 390 changed files with 2,441 additions and 2,838 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -16,19 +16,18 @@

package com.netflix.spinnaker.orca.applications.pipelines

import groovy.transform.CompileStatic
import com.netflix.spinnaker.orca.applications.tasks.UpsertApplicationTask
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 groovy.transform.CompileStatic
import org.springframework.stereotype.Component

@Component
@CompileStatic
class CreateApplicationStage implements StageDefinitionBuilder {
@Override
<T extends Execution<T>> void taskGraph(Stage<T> stage, TaskNode.Builder builder) {
void taskGraph(Stage stage, TaskNode.Builder builder) {
builder
.withTask("createApplication", UpsertApplicationTask)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,20 +16,19 @@

package com.netflix.spinnaker.orca.applications.pipelines

import groovy.transform.CompileStatic
import com.netflix.spinnaker.orca.applications.tasks.DeleteApplicationTask
import com.netflix.spinnaker.orca.applications.tasks.VerifyApplicationHasNoDependenciesTask
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 groovy.transform.CompileStatic
import org.springframework.stereotype.Component

@Component
@CompileStatic
class DeleteApplicationStage implements StageDefinitionBuilder {
@Override
<T extends Execution<T>> void taskGraph(Stage<T> stage, TaskNode.Builder builder) {
void taskGraph(Stage stage, TaskNode.Builder builder) {
builder
.withTask("verifyNoDependencies", VerifyApplicationHasNoDependenciesTask)
.withTask("deleteApplication", DeleteApplicationTask)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,16 +23,14 @@ import com.netflix.spinnaker.orca.TaskResult
import com.netflix.spinnaker.orca.front50.Front50Service
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 groovy.transform.CompileStatic
import org.springframework.beans.factory.annotation.Autowired
import org.springframework.stereotype.Component

@Component
class DeleteProjectStage implements StageDefinitionBuilder {
@Override
<T extends Execution<T>> void taskGraph(Stage<T> stage, TaskNode.Builder builder) {
void taskGraph(Stage stage, TaskNode.Builder builder) {
builder
.withTask("deleteProject", DeleteProjectTask)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,19 +16,18 @@

package com.netflix.spinnaker.orca.applications.pipelines

import groovy.transform.CompileStatic
import com.netflix.spinnaker.orca.applications.tasks.UpsertApplicationTask
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 groovy.transform.CompileStatic
import org.springframework.stereotype.Component

@Component
@CompileStatic
class UpdateApplicationStage implements StageDefinitionBuilder {
@Override
<T extends Execution<T>> void taskGraph(Stage<T> stage, TaskNode.Builder builder) {
void taskGraph(Stage stage, TaskNode.Builder builder) {
builder
.withTask("updateApplication", UpsertApplicationTask)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,19 +16,18 @@

package com.netflix.spinnaker.orca.applications.pipelines

import groovy.transform.CompileStatic
import com.netflix.spinnaker.orca.applications.tasks.UpsertApplicationTask
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 groovy.transform.CompileStatic
import org.springframework.stereotype.Component

@Component
@CompileStatic
class UpsertApplicationStage implements StageDefinitionBuilder {
@Override
<T extends Execution<T>> void taskGraph(Stage<T> stage, TaskNode.Builder builder) {
void taskGraph(Stage stage, TaskNode.Builder builder) {
builder
.withTask("upsertApplication", UpsertApplicationTask)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,17 +24,15 @@ import com.netflix.spinnaker.orca.TaskResult
import com.netflix.spinnaker.orca.front50.Front50Service
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 groovy.transform.CompileStatic
import groovy.util.logging.Slf4j
import org.springframework.beans.factory.annotation.Autowired
import org.springframework.stereotype.Component

@Component
class UpsertProjectStage implements StageDefinitionBuilder {
@Override
<T extends Execution<T>> void taskGraph(Stage<T> stage, TaskNode.Builder builder) {
void taskGraph(Stage stage, TaskNode.Builder builder) {
builder
.withTask("upsertProject", UpsertProjectTask)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,11 @@ import com.netflix.spinnaker.orca.bakery.tasks.CreateBakeTask
import com.netflix.spinnaker.orca.bakery.tasks.MonitorBakeTask
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 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

@Slf4j
Expand All @@ -43,7 +41,7 @@ class BakeStage implements StageDefinitionBuilder, RestartableStage {
public static final String PIPELINE_CONFIG_TYPE = "bake"

@Override
<T extends Execution<T>> void taskGraph(Stage<T> stage, TaskNode.Builder builder) {
void taskGraph(Stage stage, TaskNode.Builder builder) {
if (isTopLevelStage(stage)) {
builder
.withTask("completeParallel", CompleteParallelBakeTask)
Expand All @@ -57,8 +55,8 @@ class BakeStage implements StageDefinitionBuilder, RestartableStage {

@Override
@Nonnull
<T extends Execution<T>> List<Stage<T>> parallelStages(
@Nonnull Stage<T> stage
List<Stage> parallelStages(
@Nonnull Stage stage
) {
if (isTopLevelStage(stage)) {
return parallelContexts(stage).collect { context ->
Expand All @@ -69,12 +67,12 @@ class BakeStage implements StageDefinitionBuilder, RestartableStage {
}
}

private boolean isTopLevelStage(Stage<?> stage) {
private boolean isTopLevelStage(Stage stage) {
stage.parentStageId == null
}

@CompileDynamic
<T extends Execution<T>> Collection<Map<String, Object>> parallelContexts(Stage<T> stage) {
Collection<Map<String, Object>> parallelContexts(Stage stage) {
Set<String> deployRegions = stage.context.region ? [stage.context.region] as Set<String> : []
deployRegions.addAll(stage.context.regions as Set<String> ?: [])

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import com.netflix.spinnaker.orca.TaskResult
import com.netflix.spinnaker.orca.bakery.api.BakeRequest
import com.netflix.spinnaker.orca.bakery.api.BakeStatus
import com.netflix.spinnaker.orca.bakery.api.BakeryService
import com.netflix.spinnaker.orca.pipeline.model.Pipeline
import com.netflix.spinnaker.orca.pipeline.model.Stage
import com.netflix.spinnaker.orca.pipeline.util.OperatingSystem
import com.netflix.spinnaker.orca.pipeline.util.PackageInfo
Expand All @@ -34,6 +33,7 @@ import org.springframework.beans.factory.annotation.Autowired
import org.springframework.beans.factory.annotation.Value
import org.springframework.stereotype.Component
import retrofit.RetrofitError
import static com.netflix.spinnaker.orca.pipeline.model.Execution.ExecutionType.PIPELINE

@Component
@CompileStatic
Expand Down Expand Up @@ -114,8 +114,8 @@ class CreateBakeTask implements RetryableTask {
if (stage.context.rebake == true) {
return true
}
if (stage.execution instanceof Pipeline) {
Map trigger = ((Pipeline) stage.execution).trigger
if (stage.execution.type == PIPELINE) {
Map trigger = stage.execution.trigger
return trigger?.rebake == true
}
return false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ class BakeStageSpec extends Specification {
}
}

def bakeStage = new Stage<>(pipeline, "bake", "Bake!", bakeStageContext)
def bakeStage = new Stage(pipeline, "bake", "Bake!", bakeStageContext)
def builder = Spy(BakeStage, {
(0..1) * now() >> {
use([TimeCategory]) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import com.netflix.spinnaker.orca.ExecutionStatus
import com.netflix.spinnaker.orca.bakery.api.Bake
import com.netflix.spinnaker.orca.bakery.api.BakeStatus
import com.netflix.spinnaker.orca.bakery.api.BakeryService
import com.netflix.spinnaker.orca.pipeline.model.Pipeline
import com.netflix.spinnaker.orca.pipeline.model.Execution
import com.netflix.spinnaker.orca.pipeline.model.Stage
import retrofit.RetrofitError
import retrofit.client.Response
Expand All @@ -36,7 +36,7 @@ class CompletedBakeTaskSpec extends Specification {

@Subject task = new CompletedBakeTask()

@Shared Pipeline pipeline = pipeline()
@Shared Execution pipeline = pipeline()

@Shared notFoundError = RetrofitError.httpError(
null,
Expand All @@ -52,7 +52,7 @@ class CompletedBakeTaskSpec extends Specification {
}

and:
def stage = new Stage<>(pipeline, "bake", [region: region, status: new BakeStatus(resourceId: bakeId)])
def stage = new Stage(pipeline, "bake", [region: region, status: new BakeStatus(resourceId: bakeId)])

when:
def result = task.execute(stage)
Expand All @@ -76,7 +76,7 @@ class CompletedBakeTaskSpec extends Specification {
}

and:
def stage = new Stage<>(pipeline, "bake", [region: region, status: new BakeStatus(resourceId: bakeId)])
def stage = new Stage(pipeline, "bake", [region: region, status: new BakeStatus(resourceId: bakeId)])

when:
task.execute(stage)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import com.netflix.spinnaker.orca.bakery.api.BakeRequest
import com.netflix.spinnaker.orca.bakery.api.BakeStatus
import com.netflix.spinnaker.orca.bakery.api.BakeryService
import com.netflix.spinnaker.orca.jackson.OrcaObjectMapper
import com.netflix.spinnaker.orca.pipeline.model.Pipeline
import com.netflix.spinnaker.orca.pipeline.model.Execution
import com.netflix.spinnaker.orca.pipeline.model.Stage
import retrofit.RetrofitError
import retrofit.client.Response
Expand Down Expand Up @@ -58,7 +58,7 @@ class CreateBakeTaskSpec extends Specification {
baseLabel: BakeRequest.Label.release.name()
]

Pipeline pipeline = pipeline {
Execution pipeline = pipeline {
stage {
type = "bake"
context = bakeConfig
Expand Down Expand Up @@ -246,7 +246,7 @@ class CreateBakeTaskSpec extends Specification {
def "finds package details from context and trigger"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -282,7 +282,7 @@ class CreateBakeTaskSpec extends Specification {
def "fails if pipeline trigger or context includes artifacts but no artifact for the bake package"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -317,7 +317,7 @@ class CreateBakeTaskSpec extends Specification {
[fileName: 'hodor.1.2.nupkg']
]
]
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: buildInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -367,7 +367,7 @@ class CreateBakeTaskSpec extends Specification {
def "build info with url yields bake stage output containing build host, job and build number"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -402,7 +402,7 @@ class CreateBakeTaskSpec extends Specification {
def "build info with url and scm yields bake stage output containing build host, job, build number and commit hash"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -435,7 +435,7 @@ class CreateBakeTaskSpec extends Specification {
def "build info with url and two scms yields bake stage output containing build host, job, build number and correctly-chosen commit hash"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -468,7 +468,7 @@ class CreateBakeTaskSpec extends Specification {
def "build info with url and master and develop scms yields bake stage output containing build host, job, build number and first commit hash"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -501,7 +501,7 @@ class CreateBakeTaskSpec extends Specification {
def "build info without url yields bake stage output without build host, job, build number and commit hash"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -536,7 +536,7 @@ class CreateBakeTaskSpec extends Specification {
def "build info with url yields bake request containing build host, job and build number"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -574,7 +574,7 @@ class CreateBakeTaskSpec extends Specification {
def "build info with url but without extractBuildDetails yields bake request without build host, job, build number, and commit hash"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -611,7 +611,7 @@ class CreateBakeTaskSpec extends Specification {
def "build info without url yields bake request without build host, job, build number and commit hash"() {
given:
bakeConfig.buildInfo = contextInfo
Pipeline pipelineWithTrigger = pipeline {
def pipelineWithTrigger = pipeline {
trigger.putAll(buildInfo: triggerInfo)
stage {
type = "bake"
Expand Down Expand Up @@ -717,7 +717,7 @@ class CreateBakeTaskSpec extends Specification {
@Unroll
def "sets rebake query parameter to #queryParameter when trigger is #trigger"() {
given:
Pipeline pipeline = pipeline {
def pipeline = pipeline {
if (triggerConfig) {
trigger.putAll(triggerConfig)
}
Expand Down
Loading

0 comments on commit 23802ef

Please sign in to comment.