=pro #17397 improve PR validation speed tremendously (skipping things)

+ enable parallel execution
+ exclude perf tests (TODO mark more as such)
+ uses sbt-dependency-graph plugin
+ implement dependency tracking for testing of only these
+ project which could have been affected by a given PR
This commit is contained in:
Konrad Malawski 2015-05-05 19:39:56 +02:00
parent 920dc48109
commit 8b045173ad
17 changed files with 226 additions and 46 deletions

View file

@ -144,16 +144,28 @@ Example:
## How To Enforce These Guidelines?
### Make Use of Pull Request Validator
Akka uses [Jenkins GitHub pull request builder plugin](https://wiki.jenkins-ci.org/display/JENKINS/GitHub+pull+request+builder+plugin) that automatically merges the code, builds it, runs the tests and comments on the Pull Request in GitHub.
Akka uses [Jenkins GitHub pull request builder plugin](https://wiki.jenkins-ci.org/display/JENKINS/GitHub+pull+request+builder+plugin)
that automatically merges the code, builds it, runs the tests and comments on the Pull Request in GitHub.
Upon a submission of a Pull Request the Github pull request builder plugin will post a following comment:
Can one of the repo owners verify this patch?
This requires a member from a core team to start Pull Request validation process by posting comment consisting only of `OK TO TEST`. From now on, whenever new commits are pushed to the Pull Request, a validation job will be automaticaly started and the results of the validation posted to the Pull Request.
This requires a member from a core team to start Pull Request validation process by posting comment consisting only of `OK TO TEST`.
From now on, whenever new commits are pushed to the Pull Request, a validation job will be automaticaly started and the results of the validation posted to the Pull Request.
A Pull Request validation job can be started manually by posting `PLS BUILD` comment on the Pull Request.
In order to speed up PR validation times, the Akka build contains a special sbt task called `validatePullRequest`,
which is smart enough to figure out which projects should be built if a PR only has changes in some parts of the project.
For example, if your PR only touches `akka-persistence`, no `akka-remote` tests need to be run, however the task
will validate all projects that depend on `akka-persistence` (including samples).
Also, tests tagged as `PerformanceTest` and the likes of it are excluded from PR validation.
In order to force the `validatePullRequest` task to build the entire project, regardless of dependency analysis of a PRs
changes one can use the special `PLS BUILD ALL` command (typed in a comment on github, on the Pull Request), which will cause
the validator to test all projects.
## Source style
Akka uses [Scalariform](https://github.com/mdr/scalariform) to enforce some of the code style rules.

View file

@ -106,7 +106,7 @@ abstract class ClusterMetricsEnabledSpec extends MultiNodeSpec(ClusterMetricsEna
"Cluster metrics" must {
"periodically collect metrics on each node, publish to the event stream, " +
"and gossip metrics around the node ring" taggedAs LongRunningTest in within(60 seconds) {
"and gossip metrics around the node ring" in within(60 seconds) {
awaitClusterUp(roles: _*)
enterBarrier("cluster-started")
awaitAssert(clusterView.members.count(_.status == MemberStatus.Up) should ===(roles.size))
@ -117,7 +117,7 @@ abstract class ClusterMetricsEnabledSpec extends MultiNodeSpec(ClusterMetricsEna
collector.sample.metrics.size should be > (3)
enterBarrier("after")
}
"reflect the correct number of node metrics in cluster view" taggedAs LongRunningTest in within(30 seconds) {
"reflect the correct number of node metrics in cluster view" in within(30 seconds) {
runOn(node2) {
cluster.leave(node1)
}
@ -146,7 +146,7 @@ abstract class ClusterMetricsDisabledSpec extends MultiNodeSpec(ClusterMetricsDi
val metricsView = new ClusterMetricsView(cluster.system)
"Cluster metrics" must {
"not collect metrics, not publish metrics events, and not gossip metrics" taggedAs LongRunningTest in {
"not collect metrics, not publish metrics events, and not gossip metrics" in {
awaitClusterUp(roles: _*)
// TODO ensure same contract
//clusterView.clusterMetrics.size should ===(0)

View file

@ -72,7 +72,7 @@ abstract class ClusterDeathWatchSpec
}
"An actor watching a remote actor in the cluster" must {
"receive Terminated when watched node becomes Down/Removed" taggedAs LongRunningTest in within(20 seconds) {
"receive Terminated when watched node becomes Down/Removed" in within(20 seconds) {
awaitClusterUp(first, second, third, fourth)
enterBarrier("cluster-up")
@ -141,7 +141,7 @@ abstract class ClusterDeathWatchSpec
}
"receive Terminated when watched path doesn't exist" taggedAs LongRunningTest ignore {
"receive Terminated when watched path doesn't exist" ignore {
Thread.sleep(5000)
runOn(first) {
val path = RootActorPath(second) / "user" / "non-existing"
@ -158,7 +158,7 @@ abstract class ClusterDeathWatchSpec
enterBarrier("after-2")
}
"be able to watch actor before node joins cluster, ClusterRemoteWatcher takes over from RemoteWatcher" taggedAs LongRunningTest in within(20 seconds) {
"be able to watch actor before node joins cluster, ClusterRemoteWatcher takes over from RemoteWatcher" in within(20 seconds) {
runOn(fifth) {
system.actorOf(Props(new Actor { def receive = Actor.emptyBehavior }).withDeploy(Deploy.local), name = "subject5")
}
@ -210,7 +210,7 @@ abstract class ClusterDeathWatchSpec
enterBarrier("after-3")
}
"be able to shutdown system when using remote deployed actor on node that crash" taggedAs LongRunningTest in within(20 seconds) {
"be able to shutdown system when using remote deployed actor on node that crash" in within(20 seconds) {
// fourth actor system will be shutdown, not part of testConductor any more
// so we can't use barriers to synchronize with it
val firstAddress = address(first)

View file

@ -32,7 +32,7 @@ abstract class NodeUpSpec
import ClusterEvent._
"A cluster node that is joining another cluster" must {
"not be able to join a node that is not a cluster member" taggedAs LongRunningTest in {
"not be able to join a node that is not a cluster member" in {
runOn(first) {
cluster.join(second)
@ -45,12 +45,12 @@ abstract class NodeUpSpec
enterBarrier("after-0")
}
"be moved to UP by the leader after a convergence" taggedAs LongRunningTest in {
"be moved to UP by the leader after a convergence" in {
awaitClusterUp(first, second)
enterBarrier("after-1")
}
"be unaffected when joining again" taggedAs LongRunningTest in {
"be unaffected when joining again" in {
val unexpected = new AtomicReference[SortedSet[Member]](SortedSet.empty)
cluster.subscribe(system.actorOf(Props(new Actor {

View file

@ -132,12 +132,12 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
Await.result(router ? GetRoutees, timeout.duration).asInstanceOf[Routees].routees
"A cluster router with a RoundRobin router" must {
"start cluster with 2 nodes" taggedAs LongRunningTest in {
"start cluster with 2 nodes" in {
awaitClusterUp(first, second)
enterBarrier("after-1")
}
"deploy routees to the member nodes in the cluster" taggedAs LongRunningTest in {
"deploy routees to the member nodes in the cluster" in {
runOn(first) {
router1.isInstanceOf[RoutedActorRef] should ===(true)
@ -162,7 +162,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
enterBarrier("after-2")
}
"lookup routees on the member nodes in the cluster" taggedAs LongRunningTest in {
"lookup routees on the member nodes in the cluster" in {
// cluster consists of first and second
@ -193,7 +193,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
enterBarrier("after-3")
}
"deploy routees to new nodes in the cluster" taggedAs LongRunningTest in {
"deploy routees to new nodes in the cluster" in {
// add third and fourth
awaitClusterUp(first, second, third, fourth)
@ -216,7 +216,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
enterBarrier("after-4")
}
"lookup routees on new nodes in the cluster" taggedAs LongRunningTest in {
"lookup routees on new nodes in the cluster" in {
// cluster consists of first, second, third and fourth
@ -238,7 +238,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
enterBarrier("after-5")
}
"deploy routees to only remote nodes when allow-local-routees = off" taggedAs LongRunningTest in {
"deploy routees to only remote nodes when allow-local-routees = off" in {
runOn(first) {
// max-nr-of-instances-per-node=1 times 3 nodes
@ -261,7 +261,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
enterBarrier("after-6")
}
"deploy routees to specified node role" taggedAs LongRunningTest in {
"deploy routees to specified node role" in {
runOn(first) {
awaitAssert(currentRoutees(router5).size should ===(2))
@ -283,7 +283,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
enterBarrier("after-7")
}
"deploy programatically defined routees to the member nodes in the cluster" taggedAs LongRunningTest in {
"deploy programatically defined routees to the member nodes in the cluster" in {
runOn(first) {
router2.isInstanceOf[RoutedActorRef] should ===(true)
@ -309,7 +309,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
enterBarrier("after-8")
}
"remove routees for unreachable nodes, and add when reachable again" taggedAs LongRunningTest in within(30.seconds) {
"remove routees for unreachable nodes, and add when reachable again" in within(30.seconds) {
// myservice is already running
@ -334,7 +334,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
enterBarrier("after-9")
}
"deploy programatically defined routees to other node when a node becomes down" taggedAs LongRunningTest in {
"deploy programatically defined routees to other node when a node becomes down" in {
muteMarkingAsUnreachable()
runOn(first) {

View file

@ -21,3 +21,5 @@ MimaKeys.previousArtifact := akkaPreviousArtifact("akka-persistence-experimental
fork in Test := true
javaOptions in Test := MultiNode.defaultMultiJvmOptions

View file

@ -56,7 +56,7 @@ class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec)
override def verifySystemShutdown = true
"A new remote actor" must {
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" in {
runOn(master) {
val actor = system.actorOf(Props[SomeActor], "service-hello")
@ -71,7 +71,7 @@ class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec)
enterBarrier("done")
}
"be locally instantiated on a remote node (with null parameter) and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
"be locally instantiated on a remote node (with null parameter) and be able to communicate through its RemoteActorRef" in {
runOn(master) {
val actor = system.actorOf(Props(classOf[SomeActorWithParam], null), "service-hello-null")
@ -86,7 +86,7 @@ class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec)
enterBarrier("done")
}
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef (with deployOnAll)" taggedAs LongRunningTest in {
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef (with deployOnAll)" in {
runOn(master) {
val actor = system.actorOf(Props[SomeActor], "service-hello2")
@ -101,7 +101,7 @@ class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec)
enterBarrier("done")
}
"be able to shutdown system when using remote deployed actor" taggedAs LongRunningTest in within(20 seconds) {
"be able to shutdown system when using remote deployed actor" in within(20 seconds) {
runOn(master) {
val actor = system.actorOf(Props[SomeActor], "service-hello3")
actor.isInstanceOf[RemoteActorRef] should ===(true)

View file

@ -54,7 +54,7 @@ abstract class RemoteDeliverySpec
"Remoting with TCP" must {
"not drop messages under normal circumstances" taggedAs LongRunningTest in {
"not drop messages under normal circumstances" in {
system.actorOf(Props[Postman], "postman-" + myself.name)
enterBarrier("actors-started")

View file

@ -111,7 +111,7 @@ abstract class RemoteNodeDeathWatchSpec
"RemoteNodeDeathWatch (" + scenario + ")" must {
"receive Terminated when remote actor is stopped" taggedAs LongRunningTest in {
"receive Terminated when remote actor is stopped" in {
runOn(first) {
val watcher = system.actorOf(Props(classOf[ProbeActor], testActor), "watcher1")
enterBarrier("actors-started-1")
@ -155,7 +155,7 @@ abstract class RemoteNodeDeathWatchSpec
enterBarrier("after-1")
}
"cleanup after watch/unwatch" taggedAs LongRunningTest in {
"cleanup after watch/unwatch" in {
runOn(first) {
val watcher = system.actorOf(Props(classOf[ProbeActor], testActor), "watcher2")
enterBarrier("actors-started-2")
@ -188,7 +188,7 @@ abstract class RemoteNodeDeathWatchSpec
enterBarrier("after-2")
}
"cleanup after bi-directional watch/unwatch" taggedAs LongRunningTest in {
"cleanup after bi-directional watch/unwatch" in {
runOn(first, second) {
val watcher = system.actorOf(Props(classOf[ProbeActor], testActor), "watcher3")
system.actorOf(Props(classOf[ProbeActor], testActor), "subject3")
@ -220,7 +220,7 @@ abstract class RemoteNodeDeathWatchSpec
enterBarrier("after-3")
}
"cleanup after bi-directional watch/stop/unwatch" taggedAs LongRunningTest in {
"cleanup after bi-directional watch/stop/unwatch" in {
runOn(first, second) {
val watcher1 = system.actorOf(Props(classOf[ProbeActor], testActor), "w1")
val watcher2 = system.actorOf(Props(classOf[ProbeActor], testActor), "w2")
@ -267,7 +267,7 @@ abstract class RemoteNodeDeathWatchSpec
enterBarrier("after-4")
}
"cleanup after stop" taggedAs LongRunningTest in {
"cleanup after stop" in {
runOn(first) {
val p1, p2, p3 = TestProbe()
val a1 = system.actorOf(Props(classOf[ProbeActor], p1.ref), "a1")
@ -359,7 +359,7 @@ abstract class RemoteNodeDeathWatchSpec
enterBarrier("after-5")
}
"receive Terminated when watched node crash" taggedAs LongRunningTest in {
"receive Terminated when watched node crash" in {
runOn(first) {
val watcher = system.actorOf(Props(classOf[ProbeActor], testActor), "watcher6")
val watcher2 = system.actorOf(Props(classOf[ProbeActor], system.deadLetters))
@ -407,7 +407,7 @@ abstract class RemoteNodeDeathWatchSpec
enterBarrier("after-6")
}
"cleanup when watching node crash" taggedAs LongRunningTest in {
"cleanup when watching node crash" in {
runOn(third) {
val watcher = system.actorOf(Props(classOf[ProbeActor], testActor), "watcher7")
enterBarrier("actors-started-7")

View file

@ -79,7 +79,7 @@ class RemoteRoundRobinSpec extends MultiNodeSpec(RemoteRoundRobinMultiJvmSpec)
def initialParticipants = roles.size
"A remote round robin pool" must {
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" in {
runOn(first, second, third) {
enterBarrier("start", "broadcast-end", "end")
@ -120,7 +120,7 @@ class RemoteRoundRobinSpec extends MultiNodeSpec(RemoteRoundRobinMultiJvmSpec)
}
"A remote round robin pool with resizer" must {
"be locally instantiated on a remote node after several resize rounds" taggedAs LongRunningTest in within(5 seconds) {
"be locally instantiated on a remote node after several resize rounds" in within(5 seconds) {
runOn(first, second, third) {
enterBarrier("start", "broadcast-end", "end")
@ -163,7 +163,7 @@ class RemoteRoundRobinSpec extends MultiNodeSpec(RemoteRoundRobinMultiJvmSpec)
}
"A remote round robin group" must {
"send messages with actor selection to remote paths" taggedAs LongRunningTest in {
"send messages with actor selection to remote paths" in {
runOn(first, second, third) {
system.actorOf(Props[SomeActor], name = "target-" + myself.name)

View file

@ -1,6 +1,6 @@
name := "akka-sample-persistence-scala"
version := "2.3-SNAPSHOT"
version := "2.4-SNAPSHOT"
scalaVersion := "2.11.5"

View file

@ -1 +1,3 @@
enablePlugins(akka.RootSettings)
net.virtualvoid.sbt.graph.Plugin.graphSettings

View file

@ -29,6 +29,8 @@ object AkkaBuild extends Build {
val enableMiMa = false
val parallelExecutionByDefault = false // TODO: enable this once we're sure it doesn not break things
lazy val buildSettings = Seq(
organization := "com.typesafe.akka",
version := "2.4-SNAPSHOT",
@ -42,7 +44,7 @@ object AkkaBuild extends Build {
settings = parentSettings ++ Release.settings ++ unidocSettings ++
SphinxDoc.akkaSettings ++ Dist.settings ++ s3Settings ++ scaladocSettings ++
GraphiteBuildEvents.settings ++ Protobuf.settings ++ Unidoc.settings(Seq(samples), Seq(remoteTests)) ++ Seq(
parallelExecution in GlobalScope := System.getProperty("akka.parallelExecution", "false").toBoolean,
parallelExecution in GlobalScope := System.getProperty("akka.parallelExecution", parallelExecutionByDefault.toString).toBoolean,
Dist.distExclude := Seq(actorTests.id, docs.id, samples.id, osgi.id),
S3.host in S3.upload := "downloads.typesafe.com.s3.amazonaws.com",
@ -327,7 +329,7 @@ object AkkaBuild extends Build {
* Test settings
*/
parallelExecution in Test := System.getProperty("akka.parallelExecution", "false").toBoolean,
parallelExecution in Test := System.getProperty("akka.parallelExecution", parallelExecutionByDefault.toString).toBoolean,
logBuffered in Test := System.getProperty("akka.logBufferedTests", "false").toBoolean,
// show full stack traces and test case durations

View file

@ -1,6 +1,11 @@
package akka
object GitHub {
def envTokenOrThrow: String =
sys.env.getOrElse("PR_VALIDATOR_GH_TOKEN",
throw new Exception("No PR_VALIDATOR_GH_TOKEN env var provided, unable to reach github!"))
def url(v: String): String = {
val branch = if (v.endsWith("SNAPSHOT")) "master" else "v" + v
"http://github.com/akka/akka/tree/" + branch

View file

@ -65,7 +65,7 @@ object Sample {
MiMa.projectSettings ++
Publish.projectSettings ++
ValidatePullRequest.projectSettings
): _*)
): _*).configs(ValidatePullRequest.ValidatePR)
private implicit class RichLoadedDefinitions(ld: LoadedDefinitions) {
def copy(projects: Seq[Project]) =

View file

@ -3,19 +3,171 @@
*/
package akka
import sbt._
import sbt.Keys._
import com.typesafe.tools.mima.plugin.MimaKeys.reportBinaryIssues
import net.virtualvoid.sbt.graph.IvyGraphMLDependencies
import net.virtualvoid.sbt.graph.IvyGraphMLDependencies.ModuleId
import org.kohsuke.github._
import sbt.Keys._
import sbt._
import scala.collection.immutable
import scala.util.matching.Regex
object ValidatePullRequest extends AutoPlugin {
val ValidatePR = config("pr-validation") extend Test
override lazy val projectConfigurations = Seq(ValidatePR)
/*
Assumptions:
Env variables set "by Jenkins" are assumed to come from this plugin:
https://wiki.jenkins-ci.org/display/JENKINS/GitHub+pull+request+builder+plugin
*/
// settings
val PullIdEnvVarName = "ghprbPullId" // Set by "GitHub pull request builder plugin"
val TargetBranchEnvVarName = "PR_TARGET_BRANCH"
val TargetBranchJenkinsEnvVarName = "ghprbTargetBranch"
val targetBranch = settingKey[String]("Branch with which the PR changes should be diffed against")
val SourceBranchEnvVarName = "PR_SOURCE_BRANCH"
val SourcePullIdJenkinsEnvVarName = "ghprbPullId" // used to obtain branch name in form of "pullreq/17397"
val sourceBranch = settingKey[String]("Branch containing the changes of this PR")
// asking github comments if this PR should be PLS BUILD ALL
val githubEnforcedBuildAll = taskKey[Boolean]("Checks via GitHub API if comments included the PLS BUILD ALL keyword")
val buildAllKeyword = taskKey[Regex]("Magic phrase to be used to trigger building of the entire project instead of analysing dependencies")
// determining touched dirs and projects
val changedDirectories = taskKey[immutable.Set[String]]("List of touched modules in this PR branch")
val projectIsAffectedByChanges = taskKey[Boolean]("True if this project is affected by the PR and should be rebuilt")
// running validation
val validatePullRequest = taskKey[Unit]("Additional tasks for pull request validation")
override def trigger = allRequirements
override lazy val projectSettings = Seq(
validatePullRequest := (),
validatePullRequest <<= validatePullRequest.dependsOn(test in Test),
override lazy val projectSettings = inConfig(ValidatePR)(Defaults.testTasks) ++ Seq(
testOptions in ValidatePR += Tests.Argument("-l akka.testkit.PerformanceTest"),
testOptions in ValidatePR += Tests.Argument("-l akka.testkit.LongRunningTest"),
testOptions in ValidatePR += Tests.Argument("-l akka.testkit.TimingTest"),
targetBranch in ValidatePR := {
sys.env.get(TargetBranchEnvVarName) orElse
sys.env.get(TargetBranchJenkinsEnvVarName) getOrElse // Set by "GitHub pull request builder plugin"
"master"
},
sourceBranch in ValidatePR := {
sys.env.get(SourceBranchEnvVarName) orElse
sys.env.get(SourcePullIdJenkinsEnvVarName).map("pullreq/" + _) getOrElse // Set by "GitHub pull request builder plugin"
"HEAD"
},
changedDirectories in ValidatePR := {
val log = streams.value.log
val targetId = (targetBranch in ValidatePR).value
val prId = (sourceBranch in ValidatePR).value
// TODO could use jgit
log.info(s"Comparing [$targetId] with [$prId] to determine changed modules in PR...")
val gitOutput = "git diff %s..%s --name-only".format(targetId, prId).!!.split("\n")
val moduleNames =
gitOutput
.map(l l.trim.takeWhile(_ != '/'))
.filter(_ startsWith "akka-")
.toSet
log.info("Detected changes in directories: " + moduleNames.mkString("[", ", ", "]"))
moduleNames
},
buildAllKeyword in ValidatePR := """PLS BUILD ALL""".r,
githubEnforcedBuildAll in ValidatePR := {
sys.env.get(PullIdEnvVarName).map(_.toInt) match {
case None => false // can't ask github if no issue ID given
case Some(prId) =>
val log = streams.value.log
val buildAllMagicPhrase = (buildAllKeyword in ValidatePR).value
log.info("Checking GitHub comments for PR validation options...")
try {
import scala.collection.JavaConverters._
val gh = GitHubBuilder.fromEnvironment().withOAuthToken(GitHub.envTokenOrThrow).build()
val comments = gh.getRepository("akka/akka").getIssue(prId).getComments.asScala
comments exists { c =>
val triggersBuildAll = buildAllMagicPhrase.findFirstIn(c.getBody).isDefined
if (triggersBuildAll)
log.info(s"GitHub PR comment [ ${c.getUrl} ] contains [$buildAllMagicPhrase], forcing BUILD ALL mode!")
triggersBuildAll
}
} catch {
case ex: Exception =>
log.warn("Unable to reach GitHub! Exception was: " + ex.getMessage)
false
}
}
},
projectIsAffectedByChanges in ValidatePR := (githubEnforcedBuildAll in ValidatePR).value || {
val log = streams.value.log
log.debug(s"Analysing project (for inclusion in PR validation): [${name.value}]")
// if in any scope, any of the changed modules is within this projects dependencies, we must test it:
val shouldBeBuilt = (changedDirectories in ValidatePR).value.exists { modifiedProject
Set(Compile, Test, Runtime, Provided, Optional) exists { ivyScope: sbt.Configuration
log.debug(s"Analysing [$ivyScope] scoped dependencies...")
def moduleId(artifactName: String) = ModuleId("com.typesafe.akka", artifactName + "_" + scalaBinaryVersion.value, version.value)
val modifiedModuleIds = Set(moduleId(modifiedProject), moduleId(modifiedProject + "-experimental"))
def resolutionFilename(includeScalaVersion: Boolean) =
s"%s-%s-%s.xml".format(
organization.value,
name.value + (if (includeScalaVersion) "_" + scalaBinaryVersion.value else ""),
ivyScope.toString())
def resolutionFile(includeScalaVersion: Boolean) =
target.value / "resolution-cache" / "reports" / resolutionFilename(includeScalaVersion)
val ivyReportFile = {
val f1 = resolutionFile(includeScalaVersion = true)
val f2 = resolutionFile(includeScalaVersion = false)
if (f1.exists()) f1 else f2
}
val deps = IvyGraphMLDependencies.graph(ivyReportFile.getAbsolutePath)
deps.nodes.foreach { m log.debug(" -> " + m.id) }
// if this project depends on a modified module, we must test it
deps.nodes.exists { m =>
val depends = modifiedModuleIds exists { _.name == m.id.name } // match just by name, we'd rather include too much than too little
if (depends) log.info(s"Project [${name.value}] must be verified, because depends on [${modifiedModuleIds.find(_ == m.id).get}]")
depends
}
}
}
shouldBeBuilt
},
validatePullRequest := Def.taskDyn {
val log = streams.value.log
val theVoid = Def.task { () } // when you stare into the void, the void stares back at you
if ((projectIsAffectedByChanges in ValidatePR).value) {
log.info(s"Changes in PR are affecting project [${name.value}] - proceeding with test:test")
theVoid.dependsOn(test in ValidatePR)
} else {
log.info(s"Skipping validation of [${name.value}], as PR does NOT affect this project...")
theVoid
}
}.value,
// add reportBinaryIssues to validatePullRequest on minor version maintenance branch
validatePullRequest <<= validatePullRequest.dependsOn(reportBinaryIssues)

View file

@ -29,3 +29,8 @@ addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.1.6")
libraryDependencies += "com.timgroup" % "java-statsd-client" % "2.0.0"
addSbtPlugin("com.typesafe.sbt" % "sbt-native-packager" % "1.0.0-RC1")
// for advanced PR validation features
addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4")
libraryDependencies += "org.kohsuke" % "github-api" % "1.68"