Merge pull request #1631 from sbt/wip/cached-resolution

Cached resolution (minigraph caching)
This commit is contained in:
Josh Suereth 2014-10-01 07:40:57 -04:00
commit cf6be71e00
19 changed files with 635 additions and 182 deletions

View File

@ -4,7 +4,7 @@
package sbt
import Resolver.PluginPattern
import ivyint.{ ConsolidatedResolveEngine, ConsolidatedResolveCache }
import ivyint.{ CachedResolutionResolveEngine, CachedResolutionResolveCache }
import java.io.File
import java.net.URI
@ -84,7 +84,7 @@ final class IvySbt(val configuration: IvyConfiguration) {
}
is
}
private lazy val ivy: Ivy =
private[sbt] def mkIvy: Ivy =
{
val i = new Ivy() {
private val loggerEngine = new SbtMessageLoggerEngine
@ -94,10 +94,11 @@ final class IvySbt(val configuration: IvyConfiguration) {
// We inject the deps we need before we can hook our resolve engine.
setSortEngine(new SortEngine(getSettings))
setEventManager(new EventManager())
if (configuration.updateOptions.consolidatedResolution) {
setResolveEngine(new ResolveEngine(getSettings, getEventManager, getSortEngine) with ConsolidatedResolveEngine {
val consolidatedResolveCache = IvySbt.consolidatedResolveCache
if (configuration.updateOptions.cachedResolution) {
setResolveEngine(new ResolveEngine(getSettings, getEventManager, getSortEngine) with CachedResolutionResolveEngine {
val cachedResolutionResolveCache = IvySbt.cachedResolutionResolveCache
val projectResolver = prOpt
def makeInstance = mkIvy
})
} else setResolveEngine(new ResolveEngine(getSettings, getEventManager, getSortEngine))
super.bind()
@ -108,6 +109,8 @@ final class IvySbt(val configuration: IvyConfiguration) {
i.getLoggerEngine.pushLogger(new IvyLoggerInterface(configuration.log))
i
}
private lazy val ivy: Ivy = mkIvy
// Must be the same file as is used in Update in the launcher
private lazy val ivyLockFile = new File(settings.getDefaultIvyUserDir, ".sbt.ivy.lock")
/** ========== End Configuration/Setup ============*/
@ -130,14 +133,14 @@ final class IvySbt(val configuration: IvyConfiguration) {
}
/**
* Cleans consolidated resolution cache.
* Cleans cached resolution cache.
* @param md - module descriptor of the original Ivy graph.
*/
private[sbt] def cleanConsolidatedResolutionCache(md: ModuleDescriptor, log: Logger): Unit =
private[sbt] def cleanCachedResolutionCache(md: ModuleDescriptor, log: Logger): Unit =
withIvy(log) { i =>
val prOpt = Option(i.getSettings.getResolver(ProjectResolver.InterProject)) map { case pr: ProjectResolver => pr }
if (configuration.updateOptions.consolidatedResolution) {
IvySbt.consolidatedResolveCache.clean(md, prOpt)
if (configuration.updateOptions.cachedResolution) {
IvySbt.cachedResolutionResolveCache.clean(md, prOpt)
}
}
@ -242,7 +245,7 @@ private[sbt] object IvySbt {
val DefaultIvyFilename = "ivy.xml"
val DefaultMavenFilename = "pom.xml"
val DefaultChecksums = Seq("sha1", "md5")
private[sbt] val consolidatedResolveCache: ConsolidatedResolveCache = new ConsolidatedResolveCache()
private[sbt] val cachedResolutionResolveCache: CachedResolutionResolveCache = new CachedResolutionResolveCache()
def defaultIvyFile(project: File) = new File(project, DefaultIvyFilename)
def defaultIvyConfiguration(project: File) = new File(project, DefaultIvyConfigFilename)
@ -270,6 +273,10 @@ private[sbt] object IvySbt {
val mainChain = makeChain("Default", "sbt-chain", resolvers)
settings.setDefaultResolver(mainChain.getName)
}
private[sbt] def isChanging(dd: DependencyDescriptor): Boolean =
dd.isChanging || isChanging(dd.getDependencyRevisionId)
private[sbt] def isChanging(module: ModuleID): Boolean =
module.revision endsWith "-SNAPSHOT"
private[sbt] def isChanging(mrid: ModuleRevisionId): Boolean =
mrid.getRevision endsWith "-SNAPSHOT"
def resolverChain(name: String, resolvers: Seq[Resolver], localOnly: Boolean, settings: IvySettings, log: Logger): DependencyResolver =

View File

@ -6,6 +6,7 @@ package sbt
import java.io.File
import scala.xml.{ Node => XNode, NodeSeq }
import collection.mutable
import ivyint.CachedResolutionResolveEngine
import org.apache.ivy.Ivy
import org.apache.ivy.core.{ IvyPatternHelper, LogOptions }
@ -69,12 +70,12 @@ object IvyActions {
}
/**
* Cleans the consolidated resolution cache, if any.
* Cleans the cached resolution cache, if any.
* This is called by clean.
*/
private[sbt] def cleanConsolidatedResolutionCache(module: IvySbt#Module, log: Logger): Unit =
private[sbt] def cleanCachedResolutionCache(module: IvySbt#Module, log: Logger): Unit =
module.withModule(log) { (ivy, md, default) =>
module.owner.cleanConsolidatedResolutionCache(md, log)
module.owner.cleanCachedResolutionCache(md, log)
}
/** Creates a Maven pom from the given Ivy configuration*/
@ -139,9 +140,9 @@ object IvyActions {
* Resolves and retrieves dependencies. 'ivyConfig' is used to produce an Ivy file and configuration.
* 'updateConfig' configures the actual resolution and retrieval process.
*/
@deprecated("Use updateEither instead.", "0.13.6")
@deprecated("This is no longer public.", "0.13.6")
def update(module: IvySbt#Module, configuration: UpdateConfiguration, log: Logger): UpdateReport =
updateEither(module, configuration, UnresolvedWarningConfiguration(), log) match {
updateEither(module, configuration, UnresolvedWarningConfiguration(), LogicalClock.unknown, None, log) match {
case Right(r) => r
case Left(w) =>
throw w.resolveException
@ -151,9 +152,25 @@ object IvyActions {
* Resolves and retrieves dependencies. 'ivyConfig' is used to produce an Ivy file and configuration.
* 'updateConfig' configures the actual resolution and retrieval process.
*/
def updateEither(module: IvySbt#Module, configuration: UpdateConfiguration,
uwconfig: UnresolvedWarningConfiguration, log: Logger): Either[UnresolvedWarning, UpdateReport] =
private[sbt] def updateEither(module: IvySbt#Module, configuration: UpdateConfiguration,
uwconfig: UnresolvedWarningConfiguration, logicalClock: LogicalClock, depDir: Option[File], log: Logger): Either[UnresolvedWarning, UpdateReport] =
module.withModule(log) {
case (ivy, md, default) if module.owner.configuration.updateOptions.cachedResolution =>
ivy.getResolveEngine match {
case x: CachedResolutionResolveEngine =>
val resolveOptions = new ResolveOptions
val resolveId = ResolveOptions.getDefaultResolveId(md)
resolveOptions.setResolveId(resolveId)
x.customResolve(md, logicalClock, resolveOptions, depDir getOrElse { sys.error("dependency base directory is not specified") }, log) match {
case Left(x) =>
Left(UnresolvedWarning(x, uwconfig))
case Right(uReport) =>
configuration.retrieve match {
case Some(rConf) => Right(retrieve(ivy, uReport, rConf))
case None => Right(uReport)
}
}
}
case (ivy, md, default) =>
val (report, err) = resolve(configuration.logging)(ivy, md, default)
err match {

View File

@ -76,8 +76,12 @@ object IvyRetrieve {
val callerConfigurations = caller.getCallerConfigurations.toArray.toVector collect {
case x if nonEmptyString(x).isDefined => x
}
val extraAttributes = toExtraAttributes(caller.getDependencyDescriptor.getExtraAttributes)
new Caller(m, callerConfigurations, extraAttributes)
val ddOpt = Option(caller.getDependencyDescriptor)
val (extraAttributes, isForce, isChanging, isTransitive) = ddOpt match {
case Some(dd) => (toExtraAttributes(dd.getExtraAttributes), dd.isForce, dd.isChanging, dd.isTransitive)
case None => (Map.empty[String, String], false, false, true)
}
new Caller(m, callerConfigurations, extraAttributes, isForce, isChanging, isTransitive)
}
val revId = dep.getResolvedId
val moduleId = toModuleID(revId)
@ -160,19 +164,11 @@ object IvyRetrieve {
*/
def findPath(target: IvyNode, from: ModuleRevisionId): List[IvyNode] = {
def doFindPath(current: IvyNode, path: List[IvyNode]): List[IvyNode] = {
val callers = current.getAllRealCallers.toList
// Ivy actually returns non-direct callers here.
// Ivy actually returns mix of direct and non-direct callers here.
// that's why we have to calculate all possible paths below and pick the longest path.
val directCallers = callers filter { caller =>
val md = caller.getModuleDescriptor
val dd = md.getDependencies.toList find { dd =>
(dd.getDependencyRevisionId == current.getId) &&
(dd.getParentRevisionId == caller.getModuleRevisionId)
}
dd.isDefined
}
val directCallersRevId = (directCallers map { _.getModuleRevisionId }).distinct
val paths: List[List[IvyNode]] = ((directCallersRevId map { revId =>
val callers = current.getAllRealCallers.toList
val callersRevId = (callers map { _.getModuleRevisionId }).distinct
val paths: List[List[IvyNode]] = ((callersRevId map { revId =>
val node = current.findNode(revId)
if (revId == from) node :: path
else if (node == node.getRoot) Nil

View File

@ -0,0 +1,110 @@
package sbt
import java.io.File
import java.net.URL
import org.json4s._
import org.apache.ivy.core
import core.module.descriptor.ModuleDescriptor
private[sbt] object JsonUtil {
def parseUpdateReport(md: ModuleDescriptor, path: File, cachedDescriptor: File, log: Logger): UpdateReport =
{
import org.json4s._
implicit val formats = native.Serialization.formats(NoTypeHints) +
new ConfigurationSerializer +
new ArtifactSerializer +
new FileSerializer
try {
val json = jawn.support.json4s.Parser.parseFromFile(path)
fromLite(json.get.extract[UpdateReportLite], cachedDescriptor)
} catch {
case e: Throwable =>
log.error("Unable to parse mini graph: " + path.toString)
throw e
}
}
def writeUpdateReport(ur: UpdateReport, graphPath: File): Unit =
{
implicit val formats = native.Serialization.formats(NoTypeHints) +
new ConfigurationSerializer +
new ArtifactSerializer +
new FileSerializer
import native.Serialization.write
val str = write(toLite(ur))
IO.write(graphPath, str, IO.utf8)
}
def toLite(ur: UpdateReport): UpdateReportLite =
UpdateReportLite(ur.configurations map { cr =>
ConfigurationReportLite(cr.configuration, cr.details)
})
def fromLite(lite: UpdateReportLite, cachedDescriptor: File): UpdateReport =
{
val stats = new UpdateStats(0L, 0L, 0L, false)
val configReports = lite.configurations map { cr =>
val details = cr.details
val modules = details flatMap {
_.modules filter { mr =>
!mr.evicted && mr.problem.isEmpty
}
}
val evicted = details flatMap {
_.modules filter { mr =>
mr.evicted
}
} map { _.module }
new ConfigurationReport(cr.configuration, modules, details, evicted)
}
new UpdateReport(cachedDescriptor, configReports, stats)
}
}
private[sbt] case class UpdateReportLite(configurations: Seq[ConfigurationReportLite])
private[sbt] case class ConfigurationReportLite(configuration: String, details: Seq[OrganizationArtifactReport])
private[sbt] class FileSerializer extends CustomSerializer[File](format => (
{
case JString(s) => new File(s)
},
{
case x: File => JString(x.toString)
}
))
private[sbt] class ConfigurationSerializer extends CustomSerializer[Configuration](format => (
{
case JString(s) => new Configuration(s)
},
{
case x: Configuration => JString(x.name)
}
))
private[sbt] class ArtifactSerializer extends CustomSerializer[Artifact](format => (
{
case json: JValue =>
implicit val fmt = format
Artifact(
(json \ "name").extract[String],
(json \ "type").extract[String],
(json \ "extension").extract[String],
(json \ "classifier").extract[Option[String]],
(json \ "configurations").extract[List[Configuration]],
(json \ "url").extract[Option[URL]],
(json \ "extraAttributes").extract[Map[String, String]]
)
},
{
case x: Artifact =>
import DefaultJsonFormats.{ OptionWriter, StringWriter, mapWriter }
val optStr = implicitly[Writer[Option[String]]]
val mw = implicitly[Writer[Map[String, String]]]
JObject(JField("name", JString(x.name)) ::
JField("type", JString(x.`type`)) ::
JField("extension", JString(x.extension)) ::
JField("classifier", optStr.write(x.classifier)) ::
JField("configurations", JArray(x.configurations.toList map { x => JString(x.name) })) ::
JField("url", optStr.write(x.url map { _.toString })) ::
JField("extraAttributes", mw.write(x.extraAttributes)) ::
Nil)
}
))

View File

@ -0,0 +1,16 @@
package sbt
/**
* Represents a logical time point for dependency resolution.
* This is used to cache dependencies across subproject resolution which may change over time.
*/
trait LogicalClock {
def toString: String
}
object LogicalClock {
def apply(x: String): LogicalClock = new LogicalClock {
override def toString: String = x
}
def unknown: LogicalClock = apply("unknown")
}

View File

@ -15,22 +15,31 @@ final class UpdateOptions private[sbt] (
/** If set to true, check all resolvers for snapshots. */
val latestSnapshots: Boolean,
/** If set to true, use consolidated resolution. */
val consolidatedResolution: Boolean) {
val consolidatedResolution: Boolean,
/** If set to true, use cached resolution. */
val cachedResolution: Boolean) {
def withCircularDependencyLevel(circularDependencyLevel: CircularDependencyLevel): UpdateOptions =
copy(circularDependencyLevel = circularDependencyLevel)
def withLatestSnapshots(latestSnapshots: Boolean): UpdateOptions =
copy(latestSnapshots = latestSnapshots)
@deprecated("Use withCachedResolution instead.", "0.13.7")
def withConsolidatedResolution(consolidatedResolution: Boolean): UpdateOptions =
copy(consolidatedResolution = consolidatedResolution)
copy(consolidatedResolution = consolidatedResolution,
cachedResolution = consolidatedResolution)
def withCachedResolution(cachedResoluton: Boolean): UpdateOptions =
copy(cachedResolution = cachedResoluton,
consolidatedResolution = cachedResolution)
private[sbt] def copy(
circularDependencyLevel: CircularDependencyLevel = this.circularDependencyLevel,
latestSnapshots: Boolean = this.latestSnapshots,
consolidatedResolution: Boolean = this.consolidatedResolution): UpdateOptions =
consolidatedResolution: Boolean = this.consolidatedResolution,
cachedResolution: Boolean = this.cachedResolution): UpdateOptions =
new UpdateOptions(circularDependencyLevel,
latestSnapshots,
consolidatedResolution)
consolidatedResolution,
cachedResolution)
}
object UpdateOptions {
@ -38,5 +47,6 @@ object UpdateOptions {
new UpdateOptions(
circularDependencyLevel = CircularDependencyLevel.Warn,
latestSnapshots = true,
consolidatedResolution = false)
consolidatedResolution = false,
cachedResolution = false)
}

View File

@ -187,7 +187,10 @@ object ModuleReport {
final class Caller(
val caller: ModuleID,
val callerConfigurations: Seq[String],
val callerExtraAttributes: Map[String, String]) {
val callerExtraAttributes: Map[String, String],
val isForceDependency: Boolean,
val isChangingDependency: Boolean,
val isTransitiveDependency: Boolean) {
override def toString: String =
s"$caller"
}

View File

@ -0,0 +1,314 @@
package sbt
package ivyint
import java.util.Date
import java.net.URL
import java.io.File
import collection.concurrent
import collection.immutable.ListMap
import org.apache.ivy.Ivy
import org.apache.ivy.core
import core.resolve._
import core.module.id.{ ModuleRevisionId, ModuleId => IvyModuleId }
import core.report.{ ResolveReport, ConfigurationResolveReport, DownloadReport }
import core.module.descriptor.{ DefaultModuleDescriptor, ModuleDescriptor, DependencyDescriptor, Configuration => IvyConfiguration }
import core.{ IvyPatternHelper, LogOptions }
import org.apache.ivy.util.Message
import org.apache.ivy.plugins.latest.{ ArtifactInfo => IvyArtifactInfo }
private[sbt] object CachedResolutionResolveCache {
def createID(organization: String, name: String, revision: String) =
ModuleRevisionId.newInstance(organization, name, revision)
def sbtOrgTemp = "org.scala-sbt.temp"
}
private[sbt] class CachedResolutionResolveCache() {
import CachedResolutionResolveCache._
val updateReportCache: concurrent.Map[ModuleRevisionId, Either[ResolveException, UpdateReport]] = concurrent.TrieMap()
val resolveReportCache: concurrent.Map[ModuleRevisionId, ResolveReport] = concurrent.TrieMap()
val resolvePropertiesCache: concurrent.Map[ModuleRevisionId, String] = concurrent.TrieMap()
val directDependencyCache: concurrent.Map[ModuleRevisionId, Vector[DependencyDescriptor]] = concurrent.TrieMap()
val conflictCache: concurrent.Map[(ModuleID, ModuleID), (Vector[ModuleID], Vector[ModuleID], String)] = concurrent.TrieMap()
val maxConflictCacheSize: Int = 10000
def clean(md0: ModuleDescriptor, prOpt: Option[ProjectResolver]): Unit = {
val mrid0 = md0.getModuleRevisionId
val mds =
if (mrid0.getOrganisation == sbtOrgTemp) Vector(md0)
else buildArtificialModuleDescriptors(md0, prOpt) map { _._1 }
mds foreach { md =>
updateReportCache.remove(md.getModuleRevisionId)
directDependencyCache.remove(md.getModuleRevisionId)
}
}
def directDependencies(md0: ModuleDescriptor): Vector[DependencyDescriptor] =
directDependencyCache.getOrElseUpdate(md0.getModuleRevisionId, md0.getDependencies.toVector)
def buildArtificialModuleDescriptors(md0: ModuleDescriptor, prOpt: Option[ProjectResolver]): Vector[(DefaultModuleDescriptor, Boolean)] =
{
def expandInternalDeps(dep: DependencyDescriptor): Vector[DependencyDescriptor] =
prOpt map {
_.getModuleDescriptor(dep.getDependencyRevisionId) match {
case Some(internal) => directDependencies(internal) flatMap expandInternalDeps
case _ => Vector(dep)
}
} getOrElse Vector(dep)
val expanded = directDependencies(md0) flatMap expandInternalDeps
val rootModuleConfigs = md0.getConfigurations.toVector
expanded map { buildArtificialModuleDescriptor(_, rootModuleConfigs, prOpt) }
}
def buildArtificialModuleDescriptor(dd: DependencyDescriptor, rootModuleConfigs: Vector[IvyConfiguration], prOpt: Option[ProjectResolver]): (DefaultModuleDescriptor, Boolean) =
{
val mrid = dd.getDependencyRevisionId
val confMap = (dd.getModuleConfigurations map { conf =>
conf + "->(" + dd.getDependencyConfigurations(conf).mkString(",") + ")"
})
val depsString = mrid.toString + ";" + confMap.mkString(";")
val sha1 = Hash.toHex(Hash(depsString))
val md1 = new DefaultModuleDescriptor(createID(sbtOrgTemp, "temp-resolve-" + sha1, "1.0"), "release", null, false) with ArtificialModuleDescriptor {
def targetModuleRevisionId: ModuleRevisionId = mrid
}
for {
conf <- rootModuleConfigs
} yield md1.addConfiguration(conf)
md1.addDependency(dd)
(md1, IvySbt.isChanging(dd))
}
def getOrElseUpdateMiniGraph(md: ModuleDescriptor, changing0: Boolean, logicalClock: LogicalClock, miniGraphPath: File, cachedDescriptor: File, log: Logger)(f: => Either[ResolveException, UpdateReport]): Either[ResolveException, UpdateReport] =
{
import Path._
val mrid = md.getResolvedModuleRevisionId
val (pathOrg, pathName, pathRevision) = md match {
case x: ArtificialModuleDescriptor =>
val tmrid = x.targetModuleRevisionId
(tmrid.getOrganisation, tmrid.getName, tmrid.getRevision + "_" + mrid.getName)
case _ =>
(mrid.getOrganisation, mrid.getName, mrid.getRevision)
}
val staticGraphDirectory = miniGraphPath / "static"
val dynamicGraphDirectory = miniGraphPath / "dynamic"
val staticGraphPath = staticGraphDirectory / pathOrg / pathName / pathRevision / "graphs" / "graph.json"
val dynamicGraphPath = dynamicGraphDirectory / logicalClock.toString / pathOrg / pathName / pathRevision / "graphs" / "graph.json"
def loadMiniGraphFromFile: Option[Either[ResolveException, UpdateReport]] =
(if (staticGraphPath.exists) Some(staticGraphPath)
else if (dynamicGraphPath.exists) Some(dynamicGraphPath)
else None) map { path =>
log.debug(s"parsing ${path.getAbsolutePath.toString}")
val ur = JsonUtil.parseUpdateReport(md, path, cachedDescriptor, log)
updateReportCache(md.getModuleRevisionId) = Right(ur)
Right(ur)
}
(updateReportCache.get(mrid) orElse loadMiniGraphFromFile) match {
case Some(result) => result
case None =>
f match {
case Right(ur) =>
val changing = changing0 || (ur.configurations exists { cr =>
cr.details exists { oar =>
oar.modules exists { mr =>
IvySbt.isChanging(mr.module) || (mr.callers exists { _.isChangingDependency })
}
}
})
IO.createDirectory(miniGraphPath)
val gp = if (changing) dynamicGraphPath
else staticGraphPath
log.debug(s"saving minigraph to $gp")
JsonUtil.writeUpdateReport(ur, gp)
// don't cache dynamic graphs in memory.
if (!changing) {
updateReportCache(md.getModuleRevisionId) = Right(ur)
}
Right(ur)
case Left(re) =>
if (!changing0) {
updateReportCache(md.getModuleRevisionId) = Left(re)
}
Left(re)
}
}
}
def getOrElseUpdateConflict(cf0: ModuleID, cf1: ModuleID, conflicts: Vector[ModuleReport])(f: => (Vector[ModuleReport], Vector[ModuleReport], String)): (Vector[ModuleReport], Vector[ModuleReport]) =
{
def reconstructReports(surviving: Vector[ModuleID], evicted: Vector[ModuleID], mgr: String): (Vector[ModuleReport], Vector[ModuleReport]) = {
val moduleIdMap = Map(conflicts map { x => x.module -> x }: _*)
(surviving map moduleIdMap, evicted map moduleIdMap map { _.copy(evicted = true, evictedReason = Some(mgr.toString)) })
}
(conflictCache get ((cf0, cf1))) match {
case Some((surviving, evicted, mgr)) => reconstructReports(surviving, evicted, mgr)
case _ =>
(conflictCache get ((cf1, cf0))) match {
case Some((surviving, evicted, mgr)) => reconstructReports(surviving, evicted, mgr)
case _ =>
val (surviving, evicted, mgr) = f
if (conflictCache.size > maxConflictCacheSize) {
conflictCache.remove(conflictCache.head._1)
}
conflictCache((cf0, cf1)) = (surviving map { _.module }, evicted map { _.module }, mgr)
(surviving, evicted)
}
}
}
}
private[sbt] trait ArtificialModuleDescriptor { self: DefaultModuleDescriptor =>
def targetModuleRevisionId: ModuleRevisionId
}
private[sbt] trait CachedResolutionResolveEngine extends ResolveEngine {
import CachedResolutionResolveCache._
private[sbt] def cachedResolutionResolveCache: CachedResolutionResolveCache
private[sbt] def projectResolver: Option[ProjectResolver]
private[sbt] def makeInstance: Ivy
// Return sbt's UpdateReport.
def customResolve(md0: ModuleDescriptor, logicalClock: LogicalClock, options0: ResolveOptions, depDir: File, log: Logger): Either[ResolveException, UpdateReport] = {
import Path._
val miniGraphPath = depDir / "module"
val cachedDescriptor = getSettings.getResolutionCacheManager.getResolvedIvyFileInCache(md0.getModuleRevisionId)
val cache = cachedResolutionResolveCache
val mds = cache.buildArtificialModuleDescriptors(md0, projectResolver)
def doWork(md: ModuleDescriptor): Either[ResolveException, UpdateReport] =
{
val options1 = new ResolveOptions(options0)
val i = makeInstance
var rr = i.resolve(md, options1)
if (!rr.hasError) Right(IvyRetrieve.updateReport(rr, cachedDescriptor))
else {
val messages = rr.getAllProblemMessages.toArray.map(_.toString).distinct
val failedPaths = ListMap(rr.getUnresolvedDependencies map { node =>
val m = IvyRetrieve.toModuleID(node.getId)
val path = IvyRetrieve.findPath(node, md.getModuleRevisionId) map { x =>
IvyRetrieve.toModuleID(x.getId)
}
log.debug("- Unresolved path " + path.toString)
m -> path
}: _*)
val failed = failedPaths.keys.toSeq
Left(new ResolveException(messages, failed, failedPaths))
}
}
val results = mds map {
case (md, changing) =>
cache.getOrElseUpdateMiniGraph(md, changing, logicalClock, miniGraphPath, cachedDescriptor, log) {
doWork(md)
}
}
val uReport = mergeResults(md0, results, log)
val cacheManager = getSettings.getResolutionCacheManager
cacheManager.saveResolvedModuleDescriptor(md0)
val prop0 = ""
val ivyPropertiesInCache0 = cacheManager.getResolvedIvyPropertiesInCache(md0.getResolvedModuleRevisionId)
IO.write(ivyPropertiesInCache0, prop0)
uReport
}
def mergeResults(md0: ModuleDescriptor, results: Vector[Either[ResolveException, UpdateReport]], log: Logger): Either[ResolveException, UpdateReport] =
if (results exists { _.isLeft }) Left(mergeErrors(md0, results collect { case Left(re) => re }, log))
else Right(mergeReports(md0, results collect { case Right(ur) => ur }, log))
def mergeErrors(md0: ModuleDescriptor, errors: Vector[ResolveException], log: Logger): ResolveException =
{
val messages = errors flatMap { _.messages }
val failed = errors flatMap { _.failed }
val failedPaths = errors flatMap {
_.failedPaths.toList map {
case (failed, paths) =>
if (paths.isEmpty) (failed, paths)
else (failed, List(IvyRetrieve.toModuleID(md0.getResolvedModuleRevisionId)) ::: paths.toList.tail)
}
}
new ResolveException(messages, failed, ListMap(failedPaths: _*))
}
def mergeReports(md0: ModuleDescriptor, reports: Vector[UpdateReport], log: Logger): UpdateReport =
{
val cachedDescriptor = getSettings.getResolutionCacheManager.getResolvedIvyFileInCache(md0.getModuleRevisionId)
val rootModuleConfigs = md0.getConfigurations.toVector
val stats = new UpdateStats(0L, 0L, 0L, false)
val configReports = rootModuleConfigs map { conf =>
val crs = reports flatMap { _.configurations filter { _.configuration == conf.getName } }
mergeConfigurationReports(conf.getName, crs, log)
}
new UpdateReport(cachedDescriptor, configReports, stats, Map.empty)
}
def mergeConfigurationReports(rootModuleConf: String, reports: Vector[ConfigurationReport], log: Logger): ConfigurationReport =
{
// get the details right, and the rest could be derived
val details = mergeOrganizationArtifactReports(rootModuleConf, reports flatMap { _.details }, log)
val modules = details flatMap {
_.modules filter { mr =>
!mr.evicted && mr.problem.isEmpty
}
}
val evicted = details flatMap {
_.modules filter { mr =>
mr.evicted
}
} map { _.module }
new ConfigurationReport(rootModuleConf, modules, details, evicted)
}
def mergeOrganizationArtifactReports(rootModuleConf: String, reports0: Vector[OrganizationArtifactReport], log: Logger): Vector[OrganizationArtifactReport] =
(reports0 groupBy { oar => (oar.organization, oar.name) }).toSeq.toVector flatMap {
case ((org, name), xs) =>
if (xs.size < 2) xs
else Vector(new OrganizationArtifactReport(org, name, mergeModuleReports(rootModuleConf, xs flatMap { _.modules }, log)))
}
def mergeModuleReports(rootModuleConf: String, modules: Vector[ModuleReport], log: Logger): Vector[ModuleReport] =
{
val merged = (modules groupBy { m => (m.module.organization, m.module.name, m.module.revision) }).toSeq.toVector flatMap {
case ((org, name, version), xs) =>
if (xs.size < 2) xs
else Vector(xs.head.copy(evicted = xs exists { _.evicted }, callers = xs flatMap { _.callers }))
}
val conflicts = merged filter { m => !m.evicted && m.problem.isEmpty }
if (conflicts.size < 2) merged
else resolveConflict(rootModuleConf, conflicts, log) match {
case (survivor, evicted) =>
survivor ++ evicted ++ (merged filter { m => m.evicted || m.problem.isDefined })
}
}
def resolveConflict(rootModuleConf: String, conflicts: Vector[ModuleReport], log: Logger): (Vector[ModuleReport], Vector[ModuleReport]) =
{
import org.apache.ivy.plugins.conflict.{ NoConflictManager, StrictConflictManager, LatestConflictManager }
val head = conflicts.head
val organization = head.module.organization
val name = head.module.name
log.debug(s"- conflict in $rootModuleConf:$organization:$name " + (conflicts map { _.module }).mkString("(", ", ", ")"))
def useLatest(lcm: LatestConflictManager): (Vector[ModuleReport], Vector[ModuleReport], String) =
conflicts find { m =>
m.callers.exists { _.isForceDependency }
} match {
case Some(m) =>
(Vector(m), conflicts filterNot { _ == m } map { _.copy(evicted = true, evictedReason = Some(lcm.toString)) }, lcm.toString)
case None =>
val strategy = lcm.getStrategy
val infos = conflicts map { ModuleReportArtifactInfo(_) }
Option(strategy.findLatest(infos.toArray, None.orNull)) match {
case Some(ModuleReportArtifactInfo(m)) =>
(Vector(m), conflicts filterNot { _ == m } map { _.copy(evicted = true, evictedReason = Some(lcm.toString)) }, lcm.toString)
case _ => (conflicts, Vector(), lcm.toString)
}
}
def doResolveConflict: (Vector[ModuleReport], Vector[ModuleReport], String) =
getSettings.getConflictManager(IvyModuleId.newInstance(organization, name)) match {
case ncm: NoConflictManager => (conflicts, Vector(), ncm.toString)
case _: StrictConflictManager => sys.error((s"conflict was found in $rootModuleConf:$organization:$name " + (conflicts map { _.module }).mkString("(", ", ", ")")))
case lcm: LatestConflictManager => useLatest(lcm)
case conflictManager => sys.error(s"Unsupported conflict manager $conflictManager")
}
if (conflicts.size == 2) {
val (cf0, cf1) = (conflicts(0).module, conflicts(1).module)
val cache = cachedResolutionResolveCache
cache.getOrElseUpdateConflict(cf0, cf1, conflicts) { doResolveConflict }
} else {
val (surviving, evicted, mgr) = doResolveConflict
(surviving, evicted)
}
}
}
private[sbt] case class ModuleReportArtifactInfo(moduleReport: ModuleReport) extends IvyArtifactInfo {
override def getLastModified: Long = moduleReport.publicationDate map { _.getTime } getOrElse 0L
override def getRevision: String = moduleReport.module.revision
}

View File

@ -1,120 +0,0 @@
package sbt
package ivyint
import java.io.File
import collection.concurrent
import org.apache.ivy.core
import core.resolve._
import core.module.id.ModuleRevisionId
import core.report.ResolveReport
import core.module.descriptor.{ DefaultModuleDescriptor, ModuleDescriptor, DependencyDescriptor }
import core.{ IvyPatternHelper, LogOptions }
import org.apache.ivy.util.Message
private[sbt] object ConsolidatedResolveCache {
def createID(organization: String, name: String, revision: String) =
ModuleRevisionId.newInstance(organization, name, revision)
def sbtOrgTemp = "org.scala-sbt.temp"
}
private[sbt] class ConsolidatedResolveCache() {
import ConsolidatedResolveCache._
val resolveReportCache: concurrent.Map[ModuleRevisionId, ResolveReport] = concurrent.TrieMap()
val resolvePropertiesCache: concurrent.Map[ModuleRevisionId, String] = concurrent.TrieMap()
val directDependencyCache: concurrent.Map[ModuleDescriptor, Vector[DependencyDescriptor]] = concurrent.TrieMap()
def clean(md0: ModuleDescriptor, prOpt: Option[ProjectResolver]): Unit = {
val mrid0 = md0.getModuleRevisionId
val md1 = if (mrid0.getOrganisation == sbtOrgTemp) md0
else buildConsolidatedModuleDescriptor(md0, prOpt)
val mrid1 = md1.getModuleRevisionId
resolveReportCache.remove(mrid1)
resolvePropertiesCache.remove(mrid1)
}
def directDependencies(md0: ModuleDescriptor): Vector[DependencyDescriptor] =
directDependencyCache.getOrElseUpdate(md0, md0.getDependencies.toVector)
def buildConsolidatedModuleDescriptor(md0: ModuleDescriptor, prOpt: Option[ProjectResolver]): DefaultModuleDescriptor = {
def expandInternalDeps(dep: DependencyDescriptor): Vector[DependencyDescriptor] =
prOpt map {
_.getModuleDescriptor(dep.getDependencyRevisionId) match {
case Some(internal) => directDependencies(internal) flatMap expandInternalDeps
case _ => Vector(dep)
}
} getOrElse Vector(dep)
val expanded = directDependencies(md0) flatMap expandInternalDeps
val depStrings = expanded map { dep =>
val mrid = dep.getDependencyRevisionId
val confMap = (dep.getModuleConfigurations map { conf =>
conf + "->(" + dep.getDependencyConfigurations(conf).mkString(",") + ")"
})
mrid.toString + ";" + confMap.mkString(";")
}
val depsString = depStrings.distinct.sorted.mkString("\n")
val sha1 = Hash.toHex(Hash(depsString))
// println("sha1: " + sha1)
val md1 = new DefaultModuleDescriptor(createID(sbtOrgTemp, "temp-resolve-" + sha1, "1.0"), "release", null, false)
md1
}
}
private[sbt] trait ConsolidatedResolveEngine extends ResolveEngine {
import ConsolidatedResolveCache._
private[sbt] def consolidatedResolveCache: ConsolidatedResolveCache
private[sbt] def projectResolver: Option[ProjectResolver]
/**
* Resolve dependencies of a module described by a module descriptor.
*/
override def resolve(md0: ModuleDescriptor, options0: ResolveOptions): ResolveReport = {
val cache = consolidatedResolveCache
val cacheManager = getSettings.getResolutionCacheManager
val md1 = cache.buildConsolidatedModuleDescriptor(md0, projectResolver)
val md1mrid = md1.getModuleRevisionId
def doWork: (ResolveReport, String) = {
if (options0.getLog != LogOptions.LOG_QUIET) {
Message.info("Consolidating managed dependencies to " + md1mrid.toString + " ...")
}
md1.setLastModified(System.currentTimeMillis)
for {
x <- md0.getConfigurations
} yield md1.addConfiguration(x)
for {
x <- md0.getDependencies
} yield md1.addDependency(x)
val options1 = new ResolveOptions(options0)
options1.setOutputReport(false)
val report0 = super.resolve(md1, options1)
val ivyPropertiesInCache1 = cacheManager.getResolvedIvyPropertiesInCache(md1.getResolvedModuleRevisionId)
val prop0 =
if (ivyPropertiesInCache1.exists) IO.read(ivyPropertiesInCache1)
else ""
if (options0.isOutputReport) {
this.outputReport(report0, cacheManager, options0)
}
cache.resolveReportCache(md1mrid) = report0
cache.resolvePropertiesCache(md1mrid) = prop0
(report0, prop0)
}
val (report0, prop0) = (cache.resolveReportCache.get(md1mrid), cache.resolvePropertiesCache.get(md1mrid)) match {
case (Some(report), Some(prop)) =>
if (options0.getLog != LogOptions.LOG_QUIET) {
Message.info("Found consolidated dependency " + md1mrid.toString + " ...")
}
(report, prop)
case _ => doWork
}
cacheManager.saveResolvedModuleDescriptor(md0)
if (prop0 != "") {
val ivyPropertiesInCache0 = cacheManager.getResolvedIvyPropertiesInCache(md0.getResolvedModuleRevisionId)
IO.write(ivyPropertiesInCache0, prop0)
}
report0
}
}

View File

@ -8,10 +8,12 @@ import cross.CrossVersionUtil
trait BaseIvySpecification extends Specification {
def currentBase: File = new File(".")
def currentTarget: File = currentBase / "target" / "ivyhome"
def currentManaged: File = currentBase / "target" / "lib_managed"
def currentDependency: File = currentBase / "target" / "dependency"
def defaultModuleId: ModuleID = ModuleID("com.example", "foo", "0.1.0", Some("compile"))
lazy val ivySbt = new IvySbt(mkIvyConfiguration)
lazy val log = Logger.Null
def module(moduleId: ModuleID, deps: Seq[ModuleID], scalaFullVersion: Option[String]): IvySbt#Module = {
lazy val log = ConsoleLogger()
def module(moduleId: ModuleID, deps: Seq[ModuleID], scalaFullVersion: Option[String],
uo: UpdateOptions = UpdateOptions()): IvySbt#Module = {
val ivyScala = scalaFullVersion map { fv =>
new IvyScala(
scalaFullVersion = fv,
@ -28,10 +30,11 @@ trait BaseIvySpecification extends Specification {
dependencies = deps,
configurations = Seq(Compile, Test, Runtime),
ivyScala = ivyScala)
val ivySbt = new IvySbt(mkIvyConfiguration(uo))
new ivySbt.Module(moduleSetting)
}
def mkIvyConfiguration: IvyConfiguration = {
def mkIvyConfiguration(uo: UpdateOptions): IvyConfiguration = {
val paths = new IvyPaths(currentBase, Some(currentTarget))
val rs = Seq(DefaultMavenRepository)
val other = Nil
@ -39,13 +42,20 @@ trait BaseIvySpecification extends Specification {
val off = false
val check = Nil
val resCacheDir = currentTarget / "resolution-cache"
val uo = UpdateOptions()
new InlineIvyConfiguration(paths, rs, other, moduleConfs, off, None, check, Some(resCacheDir), uo, log)
}
def ivyUpdate(module: IvySbt#Module) = {
def ivyUpdateEither(module: IvySbt#Module): Either[UnresolvedWarning, UpdateReport] = {
// IO.delete(currentTarget)
val config = new UpdateConfiguration(None, false, UpdateLogging.Full)
IvyActions.update(module, config, log)
val retrieveConfig = new RetrieveConfiguration(currentManaged, Resolver.defaultRetrievePattern)
val config = new UpdateConfiguration(Some(retrieveConfig), false, UpdateLogging.Full)
IvyActions.updateEither(module, config, UnresolvedWarningConfiguration(), LogicalClock.unknown, Some(currentDependency), log)
}
def ivyUpdate(module: IvySbt#Module) =
ivyUpdateEither(module) match {
case Right(r) => r
case Left(w) =>
throw w.resolveException
}
}

View File

@ -0,0 +1,51 @@
package sbt
import org.specs2._
class CachedResolutionSpec extends BaseIvySpecification {
def is = args(sequential = true) ^ s2"""
This is a specification to check the cached resolution
Resolving the same module twice should
work $e1
Resolving the unsolvable module should
not work $e2
"""
def commonsIo13 = ModuleID("commons-io", "commons-io", "1.3", Some("compile"))
def mavenCayennePlugin302 = ModuleID("org.apache.cayenne.plugins", "maven-cayenne-plugin", "3.0.2", Some("compile"))
def defaultOptions = EvictionWarningOptions.default
import ShowLines._
def e1 = {
val m = module(ModuleID("com.example", "foo", "0.1.0", Some("compile")), Seq(commonsIo13), Some("2.10.2"), UpdateOptions().withCachedResolution(true))
val report = ivyUpdate(m)
val report2 = ivyUpdate(m)
println(report)
println(report.configurations.head.modules.head.artifacts)
report.configurations.size must_== 3
}
def e2 = {
log.setLevel(Level.Debug)
val m = module(ModuleID("com.example", "foo", "0.2.0", Some("compile")), Seq(mavenCayennePlugin302), Some("2.10.2"), UpdateOptions().withCachedResolution(true))
ivyUpdateEither(m) match {
case Right(_) => sys.error("this should've failed")
case Left(uw) =>
println(uw.lines.mkString("\n"))
}
ivyUpdateEither(m) match {
case Right(_) => sys.error("this should've failed 2")
case Left(uw) =>
uw.lines must contain(allOf("\n\tNote: Unresolved dependencies path:",
"\t\tfoundrylogic.vpp:vpp:2.2.1",
"\t\t +- org.apache.cayenne:cayenne-tools:3.0.2",
"\t\t +- org.apache.cayenne.plugins:maven-cayenne-plugin:3.0.2",
"\t\t +- com.example:foo:0.2.0"))
}
}
}

View File

@ -81,7 +81,8 @@ object CacheIvy {
implicit def organizationArtifactReportFormat(implicit sf: Format[String], bf: Format[Boolean], df: Format[Seq[ModuleReport]]): Format[OrganizationArtifactReport] =
wrap[OrganizationArtifactReport, (String, String, Seq[ModuleReport])](m => (m.organization, m.name, m.modules), { case (o, n, r) => OrganizationArtifactReport(o, n, r) })
implicit def callerFormat: Format[Caller] =
wrap[Caller, (ModuleID, Seq[String], Map[String, String])](c => (c.caller, c.callerConfigurations, c.callerExtraAttributes), { case (c, cc, ea) => new Caller(c, cc, ea) })
wrap[Caller, (ModuleID, Seq[String], Map[String, String], Boolean, Boolean, Boolean)](c => (c.caller, c.callerConfigurations, c.callerExtraAttributes, c.isForceDependency, c.isChangingDependency, c.isTransitiveDependency),
{ case (c, cc, ea, fd, cd, td) => new Caller(c, cc, ea, fd, cd, td) })
implicit def exclusionRuleFormat(implicit sf: Format[String]): Format[ExclusionRule] =
wrap[ExclusionRule, (String, String, String, Seq[String])](e => (e.organization, e.name, e.artifact, e.configurations), { case (o, n, a, cs) => ExclusionRule(o, n, a, cs) })
implicit def crossVersionFormat: Format[CrossVersion] = wrap(crossToInt, crossFromInt)

View File

@ -12,6 +12,7 @@ object BuildPaths {
val globalPluginsDirectory = AttributeKey[File]("global-plugins-directory", "The base directory for global sbt plugins.", DSetting)
val globalSettingsDirectory = AttributeKey[File]("global-settings-directory", "The base directory for global sbt settings.", DSetting)
val stagingDirectory = AttributeKey[File]("staging-directory", "The directory for staging remote projects.", DSetting)
val dependencyBaseDirectory = AttributeKey[File]("dependency-base-directory", "The base directory for caching dependency resolution.", DSetting)
import Path._
@ -39,6 +40,9 @@ object BuildPaths {
def getGlobalSettingsDirectory(state: State, globalBase: File): File =
fileSetting(globalSettingsDirectory, GlobalSettingsProperty, globalBase)(state)
def getDependencyDirectory(state: State, globalBase: File): File =
fileSetting(dependencyBaseDirectory, DependencyBaseProperty, defaultDependencyBase(globalBase))(state)
private[this] def fileSetting(stateKey: AttributeKey[File], property: String, default: File)(state: State): File =
getFileSetting(stateKey, property, default)(state)
@ -56,6 +60,7 @@ object BuildPaths {
sbt.cross.CrossVersionUtil.binarySbtVersion(state.configuration.provider.id.version)
private[this] def defaultStaging(globalBase: File) = globalBase / "staging"
private[this] def defaultGlobalPlugins(globalBase: File) = globalBase / PluginsDirectoryName
private[this] def defaultDependencyBase(globalBase: File) = globalBase / "dependency"
def configurationSources(base: File): Seq[File] = (base * (GlobFilter("*.sbt") - ".sbt")).get
def pluginDirectory(definitionBase: File) = definitionBase / PluginsDirectoryName
@ -77,6 +82,7 @@ object BuildPaths {
final val StagingProperty = "sbt.global.staging"
final val GlobalPluginsProperty = "sbt.global.plugins"
final val GlobalSettingsProperty = "sbt.global.settings"
final val DependencyBaseProperty = "sbt.dependency.base"
def crossPath(base: File, instance: xsbti.compile.ScalaInstance): File = base / ("scala_" + instance.version)

View File

@ -222,7 +222,7 @@ object Defaults extends BuildCommon {
crossTarget := makeCrossTarget(target.value, scalaBinaryVersion.value, sbtBinaryVersion.value, sbtPlugin.value, crossPaths.value),
clean := {
val _ = clean.value
IvyActions.cleanConsolidatedResolutionCache(ivyModule.value, streams.value.log)
IvyActions.cleanCachedResolutionCache(ivyModule.value, streams.value.log)
}
)
// must be a val: duplication detected by object identity
@ -1255,6 +1255,8 @@ object Classpaths {
case (Some(res), _, Some(decl)) if res == decl => jars
case _ => Nil
}
def intToByteArray(x: Int): Array[Byte] =
Array((x >>> 24).toByte, (x >> 16 & 0xff).toByte, (x >> 8 & 0xff).toByte, (x & 0xff).toByte)
val subScalaJars: String => Seq[File] = Defaults.unmanagedScalaInstanceOnly.value match {
case Some(si) => subUnmanaged(si.version, si.jars)
case None => sv => if (scalaProvider.version == sv) scalaProvider.jars else Nil
@ -1262,25 +1264,28 @@ object Classpaths {
val transform: UpdateReport => UpdateReport = r => substituteScalaFiles(scalaOrganization.value, r)(subScalaJars)
val uwConfig = (unresolvedWarningConfiguration in update).value
val show = Reference.display(thisProjectRef.value)
val st = state.value
val logicalClock = LogicalClock(Hash.toHex(intToByteArray(st.hashCode)))
val depDir = BuildPaths.getDependencyDirectory(st, BuildPaths.getGlobalBase(st))
cachedUpdate(s.cacheDirectory / updateCacheName.value, show, ivyModule.value, updateConfiguration.value, transform,
skip = (skip in update).value, force = isRoot, depsUpdated = depsUpdated,
uwConfig = uwConfig, log = s.log)
uwConfig = uwConfig, logicalClock = logicalClock, depDir = Some(depDir), log = s.log)
}
@deprecated("Use cachedUpdate with the variant that takes unresolvedHandler instead.", "0.13.6")
def cachedUpdate(cacheFile: File, label: String, module: IvySbt#Module, config: UpdateConfiguration,
transform: UpdateReport => UpdateReport, skip: Boolean, force: Boolean, depsUpdated: Boolean, log: Logger): UpdateReport =
cachedUpdate(cacheFile, label, module, config, transform, skip, force, depsUpdated,
UnresolvedWarningConfiguration(), log)
def cachedUpdate(cacheFile: File, label: String, module: IvySbt#Module, config: UpdateConfiguration,
UnresolvedWarningConfiguration(), LogicalClock.unknown, None, log)
private[sbt] def cachedUpdate(cacheFile: File, label: String, module: IvySbt#Module, config: UpdateConfiguration,
transform: UpdateReport => UpdateReport, skip: Boolean, force: Boolean, depsUpdated: Boolean,
uwConfig: UnresolvedWarningConfiguration, log: Logger): UpdateReport =
uwConfig: UnresolvedWarningConfiguration, logicalClock: LogicalClock, depDir: Option[File], log: Logger): UpdateReport =
{
implicit val updateCache = updateIC
type In = IvyConfiguration :+: ModuleSettings :+: UpdateConfiguration :+: HNil
def work = (_: In) match {
case conf :+: settings :+: config :+: HNil =>
log.info("Updating " + label + "...")
val r = IvyActions.updateEither(module, config, uwConfig, log) match {
val r = IvyActions.updateEither(module, config, uwConfig, logicalClock, depDir, log) match {
case Right(ur) => ur
case Left(uw) =>
import ShowLines._

View File

@ -0,0 +1,16 @@
[@eed3si9n]: https://github.com/eed3si9n
[1631]: https://github.com/sbt/sbt/pull/1631
### cached resolution (minigraph caching)
sbt 0.13.7 adds a new update option called *cached resolution*, which replaces consolidated resolution:
updateOptions := updateOptions.value.withCachedResolution(true)
Unlike consolidated resolution, which only consolidated subprojects with identical dependency graph, cached resolution create an artificial graph for each direct dependency (minigraph) for all subprojects, resolves them independently, saves them into json file, and stiches the minigraphs together.
Once the minigraphs are resolved and saved as files, dependency resolution turns into a matter of loading json file from the second run onwards, which should complete in a matter of seconds even for large projects. Also, because the files are saved under a global `~/.sbt/0.13/dependency` (or what's specified by `sbt.dependency.base` flag), the resolution result is shared across all builds.
Breaking graphs into minigraphs allows partial resolution results to be shared, which scales better for subprojects with similar but slightly different dependencies, and also for making small changes to the dependencies graph over time.
[#1631][1631] by [@eed3si9n][@eed3si9n].

View File

@ -126,7 +126,7 @@ object Sbt extends Build {
/* **** Intermediate-level Modules **** */
// Apache Ivy integration
lazy val ivySub = baseProject(file("ivy"), "Ivy") dependsOn (interfaceSub, launchInterfaceSub, crossSub, logSub % "compile;test->test", ioSub % "compile;test->test", launchSub % "test->test", collectionSub) settings (ivy, jsch, testExclusive)
lazy val ivySub = baseProject(file("ivy"), "Ivy") dependsOn (interfaceSub, launchInterfaceSub, crossSub, logSub % "compile;test->test", ioSub % "compile;test->test", launchSub % "test->test", collectionSub) settings (ivy, jsch, testExclusive, json4sNative, jawnParser, jawnJson4s)
// Runner for uniform test interface
lazy val testingSub = baseProject(file("testing"), "Testing") dependsOn (ioSub, classpathSub, logSub, launchInterfaceSub, testAgentSub) settings (testInterface)
// Testing agent for running tests in a separate process.

View File

@ -178,6 +178,9 @@ object Common {
lazy val httpclient = lib("commons-httpclient" % "commons-httpclient" % "3.1")
lazy val jsch = lib("com.jcraft" % "jsch" % "0.1.46" intransitive ())
lazy val sbinary = libraryDependencies += "org.scala-tools.sbinary" %% "sbinary" % "0.4.2"
lazy val json4sNative = lib("org.json4s" %% "json4s-native" % "3.2.10")
lazy val jawnParser = lib("org.spire-math" %% "jawn-parser" % "0.6.0")
lazy val jawnJson4s = lib("org.spire-math" %% "json4s-support" % "0.6.0")
lazy val scalaCompiler = libraryDependencies <+= scalaVersion(sv => "org.scala-lang" % "scala-compiler" % sv)
lazy val testInterface = lib("org.scala-sbt" % "test-interface" % "1.0")
private def scala211Module(name: String, moduleVersion: String) =

View File

@ -6,10 +6,13 @@ def commonSettings: Seq[Def.Setting[_]] =
libraryDependencies := Seq(
"net.sf.json-lib" % "json-lib" % "2.4" classifier "jdk15" intransitive(),
"com.typesafe.akka" %% "akka-remote" % "2.3.4" exclude("com.typesafe.akka", "akka-actor_2.10"),
"commons-io" % "commons-io" % "1.3"
"net.databinder" %% "unfiltered-uploads" % "0.8.0",
"commons-io" % "commons-io" % "1.3",
"com.typesafe" % "config" % "0.4.9-SNAPSHOT"
),
dependencyOverrides += "commons-io" % "commons-io" % "1.4",
scalaVersion := "2.10.4"
// dependencyOverrides += "commons-io" % "commons-io" % "1.4",
scalaVersion := "2.10.4",
resolvers += Resolver.sonatypeRepo("snapshots")
)
def consolidatedResolutionSettings: Seq[Def.Setting[_]] =
@ -27,9 +30,10 @@ lazy val b = project.
settings(commonSettings: _*)
lazy val c = project.
dependsOn(a).
settings(consolidatedResolutionSettings: _*).
settings(
libraryDependencies := Seq(organization.value %% "a" % version.value)
// libraryDependencies := Seq(organization.value %% "a" % version.value)
)
lazy val root = (project in file(".")).
@ -37,9 +41,13 @@ lazy val root = (project in file(".")).
organization in ThisBuild := "org.example",
version in ThisBuild := "1.0",
check := {
val acp = (externalDependencyClasspath in Compile in a).value
val bcp = (externalDependencyClasspath in Compile in b).value
if (acp == bcp) ()
else sys.error("Different classpaths are found:\n" + acp.toString + "\n" + bcp.toString)
val acp = (externalDependencyClasspath in Compile in a).value.sortBy {_.data.getName}
val bcp = (externalDependencyClasspath in Compile in b).value.sortBy {_.data.getName}
val ccp = (externalDependencyClasspath in Compile in c).value.sortBy {_.data.getName} filterNot {_.data.getName == "demo_2.10.jar"}
if (acp == bcp && acp == ccp) ()
else sys.error("Different classpaths are found:" +
"\n - a (consolidated) " + acp.toString +
"\n - b (plain) " + bcp.toString +
"\n - c (inter-project) " + ccp.toString)
}
)

View File

@ -2,4 +2,4 @@
> check
> c/run
# > c/run