mirror of https://github.com/sbt/sbt.git
Merge pull request #7644 from eed3si9n/wip/testquick
This commit is contained in:
commit
1cb310554f
|
|
@ -347,10 +347,11 @@ trait Cont:
|
|||
$input,
|
||||
codeContentHash = Digest.dummy($codeContentHash),
|
||||
extraHash = Digest.dummy($extraHash),
|
||||
tags = $tagsExpr
|
||||
tags = $tagsExpr,
|
||||
config = $cacheConfigExpr,
|
||||
)({ _ =>
|
||||
$block
|
||||
})($cacheConfigExpr)
|
||||
})
|
||||
}
|
||||
|
||||
// This will generate following code for Def.declareOutput(...):
|
||||
|
|
|
|||
|
|
@ -104,7 +104,21 @@ object Pkg:
|
|||
val sources: Seq[(HashedVirtualFileRef, String)],
|
||||
val jar: VirtualFileRef,
|
||||
val options: Seq[PackageOption]
|
||||
)
|
||||
) {
|
||||
import sbt.util.CacheImplicits.hashedVirtualFileRefToStr
|
||||
private def sourcesStr: String =
|
||||
sources
|
||||
.map { case (k, v) =>
|
||||
s"${hashedVirtualFileRefToStr(k)}=$v"
|
||||
}
|
||||
.mkString(",\n ")
|
||||
override def toString(): String = s"""Configuration(
|
||||
sources = Seq(${sourcesStr}),
|
||||
jar = ...,
|
||||
options = ...,
|
||||
)
|
||||
"""
|
||||
}
|
||||
|
||||
object Configuration:
|
||||
given IsoLList.Aux[
|
||||
|
|
|
|||
|
|
@ -307,6 +307,7 @@ object Tests {
|
|||
in.filter(t => seen.add(f(t)))
|
||||
}
|
||||
|
||||
// Called by Defaults
|
||||
def apply(
|
||||
frameworks: Map[TestFramework, Framework],
|
||||
testLoader: ClassLoader,
|
||||
|
|
@ -340,7 +341,7 @@ object Tests {
|
|||
apply(frameworks, testLoader, runners, o, config, log)
|
||||
}
|
||||
|
||||
def testTask(
|
||||
private[sbt] def testTask(
|
||||
loader: ClassLoader,
|
||||
frameworks: Map[TestFramework, Framework],
|
||||
runners: Map[TestFramework, Runner],
|
||||
|
|
|
|||
|
|
@ -101,7 +101,6 @@ import sbt.SlashSyntax0._
|
|||
import sbt.internal.inc.{
|
||||
Analysis,
|
||||
AnalyzingCompiler,
|
||||
FileAnalysisStore,
|
||||
ManagedLoggedReporter,
|
||||
MixedAnalyzingCompiler,
|
||||
ScalaInstance
|
||||
|
|
@ -140,6 +139,7 @@ import xsbti.compile.{
|
|||
TastyFiles,
|
||||
TransactionalManagerType
|
||||
}
|
||||
import sbt.internal.IncrementalTest
|
||||
|
||||
object Defaults extends BuildCommon {
|
||||
final val CacheDirectoryName = "cache"
|
||||
|
|
@ -153,18 +153,6 @@ object Defaults extends BuildCommon {
|
|||
|
||||
def lock(app: xsbti.AppConfiguration): xsbti.GlobalLock = LibraryManagement.lock(app)
|
||||
|
||||
private[sbt] def extractAnalysis(
|
||||
metadata: StringAttributeMap,
|
||||
converter: FileConverter
|
||||
): Option[CompileAnalysis] =
|
||||
def asBinary(file: File) = FileAnalysisStore.binary(file).get.asScala
|
||||
def asText(file: File) = FileAnalysisStore.text(file).get.asScala
|
||||
for
|
||||
ref <- metadata.get(Keys.analysis)
|
||||
file = converter.toPath(VirtualFileRef.of(ref)).toFile
|
||||
content <- asBinary(file).orElse(asText(file))
|
||||
yield content.getAnalysis
|
||||
|
||||
private[sbt] def globalDefaults(ss: Seq[Setting[_]]): Seq[Setting[_]] =
|
||||
Def.defaultSettings(inScope(GlobalScope)(ss))
|
||||
|
||||
|
|
@ -1323,7 +1311,8 @@ object Defaults extends BuildCommon {
|
|||
testListeners :== Nil,
|
||||
testOptions :== Nil,
|
||||
testResultLogger :== TestResultLogger.Default,
|
||||
testOnly / testFilter :== (selectedFilter _)
|
||||
testOnly / testFilter :== (IncrementalTest.selectedFilter _),
|
||||
extraTestDigests :== Nil,
|
||||
)
|
||||
)
|
||||
lazy val testTasks: Seq[Setting[_]] =
|
||||
|
|
@ -1342,7 +1331,11 @@ object Defaults extends BuildCommon {
|
|||
.storeAs(definedTestNames)
|
||||
.triggeredBy(compile)
|
||||
.value,
|
||||
testQuick / testFilter := testQuickFilter.value,
|
||||
definedTestDigests := IncrementalTest.definedTestDigestTask
|
||||
.triggeredBy(compile)
|
||||
.value,
|
||||
testQuick / testFilter := IncrementalTest.filterTask.value,
|
||||
extraTestDigests ++= IncrementalTest.extraTestDigestsTask.value,
|
||||
executeTests := {
|
||||
import sbt.TupleSyntax.*
|
||||
(
|
||||
|
|
@ -1422,7 +1415,10 @@ object Defaults extends BuildCommon {
|
|||
),
|
||||
Keys.logLevel.?.value.getOrElse(stateLogLevel),
|
||||
) +:
|
||||
new TestStatusReporter(succeededFile((test / streams).value.cacheDirectory)) +:
|
||||
TestStatusReporter(
|
||||
definedTestDigests.value,
|
||||
Def.cacheConfiguration.value,
|
||||
) +:
|
||||
(TaskZero / testListeners).value
|
||||
},
|
||||
testOptions := Tests.Listeners(testListeners.value) +: (TaskZero / testOptions).value,
|
||||
|
|
@ -1491,46 +1487,6 @@ object Defaults extends BuildCommon {
|
|||
)
|
||||
}
|
||||
|
||||
def testQuickFilter: Initialize[Task[Seq[String] => Seq[String => Boolean]]] =
|
||||
Def.task {
|
||||
val cp = (test / fullClasspath).value
|
||||
val s = (test / streams).value
|
||||
val converter = fileConverter.value
|
||||
val analyses = cp
|
||||
.flatMap(a => extractAnalysis(a.metadata, converter))
|
||||
.collect { case analysis: Analysis => analysis }
|
||||
val succeeded = TestStatus.read(succeededFile(s.cacheDirectory))
|
||||
val stamps = collection.mutable.Map.empty[String, Long]
|
||||
def stamp(dep: String): Option[Long] =
|
||||
analyses.flatMap(internalStamp(dep, _, Set.empty)).maxOption
|
||||
def internalStamp(c: String, analysis: Analysis, alreadySeen: Set[String]): Option[Long] = {
|
||||
if (alreadySeen.contains(c)) None
|
||||
else
|
||||
def computeAndStoreStamp: Option[Long] = {
|
||||
import analysis.{ apis, relations }
|
||||
val internalDeps = relations
|
||||
.internalClassDeps(c)
|
||||
.flatMap(internalStamp(_, analysis, alreadySeen + c))
|
||||
val externalDeps = relations.externalDeps(c).flatMap(stamp)
|
||||
val classStamps = relations.productClassName.reverse(c).flatMap { pc =>
|
||||
apis.internal.get(pc).map(_.compilationTimestamp)
|
||||
}
|
||||
val maxStamp = (internalDeps ++ externalDeps ++ classStamps).maxOption
|
||||
maxStamp.foreach(maxStamp => stamps(c) = maxStamp)
|
||||
maxStamp
|
||||
}
|
||||
stamps.get(c).orElse(computeAndStoreStamp)
|
||||
}
|
||||
def noSuccessYet(test: String) = succeeded.get(test) match {
|
||||
case None => true
|
||||
case Some(ts) => stamps.synchronized(stamp(test)).exists(_ > ts)
|
||||
}
|
||||
args =>
|
||||
for (filter <- selectedFilter(args))
|
||||
yield (test: String) => filter(test) && noSuccessYet(test)
|
||||
}
|
||||
def succeededFile(dir: File) = dir / "succeeded_tests"
|
||||
|
||||
@nowarn
|
||||
def inputTests(key: InputKey[_]): Initialize[InputTask[Unit]] =
|
||||
inputTests0.mapReferenced(Def.mapScope(_ in key.key))
|
||||
|
|
@ -1747,21 +1703,6 @@ object Defaults extends BuildCommon {
|
|||
result
|
||||
}
|
||||
|
||||
def selectedFilter(args: Seq[String]): Seq[String => Boolean] = {
|
||||
def matches(nfs: Seq[NameFilter], s: String) = nfs.exists(_.accept(s))
|
||||
|
||||
val (excludeArgs, includeArgs) = args.partition(_.startsWith("-"))
|
||||
|
||||
val includeFilters = includeArgs map GlobFilter.apply
|
||||
val excludeFilters = excludeArgs.map(_.substring(1)).map(GlobFilter.apply)
|
||||
|
||||
(includeFilters, excludeArgs) match {
|
||||
case (Nil, Nil) => Seq(const(true))
|
||||
case (Nil, _) => Seq((s: String) => !matches(excludeFilters, s))
|
||||
case _ =>
|
||||
includeFilters.map(f => (s: String) => (f.accept(s) && !matches(excludeFilters, s)))
|
||||
}
|
||||
}
|
||||
def detectTests: Initialize[Task[Seq[TestDefinition]]] =
|
||||
Def.task {
|
||||
Tests.discover(loadedTestFrameworks.value.values.toList, compile.value, streams.value.log)._1
|
||||
|
|
@ -2625,7 +2566,7 @@ object Defaults extends BuildCommon {
|
|||
val cachedAnalysisMap: Map[VirtualFile, CompileAnalysis] = (
|
||||
for
|
||||
attributed <- cp
|
||||
analysis <- extractAnalysis(attributed.metadata, converter)
|
||||
analysis <- BuildDef.extractAnalysis(attributed.metadata, converter)
|
||||
yield (converter.toVirtualFile(attributed.data), analysis)
|
||||
).toMap
|
||||
val cachedPerEntryDefinesClassLookup: VirtualFile => DefinesClass =
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ import sbt.librarymanagement._
|
|||
import sbt.librarymanagement.ivy.{ Credentials, IvyConfiguration, IvyPaths, UpdateOptions }
|
||||
import sbt.nio.file.Glob
|
||||
import sbt.testing.Framework
|
||||
import sbt.util.{ cacheLevel, ActionCacheStore, Level, Logger, LoggerContext }
|
||||
import sbt.util.{ cacheLevel, ActionCacheStore, Digest, Level, Logger, LoggerContext }
|
||||
import xsbti.{ HashedVirtualFileRef, VirtualFile, VirtualFileRef }
|
||||
import xsbti.compile._
|
||||
import xsbti.compile.analysis.ReadStamps
|
||||
|
|
@ -349,8 +349,10 @@ object Keys {
|
|||
// Test Keys
|
||||
val testLoader = taskKey[ClassLoader]("Provides the class loader used for testing.").withRank(DTask)
|
||||
val loadedTestFrameworks = taskKey[Map[TestFramework, Framework]]("Loads Framework definitions from the test loader.").withRank(DTask)
|
||||
@cacheLevel(include = Array.empty)
|
||||
val definedTests = taskKey[Seq[TestDefinition]]("Provides the list of defined tests.").withRank(BMinusTask)
|
||||
val definedTestNames = taskKey[Seq[String]]("Provides the set of defined test names.").withRank(BMinusTask)
|
||||
val definedTestDigests = taskKey[Map[String, Digest]]("Provides a unique digest of defined tests.").withRank(DTask)
|
||||
val executeTests = taskKey[Tests.Output]("Executes all tests, producing a report.").withRank(CTask)
|
||||
val test = taskKey[Unit]("Executes all tests.").withRank(APlusTask)
|
||||
val testOnly = inputKey[Unit]("Executes the tests provided as arguments or all tests if no arguments are provided.").withRank(ATask)
|
||||
|
|
@ -364,6 +366,7 @@ object Keys {
|
|||
val testResultLogger = settingKey[TestResultLogger]("Logs results after a test task completes.").withRank(DTask)
|
||||
val testGrouping = taskKey[Seq[Tests.Group]]("Collects discovered tests into groups. Whether to fork and the options for forking are configurable on a per-group basis.").withRank(BMinusTask)
|
||||
val isModule = AttributeKey[Boolean]("isModule", "True if the target is a module.", DSetting)
|
||||
val extraTestDigests = taskKey[Seq[Digest]]("Extra digests that would invalidate test caching").withRank(DTask)
|
||||
|
||||
// Classpath/Dependency Management Keys
|
||||
type Classpath = Def.Classpath
|
||||
|
|
|
|||
|
|
@ -194,10 +194,7 @@ object RemoteCache {
|
|||
.withResolvers(rs)
|
||||
}
|
||||
)
|
||||
) ++ inConfig(Compile)(
|
||||
configCacheSettings(compileArtifact(Compile, cachedCompileClassifier))
|
||||
)
|
||||
++ inConfig(Test)(configCacheSettings(testArtifact(Test, cachedTestClassifier))))
|
||||
) ++ inConfig(Compile)(configCacheSettings(compileArtifact(Compile, cachedCompileClassifier))))
|
||||
|
||||
def getResourceFilePaths() = Def.task {
|
||||
val syncDir = crossTarget.value / (prefix(configuration.value.name) + "sync")
|
||||
|
|
@ -383,19 +380,6 @@ object RemoteCache {
|
|||
)
|
||||
}
|
||||
|
||||
def testArtifact(
|
||||
configuration: Configuration,
|
||||
classifier: String
|
||||
): Def.Initialize[Task[TestRemoteCacheArtifact]] = Def.task {
|
||||
TestRemoteCacheArtifact(
|
||||
Artifact(moduleName.value, classifier),
|
||||
configuration / packageCache,
|
||||
(configuration / classDirectory).value,
|
||||
(configuration / compileAnalysisFile).value,
|
||||
Defaults.succeededFile((configuration / test / streams).value.cacheDirectory)
|
||||
)
|
||||
}
|
||||
|
||||
private def toVersion(v: String): String = s"0.0.0-$v"
|
||||
|
||||
private lazy val doption = new DownloadOptions
|
||||
|
|
|
|||
|
|
@ -14,9 +14,10 @@ import Keys.{ organization, thisProject, autoGeneratedProject }
|
|||
import Def.Setting
|
||||
// import sbt.ProjectExtra.apply
|
||||
import sbt.io.Hash
|
||||
import sbt.internal.util.Attributed
|
||||
import sbt.internal.inc.ReflectUtilities
|
||||
import xsbti.FileConverter
|
||||
import sbt.internal.util.{ Attributed, StringAttributeMap }
|
||||
import sbt.internal.inc.{ FileAnalysisStore, ReflectUtilities }
|
||||
import xsbti.{ FileConverter, VirtualFileRef }
|
||||
import xsbti.compile.CompileAnalysis
|
||||
|
||||
trait BuildDef {
|
||||
def projectDefinitions(@deprecated("unused", "") baseDirectory: File): Seq[Project] = projects
|
||||
|
|
@ -33,7 +34,7 @@ trait BuildDef {
|
|||
def rootProject: Option[Project] = None
|
||||
}
|
||||
|
||||
private[sbt] object BuildDef {
|
||||
private[sbt] object BuildDef:
|
||||
val defaultEmpty: BuildDef = new BuildDef { override def projects = Nil }
|
||||
|
||||
val default: BuildDef = new BuildDef {
|
||||
|
|
@ -78,5 +79,19 @@ private[sbt] object BuildDef {
|
|||
in: Seq[Attributed[_]],
|
||||
converter: FileConverter
|
||||
): Seq[xsbti.compile.CompileAnalysis] =
|
||||
in.flatMap(a => Defaults.extractAnalysis(a.metadata, converter))
|
||||
}
|
||||
in.flatMap(a => extractAnalysis(a.metadata, converter))
|
||||
|
||||
private[sbt] def extractAnalysis(
|
||||
metadata: StringAttributeMap,
|
||||
converter: FileConverter
|
||||
): Option[CompileAnalysis] =
|
||||
import sbt.OptionSyntax.*
|
||||
def asBinary(file: File) = FileAnalysisStore.binary(file).get.asScala
|
||||
def asText(file: File) = FileAnalysisStore.text(file).get.asScala
|
||||
for
|
||||
ref <- metadata.get(Keys.analysis)
|
||||
file = converter.toPath(VirtualFileRef.of(ref)).toFile
|
||||
content <- asBinary(file).orElse(asText(file))
|
||||
yield content.getAnalysis
|
||||
|
||||
end BuildDef
|
||||
|
|
|
|||
|
|
@ -0,0 +1,218 @@
|
|||
/*
|
||||
* sbt
|
||||
* Copyright 2023, Scala center
|
||||
* Copyright 2011 - 2022, Lightbend, Inc.
|
||||
* Copyright 2008 - 2010, Mark Harrah
|
||||
* Licensed under Apache License 2.0 (see LICENSE)
|
||||
*/
|
||||
|
||||
package sbt
|
||||
package internal
|
||||
|
||||
import java.io.File
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import Keys.{ test, fileConverter, fullClasspath, streams }
|
||||
import sbt.Def.Initialize
|
||||
import sbt.internal.inc.Analysis
|
||||
import sbt.internal.util.Attributed
|
||||
import sbt.internal.util.Types.const
|
||||
import sbt.io.{ GlobFilter, IO, NameFilter }
|
||||
import sbt.protocol.testing.TestResult
|
||||
import sbt.SlashSyntax0.*
|
||||
import sbt.util.{ ActionCache, BuildWideCacheConfiguration, CacheLevelTag, Digest }
|
||||
import sbt.util.CacheImplicits.given
|
||||
import scala.collection.concurrent
|
||||
import scala.collection.mutable
|
||||
import scala.collection.SortedSet
|
||||
import xsbti.{ FileConverter, HashedVirtualFileRef, VirtualFileRef }
|
||||
|
||||
object IncrementalTest:
|
||||
def filterTask: Initialize[Task[Seq[String] => Seq[String => Boolean]]] =
|
||||
Def.task {
|
||||
val cp = (Keys.test / fullClasspath).value
|
||||
val s = (Keys.test / streams).value
|
||||
val digests = (Keys.definedTestDigests).value
|
||||
val config = Def.cacheConfiguration.value
|
||||
def hasCachedSuccess(ts: Digest): Boolean =
|
||||
val input = cacheInput(ts)
|
||||
ActionCache.exists(input._1, input._2, input._3, config)
|
||||
def hasSucceeded(className: String): Boolean = digests.get(className) match
|
||||
case None => false
|
||||
case Some(ts) => hasCachedSuccess(ts)
|
||||
args =>
|
||||
for filter <- selectedFilter(args)
|
||||
yield (test: String) => filter(test) && !hasSucceeded(test)
|
||||
}
|
||||
|
||||
// cache the test digests against the fullClasspath.
|
||||
def definedTestDigestTask: Initialize[Task[Map[String, Digest]]] = Def.cachedTask {
|
||||
val cp = (Keys.test / fullClasspath).value
|
||||
val testNames = Keys.definedTests.value.map(_.name).toVector.distinct
|
||||
val converter = fileConverter.value
|
||||
val extra = Keys.extraTestDigests.value
|
||||
val stamper = ClassStamper(cp, converter)
|
||||
// TODO: Potentially do something about JUnit 5 and others which might not use class name
|
||||
Map((testNames.flatMap: name =>
|
||||
stamper.transitiveStamp(name, extra) match
|
||||
case Some(ts) => Seq(name -> ts)
|
||||
case None => Nil
|
||||
): _*)
|
||||
}
|
||||
|
||||
def extraTestDigestsTask: Initialize[Task[Seq[Digest]]] = Def.cachedTask {
|
||||
// by default this captures JVM version
|
||||
val extraInc = Keys.extraIncOptions.value
|
||||
// throw in any information useful for runtime invalidation
|
||||
val salt = s"""${extraInc.mkString(",")}
|
||||
"""
|
||||
Vector(Digest.sha256Hash(salt.getBytes("UTF-8")))
|
||||
}
|
||||
|
||||
def selectedFilter(args: Seq[String]): Seq[String => Boolean] =
|
||||
def matches(nfs: Seq[NameFilter], s: String) = nfs.exists(_.accept(s))
|
||||
val (excludeArgs, includeArgs) = args.partition(_.startsWith("-"))
|
||||
val includeFilters = includeArgs.map(GlobFilter.apply)
|
||||
val excludeFilters = excludeArgs.map(_.substring(1)).map(GlobFilter.apply)
|
||||
(includeFilters, excludeArgs) match
|
||||
case (Nil, Nil) => Seq(const(true))
|
||||
case (Nil, _) => Seq((s: String) => !matches(excludeFilters, s))
|
||||
case _ =>
|
||||
includeFilters.map(f => (s: String) => (f.accept(s) && !matches(excludeFilters, s)))
|
||||
|
||||
private[sbt] def cacheInput(value: Digest): (Unit, Digest, Digest) =
|
||||
((), value, Digest.zero)
|
||||
end IncrementalTest
|
||||
|
||||
private[sbt] class TestStatusReporter(
|
||||
digests: Map[String, Digest],
|
||||
cacheConfiguration: BuildWideCacheConfiguration,
|
||||
) extends TestsListener:
|
||||
// int value to represent success
|
||||
private final val successfulTest = 0
|
||||
|
||||
def doInit(): Unit = ()
|
||||
def startGroup(name: String): Unit = ()
|
||||
def testEvent(event: TestEvent): Unit = ()
|
||||
def endGroup(name: String, t: Throwable): Unit = ()
|
||||
|
||||
/**
|
||||
* If the test has succeeded, record the fact that it has
|
||||
* using its unique digest, so we can skip the test later.
|
||||
*/
|
||||
def endGroup(name: String, result: TestResult): Unit =
|
||||
if result == TestResult.Passed then
|
||||
digests.get(name) match
|
||||
case Some(ts) =>
|
||||
// treat each test suite as a successful action that returns 0
|
||||
val input = IncrementalTest.cacheInput(ts)
|
||||
ActionCache.cache(
|
||||
key = input._1,
|
||||
codeContentHash = input._2,
|
||||
extraHash = input._3,
|
||||
tags = CacheLevelTag.all.toList,
|
||||
config = cacheConfiguration,
|
||||
): (_) =>
|
||||
ActionCache.actionResult(successfulTest)
|
||||
case None => ()
|
||||
else ()
|
||||
def doComplete(finalResult: TestResult): Unit = ()
|
||||
end TestStatusReporter
|
||||
|
||||
private[sbt] object TestStatus:
|
||||
import java.util.Properties
|
||||
def read(f: File): concurrent.Map[String, Digest] =
|
||||
import scala.jdk.CollectionConverters.*
|
||||
val props = Properties()
|
||||
IO.load(props, f)
|
||||
val result = ConcurrentHashMap[String, Digest]()
|
||||
props.asScala.iterator.foreach { case (k, v) => result.put(k, Digest(v)) }
|
||||
result.asScala
|
||||
|
||||
def write(map: collection.Map[String, Digest], label: String, f: File): Unit =
|
||||
IO.writeLines(
|
||||
f,
|
||||
s"# $label" ::
|
||||
map.toList.sortBy(_._1).map { case (k, v) =>
|
||||
s"$k=$v"
|
||||
}
|
||||
)
|
||||
end TestStatus
|
||||
|
||||
/**
|
||||
* ClassStamper provides `transitiveStamp` method to calculate a unique
|
||||
* fingerprint, which will be used for runtime invalidation.
|
||||
*/
|
||||
class ClassStamper(
|
||||
classpath: Seq[Attributed[HashedVirtualFileRef]],
|
||||
converter: FileConverter,
|
||||
):
|
||||
private val stamps = mutable.Map.empty[String, SortedSet[Digest]]
|
||||
private val vfStamps = mutable.Map.empty[VirtualFileRef, Digest]
|
||||
private lazy val analyses = classpath
|
||||
.flatMap(a => BuildDef.extractAnalysis(a.metadata, converter))
|
||||
.collect { case analysis: Analysis => analysis }
|
||||
|
||||
/**
|
||||
* Given a classpath and a class name, this tries to create a SHA-256 digest.
|
||||
* @param className className to stamp
|
||||
* @param extraHashes additional information to include into the returning digest
|
||||
*/
|
||||
private[sbt] def transitiveStamp(className: String, extaHashes: Seq[Digest]): Option[Digest] =
|
||||
val digests = SortedSet(analyses.flatMap(internalStamp(className, _, Set.empty)): _*)
|
||||
if digests.nonEmpty then Some(Digest.sha256Hash(digests.toSeq ++ extaHashes: _*))
|
||||
else None
|
||||
|
||||
private def internalStamp(
|
||||
className: String,
|
||||
analysis: Analysis,
|
||||
alreadySeen: Set[String],
|
||||
): SortedSet[Digest] =
|
||||
if alreadySeen.contains(className) then SortedSet.empty
|
||||
else
|
||||
stamps.get(className) match
|
||||
case Some(xs) => xs
|
||||
case _ =>
|
||||
import analysis.relations
|
||||
val internalDeps = relations
|
||||
.internalClassDeps(className)
|
||||
.flatMap: otherCN =>
|
||||
internalStamp(otherCN, analysis, alreadySeen + className)
|
||||
val internalJarDeps = relations
|
||||
.externalDeps(className)
|
||||
.flatMap: libClassName =>
|
||||
transitiveStamp(libClassName, Nil)
|
||||
val externalDeps = relations
|
||||
.externalDeps(className)
|
||||
.flatMap: libClassName =>
|
||||
relations.libraryClassName
|
||||
.reverse(libClassName)
|
||||
.map(stampVf)
|
||||
val classDigests = relations.productClassName
|
||||
.reverse(className)
|
||||
.flatMap: prodClassName =>
|
||||
relations
|
||||
.definesClass(prodClassName)
|
||||
.flatMap: sourceFile =>
|
||||
relations
|
||||
.products(sourceFile)
|
||||
.map(stampVf)
|
||||
// TODO: substitue the above with
|
||||
// val classDigests = relations.productClassName
|
||||
// .reverse(className)
|
||||
// .flatMap: prodClassName =>
|
||||
// analysis.apis.internal
|
||||
// .get(prodClassName)
|
||||
// .map: analyzed =>
|
||||
// 0L // analyzed.??? we need a hash here
|
||||
val xs = SortedSet(
|
||||
(internalDeps union internalJarDeps union externalDeps union classDigests).toSeq: _*
|
||||
)
|
||||
if xs.nonEmpty then stamps(className) = xs
|
||||
else ()
|
||||
xs
|
||||
def stampVf(vf: VirtualFileRef): Digest =
|
||||
vf match
|
||||
case h: HashedVirtualFileRef => Digest(h)
|
||||
case _ =>
|
||||
vfStamps.getOrElseUpdate(vf, Digest.sha256Hash(converter.toPath(vf)))
|
||||
end ClassStamper
|
||||
|
|
@ -8,13 +8,17 @@
|
|||
|
||||
package sbt
|
||||
|
||||
import sbt.internal.IncrementalTest
|
||||
|
||||
object DefaultsTest extends verify.BasicTestSuite {
|
||||
|
||||
test("`selectedFilter` should return all tests for an empty list") {
|
||||
val expected = Map("Test1" -> true, "Test2" -> true)
|
||||
val filter = List.empty[String]
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -22,7 +26,9 @@ object DefaultsTest extends verify.BasicTestSuite {
|
|||
val expected = Map("Test1" -> true, "Test2" -> false, "Foo" -> false)
|
||||
val filter = List("Test1", "foo")
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -30,7 +36,9 @@ object DefaultsTest extends verify.BasicTestSuite {
|
|||
val expected = Map("Test1" -> true, "Test2" -> true, "Foo" -> false)
|
||||
val filter = List("Test*")
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -38,7 +46,9 @@ object DefaultsTest extends verify.BasicTestSuite {
|
|||
val expected = Map("Test1" -> true, "Test2" -> false, "Foo" -> false)
|
||||
val filter = List("Test*", "-Test2")
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -46,7 +56,9 @@ object DefaultsTest extends verify.BasicTestSuite {
|
|||
val expected = Map("Test1" -> true, "Test2" -> false, "Foo" -> true)
|
||||
val filter = List("-Test2")
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -54,7 +66,9 @@ object DefaultsTest extends verify.BasicTestSuite {
|
|||
val expected = Map("Test1" -> true, "Test2" -> true, "Foo" -> false)
|
||||
val filter = List("Test*", "-F*")
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -62,7 +76,9 @@ object DefaultsTest extends verify.BasicTestSuite {
|
|||
val expected = Map("Test1" -> true, "Test2" -> true, "Foo" -> false)
|
||||
val filter = List("T*1", "T*2", "-F*")
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -70,7 +86,9 @@ object DefaultsTest extends verify.BasicTestSuite {
|
|||
val expected = Map("Test1" -> true, "Test2" -> true, "AAA" -> false, "Foo" -> false)
|
||||
val filter = List("-A*", "-F*")
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -78,7 +96,9 @@ object DefaultsTest extends verify.BasicTestSuite {
|
|||
val expected = Map("Test1" -> false, "Test2" -> false, "Test3" -> true)
|
||||
val filter = List("T*", "-T*1", "-T*2")
|
||||
assert(
|
||||
expected.map(t => (t._1, Defaults.selectedFilter(filter).exists(fn => fn(t._1)))) == expected
|
||||
expected.map(t =>
|
||||
(t._1, IncrementalTest.selectedFilter(filter).exists(fn => fn(t._1)))
|
||||
) == expected
|
||||
)
|
||||
}
|
||||
}
|
||||
|
|
@ -93,6 +93,7 @@ final class TestFramework(val implClassNames: String*) extends Serializable {
|
|||
def create(loader: ClassLoader, log: ManagedLogger): Option[Framework] =
|
||||
createFramework(loader, log, implClassNames.toList)
|
||||
}
|
||||
|
||||
final class TestDefinition(
|
||||
val name: String,
|
||||
val fingerprint: Fingerprint,
|
||||
|
|
@ -108,7 +109,7 @@ final class TestDefinition(
|
|||
override def hashCode: Int = (name.hashCode, TestFramework.hashCode(fingerprint)).hashCode
|
||||
}
|
||||
|
||||
final class TestRunner(
|
||||
private[sbt] final class TestRunner(
|
||||
delegate: Runner,
|
||||
listeners: Vector[TestReportListener],
|
||||
log: ManagedLogger
|
||||
|
|
@ -214,7 +215,7 @@ object TestFramework {
|
|||
case _ => f.toString
|
||||
}
|
||||
|
||||
def testTasks(
|
||||
private[sbt] def testTasks(
|
||||
frameworks: Map[TestFramework, Framework],
|
||||
runners: Map[TestFramework, Runner],
|
||||
testLoader: ClassLoader,
|
||||
|
|
|
|||
|
|
@ -1,53 +0,0 @@
|
|||
/*
|
||||
* sbt
|
||||
* Copyright 2023, Scala center
|
||||
* Copyright 2011 - 2022, Lightbend, Inc.
|
||||
* Copyright 2008 - 2010, Mark Harrah
|
||||
* Licensed under Apache License 2.0 (see LICENSE)
|
||||
*/
|
||||
|
||||
package sbt
|
||||
|
||||
import java.io.File
|
||||
|
||||
import sbt.io.IO
|
||||
import sbt.protocol.testing.TestResult
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
import scala.collection.concurrent
|
||||
|
||||
// Assumes exclusive ownership of the file.
|
||||
private[sbt] class TestStatusReporter(f: File) extends TestsListener {
|
||||
private lazy val succeeded: concurrent.Map[String, Long] = TestStatus.read(f)
|
||||
|
||||
def doInit(): Unit = ()
|
||||
def startGroup(name: String): Unit = { succeeded remove name; () }
|
||||
def testEvent(event: TestEvent): Unit = ()
|
||||
def endGroup(name: String, t: Throwable): Unit = ()
|
||||
def endGroup(name: String, result: TestResult): Unit = {
|
||||
if (result == TestResult.Passed)
|
||||
succeeded(name) = System.currentTimeMillis
|
||||
}
|
||||
def doComplete(finalResult: TestResult): Unit = {
|
||||
TestStatus.write(succeeded, "Successful Tests", f)
|
||||
}
|
||||
}
|
||||
|
||||
private[sbt] object TestStatus {
|
||||
import java.util.Properties
|
||||
def read(f: File): concurrent.Map[String, Long] = {
|
||||
import scala.jdk.CollectionConverters.*
|
||||
val properties = new Properties
|
||||
IO.load(properties, f)
|
||||
val result = new ConcurrentHashMap[String, Long]()
|
||||
properties.asScala.iterator.foreach { case (k, v) => result.put(k, v.toLong) }
|
||||
result.asScala
|
||||
}
|
||||
|
||||
def write(map: collection.Map[String, Long], label: String, f: File): Unit = {
|
||||
val properties = new Properties
|
||||
for ((test, lastSuccessTime) <- map)
|
||||
properties.setProperty(test, lastSuccessTime.toString)
|
||||
IO.write(properties, label, f)
|
||||
}
|
||||
}
|
||||
|
|
@ -31,24 +31,19 @@ object ActionCache:
|
|||
* Even if the input tasks are the same, the code part needs to be tracked.
|
||||
* - extraHash: Reserved for later, which we might use to invalidate the cache.
|
||||
* - tags: Tags to track cache level.
|
||||
* - action: The actual action to be cached.
|
||||
* - config: The configuration that's used to store where the cache backends are.
|
||||
* - action: The actual action to be cached.
|
||||
*/
|
||||
def cache[I: HashWriter, O: JsonFormat: ClassTag](
|
||||
key: I,
|
||||
codeContentHash: Digest,
|
||||
extraHash: Digest,
|
||||
tags: List[CacheLevelTag],
|
||||
config: BuildWideCacheConfiguration,
|
||||
)(
|
||||
action: I => InternalActionResult[O],
|
||||
)(
|
||||
config: BuildWideCacheConfiguration
|
||||
): O =
|
||||
import config.*
|
||||
val input =
|
||||
Digest.sha256Hash(codeContentHash, extraHash, Digest.dummy(Hasher.hashUnsafe[I](key)))
|
||||
val valuePath = s"value/${input}.json"
|
||||
|
||||
def organicTask: O =
|
||||
// run action(...) and combine the newResult with outputs
|
||||
val InternalActionResult(result, outputs) =
|
||||
|
|
@ -69,38 +64,80 @@ object ActionCache:
|
|||
result
|
||||
else
|
||||
cacheEventLog.append(ActionCacheEvent.OnsiteTask)
|
||||
val input = mkInput(key, codeContentHash, extraHash)
|
||||
val valueFile = StringVirtualFile1(s"value/${input}.json", CompactPrinter(json))
|
||||
val newOutputs = Vector(valueFile) ++ outputs.toVector
|
||||
store.put(UpdateActionResultRequest(input, newOutputs, exitCode = 0)) match
|
||||
case Right(cachedResult) =>
|
||||
syncBlobs(cachedResult.outputFiles)
|
||||
store.syncBlobs(cachedResult.outputFiles, outputDirectory)
|
||||
result
|
||||
case Left(e) => throw e
|
||||
|
||||
get(key, codeContentHash, extraHash, tags, config) match
|
||||
case Some(value) => value
|
||||
case None => organicTask
|
||||
end cache
|
||||
|
||||
/**
|
||||
* Retrieves the cached value.
|
||||
*/
|
||||
def get[I: HashWriter, O: JsonFormat: ClassTag](
|
||||
key: I,
|
||||
codeContentHash: Digest,
|
||||
extraHash: Digest,
|
||||
tags: List[CacheLevelTag],
|
||||
config: BuildWideCacheConfiguration,
|
||||
): Option[O] =
|
||||
import config.store
|
||||
def valueFromStr(str: String, origin: Option[String]): O =
|
||||
cacheEventLog.append(ActionCacheEvent.Found(origin.getOrElse("unknown")))
|
||||
config.cacheEventLog.append(ActionCacheEvent.Found(origin.getOrElse("unknown")))
|
||||
val json = Parser.parseUnsafe(str)
|
||||
Converter.fromJsonUnsafe[O](json)
|
||||
|
||||
def syncBlobs(refs: Seq[HashedVirtualFileRef]): Seq[Path] =
|
||||
store.syncBlobs(refs, config.outputDirectory)
|
||||
|
||||
val getRequest =
|
||||
GetActionResultRequest(input, inlineStdout = false, inlineStderr = false, Vector(valuePath))
|
||||
store.get(getRequest) match
|
||||
findActionResult(key, codeContentHash, extraHash, config) match
|
||||
case Right(result) =>
|
||||
// some protocol can embed values into the result
|
||||
result.contents.headOption match
|
||||
case Some(head) =>
|
||||
syncBlobs(result.outputFiles)
|
||||
store.syncBlobs(result.outputFiles, config.outputDirectory)
|
||||
val str = String(head.array(), StandardCharsets.UTF_8)
|
||||
valueFromStr(str, result.origin)
|
||||
Some(valueFromStr(str, result.origin))
|
||||
case _ =>
|
||||
val paths = syncBlobs(result.outputFiles)
|
||||
if paths.isEmpty then organicTask
|
||||
else valueFromStr(IO.read(paths.head.toFile()), result.origin)
|
||||
case Left(_) => organicTask
|
||||
end cache
|
||||
val paths = store.syncBlobs(result.outputFiles, config.outputDirectory)
|
||||
if paths.isEmpty then None
|
||||
else Some(valueFromStr(IO.read(paths.head.toFile()), result.origin))
|
||||
case Left(_) => None
|
||||
|
||||
/**
|
||||
* Checks if the ActionResult exists in the cache.
|
||||
*/
|
||||
def exists[I: HashWriter](
|
||||
key: I,
|
||||
codeContentHash: Digest,
|
||||
extraHash: Digest,
|
||||
config: BuildWideCacheConfiguration,
|
||||
): Boolean =
|
||||
findActionResult(key, codeContentHash, extraHash, config) match
|
||||
case Right(_) => true
|
||||
case Left(_) => false
|
||||
|
||||
inline private[sbt] def findActionResult[I: HashWriter, O](
|
||||
key: I,
|
||||
codeContentHash: Digest,
|
||||
extraHash: Digest,
|
||||
config: BuildWideCacheConfiguration,
|
||||
): Either[Throwable, ActionResult] =
|
||||
val input = mkInput(key, codeContentHash, extraHash)
|
||||
val valuePath = s"value/${input}.json"
|
||||
val getRequest =
|
||||
GetActionResultRequest(input, inlineStdout = false, inlineStderr = false, Vector(valuePath))
|
||||
config.store.get(getRequest)
|
||||
|
||||
private inline def mkInput[I: HashWriter](
|
||||
key: I,
|
||||
codeContentHash: Digest,
|
||||
extraHash: Digest
|
||||
): Digest =
|
||||
Digest.sha256Hash(codeContentHash, extraHash, Digest.dummy(Hasher.hashUnsafe[I](key)))
|
||||
|
||||
def manifestFromFile(manifest: Path): Manifest =
|
||||
import sbt.internal.util.codec.ManifestCodec.given
|
||||
|
|
@ -146,6 +183,9 @@ object ActionCache:
|
|||
IO.zip((allPaths ++ Seq(mPath)).flatMap(rebase), zipPath.toFile(), Some(default2010Timestamp))
|
||||
conv.toVirtualFile(zipPath)
|
||||
|
||||
inline def actionResult[A1](inline value: A1): InternalActionResult[A1] =
|
||||
InternalActionResult(value, Nil)
|
||||
|
||||
/**
|
||||
* Represents a value and output files, used internally by the macro.
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -26,6 +26,10 @@ object Digest:
|
|||
def toBytes: Array[Byte] = parse(d)._4
|
||||
def sizeBytes: Long = parse(d)._3
|
||||
|
||||
given digestOrd(using ord: Ordering[String]): Ordering[Digest] with
|
||||
def compare(x: Digest, y: Digest) =
|
||||
ord.compare(x, y)
|
||||
|
||||
def apply(s: String): Digest =
|
||||
validateString(s)
|
||||
s
|
||||
|
|
|
|||
|
|
@ -43,10 +43,10 @@ object ActionCacheTest extends BasicTestSuite:
|
|||
IO.withTemporaryDirectory: (tempDir) =>
|
||||
val config = getCacheConfig(cache, tempDir)
|
||||
val v1 =
|
||||
ActionCache.cache((1, 1), Digest.zero, Digest.zero, tags)(action)(config)
|
||||
ActionCache.cache((1, 1), Digest.zero, Digest.zero, tags, config)(action)
|
||||
assert(v1 == 2)
|
||||
val v2 =
|
||||
ActionCache.cache((1, 1), Digest.zero, Digest.zero, tags)(action)(config)
|
||||
ActionCache.cache((1, 1), Digest.zero, Digest.zero, tags, config)(action)
|
||||
assert(v2 == 2)
|
||||
// check that the action has been invoked only once
|
||||
assert(called == 1)
|
||||
|
|
@ -65,7 +65,7 @@ object ActionCacheTest extends BasicTestSuite:
|
|||
}
|
||||
val config = getCacheConfig(cache, tempDir)
|
||||
val v1 =
|
||||
ActionCache.cache((1, 1), Digest.zero, Digest.zero, tags)(action)(config)
|
||||
ActionCache.cache((1, 1), Digest.zero, Digest.zero, tags, config)(action)
|
||||
assert(v1 == 2)
|
||||
// ActionResult only contains the reference to the files.
|
||||
// To retrieve them, separately call readBlobs or syncBlobs.
|
||||
|
|
@ -75,7 +75,7 @@ object ActionCacheTest extends BasicTestSuite:
|
|||
assert(content == "2")
|
||||
|
||||
val v2 =
|
||||
ActionCache.cache((1, 1), Digest.zero, Digest.zero, tags)(action)(config)
|
||||
ActionCache.cache((1, 1), Digest.zero, Digest.zero, tags, config)(action)
|
||||
assert(v2 == 2)
|
||||
// check that the action has been invoked only once
|
||||
assert(called == 1)
|
||||
|
|
|
|||
Loading…
Reference in New Issue