Skip to content

Commit

Permalink
Merge pull request #97 from abayer/jenkins-41243
Browse files Browse the repository at this point in the history
[JENKINS-41243] Slightly faster tagging of synthetic stages
  • Loading branch information
abayer committed Jan 25, 2017
2 parents 8ba5af1 + 6989f43 commit a8bc15a
Show file tree
Hide file tree
Showing 9 changed files with 233 additions and 30 deletions.
Expand Up @@ -27,6 +27,14 @@ package org.jenkinsci.plugins.pipeline.modeldefinition


public class SyntheticStageNames {
public static List<String> preStages() {
return [checkout(), agentSetup(), toolInstall()]
}

public static List<String> postStages() {
return [postBuild()]
}

public static String checkout() {
return "Declarative: Checkout SCM"
}
Expand All @@ -42,8 +50,4 @@ public class SyntheticStageNames {
public static String postBuild() {
return "Declarative: Post Build Actions"
}

public static String notifications() {
return "Declarative: Notifications"
}
}
Expand Up @@ -33,7 +33,6 @@ import hudson.ExtensionList
import hudson.model.Describable
import hudson.model.Descriptor
import org.apache.commons.codec.digest.DigestUtils
import org.apache.commons.lang.reflect.FieldUtils
import org.jenkinsci.plugins.pipeline.StageStatus
import org.jenkinsci.plugins.pipeline.StageTagsMetadata
import org.jenkinsci.plugins.pipeline.SyntheticStage
Expand All @@ -60,7 +59,7 @@ import org.jenkinsci.plugins.workflow.support.steps.StageStep
import javax.annotation.Nullable
import java.lang.reflect.ParameterizedType
import java.lang.reflect.Type
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeUnit

// TODO: Prune like mad once we have step-in-groovy and don't need these static whitelisted wrapper methods.
/**
Expand Down Expand Up @@ -191,15 +190,19 @@ public class Utils {
}
}

static void attachExecutionModel(CpsScript script) {
static void attachDeclarativeActions(CpsScript script) {
WorkflowRun r = script.$build()
ModelASTPipelineDef model = Converter.parseFromCpsScript(script)

ModelASTStages stages = model.stages

stages.removeSourceLocation()

r.addAction(new ExecutionModelAction(stages))
if (r.getAction(SyntheticStageGraphListener.GraphListenerAction.class) == null) {
r.addAction(new SyntheticStageGraphListener.GraphListenerAction())
}
if (r.getAction(ExecutionModelAction.class) == null) {
r.addAction(new ExecutionModelAction(stages))
}
}

static Predicate<FlowNode> endNodeForStage(final StepStartNode startNode) {
Expand Down Expand Up @@ -295,16 +298,6 @@ public class Utils {
return getTagMetadata(SyntheticStage.class)
}

/**
* Marks the containing stage with this name as a synthetic stage, with the appropriate context.
*
* @param stageName
* @param context
*/
static void markSyntheticStage(String stageName, String context) {
markStageWithTag(stageName, getSyntheticStageMetadata().tagName, context)
}

static void markStageFailedAndContinued(String stageName) {
markStageWithTag(stageName, getStageStatusMetadata().tagName, getStageStatusMetadata().failedAndContinued)
}
Expand Down
@@ -0,0 +1,108 @@
/*
* The MIT License
*
* Copyright (c) 2017, CloudBees, Inc.
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
* in the Software without restriction, including without limitation the rights
* to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
* copies of the Software, and to permit persons to whom the Software is
* furnished to do so, subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in
* all copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
* FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
* AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
* LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
* OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
* THE SOFTWARE.
*/

package org.jenkinsci.plugins.pipeline.modeldefinition;

import hudson.model.InvisibleAction;
import hudson.model.Run;
import jenkins.model.RunAction2;
import org.jenkinsci.plugins.pipeline.SyntheticStage;
import org.jenkinsci.plugins.workflow.actions.LabelAction;
import org.jenkinsci.plugins.workflow.actions.TagsAction;
import org.jenkinsci.plugins.workflow.cps.nodes.StepStartNode;
import org.jenkinsci.plugins.workflow.flow.FlowExecution;
import org.jenkinsci.plugins.workflow.flow.GraphListener;
import org.jenkinsci.plugins.workflow.graph.FlowNode;
import org.jenkinsci.plugins.workflow.job.WorkflowRun;
import org.jenkinsci.plugins.workflow.support.steps.StageStep;

import java.io.IOException;
import java.util.logging.Logger;

import static java.util.logging.Level.WARNING;

public final class SyntheticStageGraphListener implements GraphListener {
private static final Logger LOGGER = Logger.getLogger(SyntheticStageGraphListener.class.getName());

@Override
public void onNewHead(FlowNode node) {
if (node != null && node instanceof StepStartNode &&
((StepStartNode) node).getDescriptor() instanceof StageStep.DescriptorImpl) {
LabelAction label = node.getPersistentAction(LabelAction.class);
if (label != null &&
(SyntheticStageNames.preStages().contains(label.getDisplayName()) ||
SyntheticStageNames.postStages().contains(label.getDisplayName()))) {
if (SyntheticStageNames.preStages().contains(label.getDisplayName())) {
attachTag(node, SyntheticStage.getPre());
}
if (SyntheticStageNames.postStages().contains(label.getDisplayName())) {
attachTag(node, SyntheticStage.getPost());
}
}
}
}

private void attachTag(FlowNode currentNode, String syntheticContext) {
TagsAction tagsAction = currentNode.getPersistentAction(TagsAction.class);
if (tagsAction == null) {
tagsAction = new TagsAction();
tagsAction.addTag(SyntheticStage.TAG_NAME, syntheticContext);
currentNode.addAction(tagsAction);
} else if (tagsAction.getTagValue(SyntheticStage.TAG_NAME) == null) {
tagsAction.addTag(SyntheticStage.TAG_NAME, syntheticContext);
try {
currentNode.save();
} catch (IOException e) {
LOGGER.log(WARNING, "failed to save actions for FlowNode id=" + currentNode.getId(), e);
}
}
}

public static class GraphListenerAction extends InvisibleAction implements RunAction2 {
@Override
public void onLoad(Run<?, ?> r) {
if (r != null && r instanceof WorkflowRun) {
WorkflowRun run = (WorkflowRun) r;
attachListener(run);
}
}

@Override
public void onAttached(Run<?, ?> r) {
if (r != null && r instanceof WorkflowRun) {
WorkflowRun run = (WorkflowRun) r;
attachListener(run);
}
}

private void attachListener(WorkflowRun run) {
if (run != null) {
FlowExecution exec = run.getExecution();
if (exec != null && !exec.isComplete()) {
exec.addListener(new SyntheticStageGraphListener());
}
}
}
}
}
Expand Up @@ -29,7 +29,6 @@ import hudson.FilePath
import hudson.Launcher
import hudson.model.Result
import org.jenkinsci.plugins.pipeline.modeldefinition.model.*
import org.jenkinsci.plugins.pipeline.modeldefinition.options.impl.SkipDefaultCheckout
import org.jenkinsci.plugins.pipeline.modeldefinition.steps.CredentialWrapper
import org.jenkinsci.plugins.pipeline.modeldefinition.when.DeclarativeStageConditional
import org.jenkinsci.plugins.workflow.cps.CpsScript
Expand All @@ -50,8 +49,6 @@ public class ModelInterpreter implements Serializable {
}

def call(CpsClosure closure) {
// Attach the stages model to the run for introspection etc.
Utils.attachExecutionModel(script)

ClosureModelTranslator m = new ClosureModelTranslator(Root.class, script)

Expand All @@ -63,6 +60,8 @@ public class ModelInterpreter implements Serializable {
Throwable firstError

if (root != null) {
// Attach the stages model to the run for introspection etc.
Utils.attachDeclarativeActions(script)
boolean postBuildRun = false

try {
Expand Down Expand Up @@ -249,7 +248,6 @@ public class ModelInterpreter implements Serializable {
def toolsList = tools.getToolEntries()
if (!Utils.withinAStage()) {
script.stage(SyntheticStageNames.toolInstall()) {
Utils.markSyntheticStage(SyntheticStageNames.toolInstall(), Utils.getSyntheticStageMetadata().pre)
toolEnv = actualToolsInstall(toolsList)
}
} else {
Expand Down Expand Up @@ -408,7 +406,6 @@ public class ModelInterpreter implements Serializable {
Throwable stageError = null
if (root.hasSatisfiedConditions(root.post, script.getProperty("currentBuild"))) {
script.stage(SyntheticStageNames.postBuild()) {
Utils.markSyntheticStage(SyntheticStageNames.postBuild(), Utils.getSyntheticStageMetadata().post)
stageError = runPostConditions(root.post, root.agent, stageError)
}
}
Expand Down
Expand Up @@ -43,7 +43,6 @@ public class DockerPipelineFromDockerfileScript extends AbstractDockerPipelineSc
def img = null
if (!Utils.withinAStage()) {
script.stage(SyntheticStageNames.agentSetup()) {
Utils.markSyntheticStage(SyntheticStageNames.agentSetup(), Utils.getSyntheticStageMetadata().pre)
try {
img = buildImage().call()
} catch (Exception e) {
Expand Down
Expand Up @@ -41,7 +41,6 @@ public class DockerPipelineScript extends AbstractDockerPipelineScript<DockerPip
return {
if (!Utils.withinAStage()) {
script.stage(SyntheticStageNames.agentSetup()) {
Utils.markSyntheticStage(SyntheticStageNames.agentSetup(), Utils.getSyntheticStageMetadata().pre)
try {
script.getProperty("docker").image(describable.image).pull()
} catch (Exception e) {
Expand Down
Expand Up @@ -50,7 +50,6 @@ public class LabelScript extends DeclarativeAgentScript<Label> {
SkipDefaultCheckout skip = (SkipDefaultCheckout)root.options?.options?.get("skipDefaultCheckout")
if (!skip?.isSkipDefaultCheckout() && Utils.hasScmContext(script)) {
script.stage(SyntheticStageNames.checkout()) {
Utils.markSyntheticStage(SyntheticStageNames.checkout(), Utils.getSyntheticStageMetadata().pre)
script.checkout script.scm
}
}
Expand Down
Expand Up @@ -422,17 +422,17 @@ public void failureBeforeStages() throws Exception {
.go();
}

private Predicate<FlowNode> syntheticStagePredicate(String stageName,
public static Predicate<FlowNode> syntheticStagePredicate(String stageName,
String context) {
return stageTagPredicate(stageName, Utils.getSyntheticStageMetadata().getTagName(), context);
}

private Predicate<FlowNode> stageStatusPredicate(String stageName,
public static Predicate<FlowNode> stageStatusPredicate(String stageName,
String stageStatus) {
return stageTagPredicate(stageName, Utils.getStageStatusMetadata().getTagName(), stageStatus);
}

private Predicate<FlowNode> stageTagPredicate(final String stageName,
public static Predicate<FlowNode> stageTagPredicate(final String stageName,
final String tagName,
final String tagValue) {
return new Predicate<FlowNode>() {
Expand Down
@@ -0,0 +1,104 @@
/*
* The MIT License
*
* Copyright (c) 2017, CloudBees, Inc.
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
* in the Software without restriction, including without limitation the rights
* to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
* copies of the Software, and to permit persons to whom the Software is
* furnished to do so, subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in
* all copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
* FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
* AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
* LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
* OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
* THE SOFTWARE.
*/
package org.jenkinsci.plugins.pipeline.modeldefinition;

import org.jenkinsci.plugins.pipeline.SyntheticStage;
import org.jenkinsci.plugins.workflow.cps.CpsFlowDefinition;
import org.jenkinsci.plugins.workflow.flow.FlowExecution;
import org.jenkinsci.plugins.workflow.graph.FlowNode;
import org.jenkinsci.plugins.workflow.graphanalysis.DepthFirstScanner;
import org.jenkinsci.plugins.workflow.job.WorkflowJob;
import org.jenkinsci.plugins.workflow.job.WorkflowRun;
import org.jenkinsci.plugins.workflow.test.steps.SemaphoreStep;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runners.model.Statement;
import org.jvnet.hudson.test.BuildWatcher;
import org.jvnet.hudson.test.Issue;
import org.jvnet.hudson.test.RestartableJenkinsRule;

import java.util.Collection;

import static org.jenkinsci.plugins.pipeline.modeldefinition.BasicModelDefTest.syntheticStagePredicate;
import static org.junit.Assert.assertNotNull;

public class SyntheticStageGraphListenerTest {

@ClassRule
public static BuildWatcher buildWatcher = new BuildWatcher();

@Rule
public RestartableJenkinsRule rr = new RestartableJenkinsRule();

@Issue("JENKINS-41243")
@Test
public void resume() throws Exception {
rr.addStep(new Statement() {
@Override
public void evaluate() throws Throwable {
WorkflowJob p = rr.j.createProject(WorkflowJob.class, "p");
p.setDefinition(new CpsFlowDefinition("pipeline {\n" +
" agent any\n" +
" stages {\n" +
" stage('x') {\n" +
" steps {\n" +
" script {\n" +
" semaphore 'wait'\n" +
" }\n" +
" }\n" +
" }\n" +
" }\n" +
" post {\n" +
" always {\n" +
" echo 'Hi there'\n" +
" }\n" +
" }\n" +
"}", true));
WorkflowRun b = p.scheduleBuild2(0).waitForStart();
SemaphoreStep.waitForStart("wait/1", b);
}
});
rr.addStep(new Statement() {
@Override
public void evaluate() throws Throwable {
WorkflowJob p = rr.j.jenkins.getItemByFullName("p", WorkflowJob.class);
WorkflowRun b = p.getBuildByNumber(1);
SemaphoreStep.success("wait/1", null);
rr.j.assertBuildStatusSuccess(rr.j.waitForCompletion(b));
rr.j.assertLogContains("[Pipeline] { (" + SyntheticStageNames.postBuild() + ")", b);
FlowExecution execution = b.getExecution();

Collection<FlowNode> heads = execution.getCurrentHeads();

DepthFirstScanner scanner = new DepthFirstScanner();

assertNotNull(scanner.findFirstMatch(heads, null, syntheticStagePredicate(SyntheticStageNames.postBuild(), SyntheticStage.getPost())));
SemaphoreStep.success("wait/2", null);
rr.j.buildAndAssertSuccess(p);
}
});
}

}

0 comments on commit a8bc15a

Please sign in to comment.