From f126206231a00812c7321232d316bbbc1e5994a7 Mon Sep 17 00:00:00 2001 From: Ethan Atkins Date: Fri, 19 Jul 2019 21:39:12 -0700 Subject: [PATCH] Fix incremental task evaluation semantics While writing documentation for the new file management/incremental task evaluation features, I realized that incremental task evaluation did not have the correct semantics. The problem was that calls to `.previous` are not scoped within the current task. By this, I mean that say there are tasks foo and bar and that the defintion of bar looks like bar := { val current = foo.value foo.previous match { case Some(v) if v == current => // value hasn't changed case _ => process(current) } } The problem is that foo.previous is stored in effectively (foo / streams).value.cacheDirectory / "previous". This means that it is completely decoupled from foo. Now, suppose that the user runs something like: > set foo := 1 > bar // processes the value 1 > set foo := 2 > foo > bar // does not process the new value 2 because foo was called, which updates the previous value This is not an unrealistic scenario and is, in fact, common if the incremental task evaluation is changed across multiple processing steps. For example, in the make-clone scripted test, the linkLib task processes the outputs of the compileLib task. If compileLib is invoked separately from linkLib, then when we next call linkLib, it might not do anything even if there was recompilation of objects because the objects hadn't changed since the last time we called compileLib. To fix this, I generalizedthe previous cache so that it can be keyed on two tasks, one is the task whose value is being stored (foo in the example above) and the other is the task in which the stored task value is retrieved (bar in the example above). When the two tasks are the same, the behavior is the same as before. Currently the previous value for foo might be stored somewhere like: base_directory/target/streams/_global/_global/foo/previous Now, if foo is stored with respect to bar, it might be stored in base_directory/target/streams/_global/_global/bar/previous-dependencies/_global/_gloal/foo/previous By storing the files this way, it is easy to remove all of the previous values for the dependencies of a task. In addition to changing how the files are stored on disk, we have to store the references in memory differently. A given task can now have multiple previous references (if, say, two tasks bar and baz both depend on the previous value). When we complete the results, we first have to collect all of the successful tasks. Then for each successful task, we find all of its references. For each of the references, we only complete the value if the scope in which the task value is used is successful. In the actual implemenation in Previous.scala, there are a number places where we have to cast to ScopedKey[Task[Any]]. This is due to limitations of ScopedKey and Task being type invariant. These casts are all safe because we never try to get the value of anything, we only use the portion of the apis of these types that are independent of the value type. Structural typing where ScopedKey[Task[_]] gets inferred to ScopedKey[Task[x]] forSome x is a big part of why we have problems with type inference. --- .../src/main/scala/sbt/Previous.scala | 121 ++++++++++++++---- .../scala/sbt/internal/BuildStructure.scala | 33 ++++- main/src/main/scala/sbt/internal/Clean.scala | 2 + .../scala/sbt/internal/FileChangesMacro.scala | 65 ++++++++++ .../sbt/internal/nio/CheckBuildSources.scala | 5 +- main/src/main/scala/sbt/nio/Keys.scala | 11 +- main/src/main/scala/sbt/nio/Settings.scala | 7 +- sbt/src/main/scala/package.scala | 7 + sbt/src/sbt-test/nio/clean/build.sbt | 16 ++- sbt/src/sbt-test/nio/diff/build.sbt | 11 +- sbt/src/sbt-test/nio/diff/test | 6 +- sbt/src/sbt-test/nio/file-hashes/build.sbt | 16 ++- .../nio/file-hashes/changes/Foo-bad.txt | 1 + .../sbt-test/nio/file-hashes/changes/Foo.txt | 1 + sbt/src/sbt-test/nio/file-hashes/test | 10 ++ sbt/src/sbt-test/nio/last-modified/build.sbt | 27 +++- sbt/src/sbt-test/nio/last-modified/test | 14 +- sbt/src/sbt-test/nio/make-clone/build.sbt | 27 ++-- sbt/src/sbt-test/nio/make-clone/changes/bad.c | 1 + sbt/src/sbt-test/nio/make-clone/test | 6 + .../sbt-test/nio/multiple-inputs/bar/bar.md | 0 .../sbt-test/nio/multiple-inputs/build.sbt | 20 +++ .../nio/multiple-inputs/changes/bad.md | 1 + .../nio/multiple-inputs/changes/updated.md | 1 + .../sbt-test/nio/multiple-inputs/foo/foo.md | 1 + sbt/src/sbt-test/nio/multiple-inputs/test | 17 +++ .../sbt-test/nio/multiple-outputs/bar/bar.md | 0 .../sbt-test/nio/multiple-outputs/build.sbt | 32 +++++ .../nio/multiple-outputs/changes/bad.md | 1 + .../nio/multiple-outputs/changes/updated.md | 1 + .../sbt-test/nio/multiple-outputs/foo/foo.md | 1 + sbt/src/sbt-test/nio/multiple-outputs/test | 23 ++++ 32 files changed, 407 insertions(+), 78 deletions(-) create mode 100644 main/src/main/scala/sbt/internal/FileChangesMacro.scala create mode 100644 sbt/src/sbt-test/nio/file-hashes/changes/Foo-bad.txt create mode 100644 sbt/src/sbt-test/nio/file-hashes/changes/Foo.txt create mode 100644 sbt/src/sbt-test/nio/make-clone/changes/bad.c create mode 100644 sbt/src/sbt-test/nio/multiple-inputs/bar/bar.md create mode 100644 sbt/src/sbt-test/nio/multiple-inputs/build.sbt create mode 100644 sbt/src/sbt-test/nio/multiple-inputs/changes/bad.md create mode 100644 sbt/src/sbt-test/nio/multiple-inputs/changes/updated.md create mode 100644 sbt/src/sbt-test/nio/multiple-inputs/foo/foo.md create mode 100644 sbt/src/sbt-test/nio/multiple-inputs/test create mode 100644 sbt/src/sbt-test/nio/multiple-outputs/bar/bar.md create mode 100644 sbt/src/sbt-test/nio/multiple-outputs/build.sbt create mode 100644 sbt/src/sbt-test/nio/multiple-outputs/changes/bad.md create mode 100644 sbt/src/sbt-test/nio/multiple-outputs/changes/updated.md create mode 100644 sbt/src/sbt-test/nio/multiple-outputs/foo/foo.md create mode 100644 sbt/src/sbt-test/nio/multiple-outputs/test diff --git a/main-settings/src/main/scala/sbt/Previous.scala b/main-settings/src/main/scala/sbt/Previous.scala index 679d960e3..91340613f 100644 --- a/main-settings/src/main/scala/sbt/Previous.scala +++ b/main-settings/src/main/scala/sbt/Previous.scala @@ -10,7 +10,8 @@ package sbt import sbt.Def.{ Initialize, ScopedKey } import sbt.Previous._ import sbt.Scope.Global -import sbt.internal.util.{ IMap, RMap, ~> } +import sbt.internal.util._ +import sbt.std.TaskExtra._ import sbt.util.StampedFormat import sjsonnew.JsonFormat @@ -20,34 +21,42 @@ import scala.util.control.NonFatal * Reads the previous value of tasks on-demand. The read values are cached so that they are only read once per task execution. * `referenced` provides the `Format` to use for each key. */ -private[sbt] final class Previous(streams: Streams, referenced: IMap[ScopedTaskKey, Referenced]) { - private[this] val map = referenced.mapValues(toValue) - private[this] def toValue = λ[Referenced ~> ReferencedValue](new ReferencedValue(_)) +private[sbt] final class Previous(streams: Streams, referenced: IMap[Previous.Key, Referenced]) { + private[this] var map = IMap.empty[Previous.Key, ReferencedValue] + // We can't use mapValues to transform the map because mapValues is lazy and evaluates the + // transformation function every time a value is fetched from the map, defeating the entire + // purpose of ReferencedValue. + for (referenced.TPair(k, v) <- referenced.toTypedSeq) map = map.put(k, new ReferencedValue(v)) private[this] final class ReferencedValue[T](referenced: Referenced[T]) { - import referenced.{ stamped, task } - lazy val previousValue: Option[T] = { - try Option(streams(task).cacheStoreFactory.make(StreamName).read[T]()(stamped)) - catch { case NonFatal(_) => None } - } + lazy val previousValue: Option[T] = referenced.read(streams) } /** Used by the .previous runtime implementation to get the previous value for task `key`. */ - private def get[T](key: ScopedKey[Task[T]]): Option[T] = + private def get[T](key: Key[T]): Option[T] = map.get(key).flatMap(_.previousValue) } object Previous { import sjsonnew.BasicJsonProtocol.StringJsonFormat private[sbt] type ScopedTaskKey[T] = ScopedKey[Task[T]] + private type AnyTaskKey = ScopedTaskKey[Any] private type Streams = sbt.std.Streams[ScopedKey[_]] /** The stream where the task value is persisted. */ private final val StreamName = "previous" + private[sbt] final val DependencyDirectory = "previous-dependencies" /** Represents a reference task.previous*/ - private[sbt] final class Referenced[T](val task: ScopedKey[Task[T]], val format: JsonFormat[T]) { - lazy val stamped = StampedFormat.withStamp(task.key.manifest.toString)(format) + private[sbt] final class Referenced[T](val key: Key[T], val format: JsonFormat[T]) { + def this(task: ScopedTaskKey[T], format: JsonFormat[T]) = this(Key(task, task), format) + @deprecated("unused", "1.3.0") + private[sbt] def task: ScopedKey[Task[T]] = key.task + lazy val stamped: JsonFormat[T] = + StampedFormat.withStamp(key.task.key.manifest.toString)(format) def setTask(newTask: ScopedKey[Task[T]]) = new Referenced(newTask, format) + private[sbt] def read(streams: Streams): Option[T] = + try Option(streams(key.cacheKey).cacheStoreFactory.make(StreamName).read[T]()(stamped)) + catch { case NonFatal(_) => None } } private[sbt] val references = SettingKey[References]( @@ -61,16 +70,41 @@ object Previous { KeyRanks.Invisible ) + private[sbt] class Key[T](val task: ScopedKey[Task[T]], val enclosing: AnyTaskKey) { + override def equals(o: Any): Boolean = o match { + case that: Key[_] => this.task == that.task && this.enclosing == that.enclosing + case _ => false + } + override def hashCode(): Int = (task.## * 31) ^ enclosing.## + def cacheKey: AnyTaskKey = { + if (task == enclosing) task + else { + val am = enclosing.scope.extra match { + case Select(a) => a.put(scopedKeyAttribute, task.asInstanceOf[AnyTaskKey]) + case _ => AttributeMap.empty.put(scopedKeyAttribute, task.asInstanceOf[AnyTaskKey]) + } + Def.ScopedKey(enclosing.scope.copy(extra = Select(am)), enclosing.key) + } + }.asInstanceOf[AnyTaskKey] + } + private[sbt] object Key { + def apply[T, U](key: ScopedKey[Task[T]], enclosing: ScopedKey[Task[U]]): Key[T] = + new Key(key, enclosing.asInstanceOf[AnyTaskKey]) + } + /** Records references to previous task value. This should be completely populated after settings finish loading. */ private[sbt] final class References { - private[this] var map = IMap.empty[ScopedTaskKey, Referenced] + private[this] var map = IMap.empty[Key, Referenced] + @deprecated("unused", "1.3.0") + def recordReference[T](key: ScopedKey[Task[T]], format: JsonFormat[T]): Unit = + recordReference(Key(key, key), format) // TODO: this arbitrarily chooses a JsonFormat. // The need to choose is a fundamental problem with this approach, but this should at least make a stable choice. - def recordReference[T](key: ScopedKey[Task[T]], format: JsonFormat[T]): Unit = synchronized { + def recordReference[T](key: Key[T], format: JsonFormat[T]): Unit = synchronized { map = map.put(key, new Referenced(key, format)) } - def getReferences: IMap[ScopedTaskKey, Referenced] = synchronized { map } + def getReferences: IMap[Key, Referenced] = synchronized { map } } /** Persists values of tasks t where there is some task referencing it via t.previous. */ @@ -80,27 +114,60 @@ object Previous { streams: Streams ): Unit = { val map = referenced.getReferences - def impl[T](key: ScopedKey[_], result: T): Unit = - for (i <- map.get(key.asInstanceOf[ScopedTaskKey[T]])) { - val out = streams.apply(i.task).cacheStoreFactory.make(StreamName) - try out.write(result)(i.stamped) - catch { case NonFatal(_) => } - } + val reverse = map.keys.groupBy(_.task) + // We first collect all of the successful tasks and write their scoped key into a map + // along with their values. + val successfulTaskResults = (for { + results.TPair(task, Value(v)) <- results.toTypedSeq + key <- task.info.attributes.get(Def.taskDefinitionKey).asInstanceOf[Option[AnyTaskKey]] + } yield key -> v).toMap + // We then traverse the successful results and look up all of the referenced values for + // each of these tasks. This can be a many to one relationship if multiple tasks refer + // the previous value of another task. For each reference we find, we check if the task has + // been successfully evaluated. If so, we write it to the appropriate previous cache for + // the completed task. for { - results.TPair(Task(info, _), Value(result)) <- results.toTypedSeq - key <- info.attributes get Def.taskDefinitionKey - } impl(key, result) + (k, v) <- successfulTaskResults + keys <- reverse.get(k) + key <- keys if successfulTaskResults.contains(key.enclosing) + ref <- map.get(key.asInstanceOf[Key[Any]]) + } { + val out = streams(key.cacheKey).cacheStoreFactory.make(StreamName) + try out.write(v)(ref.stamped) + catch { case NonFatal(_) => } + } } + private[sbt] val scopedKeyAttribute = AttributeKey[AnyTaskKey]( + "previous-scoped-key-attribute", + "Specifies a scoped key for a task on which .previous is called. Used to " + + "set the cache directory for the task-specific previous value: see Previous.runtimeInEnclosingTask." + ) /** Public as a macro implementation detail. Do not call directly. */ def runtime[T](skey: TaskKey[T])(implicit format: JsonFormat[T]): Initialize[Task[Option[T]]] = { val inputs = (cache in Global) zip Def.validated(skey, selfRefOk = true) zip (references in Global) inputs { case ((prevTask, resolved), refs) => - refs.recordReference(resolved, format) // always evaluated on project load - import std.TaskExtra._ - prevTask.map(_ get resolved) // evaluated if this task is evaluated + val key = Key(resolved, resolved) + refs.recordReference(key, format) // always evaluated on project load + prevTask.map(_.get(key)) // evaluated if this task is evaluated + } + } + + /** Public as a macro implementation detail. Do not call directly. */ + def runtimeInEnclosingTask[T](skey: TaskKey[T])( + implicit format: JsonFormat[T] + ): Initialize[Task[Option[T]]] = { + val inputs = (cache in Global) + .zip(Def.validated(skey, selfRefOk = true)) + .zip(references in Global) + .zip(Def.resolvedScoped) + inputs { + case (((prevTask, resolved), refs), inTask: ScopedKey[Task[_]] @unchecked) => + val key = Key(resolved, inTask) + refs.recordReference(key, format) // always evaluated on project load + prevTask.map(_.get(key)) // evaluated if this task is evaluated } } } diff --git a/main/src/main/scala/sbt/internal/BuildStructure.scala b/main/src/main/scala/sbt/internal/BuildStructure.scala index f7af061dd..581870263 100644 --- a/main/src/main/scala/sbt/internal/BuildStructure.scala +++ b/main/src/main/scala/sbt/internal/BuildStructure.scala @@ -15,6 +15,7 @@ import Def.{ ScopeLocal, ScopedKey, Setting, displayFull } import BuildPaths.outputDirectory import Scope.GlobalScope import BuildStreams.Streams +import sbt.LocalRootProject import sbt.io.syntax._ import sbt.internal.util.{ AttributeEntry, AttributeKey, AttributeMap, Attributed, Settings } import sbt.internal.util.Attributed.data @@ -291,6 +292,7 @@ object BuildStreams { final val GlobalPath = "_global" final val BuildUnitPath = "_build" final val StreamsDirectory = "streams" + private final val RootPath = "_root" def mkStreams( units: Map[URI, LoadedBuildUnit], @@ -338,14 +340,39 @@ object BuildStreams { pathComponent(scope.config, scoped, "config")(_.name) :: pathComponent(scope.task, scoped, "task")(_.label) :: pathComponent(scope.extra, scoped, "extra")(showAMap) :: - scoped.key.label :: - Nil + scoped.key.label :: previousComponent(scope.extra) } + private def previousComponent(value: ScopeAxis[AttributeMap]): List[String] = + value match { + case Select(am) => + am.get(Previous.scopedKeyAttribute) match { + case Some(sk) => + val project = sk.scope.project match { + case Zero => GlobalPath + case Select(BuildRef(_)) => BuildUnitPath + case Select(ProjectRef(_, id)) => id + case Select(LocalProject(id)) => id + case Select(RootProject(_)) => RootPath + case Select(LocalRootProject) => LocalRootProject.toString + case Select(ThisBuild) | Select(ThisProject) | This => + // Don't want to crash if somehow an unresolved key makes it in here. + This.toString + } + List(Previous.DependencyDirectory, project) ++ nonProjectPath(sk) + case _ => Nil + } + case _ => Nil + } def showAMap(a: AttributeMap): String = a.entries.toStream .sortBy(_.key.label) - .map { case AttributeEntry(key, value) => s"${key.label}=$value" } + .flatMap { + // The Previous.scopedKeyAttribute is an implementation detail that allows us to get a + // more specific cache directory for a task stream. + case AttributeEntry(key, _) if key == Previous.scopedKeyAttribute => Nil + case AttributeEntry(key, value) => s"${key.label}=$value" :: Nil + } .mkString(" ") def projectPath( diff --git a/main/src/main/scala/sbt/internal/Clean.scala b/main/src/main/scala/sbt/internal/Clean.scala index 1e9f02e22..29a8064f4 100644 --- a/main/src/main/scala/sbt/internal/Clean.scala +++ b/main/src/main/scala/sbt/internal/Clean.scala @@ -139,7 +139,9 @@ private[sbt] object Clean { // We do not want to inadvertently delete files that are not in the target directory. val excludeFilter: Path => Boolean = path => !path.startsWith(targetDir) || filter(path) val delete = cleanDelete(scope).value + val st = streams.in(scope).value taskKey.previous.foreach(_.toSeqPath.foreach(p => if (!excludeFilter(p)) delete(p))) + delete(st.cacheDirectory.toPath / Previous.DependencyDirectory) } } tag Tags.Clean private[this] def tryDelete(debug: String => Unit): Path => Unit = path => { diff --git a/main/src/main/scala/sbt/internal/FileChangesMacro.scala b/main/src/main/scala/sbt/internal/FileChangesMacro.scala new file mode 100644 index 000000000..e103a54f6 --- /dev/null +++ b/main/src/main/scala/sbt/internal/FileChangesMacro.scala @@ -0,0 +1,65 @@ +/* + * sbt + * Copyright 2011 - 2018, Lightbend, Inc. + * Copyright 2008 - 2010, Mark Harrah + * Licensed under Apache License 2.0 (see LICENSE) + */ + +package sbt +package internal + +import java.nio.file.{ Path => NioPath } + +import sbt.nio.FileStamp +import sbt.nio.Keys._ +import sbt.nio.file.ChangedFiles + +import scala.annotation.compileTimeOnly +import scala.language.experimental.macros +import scala.reflect.macros.blackbox + +/** + * Provides extension methods to `TaskKey[T]` that can be use to fetch the input and output file + * dependency changes for a task. Nothing in this object is intended to be called directly but, + * because there are macro definitions, some of the definitions must be public. + * + */ +object FileChangesMacro { + private[sbt] sealed abstract class TaskOps[T](val taskKey: TaskKey[T]) { + @compileTimeOnly( + "`changedInputFiles` can only be called on a task within a task definition macro, such as :=, +=, ++=, or Def.task." + ) + def changedInputFiles: Option[ChangedFiles] = macro changedInputFilesImpl[T] + @compileTimeOnly( + "`changedOutputFiles` can only be called on a task within a task definition macro, such as :=, +=, ++=, or Def.task." + ) + def changedOutputFiles: Option[ChangedFiles] = macro changedOutputFilesImpl[T] + } + def changedInputFilesImpl[T: c.WeakTypeTag](c: blackbox.Context): c.Expr[Option[ChangedFiles]] = { + impl[T](c)(c.universe.reify(changedInputFiles), c.universe.reify(inputFileStamps)) + } + def changedOutputFilesImpl[T: c.WeakTypeTag]( + c: blackbox.Context + ): c.Expr[Option[ChangedFiles]] = { + impl[T](c)(c.universe.reify(changedOutputFiles), c.universe.reify(outputFileStamps)) + } + private def impl[T: c.WeakTypeTag]( + c: blackbox.Context + )( + changeKey: c.Expr[TaskKey[Seq[(NioPath, FileStamp)] => Option[ChangedFiles]]], + mapKey: c.Expr[TaskKey[Seq[(NioPath, FileStamp)]]] + ): c.Expr[Option[ChangedFiles]] = { + import c.universe._ + val taskTpe = c.weakTypeOf[TaskKey[T]] + lazy val err = "Couldn't expand file change macro." + val taskKey = c.Expr[TaskKey[T]](c.macroApplication match { + case Select(Apply(_, k :: Nil), _) if k.tpe <:< taskTpe => k + case _ => c.abort(c.enclosingPosition, err) + }) + reify { + val changes = (changeKey.splice in taskKey.splice).value + import sbt.nio.FileStamp.Formats._ + Previous.runtimeInEnclosingTask(mapKey.splice in taskKey.splice).value.flatMap(changes) + } + } +} diff --git a/main/src/main/scala/sbt/internal/nio/CheckBuildSources.scala b/main/src/main/scala/sbt/internal/nio/CheckBuildSources.scala index 3aeaf5b57..0ee250362 100644 --- a/main/src/main/scala/sbt/internal/nio/CheckBuildSources.scala +++ b/main/src/main/scala/sbt/internal/nio/CheckBuildSources.scala @@ -22,8 +22,11 @@ private[sbt] object CheckBuildSources { (onChangedBuildSource in Scope.Global).value match { case IgnoreSourceChanges => new StateTransform(st) case o => + import sbt.nio.FileStamp.Formats._ logger.debug("Checking for meta build source updates") - (changedInputFiles in checkBuildSources).value match { + val previous = (inputFileStamps in checkBuildSources).previous + val changes = (changedInputFiles in checkBuildSources).value + previous.flatMap(changes) match { case Some(cf: ChangedFiles) if !firstTime => val rawPrefix = s"build source files have changed\n" + (if (cf.created.nonEmpty) s"new files: ${cf.created.mkString("\n ", "\n ", "\n")}" diff --git a/main/src/main/scala/sbt/nio/Keys.scala b/main/src/main/scala/sbt/nio/Keys.scala index 71c5a38bb..d991691fc 100644 --- a/main/src/main/scala/sbt/nio/Keys.scala +++ b/main/src/main/scala/sbt/nio/Keys.scala @@ -29,7 +29,8 @@ object Keys { case object ReloadOnSourceChanges extends WatchBuildSourceOption val allInputFiles = taskKey[Seq[Path]]("All of the file inputs for a task excluding directories and hidden files.") - val changedInputFiles = taskKey[Option[ChangedFiles]]("The changed files for a task") + val changedInputFiles = + taskKey[Seq[(Path, FileStamp)] => Option[ChangedFiles]]("The changed files for a task") val fileInputs = settingKey[Seq[Glob]]( "The file globs that are used by a task. This setting will generally be scoped per task. It will also be used to determine the sources to watch during continuous execution." ) @@ -41,7 +42,9 @@ object Keys { val allOutputFiles = taskKey[Seq[Path]]("All of the file outputs for a task excluding directories and hidden files.") val changedOutputFiles = - taskKey[Option[ChangedFiles]]("The files that have changed since the last task run.") + taskKey[Seq[(Path, FileStamp)] => Option[ChangedFiles]]( + "The files that have changed since the last task run." + ) val outputFileStamper = settingKey[FileStamper]( "Toggles the file stamping implementation used to determine whether or not a file has been modified." ) @@ -130,10 +133,10 @@ object Keys { private[sbt] val dynamicFileOutputs = taskKey[Seq[Path]]("The outputs of a task").withRank(Invisible) - private[sbt] val inputFileStamps = + val inputFileStamps = taskKey[Seq[(Path, FileStamp)]]("Retrieves the hashes for a set of task input files") .withRank(Invisible) - private[sbt] val outputFileStamps = + val outputFileStamps = taskKey[Seq[(Path, FileStamp)]]("Retrieves the hashes for a set of task output files") .withRank(Invisible) private[sbt] type FileAttributeMap = diff --git a/main/src/main/scala/sbt/nio/Settings.scala b/main/src/main/scala/sbt/nio/Settings.scala index c8113d068..f150b9a01 100644 --- a/main/src/main/scala/sbt/nio/Settings.scala +++ b/main/src/main/scala/sbt/nio/Settings.scala @@ -24,8 +24,8 @@ import sbt.std.TaskExtra._ import sjsonnew.JsonFormat import scala.collection.JavaConverters._ -import scala.collection.mutable import scala.collection.immutable.VectorBuilder +import scala.collection.mutable private[sbt] object Settings { private[sbt] def inject(transformed: Seq[Def.Setting[_]]): Seq[Def.Setting[_]] = { @@ -227,13 +227,14 @@ private[sbt] object Settings { }) :: Nil private[this] def changedFilesImpl( scopedKey: Def.ScopedKey[_], - changeKey: TaskKey[Option[ChangedFiles]], + changeKey: TaskKey[Seq[(Path, FileStamp)] => Option[ChangedFiles]], stampKey: TaskKey[Seq[(Path, FileStamp)]] ): Def.Setting[_] = addTaskDefinition(changeKey in scopedKey.scope := { val current = (stampKey in scopedKey.scope).value - (stampKey in scopedKey.scope).previous.flatMap(changedFiles(_, current)) + previous => changedFiles(previous, current) }) + private[sbt] def changedFiles( previous: Seq[(Path, FileStamp)], current: Seq[(Path, FileStamp)] diff --git a/sbt/src/main/scala/package.scala b/sbt/src/main/scala/package.scala index 00ecb2672..01d133947 100644 --- a/sbt/src/main/scala/package.scala +++ b/sbt/src/main/scala/package.scala @@ -9,6 +9,8 @@ import sbt.nio.FileStamp import sjsonnew.JsonFormat import java.nio.file.{ Path => NioPath } +import sbt.internal.FileChangesMacro + import scala.language.experimental.macros package object sbt @@ -33,6 +35,11 @@ package object sbt implicit def fileToRichFile(file: File): sbt.io.RichFile = new sbt.io.RichFile(file) implicit def filesToFinder(cc: Traversable[File]): sbt.io.PathFinder = sbt.io.PathFinder.strict(cc) + /* + * Provides macro extension methods. Because the extension methods are all macros, no instance + * of FileChangesMacro.TaskOps is ever made which is why it is ok to use `???`. + */ + implicit def taskToTaskOpts[T](t: TaskKey[T]): FileChangesMacro.TaskOps[T] = ??? implicit val fileStampJsonFormatter: JsonFormat[Seq[(NioPath, FileStamp)]] = FileStamp.Formats.seqPathFileStampJsonFormatter implicit val pathJsonFormatter: JsonFormat[Seq[NioPath]] = FileStamp.Formats.seqPathJsonFormatter diff --git a/sbt/src/sbt-test/nio/clean/build.sbt b/sbt/src/sbt-test/nio/clean/build.sbt index 1c98f43ae..30d65a36b 100644 --- a/sbt/src/sbt-test/nio/clean/build.sbt +++ b/sbt/src/sbt-test/nio/clean/build.sbt @@ -9,7 +9,7 @@ copyFile / target := baseDirectory.value / "out" copyFile := Def.task { val prev = copyFile.previous - val changes: Option[Seq[Path]] = (copyFile / changedInputFiles).value.map { + val changes: Option[Seq[Path]] = copyFile.changedInputFiles.map { case ChangedFiles(c, _, u) => c ++ u } prev match { @@ -35,9 +35,15 @@ checkOutDirectoryHasFile := { assert(result == Seq(baseDirectory.value / "out" / "Foo.txt")) } -val checkCount = inputKey[Unit]("Check that the expected number of evaluations have run.") -checkCount := Def.inputTask { - val expected = Def.spaceDelimited("").parsed.head.toInt +commands += Command.single("checkCount") { (s, digits) => + s"writeCount $digits" :: "checkCountImpl" :: s +} + +val writeCount = inputKey[Unit]("writes the count to a file") +writeCount := IO.write(baseDirectory.value / "expectedCount", Def.spaceDelimited().parsed.head) +val checkCountImpl = taskKey[Unit]("Check that the expected number of evaluations have run.") +checkCountImpl := { + val expected = IO.read(baseDirectory.value / "expectedCount").toInt val previous = copyFile.previous.getOrElse(0) assert(previous == expected) -}.evaluated +} diff --git a/sbt/src/sbt-test/nio/diff/build.sbt b/sbt/src/sbt-test/nio/diff/build.sbt index 69a0927f7..3bf3d6dae 100644 --- a/sbt/src/sbt-test/nio/diff/build.sbt +++ b/sbt/src/sbt-test/nio/diff/build.sbt @@ -4,8 +4,9 @@ val fileInputTask = taskKey[Unit]("task with file inputs") fileInputTask / fileInputs += Glob(baseDirectory.value / "base", "*.md") -fileInputTask := Def.taskDyn { - if ((fileInputTask / changedInputFiles).value.fold(false)(_.updated.nonEmpty)) - Def.task(assert(true)) - else Def.task(assert(false)) -}.value +fileInputTask := { + if (fileInputTask.changedInputFiles.fold(false)( + _.created.exists(_.getFileName.toString.startsWith("foo")) + )) assert(false) + assert(true) +} diff --git a/sbt/src/sbt-test/nio/diff/test b/sbt/src/sbt-test/nio/diff/test index 1a1fd1c11..3ca2bbc80 100644 --- a/sbt/src/sbt-test/nio/diff/test +++ b/sbt/src/sbt-test/nio/diff/test @@ -1,5 +1,9 @@ --> fileInputTask +> fileInputTask $ copy-file changes/Bar.md base/Bar.md > fileInputTask + +$ copy-file changes/Bar.md base/foo.md + +-> fileInputTask diff --git a/sbt/src/sbt-test/nio/file-hashes/build.sbt b/sbt/src/sbt-test/nio/file-hashes/build.sbt index eba843fca..d9d9186c7 100644 --- a/sbt/src/sbt-test/nio/file-hashes/build.sbt +++ b/sbt/src/sbt-test/nio/file-hashes/build.sbt @@ -7,20 +7,22 @@ foo / fileInputs := Seq( ) val checkModified = taskKey[Unit]("check that modified files are returned") -checkModified := Def.taskDyn { - val modified = (foo / changedInputFiles).value.map(_.updated).getOrElse(Nil) +checkModified := { + val changes = foo.changedInputFiles + val modified = changes.map(_.updated).getOrElse(Nil) + println(modified) val allFiles = (foo / allInputFiles).value - if (modified.isEmpty) Def.task(assert(true)) - else Def.task { + if (modified.isEmpty) assert(true) + else { assert(modified != allFiles) assert(modified == Seq((baseDirectory.value / "base" / "Bar.md").toPath)) } -}.value +} val checkRemoved = taskKey[Unit]("check that removed files are returned") checkRemoved := Def.taskDyn { val files = (foo / allInputFiles).value - val removed = (foo / changedInputFiles).value.map(_.deleted).getOrElse(Nil) + val removed = foo.changedInputFiles.map(_.deleted).getOrElse(Nil) if (removed.isEmpty) Def.task(assert(true)) else Def.task { assert(files == Seq((baseDirectory.value / "base" / "Foo.txt").toPath)) @@ -31,7 +33,7 @@ checkRemoved := Def.taskDyn { val checkAdded = taskKey[Unit]("check that modified files are returned") checkAdded := Def.taskDyn { val files = (foo / allInputFiles).value - val added = (foo / changedInputFiles).value.map(_.created).getOrElse(Nil) + val added = foo.changedInputFiles.map(_.created).getOrElse(Nil) if (added.isEmpty || (files.toSet == added.toSet)) Def.task(assert(true)) else Def.task { val base = baseDirectory.value / "base" diff --git a/sbt/src/sbt-test/nio/file-hashes/changes/Foo-bad.txt b/sbt/src/sbt-test/nio/file-hashes/changes/Foo-bad.txt new file mode 100644 index 000000000..551f4d337 --- /dev/null +++ b/sbt/src/sbt-test/nio/file-hashes/changes/Foo-bad.txt @@ -0,0 +1 @@ +fooo \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/file-hashes/changes/Foo.txt b/sbt/src/sbt-test/nio/file-hashes/changes/Foo.txt new file mode 100644 index 000000000..191028156 --- /dev/null +++ b/sbt/src/sbt-test/nio/file-hashes/changes/Foo.txt @@ -0,0 +1 @@ +foo \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/file-hashes/test b/sbt/src/sbt-test/nio/file-hashes/test index 621605305..094635c99 100644 --- a/sbt/src/sbt-test/nio/file-hashes/test +++ b/sbt/src/sbt-test/nio/file-hashes/test @@ -4,6 +4,16 @@ $ copy-file changes/Bar.md base/Bar.md > checkModified +$ copy-file changes/Foo-bad.txt base/Foo.txt + +-> checkModified + +-> checkModified + +$ copy-file changes/Foo.txt base/Foo.txt + +> checkModified + > checkRemoved $ delete base/Bar.md diff --git a/sbt/src/sbt-test/nio/last-modified/build.sbt b/sbt/src/sbt-test/nio/last-modified/build.sbt index 58678dcf0..6efdd949a 100644 --- a/sbt/src/sbt-test/nio/last-modified/build.sbt +++ b/sbt/src/sbt-test/nio/last-modified/build.sbt @@ -1,17 +1,32 @@ import sbt.nio.Keys._ +import scala.util.Try + val fileInputTask = taskKey[Unit]("task with file inputs") fileInputTask / fileInputs += (baseDirectory.value / "base").toGlob / "*.md" fileInputTask / inputFileStamper := sbt.nio.FileStamper.LastModified -fileInputTask := Def.taskDyn { - (fileInputTask / changedInputFiles).value match { - case Some(ChangedFiles(_, _, u)) if u.nonEmpty => Def.task(assert(true)) - case None => Def.task(assert(false)) - } -}.value +fileInputTask := { + /* + * Normally we'd use an input task for this kind of thing, but input tasks don't work with + * incremental task evaluation so, instead, we manually set the input in a file. As a result, + * most of the test commands have to be split into two: one to set the expected result and one + * to validate it. + */ + val expectedChanges = + Try(IO.read(baseDirectory.value / "expected").split(" ").toSeq.filterNot(_.isEmpty)) + .getOrElse(Nil) + .map(baseDirectory.value.toPath / "base" / _) + val actual = fileInputTask.changedInputFiles.toSeq.flatMap(_.updated) + assert(actual.toSet == expectedChanges.toSet) +} + +val setExpected = inputKey[Unit]("Writes a space separated list of files") +setExpected := { + IO.write(baseDirectory.value / "expected", Def.spaceDelimited().parsed.mkString(" ")) +} val setLastModified = taskKey[Unit]("Reset the last modified time") setLastModified := { diff --git a/sbt/src/sbt-test/nio/last-modified/test b/sbt/src/sbt-test/nio/last-modified/test index 15dab9326..8c2ffd07f 100644 --- a/sbt/src/sbt-test/nio/last-modified/test +++ b/sbt/src/sbt-test/nio/last-modified/test @@ -1,8 +1,10 @@ --> fileInputTask +> fileInputTask $ touch base/Bar.md -# this should succeed even though the contents didn't change +# The change to Bar.md should be detected since we set last modified instead of hash +> setExpected Bar.md + > fileInputTask $ copy-file changes/Bar.md base/Bar.md @@ -18,9 +20,13 @@ $ copy-file changes/Bar2.md base/Bar.md > setLastModified -# this should fail even though we changed the file with a copy --> fileInputTask +# Since we reverted to the previous last modified time, there should be no changes +> setExpected + +> fileInputTask $ touch base/Bar.md +> setExpected Bar.md + > fileInputTask diff --git a/sbt/src/sbt-test/nio/make-clone/build.sbt b/sbt/src/sbt-test/nio/make-clone/build.sbt index 05327cb25..c6efc7d5a 100644 --- a/sbt/src/sbt-test/nio/make-clone/build.sbt +++ b/sbt/src/sbt-test/nio/make-clone/build.sbt @@ -11,18 +11,20 @@ compileLib / fileInputs := { } compileLib / target := baseDirectory.value / "out" / "objects" compileLib := { + val objectDir: Path = (compileLib / target).value.toPath / "objects" + def objectPath(path: Path): Path = { + val name = path.getFileName.toString + objectDir.resolve(name.substring(0, name.lastIndexOf('.')) + ".o") + } val allFiles: Seq[Path] = (compileLib / allInputFiles).value - val changedFiles: Option[Seq[Path]] = (compileLib / changedInputFiles).value match { - case Some(ChangedFiles(c, _, u)) => Some(c ++ u) + val changedFiles: Option[Seq[Path]] = compileLib.changedInputFiles match { + case Some(ChangedFiles(c, d, u)) => + d.foreach(p => Files.deleteIfExists(objectPath(p))) + Some(c ++ u) case None => None } val include = (compileLib / sourceDirectory).value / "include" - val objectDir: Path = (compileLib / target).value.toPath / "objects" val logger = streams.value.log - def objectFileName(path: Path): String = { - val name = path.getFileName.toString - name.substring(0, name.lastIndexOf('.')) + ".o" - } compileLib.previous match { case Some(outputs: Seq[Path]) if changedFiles.isEmpty => logger.info("Not compiling libfoo: no inputs have changed.") @@ -34,20 +36,21 @@ compileLib := { if (changedFiles.fold(false)(_.exists(extensionFilter("h")))) allFiles.filter(extensionFilter("c")) else changedFiles.getOrElse(allFiles).filter(extensionFilter("c")) - cFiles.map { file => - val outFile = objectDir.resolve(objectFileName(file)) + cFiles.sorted.foreach { file => + val outFile = objectPath(file) logger.info(s"Compiling $file to $outFile") (Seq("gcc") ++ compileOpts.value ++ Seq("-c", file.toString, s"-I$include", "-o", outFile.toString)).!! outFile } + allFiles.filter(extensionFilter("c")).map(objectPath) } } val linkLib = taskKey[Path]("") linkLib / target := baseDirectory.value / "out" / "lib" linkLib := { - val changedObjects = (compileLib / changedOutputFiles).value + val changedObjects = compileLib.changedOutputFiles val outPath = (linkLib / target).value.toPath val allObjects = (compileLib / allOutputFiles).value.map(_.toString) val logger = streams.value.log @@ -76,8 +79,8 @@ compileMain / fileInputs := (compileMain / sourceDirectory).value.toGlob / "main compileMain / target := baseDirectory.value / "out" / "main" compileMain := { val library = linkLib.value - val changed: Boolean = (compileMain / changedInputFiles).value.nonEmpty || - (linkLib / changedOutputFiles).value.nonEmpty + val changed: Boolean = compileMain.changedInputFiles.nonEmpty || + linkLib.changedOutputFiles.nonEmpty val include = (compileLib / sourceDirectory).value / "include" val logger = streams.value.log val outDir = (compileMain / target).value.toPath diff --git a/sbt/src/sbt-test/nio/make-clone/changes/bad.c b/sbt/src/sbt-test/nio/make-clone/changes/bad.c new file mode 100644 index 000000000..4e0b2da04 --- /dev/null +++ b/sbt/src/sbt-test/nio/make-clone/changes/bad.c @@ -0,0 +1 @@ +int \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/make-clone/test b/sbt/src/sbt-test/nio/make-clone/test index 11c715993..235a39f76 100644 --- a/sbt/src/sbt-test/nio/make-clone/test +++ b/sbt/src/sbt-test/nio/make-clone/test @@ -20,6 +20,12 @@ > checkOutput 2 8 +$ copy-file changes/bad.c src/lib/bad.c + $ copy-file changes/lib.c src/lib/lib.c +-> checkOutput 2 4 + +$ delete src/lib/bad.c + > checkOutput 2 4 diff --git a/sbt/src/sbt-test/nio/multiple-inputs/bar/bar.md b/sbt/src/sbt-test/nio/multiple-inputs/bar/bar.md new file mode 100644 index 000000000..e69de29bb diff --git a/sbt/src/sbt-test/nio/multiple-inputs/build.sbt b/sbt/src/sbt-test/nio/multiple-inputs/build.sbt new file mode 100644 index 000000000..4610a069f --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-inputs/build.sbt @@ -0,0 +1,20 @@ +val foo = taskKey[Unit]("dummy task with inputs") +foo / fileInputs += baseDirectory.value.toGlob / "foo" / * + +val bar = taskKey[Unit]("dummy task with inputs") +bar / fileInputs += baseDirectory.value.toGlob / "bar" / * + +val check = taskKey[Unit]("check expected changes") +check := { + foo.changedInputFiles.toSeq.flatMap(_.updated) ++ + bar.changedInputFiles.toSeq.flatMap(_.updated) match { + case Nil => + val contents = IO.read(baseDirectory.value / "foo" / "foo.md") + assert(contents == "foo", s"expected 'foo', got '$contents") + case Seq(f, b) => + val fContents = IO.read(f.toFile) + assert(fContents == "updated", s"expected 'updated', got '$fContents' for $f") + val bContents = IO.read(b.toFile) + assert(bContents == "updated", s"expected 'updated', got '$fContents' for $b") + } +} diff --git a/sbt/src/sbt-test/nio/multiple-inputs/changes/bad.md b/sbt/src/sbt-test/nio/multiple-inputs/changes/bad.md new file mode 100644 index 000000000..44d6628cd --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-inputs/changes/bad.md @@ -0,0 +1 @@ +bad \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/multiple-inputs/changes/updated.md b/sbt/src/sbt-test/nio/multiple-inputs/changes/updated.md new file mode 100644 index 000000000..f55556eed --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-inputs/changes/updated.md @@ -0,0 +1 @@ +updated \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/multiple-inputs/foo/foo.md b/sbt/src/sbt-test/nio/multiple-inputs/foo/foo.md new file mode 100644 index 000000000..191028156 --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-inputs/foo/foo.md @@ -0,0 +1 @@ +foo \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/multiple-inputs/test b/sbt/src/sbt-test/nio/multiple-inputs/test new file mode 100644 index 000000000..348eb304b --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-inputs/test @@ -0,0 +1,17 @@ +> check + +$ copy-file changes/bad.md foo/foo.md + +$ copy-file changes/updated.md bar/bar.md + +-> check + +-> check + +$ copy-file changes/updated.md foo/foo.md + +> check + +# the changes should be empty now but the content of foo/foo.md is no longer "foo" +-> check + diff --git a/sbt/src/sbt-test/nio/multiple-outputs/bar/bar.md b/sbt/src/sbt-test/nio/multiple-outputs/bar/bar.md new file mode 100644 index 000000000..e69de29bb diff --git a/sbt/src/sbt-test/nio/multiple-outputs/build.sbt b/sbt/src/sbt-test/nio/multiple-outputs/build.sbt new file mode 100644 index 000000000..c31640795 --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-outputs/build.sbt @@ -0,0 +1,32 @@ +import java.nio.file.Path + +val foo = taskKey[Seq[Path]]("dummy task with inputs") +foo := fileTreeView.value.list(baseDirectory.value.toGlob / "foo" / *).map(_._1) + +val bar = taskKey[Seq[Path]]("dummy task with inputs") +bar := fileTreeView.value.list(baseDirectory.value.toGlob / "bar" / *).map(_._1) + +val check = taskKey[Unit]("check expected changes") +check := { + foo.changedOutputFiles.toSeq.flatMap(_.updated) ++ + bar.changedOutputFiles.toSeq.flatMap(_.updated) match { + case Nil => + val contents = IO.read(baseDirectory.value / "foo" / "foo.md") + assert(contents == "foo", s"expected 'foo', got '$contents") + case Seq(f, b) => + val fContents = IO.read(f.toFile) + assert(fContents == "updated", s"expected 'updated', got '$fContents' for $f") + val bContents = IO.read(b.toFile) + assert(bContents == "updated", s"expected 'updated', got '$fContents' for $b") + } +} + +val setModified = inputKey[Unit]("set the last modified time for a file") +setModified := { + val Seq(relative, lm) = Def.spaceDelimited().parsed + // be safe in case of windows + val file = relative.split("/") match { + case Array(h, rest @ _*) => rest.foldLeft(baseDirectory.value / h)(_ / _) + } + IO.setModifiedTimeOrFalse(file, lm.toLong) +} diff --git a/sbt/src/sbt-test/nio/multiple-outputs/changes/bad.md b/sbt/src/sbt-test/nio/multiple-outputs/changes/bad.md new file mode 100644 index 000000000..44d6628cd --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-outputs/changes/bad.md @@ -0,0 +1 @@ +bad \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/multiple-outputs/changes/updated.md b/sbt/src/sbt-test/nio/multiple-outputs/changes/updated.md new file mode 100644 index 000000000..f55556eed --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-outputs/changes/updated.md @@ -0,0 +1 @@ +updated \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/multiple-outputs/foo/foo.md b/sbt/src/sbt-test/nio/multiple-outputs/foo/foo.md new file mode 100644 index 000000000..191028156 --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-outputs/foo/foo.md @@ -0,0 +1 @@ +foo \ No newline at end of file diff --git a/sbt/src/sbt-test/nio/multiple-outputs/test b/sbt/src/sbt-test/nio/multiple-outputs/test new file mode 100644 index 000000000..804dc4733 --- /dev/null +++ b/sbt/src/sbt-test/nio/multiple-outputs/test @@ -0,0 +1,23 @@ +> check + +$ copy-file changes/bad.md foo/foo.md + +$ copy-file changes/updated.md bar/bar.md + +# just in case the two of foo.md copies happen too quickly to update the last modified time +> setModified foo/foo.md 123456 + +-> check + +-> check + +$ copy-file changes/updated.md foo/foo.md + +# just in case the two of foo.md copies happen too quickly to update the last modified time +> setModified foo/foo.md 12345678 + +> check + +# the changes should be empty now but the content of foo/foo.md is no longer "foo" +-> check +