From eed7553130273ee1a735a31019cd6a06585445a4 Mon Sep 17 00:00:00 2001 From: Eugene Yokota Date: Sun, 29 Oct 2023 15:43:43 -0400 Subject: [PATCH] Start Def.declareOutput --- .../sbt/internal/util/appmacro/Cont.scala | 80 ++++++++---- .../internal/util/appmacro/ContextUtil.scala | 31 ++++- .../scala/sbt/internal/ContTestMacro.scala | 2 +- main-settings/src/main/scala/sbt/Def.scala | 10 ++ .../src/main/scala/sbt/std/InputConvert.scala | 1 + .../src/main/scala/sbt/std/InputWrapper.scala | 6 + .../src/main/scala/sbt/std/TaskMacro.scala | 7 +- main/src/main/scala/sbt/RemoteCache.scala | 16 +-- .../scala/sbt/internal/CommandExchange.scala | 1 + sbt-app/src/sbt-test/cache/basic/build.sbt | 31 +++++ sbt-app/src/sbt-test/cache/basic/test | 18 +++ .../src/main/scala/sbt/util/ActionCache.scala | 18 ++- .../scala/sbt/util/ActionCacheStore.scala | 122 +++++++++++------- .../test/scala/sbt/util/ActionCacheTest.scala | 49 +++---- 14 files changed, 280 insertions(+), 112 deletions(-) create mode 100644 sbt-app/src/sbt-test/cache/basic/build.sbt create mode 100644 sbt-app/src/sbt-test/cache/basic/test diff --git a/core-macros/src/main/scala/sbt/internal/util/appmacro/Cont.scala b/core-macros/src/main/scala/sbt/internal/util/appmacro/Cont.scala index 0604fd05ba2..8df7cdd4667 100644 --- a/core-macros/src/main/scala/sbt/internal/util/appmacro/Cont.scala +++ b/core-macros/src/main/scala/sbt/internal/util/appmacro/Cont.scala @@ -7,9 +7,10 @@ import scala.collection.mutable.ListBuffer import scala.reflect.{ ClassTag, TypeTest } import scala.quoted.* import sjsonnew.{ BasicJsonProtocol, HashWriter, JsonFormat } -import sbt.util.{ ActionCache, ActionCacheStore } +import sbt.util.{ ActionCache, ActionCacheStore, CacheConfiguration } import sbt.util.Applicative import sbt.util.Monad +import xsbti.VirtualFile import Types.Id /** @@ -27,12 +28,12 @@ trait Cont: def contMapN[A: Type, F[_], Effect[_]: Type]( tree: Expr[A], applicativeExpr: Expr[Applicative[F]], - storeExpr: Option[Expr[ActionCacheStore]], + cacheConfigExpr: Option[Expr[CacheConfiguration]], )(using iftpe: Type[F], eatpe: Type[Effect[A]], ): Expr[F[Effect[A]]] = - contMapN[A, F, Effect](tree, applicativeExpr, storeExpr, conv.idTransform) + contMapN[A, F, Effect](tree, applicativeExpr, cacheConfigExpr, conv.idTransform) /** * Implementation of a macro that provides a direct syntax for applicative functors. It is @@ -41,13 +42,13 @@ trait Cont: def contMapN[A: Type, F[_], Effect[_]: Type]( tree: Expr[A], applicativeExpr: Expr[Applicative[F]], - storeExpr: Option[Expr[ActionCacheStore]], + cacheConfigExpr: Option[Expr[CacheConfiguration]], inner: conv.TermTransform[Effect] )(using iftpe: Type[F], eatpe: Type[Effect[A]], ): Expr[F[Effect[A]]] = - contImpl[A, F, Effect](Left(tree), applicativeExpr, storeExpr, inner) + contImpl[A, F, Effect](Left(tree), applicativeExpr, cacheConfigExpr, inner) /** * Implementation of a macro that provides a direct syntax for applicative functors. It is @@ -56,12 +57,12 @@ trait Cont: def contFlatMap[A: Type, F[_], Effect[_]: Type]( tree: Expr[F[A]], applicativeExpr: Expr[Applicative[F]], - storeExpr: Option[Expr[ActionCacheStore]], + cacheConfigExpr: Option[Expr[CacheConfiguration]], )(using iftpe: Type[F], eatpe: Type[Effect[A]], ): Expr[F[Effect[A]]] = - contFlatMap[A, F, Effect](tree, applicativeExpr, storeExpr, conv.idTransform) + contFlatMap[A, F, Effect](tree, applicativeExpr, cacheConfigExpr, conv.idTransform) /** * Implementation of a macro that provides a direct syntax for applicative functors. It is @@ -70,13 +71,13 @@ trait Cont: def contFlatMap[A: Type, F[_], Effect[_]: Type]( tree: Expr[F[A]], applicativeExpr: Expr[Applicative[F]], - storeExpr: Option[Expr[ActionCacheStore]], + cacheConfigExpr: Option[Expr[CacheConfiguration]], inner: conv.TermTransform[Effect] )(using iftpe: Type[F], eatpe: Type[Effect[A]], ): Expr[F[Effect[A]]] = - contImpl[A, F, Effect](Right(tree), applicativeExpr, storeExpr, inner) + contImpl[A, F, Effect](Right(tree), applicativeExpr, cacheConfigExpr, inner) def summonAppExpr[F[_]: Type]: Expr[Applicative[F]] = import conv.qctx @@ -150,7 +151,7 @@ trait Cont: def contImpl[A: Type, F[_], Effect[_]: Type]( eitherTree: Either[Expr[A], Expr[F[A]]], applicativeExpr: Expr[Applicative[F]], - storeExprOpt: Option[Expr[ActionCacheStore]], + cacheConfigExprOpt: Option[Expr[CacheConfiguration]], inner: conv.TermTransform[Effect] )(using iftpe: Type[F], @@ -167,29 +168,46 @@ trait Cont: case Right(r) => (r, faTpe) val inputBuf = ListBuffer[Input]() + val outputBuf = ListBuffer[Output]() def makeApp(body: Term, inputs: List[Input]): Expr[F[Effect[A]]] = inputs match case Nil => pure(body) case x :: Nil => genMap(body, x) case xs => genMapN(body, xs) + // wrap body in between output var declarations and var references + def letOutput[A1: Type](outputs: List[Output])(body: Expr[A1]): Expr[(A1, Seq[VirtualFile])] = + Block( + outputs.map(_.toVarDef), + '{ + ( + $body, + List(${ Varargs[VirtualFile](outputs.map(_.toRef.asExprOf[VirtualFile])) }: _*) + ) + }.asTerm + ).asExprOf[(A1, Seq[VirtualFile])] + // no inputs, so construct F[A] via Instance.pure or pure+flatten def pure(body: Term): Expr[F[Effect[A]]] = def pure0[A1: Type](body: Expr[A1]): Expr[F[A1]] = - storeExprOpt match - case Some(storeExpr) => + cacheConfigExprOpt match + case Some(cacheConfigExpr) => val codeContentHash = Expr[Long](body.show.##) val aJsonFormat = summonJsonFormat[A1] val aClassTag = summonClassTag[A1] + val outputs = outputBuf.toList + val block = letOutput(outputBuf.toList)(body) '{ import BasicJsonProtocol.given // given HashWriter[Unit] = $inputHashWriter given JsonFormat[A1] = $aJsonFormat given ClassTag[A1] = $aClassTag - val x = ActionCache.cache((), $codeContentHash)({ _ => - ($body, Nil) - })($storeExpr) - $applicativeExpr.pure[A1] { () => x.value } + $applicativeExpr.pure[A1] { () => + val x = ActionCache.cache((), $codeContentHash)({ _ => + $block + })($cacheConfigExpr) + x.value + } } case None => '{ @@ -234,21 +252,22 @@ trait Cont: } val modifiedBody = transformWrappers(body.asTerm.changeOwner(sym), substitute, sym).asExprOf[A1] - storeExprOpt match - case Some(storesExpr) => + cacheConfigExprOpt match + case Some(cacheConfigExpr) => val codeContentHash = Expr[Long](modifiedBody.##) val paramRef = Ref(param.symbol).asExprOf[a] val inputHashWriter = summonHashWriter[a] val aJsonFormat = summonJsonFormat[A1] val aClassTag = summonClassTag[A1] + val block = letOutput(outputBuf.toList)(modifiedBody) '{ given HashWriter[a] = $inputHashWriter given JsonFormat[A1] = $aJsonFormat given ClassTag[A1] = $aClassTag ActionCache .cache($paramRef, $codeContentHash)({ _ => - ($modifiedBody, Nil) - })($storesExpr) + $block + })($cacheConfigExpr) .value }.asTerm.changeOwner(sym) case None => modifiedBody.asTerm @@ -295,21 +314,22 @@ trait Cont: } val modifiedBody = transformWrappers(body.asTerm.changeOwner(sym), substitute, sym).asExprOf[A1] - storeExprOpt match - case Some(storeExpr) => + cacheConfigExprOpt match + case Some(cacheConfigExpr) => val codeContentHash = Expr[Long](modifiedBody.##) val p0Ref = Ref(p0.symbol).asExprOf[inputTypeTpe] val inputHashWriter = summonHashWriter[inputTypeTpe] val aJsonFormat = summonJsonFormat[A1] val aClassTag = summonClassTag[A1] + val block = letOutput(outputBuf.toList)(modifiedBody) '{ given HashWriter[inputTypeTpe] = $inputHashWriter given JsonFormat[A1] = $aJsonFormat given ClassTag[A1] = $aClassTag ActionCache .cache($p0Ref, $codeContentHash)({ _ => - ($modifiedBody, Nil) - })($storeExpr) + $block + })($cacheConfigExpr) .value }.asTerm.changeOwner(sym) case None => @@ -338,14 +358,22 @@ trait Cont: case Right(_) => flatten(genMapN0[F[Effect[A]]](inner(body).asExprOf[F[Effect[A]]])) + val WrapOutputName = "wrapOutput_\u2603\u2603" // Called when transforming the tree to add an input. // For `qual` of type F[A], and a `selection` qual.value. val record = [a] => (name: String, tpe: Type[a], qual: Term, oldTree: Term) => given t: Type[a] = tpe convert[a](name, qual) transform { (replacement: Term) => - inputBuf += Input(TypeRepr.of[a], qual, replacement, freshName("q")) - oldTree + if name != WrapOutputName then + inputBuf += Input(TypeRepr.of[a], qual, replacement, freshName("q")) + oldTree + else + val output = Output(TypeRepr.of[a], qual, freshName("o"), Symbol.spliceOwner) + outputBuf += output + if cacheConfigExprOpt.isDefined then output.toAssign + else oldTree + end if } val tx = transformWrappers(expr.asTerm, record, Symbol.spliceOwner) val tr = makeApp(tx, inputBuf.toList) diff --git a/core-macros/src/main/scala/sbt/internal/util/appmacro/ContextUtil.scala b/core-macros/src/main/scala/sbt/internal/util/appmacro/ContextUtil.scala index 89a07860cd7..6fbae74beee 100644 --- a/core-macros/src/main/scala/sbt/internal/util/appmacro/ContextUtil.scala +++ b/core-macros/src/main/scala/sbt/internal/util/appmacro/ContextUtil.scala @@ -15,10 +15,6 @@ trait ContextUtil[C <: Quotes & scala.Singleton](val qctx: C, val valStart: Int) counter = counter + 1 s"$$${prefix}${counter}" - /** - * Constructs a new, synthetic, local var with type `tpe`, a unique name, initialized to - * zero-equivalent (Zero[A]), and owned by `parent`. - */ def freshValDef(parent: Symbol, tpe: TypeRepr, rhs: Term): ValDef = tpe.asType match case '[a] => @@ -61,6 +57,33 @@ trait ContextUtil[C <: Quotes & scala.Singleton](val qctx: C, val valStart: Int) override def toString: String = s"Input($tpe, $qual, $term, $name)" + /** + * Represents an output expression via Def.declareOutput + */ + final class Output( + val tpe: TypeRepr, + val term: Term, + val name: String, + val parent: Symbol, + ): + override def toString: String = + s"Output($tpe, $term, $name)" + val placeholder: Symbol = + tpe.asType match + case '[a] => + Symbol.newVal( + parent, + name, + tpe, + Flags.Mutable, + Symbol.noSymbol + ) + def toVarDef: ValDef = + ValDef(placeholder, rhs = Some('{ null }.asTerm)) + def toAssign: Term = Assign(toRef, term) + def toRef: Ref = Ref(placeholder) + end Output + trait TermTransform[F[_]]: def apply(in: Term): Term end TermTransform diff --git a/core-macros/src/test/scala/sbt/internal/ContTestMacro.scala b/core-macros/src/test/scala/sbt/internal/ContTestMacro.scala index ea1eca89183..9574b9b9b76 100644 --- a/core-macros/src/test/scala/sbt/internal/ContTestMacro.scala +++ b/core-macros/src/test/scala/sbt/internal/ContTestMacro.scala @@ -20,7 +20,7 @@ object ContTestMacro: convert1.contMapN[A, List, Id]( expr, convert1.summonAppExpr[List], - '{ Seq(inMemoryCache: ActionCacheStore) }, + None, convert1.idTransform ) diff --git a/main-settings/src/main/scala/sbt/Def.scala b/main-settings/src/main/scala/sbt/Def.scala index 8a36aebe4ec..1f49220f232 100644 --- a/main-settings/src/main/scala/sbt/Def.scala +++ b/main-settings/src/main/scala/sbt/Def.scala @@ -8,6 +8,7 @@ package sbt import java.io.File +import java.nio.file.Path import java.net.URI import scala.annotation.compileTimeOnly @@ -230,8 +231,13 @@ object Def extends Init[Scope] with TaskMacroExtra with InitializeImplicits: import language.experimental.macros + // These are here, as opposed to RemoteCahe, since we need them from TaskMacro etc private[sbt] var _cacheStore: ActionCacheStore = AggregateActionCacheStore.empty def cacheStore: ActionCacheStore = _cacheStore + private[sbt] var _outputDirectory: Option[Path] = None + def outputDirectoryForCache: Path = _outputDirectory match + case Some(dir) => dir + case None => sys.error("outputDirectoryForCache has not been set") inline def cachedTask[A1: JsonFormat](inline a1: A1): Def.Initialize[Task[A1]] = ${ TaskMacro.taskMacroImpl[A1]('a1, cached = true) } @@ -296,6 +302,10 @@ object Def extends Init[Scope] with TaskMacroExtra with InitializeImplicits: */ def promise[A]: PromiseWrap[A] = new PromiseWrap[A]() + inline def declareOutput(inline vf: VirtualFile): Unit = + InputWrapper.`wrapOutput_\u2603\u2603`[VirtualFile](vf) + + // The following conversions enable the types Initialize[T], Initialize[Task[T]], and Task[T] to // be used in task and setting macros as inputs with an ultimate result of type T diff --git a/main-settings/src/main/scala/sbt/std/InputConvert.scala b/main-settings/src/main/scala/sbt/std/InputConvert.scala index 3a830fa54b4..12a456cec1d 100644 --- a/main-settings/src/main/scala/sbt/std/InputConvert.scala +++ b/main-settings/src/main/scala/sbt/std/InputConvert.scala @@ -78,6 +78,7 @@ class FullConvert[C <: Quotes & scala.Singleton](override val qctx: C, valStart: case InputWrapper.WrapPreviousName => Converted.success(in) case InputWrapper.WrapInitName => wrapInit[A](in) case InputWrapper.WrapTaskName => wrapTask[A](in) + case InputWrapper.WrapOutputName => Converted.success(in) case _ => Converted.NotApplicable() private def wrapInit[A: Type](tree: Term): Converted = diff --git a/main-settings/src/main/scala/sbt/std/InputWrapper.scala b/main-settings/src/main/scala/sbt/std/InputWrapper.scala index 533d2bd9c7f..2da95f9bf2e 100644 --- a/main-settings/src/main/scala/sbt/std/InputWrapper.scala +++ b/main-settings/src/main/scala/sbt/std/InputWrapper.scala @@ -26,6 +26,7 @@ object InputWrapper: private[std] final val WrapTaskName = "wrapTask_\u2603\u2603" private[std] final val WrapInitName = "wrapInit_\u2603\u2603" + private[std] final val WrapOutputName = "wrapOutput_\u2603\u2603" private[std] final val WrapInitTaskName = "wrapInitTask_\u2603\u2603" private[std] final val WrapInitInputName = "wrapInitInputTask_\u2603\u2603" private[std] final val WrapInputName = "wrapInputTask_\u2603\u2603" @@ -41,6 +42,11 @@ object InputWrapper: ) def `wrapInit_\u2603\u2603`[T](@deprecated("unused", "") in: Any): T = implDetailError + @compileTimeOnly( + "`declareOutput` can only be used within a task macro, such as Def.cachedTask." + ) + def `wrapOutput_\u2603\u2603`[A](@deprecated("unused", "") in: Any): A = implDetailError + @compileTimeOnly( "`value` can only be called on a task within a task definition macro, such as :=, +=, ++=, or Def.task." ) diff --git a/main-settings/src/main/scala/sbt/std/TaskMacro.scala b/main-settings/src/main/scala/sbt/std/TaskMacro.scala index c11c0e07f63..705847c5107 100644 --- a/main-settings/src/main/scala/sbt/std/TaskMacro.scala +++ b/main-settings/src/main/scala/sbt/std/TaskMacro.scala @@ -28,6 +28,7 @@ import scala.annotation.tailrec import scala.reflect.internal.util.UndefinedPosition import scala.quoted.* import sjsonnew.{ BasicJsonProtocol, JsonFormat } +import sbt.util.CacheConfiguration object TaskMacro: final val AssignInitName = "set" @@ -60,10 +61,10 @@ object TaskMacro: case '{ if ($cond) then $thenp else $elsep } => taskIfImpl[A1](t, cached) case _ => val convert1 = new FullConvert(qctx, 0) - val storeExpr = - if cached then Some('{ Def.cacheStore }) + val cacheConfigExpr = + if cached then Some('{ CacheConfiguration(Def.cacheStore, Def.outputDirectoryForCache) }) else None - convert1.contMapN[A1, F, Id](t, convert1.appExpr, storeExpr) + convert1.contMapN[A1, F, Id](t, convert1.appExpr, cacheConfigExpr) def taskIfImpl[A1: Type](expr: Expr[A1], cached: Boolean)(using qctx: Quotes diff --git a/main/src/main/scala/sbt/RemoteCache.scala b/main/src/main/scala/sbt/RemoteCache.scala index 62476f8502e..cb48274dd2d 100644 --- a/main/src/main/scala/sbt/RemoteCache.scala +++ b/main/src/main/scala/sbt/RemoteCache.scala @@ -9,7 +9,7 @@ package sbt package internal import java.io.File -import java.nio.file.Path +import java.nio.file.{ Files, Path } import org.apache.ivy.core.module.descriptor.{ DefaultArtifact, Artifact => IArtifact } import org.apache.ivy.core.report.DownloadStatus @@ -58,7 +58,8 @@ object RemoteCache { final val cachedTestClassifier = "cached-test" final val commitLength = 10 - def globalCacheStore: ActionCacheStore = Def.cacheStore + def cacheStore: ActionCacheStore = Def.cacheStore + def outputDirectory: Path = Def.outputDirectoryForCache // TODO: cap with caffeine private[sbt] val analysisStore: mutable.Map[HashedVirtualFileRef, CompileAnalysis] = @@ -66,11 +67,13 @@ object RemoteCache { private[sbt] def setActionCacheStore(store: ActionCacheStore): Unit = Def._cacheStore = store + private[sbt] def setOutputDirectory(out: Path): Unit = + Def._outputDirectory = Some(out) private[sbt] def getCachedAnalysis(ref: HashedVirtualFileRef): CompileAnalysis = analysisStore.getOrElseUpdate( ref, { - val vfs = globalCacheStore.readBlobs(ref :: Nil) + val vfs = cacheStore.getBlobs(ref :: Nil) val vf = vfs.head IO.withTemporaryFile(vf.id, ".tmp"): file => IO.transfer(vf.input, file) @@ -93,7 +96,7 @@ object RemoteCache { ) FileAnalysisStore.binary(file).set(AnalysisContents.create(analysis, setup)) val vf = tempConverter.toVirtualFile(file.toPath) - val refs = globalCacheStore.writeBlobs(vf :: Nil) + val refs = cacheStore.putBlobs(vf :: Nil) refs.headOption match case Some(ref) => analysisStore(ref) = analysis @@ -133,10 +136,7 @@ object RemoteCache { }, cacheStores := { List( - DiskActionCacheStore( - localCacheDirectory.value.toPath, - localCacheDirectory.value.toPath.resolve("out") - ) + DiskActionCacheStore(localCacheDirectory.value.toPath()) ) }, ) diff --git a/main/src/main/scala/sbt/internal/CommandExchange.scala b/main/src/main/scala/sbt/internal/CommandExchange.scala index 60151d73ae2..ee3f7506143 100644 --- a/main/src/main/scala/sbt/internal/CommandExchange.scala +++ b/main/src/main/scala/sbt/internal/CommandExchange.scala @@ -200,6 +200,7 @@ private[sbt] final class CommandExchange { // TODO: initialize somewhere else? RemoteCache.setActionCacheStore(AggregateActionCacheStore(caches)) + RemoteCache.setOutputDirectory((s.baseDir / "target" / "out").toPath()) def onIncomingSocket(socket: Socket, instance: ServerInstance): Unit = { val name = newNetworkName diff --git a/sbt-app/src/sbt-test/cache/basic/build.sbt b/sbt-app/src/sbt-test/cache/basic/build.sbt new file mode 100644 index 00000000000..ccb49f1109d --- /dev/null +++ b/sbt-app/src/sbt-test/cache/basic/build.sbt @@ -0,0 +1,31 @@ +import sbt.internal.util.StringVirtualFile1 +import sjsonnew.BasicJsonProtocol.* + +val pure1 = taskKey[Unit]("") +val map1 = taskKey[String]("") +val mapN1 = taskKey[Unit]("") + +Global / localCacheDirectory := new File("/tmp/sbt/diskcache/") + +pure1 := (Def.cachedTask { + val output = StringVirtualFile1("a.txt", "foo") + Def.declareOutput(output) + () +}).value + +map1 := (Def.cachedTask { + pure1.value + val output1 = StringVirtualFile1("b1.txt", "foo") + val output2 = StringVirtualFile1("b2.txt", "foo") + Def.declareOutput(output1) + Def.declareOutput(output2) + "something" +}).value + +mapN1 := (Def.cachedTask { + pure1.value + map1.value + val output = StringVirtualFile1("c.txt", "foo") + Def.declareOutput(output) + () +}).value diff --git a/sbt-app/src/sbt-test/cache/basic/test b/sbt-app/src/sbt-test/cache/basic/test new file mode 100644 index 00000000000..e45a5869120 --- /dev/null +++ b/sbt-app/src/sbt-test/cache/basic/test @@ -0,0 +1,18 @@ +> startServer +> pure1 +$ exists target/out/a.txt +> clean +> pure1 +$ exists target/out/a.txt + +> clean +> map1 +$ exists target/out/a.txt +$ exists target/out/b1.txt +$ exists target/out/b2.txt + +> clean +> mapN1 +$ exists target/out/a.txt +$ exists target/out/b1.txt +$ exists target/out/c.txt diff --git a/util-cache-resolver/src/main/scala/sbt/util/ActionCache.scala b/util-cache-resolver/src/main/scala/sbt/util/ActionCache.scala index 715ad8300cb..2a0fb76a6f8 100644 --- a/util-cache-resolver/src/main/scala/sbt/util/ActionCache.scala +++ b/util-cache-resolver/src/main/scala/sbt/util/ActionCache.scala @@ -4,20 +4,30 @@ import scala.reflect.ClassTag import sjsonnew.{ HashWriter, JsonFormat } import sjsonnew.support.murmurhash.Hasher import xsbti.VirtualFile +import java.nio.file.Path object ActionCache: def cache[I: HashWriter, O: JsonFormat: ClassTag](key: I, otherInputs: Long)( action: I => (O, Seq[VirtualFile]) )( - store: ActionCacheStore + config: CacheConfiguration ): ActionValue[O] = val hash: Long = otherInputs * 13L + Hasher.hashUnsafe[I](key) val input = ActionInput(hash.toHexString) - val result: Option[ActionValue[O]] = store.read[O](input) + val store = config.store + val result: Option[ActionValue[O]] = store.get[O](input) result match - case Some(v) => v + case Some(value) => + store.syncBlobs(value.outputs, config.outputDirectory) + value // return the value case None => val (newResult, outputs) = action(key) - val value = store.write[O](input, newResult, outputs) + val value = store.put[O](input, newResult, outputs) + store.syncBlobs(value.outputs, config.outputDirectory) value end ActionCache + +class CacheConfiguration( + val store: ActionCacheStore, + val outputDirectory: Path, +) diff --git a/util-cache-resolver/src/main/scala/sbt/util/ActionCacheStore.scala b/util-cache-resolver/src/main/scala/sbt/util/ActionCacheStore.scala index 21fedf8f382..6d8472b9166 100644 --- a/util-cache-resolver/src/main/scala/sbt/util/ActionCacheStore.scala +++ b/util-cache-resolver/src/main/scala/sbt/util/ActionCacheStore.scala @@ -13,53 +13,78 @@ import sbt.internal.util.PlainVirtualFile1 import sbt.io.IO import sbt.io.syntax.* import xsbti.{ VirtualFile, VirtualFileRef } +import sbt.nio.file.FileAttributes /** * An abstration of a remote or local cache store. */ trait ActionCacheStore: - def read[A1: ClassTag: JsonFormat](input: ActionInput): Option[ActionValue[A1]] - - def write[A1: ClassTag: JsonFormat]( + /** + * Put a value and blobs to the cache store for later retrieval, + * based on the key input. + */ + def put[A1: ClassTag: JsonFormat]( key: ActionInput, value: A1, blobs: Seq[VirtualFile], ): ActionValue[A1] - def readBlobs(refs: Seq[HashedVirtualFileRef]): Seq[VirtualFile] - - def writeBlobs(blobs: Seq[VirtualFile]): Seq[HashedVirtualFileRef] + /** + * Get the value for the key from the cache store. + */ + def get[A1: ClassTag: JsonFormat](key: ActionInput): Option[ActionValue[A1]] + + /** + * Put VirtualFile blobs to the cache store for later retrieval. + */ + def putBlobs(blobs: Seq[VirtualFile]): Seq[HashedVirtualFileRef] + + /** + * Get blobs from the cache store. + */ + def getBlobs(refs: Seq[HashedVirtualFileRef]): Seq[VirtualFile] + + /** + * Materialize blobs to the output directory. + */ + def syncBlobs(refs: Seq[HashedVirtualFileRef], outputDirectory: Path): Seq[Path] end ActionCacheStore class AggregateActionCacheStore(stores: Seq[ActionCacheStore]) extends ActionCacheStore: - override def read[A1: ClassTag: JsonFormat](input: ActionInput): Option[ActionValue[A1]] = + override def get[A1: ClassTag: JsonFormat](input: ActionInput): Option[ActionValue[A1]] = var result: Option[ActionValue[A1]] = None stores.foreach: store => - if result.isEmpty then result = store.read[A1](input) + if result.isEmpty then result = store.get[A1](input) result - override def write[A1: ClassTag: JsonFormat]( + override def put[A1: ClassTag: JsonFormat]( key: ActionInput, value: A1, blobs: Seq[VirtualFile], ): ActionValue[A1] = var result: Option[ActionValue[A1]] = None stores.foreach: store => - val v = store.write[A1](key, value, blobs) + val v = store.put[A1](key, value, blobs) if result.isEmpty then result = Some(v) result.get - override def readBlobs(refs: Seq[HashedVirtualFileRef]): Seq[VirtualFile] = + override def putBlobs(blobs: Seq[VirtualFile]): Seq[HashedVirtualFileRef] = + var result: Seq[HashedVirtualFileRef] = Nil + stores.foreach: store => + val xs = store.putBlobs(blobs) + if result.isEmpty then result = xs + result + + override def getBlobs(refs: Seq[HashedVirtualFileRef]): Seq[VirtualFile] = var result: Seq[VirtualFile] = Nil stores.foreach: store => - if result.isEmpty then result = store.readBlobs(refs) + if result.isEmpty then result = store.getBlobs(refs) result - override def writeBlobs(blobs: Seq[VirtualFile]): Seq[HashedVirtualFileRef] = - var result: Seq[HashedVirtualFileRef] = Nil + override def syncBlobs(refs: Seq[HashedVirtualFileRef], outputDirectory: Path): Seq[Path] = + var result: Seq[Path] = Nil stores.foreach: store => - val xs = store.writeBlobs(blobs) - if result.isEmpty then result = xs + if result.isEmpty then result = store.syncBlobs(refs, outputDirectory) result end AggregateActionCacheStore @@ -71,56 +96,59 @@ class InMemoryActionCacheStore extends ActionCacheStore: private val underlying: mutable.Map[ActionInput, JValue] = mutable.Map.empty private val blobCache: mutable.Map[HashedVirtualFileRef, VirtualFile] = mutable.Map.empty - override def read[A1: ClassTag: JsonFormat](input: ActionInput): Option[ActionValue[A1]] = + override def get[A1: ClassTag: JsonFormat](input: ActionInput): Option[ActionValue[A1]] = underlying .get(input) .map: j => Converter.fromJsonUnsafe[ActionValue[A1]](j) - override def write[A1: ClassTag: JsonFormat]( + override def put[A1: ClassTag: JsonFormat]( key: ActionInput, value: A1, blobs: Seq[VirtualFile], ): ActionValue[A1] = - val refs = writeBlobs(blobs) + val refs = putBlobs(blobs) val v = ActionValue(value, refs) val json = Converter.toJsonUnsafe(v) underlying(key) = json v - override def readBlobs(refs: Seq[HashedVirtualFileRef]): Seq[VirtualFile] = - refs.map: r => - blobCache(r) - - override def writeBlobs(blobs: Seq[VirtualFile]): Seq[HashedVirtualFileRef] = + override def putBlobs(blobs: Seq[VirtualFile]): Seq[HashedVirtualFileRef] = blobs.map: b => val ref = HashedVirtualFileRef(b.id, b.contentHash) blobCache(ref) = b ref + override def getBlobs(refs: Seq[HashedVirtualFileRef]): Seq[VirtualFile] = + refs.map: r => + blobCache(r) + + override def syncBlobs(refs: Seq[HashedVirtualFileRef], outputDirectory: Path): Seq[Path] = + refs.map: r => + val b = blobCache(r) + val outFile = outputDirectory.resolve(r.id).toFile() + if outFile.exists() then IO.delete(outFile) + IO.transfer(b.input, outFile) + outFile.toPath + override def toString(): String = underlying.toString() end InMemoryActionCacheStore -class DiskActionCacheStore(base: Path, outDir: Path) extends ActionCacheStore: +class DiskActionCacheStore(base: Path) extends ActionCacheStore: lazy val casBase: Path = { val dir = base.resolve("cas") IO.createDirectory(dir.toFile) dir } - lazy val outBase: Path = { - IO.createDirectory(outDir.toFile) - outDir - } - lazy val acBase: Path = { val dir = base.resolve("ac") IO.createDirectory(dir.toFile) dir } - override def read[A1: ClassTag: JsonFormat](input: ActionInput): Option[ActionValue[A1]] = + override def get[A1: ClassTag: JsonFormat](input: ActionInput): Option[ActionValue[A1]] = val acFile = acBase.toFile / input.inputHash if acFile.exists then val str = IO.read(acFile) @@ -131,32 +159,38 @@ class DiskActionCacheStore(base: Path, outDir: Path) extends ActionCacheStore: catch case NonFatal(_) => None else None - override def write[A1: ClassTag: JsonFormat]( + override def put[A1: ClassTag: JsonFormat]( key: ActionInput, value: A1, blobs: Seq[VirtualFile], ): ActionValue[A1] = val acFile = acBase.toFile / key.inputHash - val refs = writeBlobs(blobs) + val refs = putBlobs(blobs) val v = ActionValue(value, refs) val json = Converter.toJsonUnsafe(v) IO.write(acFile, CompactPrinter(json)) v - override def readBlobs(refs: Seq[HashedVirtualFileRef]): Seq[VirtualFile] = - refs.flatMap: r => - val casFile = casBase.toFile / r.contentHash.toHexString - val outFile = outBase.toFile / r.id - if casFile.exists then - if outFile.exists then IO.delete(outFile) - Files.createSymbolicLink(outFile.toPath, casFile.toPath) - Some(PlainVirtualFile1(outFile.toPath, r.id)) - else None - - override def writeBlobs(blobs: Seq[VirtualFile]): Seq[HashedVirtualFileRef] = + override def putBlobs(blobs: Seq[VirtualFile]): Seq[HashedVirtualFileRef] = blobs.map: b => val ref = HashedVirtualFileRef(b.id, b.contentHash) val outFile = casBase.toFile / b.contentHash.toHexString IO.transfer(b.input, outFile) ref + + override def getBlobs(refs: Seq[HashedVirtualFileRef]): Seq[VirtualFile] = + refs.flatMap: r => + val casFile = casBase.toFile / r.contentHash.toHexString + if casFile.exists then Some(PlainVirtualFile1(casFile.toPath(), r.id)) + else None + + override def syncBlobs(refs: Seq[HashedVirtualFileRef], outputDirectory: Path): Seq[Path] = + refs.flatMap: r => + val casFile = casBase.toFile / r.contentHash.toHexString + if casFile.exists then + val outPath = outputDirectory.resolve(r.id) + Files.createDirectories(outPath.getParent()) + if outPath.toFile().exists() then IO.delete(outPath.toFile()) + Some(Files.createSymbolicLink(outPath, casFile.toPath)) + else None end DiskActionCacheStore diff --git a/util-cache-resolver/src/test/scala/sbt/util/ActionCacheTest.scala b/util-cache-resolver/src/test/scala/sbt/util/ActionCacheTest.scala index a5a094eccee..9ca2dbf80a9 100644 --- a/util-cache-resolver/src/test/scala/sbt/util/ActionCacheTest.scala +++ b/util-cache-resolver/src/test/scala/sbt/util/ActionCacheTest.scala @@ -15,9 +15,9 @@ object ActionCacheTest extends BasicTestSuite: def testHoldBlob(cache: ActionCacheStore): Unit = val in = StringVirtualFile1("a.txt", "foo") - val hashRefs = cache.writeBlobs(in :: Nil) + val hashRefs = cache.putBlobs(in :: Nil) assert(hashRefs.size == 1) - val actual = cache.readBlobs(hashRefs).head + val actual = cache.getBlobs(hashRefs).head assert(actual.id == "a.txt") test("In-memory cache can hold action value"): @@ -33,12 +33,14 @@ object ActionCacheTest extends BasicTestSuite: called += 1 (a + b, Nil) } - val v1 = ActionCache.cache[(Int, Int), Int]((1, 1), 1L)(action)(cache) - assert(v1.value == 2) - val v2 = ActionCache.cache[(Int, Int), Int]((1, 1), 1L)(action)(cache) - assert(v2.value == 2) - // check that the action has been invoked only once - assert(called == 1) + IO.withTemporaryDirectory: (tempDir) => + val config = CacheConfiguration(cache, tempDir.toPath()) + val v1 = ActionCache.cache[(Int, Int), Int]((1, 1), 1L)(action)(config) + assert(v1.value == 2) + val v2 = ActionCache.cache[(Int, Int), Int]((1, 1), 1L)(action)(config) + assert(v2.value == 2) + // check that the action has been invoked only once + assert(called == 1) test("In-memory cache can hold action value with blob"): withInMemoryCache(testActionCacheWithBlob) @@ -54,18 +56,22 @@ object ActionCacheTest extends BasicTestSuite: val out = StringVirtualFile1("a.txt", (a + b).toString) (a + b, Seq(out)) } - val v1 = ActionCache.cache[(Int, Int), Int]((1, 1), 1L)(action)(cache) - assert(v1.value == 2) - // ActionValue only contains the reference to the files. - // To retrieve them, separately call readBlobs. - val files1 = cache.readBlobs(v1.outputs) - val file1 = files1(0) - val content = IO.readStream(file1.input) - assert(content == "2") - val v2 = ActionCache.cache[(Int, Int), Int]((1, 1), 1L)(action)(cache) - assert(v2.value == 2) - // check that the action has been invoked only once - assert(called == 1) + IO.withTemporaryDirectory: (tempDir) => + val config = CacheConfiguration(cache, tempDir.toPath()) + val v1 = ActionCache.cache[(Int, Int), Int]((1, 1), 1L)(action)(config) + assert(v1.value == 2) + // ActionValue only contains the reference to the files. + // To retrieve them, separately call readBlobs or syncBlobs. + val files1 = cache.syncBlobs(v1.outputs, tempDir.toPath()) + val file1 = files1(0) + assert(file1.toFile().exists()) + val content = IO.read(file1.toFile()) + assert(content == "2") + + val v2 = ActionCache.cache[(Int, Int), Int]((1, 1), 1L)(action)(config) + assert(v2.value == 2) + // check that the action has been invoked only once + assert(called == 1) def withInMemoryCache(f: InMemoryActionCacheStore => Unit): Unit = val cache = InMemoryActionCacheStore() @@ -75,8 +81,7 @@ object ActionCacheTest extends BasicTestSuite: IO.withTemporaryDirectory( { tempDir0 => val tempDir = tempDir0.toPath - val outDir = tempDir.resolve("out") - val cache = DiskActionCacheStore(tempDir, outDir) + val cache = DiskActionCacheStore(tempDir) f(cache) }, keepDirectory = false